Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* 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.dispatcher;

import javax.ws.rs.core.UriInfo;

import java.net.URI;

public interface CoordinatorLocation
{
URI getUri(UriInfo uriInfo, String xForwardedProto);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,120 @@
/*
* 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.dispatcher;

import com.facebook.airlift.concurrent.BoundedExecutor;
import com.facebook.airlift.concurrent.ThreadPoolExecutorMBean;
import com.facebook.presto.execution.QueryManagerConfig;
import com.google.common.io.Closer;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
import org.weakref.jmx.Flatten;
import org.weakref.jmx.Managed;
import org.weakref.jmx.Nested;

import javax.annotation.PreDestroy;
import javax.inject.Inject;

import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadPoolExecutor;

import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed;
import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static java.util.concurrent.Executors.newScheduledThreadPool;

public class DispatchExecutor
{
private final Closer closer = Closer.create();

private final ListeningExecutorService executor;
private final BoundedExecutor boundedExecutor;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm. 😃

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible to also export the number of active threads in BoundedExecutor? -- this has been an operation pain when we want to monitor when BoundedExecutor is full.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BoundedExecutor does not expose the number of active threads, however the underlying executor itself does export itself as an MBean via ThreadPoolExecutorMBean

private final ListeningScheduledExecutorService scheduledExecutor;

private final DispatchExecutorMBeans mbeans;

@Inject
public DispatchExecutor(QueryManagerConfig config)
{
ExecutorService coreExecutor = newCachedThreadPool(daemonThreadsNamed("dispatcher-query-%s"));
closer.register(coreExecutor::shutdownNow);
executor = listeningDecorator(coreExecutor);
boundedExecutor = new BoundedExecutor(coreExecutor, config.getQuerySubmissionMaxThreads());

ScheduledExecutorService coreScheduledExecutor = newScheduledThreadPool(config.getQueryManagerExecutorPoolSize(), daemonThreadsNamed("dispatch-executor-%s"));
closer.register(coreScheduledExecutor::shutdownNow);
scheduledExecutor = listeningDecorator(coreScheduledExecutor);

mbeans = new DispatchExecutorMBeans(coreExecutor, coreScheduledExecutor);
}

public ListeningExecutorService getExecutor()
{
return executor;
}

public BoundedExecutor getBoundedExecutor()
{
return boundedExecutor;
}

public ListeningScheduledExecutorService getScheduledExecutor()
{
return scheduledExecutor;
}

@Managed
@Flatten
public DispatchExecutorMBeans getMbeans()
{
return mbeans;
}

@PreDestroy
public void shutdown()
throws Exception
{
closer.close();
}

public class DispatchExecutorMBeans
{
private final ThreadPoolExecutorMBean executor;
private final ThreadPoolExecutorMBean scheduledExecutor;

public DispatchExecutorMBeans(ExecutorService coreExecutor, ScheduledExecutorService coreScheduledExecutor)
{
requireNonNull(coreExecutor, "coreExecutor is null");
requireNonNull(coreScheduledExecutor, "coreScheduledExecutor is null");
executor = new ThreadPoolExecutorMBean((ThreadPoolExecutor) coreExecutor);
scheduledExecutor = new ThreadPoolExecutorMBean((ThreadPoolExecutor) coreScheduledExecutor);
}

@Managed
@Nested
public ThreadPoolExecutorMBean getExecutor()
{
return executor;
}

@Managed
@Nested
public ThreadPoolExecutorMBean getScheduledExecutor()
{
return scheduledExecutor;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.dispatcher;

import com.facebook.presto.execution.ExecutionFailureInfo;
import io.airlift.units.Duration;

import java.util.Optional;

import static java.util.Objects.requireNonNull;

public class DispatchInfo
{
private final Optional<CoordinatorLocation> coordinatorLocation;
private final Optional<ExecutionFailureInfo> failureInfo;
private final Duration elapsedTime;
private final Duration queuedTime;

public static DispatchInfo queued(Duration elapsedTime, Duration queuedTime)
{
return new DispatchInfo(Optional.empty(), Optional.empty(), elapsedTime, queuedTime);
}

public static DispatchInfo dispatched(CoordinatorLocation coordinatorLocation, Duration elapsedTime, Duration queuedTime)
{
requireNonNull(coordinatorLocation, "coordinatorLocation is null");
return new DispatchInfo(Optional.of(coordinatorLocation), Optional.empty(), elapsedTime, queuedTime);
}

public static DispatchInfo failed(ExecutionFailureInfo failureInfo, Duration elapsedTime, Duration queuedTime)
{
requireNonNull(failureInfo, "coordinatorLocation is null");
return new DispatchInfo(Optional.empty(), Optional.of(failureInfo), elapsedTime, queuedTime);
}

private DispatchInfo(Optional<CoordinatorLocation> coordinatorLocation, Optional<ExecutionFailureInfo> failureInfo, Duration elapsedTime, Duration queuedTime)
{
this.coordinatorLocation = requireNonNull(coordinatorLocation, "coordinatorLocation is null");
this.failureInfo = requireNonNull(failureInfo, "failureInfo is null");
this.elapsedTime = requireNonNull(elapsedTime, "elapsedTime is null");
this.queuedTime = requireNonNull(queuedTime, "queuedTime is null");
}

public Optional<CoordinatorLocation> getCoordinatorLocation()
{
return coordinatorLocation;
}

public Optional<ExecutionFailureInfo> getFailureInfo()
{
return failureInfo;
}

public Duration getElapsedTime()
{
return elapsedTime;
}

public Duration getQueuedTime()
{
return queuedTime;
}
}
Loading