mirror of https://github.com/apache/druid.git
use num processing threads for realtime query execution
This commit is contained in:
parent
e03c623467
commit
1ec098c010
|
@ -189,6 +189,7 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
|
|||
injectables.put("segmentPublisher", getSegmentPublisher());
|
||||
injectables.put("serverView", getServerView());
|
||||
injectables.put("serviceEmitter", getEmitter());
|
||||
injectables.put("queryExecutorService", getQueryExecutorService());
|
||||
|
||||
getJsonMapper().setInjectableValues(
|
||||
new InjectableValues()
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -92,7 +91,6 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||
public class RealtimePlumberSchool implements PlumberSchool
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(RealtimePlumberSchool.class);
|
||||
private static final ListeningExecutorService EXEC = MoreExecutors.sameThreadExecutor();
|
||||
|
||||
private final Period windowPeriod;
|
||||
private final File basePersistDirectory;
|
||||
|
@ -108,6 +106,7 @@ public class RealtimePlumberSchool implements PlumberSchool
|
|||
private volatile DataSegmentAnnouncer segmentAnnouncer = null;
|
||||
private volatile SegmentPublisher segmentPublisher = null;
|
||||
private volatile ServerView serverView = null;
|
||||
private volatile ExecutorService queryExecutorService = null;
|
||||
|
||||
@JsonCreator
|
||||
public RealtimePlumberSchool(
|
||||
|
@ -175,6 +174,12 @@ public class RealtimePlumberSchool implements PlumberSchool
|
|||
this.emitter = emitter;
|
||||
}
|
||||
|
||||
@JacksonInject("queryExecutorService")
|
||||
public void setQueryExecutorService(ExecutorService queryExecutorService)
|
||||
{
|
||||
this.queryExecutorService = queryExecutorService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics)
|
||||
{
|
||||
|
@ -262,7 +267,7 @@ public class RealtimePlumberSchool implements PlumberSchool
|
|||
|
||||
return toolchest.mergeResults(
|
||||
factory.mergeRunners(
|
||||
EXEC,
|
||||
queryExecutorService,
|
||||
FunctionalIterable
|
||||
.create(querySinks)
|
||||
.transform(
|
||||
|
@ -277,7 +282,7 @@ public class RealtimePlumberSchool implements PlumberSchool
|
|||
emitter,
|
||||
builderFn,
|
||||
factory.mergeRunners(
|
||||
EXEC,
|
||||
MoreExecutors.sameThreadExecutor(),
|
||||
Iterables.transform(
|
||||
theSink,
|
||||
new Function<FireHydrant, QueryRunner<T>>()
|
||||
|
|
|
@ -23,19 +23,24 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.concurrent.ExecutorServiceConfig;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.collect.StupidPool;
|
||||
import com.metamx.druid.initialization.ServerInit;
|
||||
import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate;
|
||||
import com.metamx.druid.query.MetricsEmittingExecutorService;
|
||||
import com.metamx.druid.query.PrioritizedExecutorService;
|
||||
import com.metamx.druid.query.QueryRunnerFactory;
|
||||
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
||||
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.skife.config.ConfigurationObjectFactory;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -45,6 +50,7 @@ public abstract class BaseServerNode<T extends QueryableNode> extends QueryableN
|
|||
private DruidProcessingConfig processingConfig = null;
|
||||
private QueryRunnerFactoryConglomerate conglomerate = null;
|
||||
private StupidPool<ByteBuffer> computeScratchPool = null;
|
||||
private ExecutorService queryExecutorService = null;
|
||||
|
||||
public BaseServerNode(
|
||||
String nodeType,
|
||||
|
@ -77,6 +83,12 @@ public abstract class BaseServerNode<T extends QueryableNode> extends QueryableN
|
|||
return processingConfig;
|
||||
}
|
||||
|
||||
public ExecutorService getQueryExecutorService()
|
||||
{
|
||||
initializeQueryExecutorService();
|
||||
return queryExecutorService;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public T setConglomerate(QueryRunnerFactoryConglomerate conglomerate)
|
||||
{
|
||||
|
@ -98,6 +110,13 @@ public abstract class BaseServerNode<T extends QueryableNode> extends QueryableN
|
|||
return (T) this;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public T setQueryExecutorService(ExecutorService queryExecutorService)
|
||||
{
|
||||
checkFieldNotSetAndSet("queryExecutorService", queryExecutorService);
|
||||
return (T) this;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public T registerQueryRunnerFactory(Class<? extends Query> queryClazz, QueryRunnerFactory factory)
|
||||
{
|
||||
|
@ -144,4 +163,24 @@ public abstract class BaseServerNode<T extends QueryableNode> extends QueryableN
|
|||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void initializeQueryExecutorService()
|
||||
{
|
||||
if (queryExecutorService == null) {
|
||||
final PrioritizedExecutorService innerExecutorService = PrioritizedExecutorService.create(
|
||||
getLifecycle(),
|
||||
getConfigFactory().buildWithReplacements(
|
||||
ExecutorServiceConfig.class, ImmutableMap.of("base_path", "druid.processing")
|
||||
)
|
||||
);
|
||||
|
||||
setQueryExecutorService(
|
||||
new MetricsEmittingExecutorService(
|
||||
innerExecutorService,
|
||||
getEmitter(),
|
||||
new ServiceMetricEvent.Builder()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -97,21 +97,13 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
|
|||
final List<Monitor> monitors = getMonitors();
|
||||
final QueryRunnerFactoryConglomerate conglomerate = getConglomerate();
|
||||
|
||||
final PrioritizedExecutorService innerExecutorService = PrioritizedExecutorService.create(
|
||||
getLifecycle(),
|
||||
getConfigFactory().buildWithReplacements(
|
||||
ExecutorServiceConfig.class, ImmutableMap.of("base_path", "druid.processing")
|
||||
)
|
||||
);
|
||||
|
||||
final ExecutorService executorService = new MetricsEmittingExecutorService(
|
||||
innerExecutorService,
|
||||
final ServerManager serverManager = new ServerManager(
|
||||
getSegmentLoader(),
|
||||
conglomerate,
|
||||
emitter,
|
||||
new ServiceMetricEvent.Builder()
|
||||
getQueryExecutorService()
|
||||
);
|
||||
|
||||
final ServerManager serverManager = new ServerManager(getSegmentLoader(), conglomerate, emitter, executorService);
|
||||
|
||||
final ZkCoordinator coordinator = new ZkCoordinator(
|
||||
getJsonMapper(),
|
||||
getConfigFactory().build(ZkCoordinatorConfig.class),
|
||||
|
@ -206,5 +198,4 @@ public class ComputeNode extends BaseServerNode<ComputeNode>
|
|||
return new ComputeNode(props, lifecycle, jsonMapper, smileMapper, configFactory);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue