/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.server;
import com.facebook.presto.OutputBuffers;
import com.facebook.presto.Session;
import com.facebook.presto.execution.LocationFactory;
import com.facebook.presto.execution.NodeTaskMap.PartitionedSplitCountTracker;
import com.facebook.presto.execution.QueryManagerConfig;
import com.facebook.presto.execution.RemoteTask;
import com.facebook.presto.execution.RemoteTaskFactory;
import com.facebook.presto.execution.TaskId;
import com.facebook.presto.execution.TaskInfo;
import com.facebook.presto.execution.TaskManagerConfig;
import com.facebook.presto.execution.TaskStatus;
import com.facebook.presto.metadata.Split;
import com.facebook.presto.operator.ForScheduler;
import com.facebook.presto.server.remotetask.HttpRemoteTask;
import com.facebook.presto.server.remotetask.RemoteTaskStats;
import com.facebook.presto.spi.Node;
import com.facebook.presto.sql.planner.PlanFragment;
import com.facebook.presto.sql.planner.plan.PlanNodeId;
import com.google.common.collect.Multimap;
import io.airlift.concurrent.BoundedExecutor;
import io.airlift.concurrent.ThreadPoolExecutorMBean;
import io.airlift.http.client.HttpClient;
import io.airlift.json.JsonCodec;
import io.airlift.units.Duration;
import org.weakref.jmx.Managed;
import org.weakref.jmx.Nested;
import javax.annotation.PreDestroy;
import javax.inject.Inject;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadPoolExecutor;
import static com.google.common.base.Preconditions.checkArgument;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
public class HttpRemoteTaskFactory
implements RemoteTaskFactory
{
private final HttpClient httpClient;
private final LocationFactory locationFactory;
private final JsonCodec<TaskStatus> taskStatusCodec;
private final JsonCodec<TaskInfo> taskInfoCodec;
private final JsonCodec<TaskUpdateRequest> taskUpdateRequestCodec;
private final Duration minErrorDuration;
private final Duration maxErrorDuration;
private final Duration taskStatusRefreshMaxWait;
private final Duration taskInfoUpdateInterval;
private final ExecutorService coreExecutor;
private final Executor executor;
private final ThreadPoolExecutorMBean executorMBean;
private final ScheduledExecutorService updateScheduledExecutor;
private final ScheduledExecutorService errorScheduledExecutor;
private final RemoteTaskStats stats;
@Inject
public HttpRemoteTaskFactory(QueryManagerConfig config,
TaskManagerConfig taskConfig,
@ForScheduler HttpClient httpClient,
LocationFactory locationFactory,
JsonCodec<TaskStatus> taskStatusCodec,
JsonCodec<TaskInfo> taskInfoCodec,
JsonCodec<TaskUpdateRequest> taskUpdateRequestCodec,
RemoteTaskStats stats)
{
this.httpClient = httpClient;
this.locationFactory = locationFactory;
this.taskStatusCodec = taskStatusCodec;
this.taskInfoCodec = taskInfoCodec;
this.taskUpdateRequestCodec = taskUpdateRequestCodec;
checkArgument(config.getRemoteTaskMaxErrorDuration().compareTo(config.getRemoteTaskMinErrorDuration()) >= 0, "max error duration is less than min error duration");
this.minErrorDuration = config.getRemoteTaskMinErrorDuration();
this.maxErrorDuration = config.getRemoteTaskMaxErrorDuration();
this.taskStatusRefreshMaxWait = taskConfig.getStatusRefreshMaxWait();
this.taskInfoUpdateInterval = taskConfig.getInfoUpdateInterval();
this.coreExecutor = newCachedThreadPool(daemonThreadsNamed("remote-task-callback-%s"));
this.executor = new BoundedExecutor(coreExecutor, config.getRemoteTaskMaxCallbackThreads());
this.executorMBean = new ThreadPoolExecutorMBean((ThreadPoolExecutor) coreExecutor);
this.stats = requireNonNull(stats, "stats is null");
this.updateScheduledExecutor = newSingleThreadScheduledExecutor(daemonThreadsNamed("task-info-update-scheduler-%s"));
this.errorScheduledExecutor = newSingleThreadScheduledExecutor(daemonThreadsNamed("remote-task-error-delay-%s"));
}
@Managed
@Nested
public ThreadPoolExecutorMBean getExecutor()
{
return executorMBean;
}
@PreDestroy
public void stop()
{
coreExecutor.shutdownNow();
updateScheduledExecutor.shutdownNow();
errorScheduledExecutor.shutdownNow();
}
@Override
public RemoteTask createRemoteTask(Session session,
TaskId taskId,
Node node,
PlanFragment fragment,
Multimap<PlanNodeId, Split> initialSplits,
OutputBuffers outputBuffers,
PartitionedSplitCountTracker partitionedSplitCountTracker,
boolean summarizeTaskInfo)
{
return new HttpRemoteTask(session,
taskId,
node.getNodeIdentifier(),
locationFactory.createTaskLocation(node, taskId),
fragment,
initialSplits,
outputBuffers,
httpClient,
executor,
updateScheduledExecutor,
errorScheduledExecutor,
minErrorDuration,
maxErrorDuration,
taskStatusRefreshMaxWait,
taskInfoUpdateInterval,
summarizeTaskInfo,
taskStatusCodec,
taskInfoCodec,
taskUpdateRequestCodec,
partitionedSplitCountTracker,
stats);
}
}