-
Notifications
You must be signed in to change notification settings - Fork 3.7k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Non querying tasks shouldn't use processing buffers / merge buffers (#…
…16887) Tasks that do not support querying or query processing i.e. supportsQueries = false do not require processing threads, processing buffers, and merge buffers.
- Loading branch information
1 parent
78775ad
commit 72fbaf2
Showing
7 changed files
with
352 additions
and
61 deletions.
There are no files selected for viewing
143 changes: 143 additions & 0 deletions
143
indexing-service/src/main/java/org/apache/druid/guice/PeonProcessingModule.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,143 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you 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 org.apache.druid.guice; | ||
|
||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
import com.google.inject.Binder; | ||
import com.google.inject.Module; | ||
import com.google.inject.Provides; | ||
import org.apache.druid.client.cache.CacheConfig; | ||
import org.apache.druid.client.cache.CachePopulator; | ||
import org.apache.druid.client.cache.CachePopulatorStats; | ||
import org.apache.druid.collections.BlockingPool; | ||
import org.apache.druid.collections.DummyBlockingPool; | ||
import org.apache.druid.collections.DummyNonBlockingPool; | ||
import org.apache.druid.collections.NonBlockingPool; | ||
import org.apache.druid.guice.annotations.Global; | ||
import org.apache.druid.guice.annotations.Merging; | ||
import org.apache.druid.guice.annotations.Smile; | ||
import org.apache.druid.indexing.common.task.Task; | ||
import org.apache.druid.java.util.common.lifecycle.Lifecycle; | ||
import org.apache.druid.java.util.common.logger.Logger; | ||
import org.apache.druid.query.DruidProcessingConfig; | ||
import org.apache.druid.query.ExecutorServiceMonitor; | ||
import org.apache.druid.query.NoopQueryProcessingPool; | ||
import org.apache.druid.query.QueryProcessingPool; | ||
import org.apache.druid.query.groupby.GroupByQueryConfig; | ||
import org.apache.druid.query.groupby.GroupByResourcesReservationPool; | ||
|
||
import java.nio.ByteBuffer; | ||
|
||
/** | ||
* This module fulfills the dependency injection of query processing and caching resources: buffer pools and | ||
* thread pools on Peon selectively. Only the peons for the tasks supporting queries need to allocate direct buffers | ||
* and thread pools. Thus, this is separate from the {@link DruidProcessingModule} to separate the needs of the peons and | ||
* the historicals | ||
* | ||
* @see DruidProcessingModule | ||
*/ | ||
public class PeonProcessingModule implements Module | ||
{ | ||
private static final Logger log = new Logger(PeonProcessingModule.class); | ||
|
||
@Override | ||
public void configure(Binder binder) | ||
{ | ||
DruidProcessingModule.registerConfigsAndMonitor(binder); | ||
} | ||
|
||
@Provides | ||
@LazySingleton | ||
public CachePopulator getCachePopulator( | ||
@Smile ObjectMapper smileMapper, | ||
CachePopulatorStats cachePopulatorStats, | ||
CacheConfig cacheConfig | ||
) | ||
{ | ||
return DruidProcessingModule.createCachePopulator(smileMapper, cachePopulatorStats, cacheConfig); | ||
} | ||
|
||
@Provides | ||
@ManageLifecycle | ||
public QueryProcessingPool getProcessingExecutorPool( | ||
Task task, | ||
DruidProcessingConfig config, | ||
ExecutorServiceMonitor executorServiceMonitor, | ||
Lifecycle lifecycle | ||
) | ||
{ | ||
if (task.supportsQueries()) { | ||
return DruidProcessingModule.createProcessingExecutorPool(config, executorServiceMonitor, lifecycle); | ||
} else { | ||
if (config.isNumThreadsConfigured()) { | ||
log.warn( | ||
"Ignoring the configured numThreads[%d] because task[%s] of type[%s] does not support queries", | ||
config.getNumThreads(), | ||
task.getId(), | ||
task.getType() | ||
); | ||
} | ||
return NoopQueryProcessingPool.instance(); | ||
} | ||
} | ||
|
||
@Provides | ||
@LazySingleton | ||
@Global | ||
public NonBlockingPool<ByteBuffer> getIntermediateResultsPool(Task task, DruidProcessingConfig config) | ||
{ | ||
if (task.supportsQueries()) { | ||
return DruidProcessingModule.createIntermediateResultsPool(config); | ||
} else { | ||
return DummyNonBlockingPool.instance(); | ||
} | ||
} | ||
|
||
@Provides | ||
@LazySingleton | ||
@Merging | ||
public BlockingPool<ByteBuffer> getMergeBufferPool(Task task, DruidProcessingConfig config) | ||
{ | ||
if (task.supportsQueries()) { | ||
return DruidProcessingModule.createMergeBufferPool(config); | ||
} else { | ||
if (config.isNumMergeBuffersConfigured()) { | ||
log.warn( | ||
"Ignoring the configured numMergeBuffers[%d] because task[%s] of type[%s] does not support queries", | ||
config.getNumThreads(), | ||
task.getId(), | ||
task.getType() | ||
); | ||
} | ||
return DummyBlockingPool.instance(); | ||
} | ||
} | ||
|
||
@Provides | ||
@LazySingleton | ||
@Merging | ||
public GroupByResourcesReservationPool getGroupByResourcesReservationPool( | ||
@Merging BlockingPool<ByteBuffer> mergeBufferPool, | ||
GroupByQueryConfig groupByQueryConfig | ||
) | ||
{ | ||
return new GroupByResourcesReservationPool(mergeBufferPool, groupByQueryConfig); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
134 changes: 134 additions & 0 deletions
134
processing/src/main/java/org/apache/druid/query/NoopQueryProcessingPool.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,134 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you 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 org.apache.druid.query; | ||
|
||
import com.google.common.util.concurrent.ListenableFuture; | ||
import org.apache.druid.error.DruidException; | ||
|
||
import java.util.Collection; | ||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.concurrent.Callable; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.TimeUnit; | ||
|
||
/** | ||
* Implementation of {@link QueryProcessingPool} that throws when any query execution task unit is submitted to it. It is | ||
* semantically shutdown from the moment it is created, and since the shutdown methods are supposed to be idempotent, | ||
* they do not throw like the execution methods | ||
*/ | ||
public class NoopQueryProcessingPool implements QueryProcessingPool | ||
{ | ||
private static final NoopQueryProcessingPool INSTANCE = new NoopQueryProcessingPool(); | ||
|
||
public static NoopQueryProcessingPool instance() | ||
{ | ||
return INSTANCE; | ||
} | ||
|
||
@Override | ||
public <T, V> ListenableFuture<T> submitRunnerTask(PrioritizedQueryRunnerCallable<T, V> task) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
@Override | ||
public <T> ListenableFuture<T> submit(Callable<T> callable) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
@Override | ||
public ListenableFuture<?> submit(Runnable runnable) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
@Override | ||
public <T> ListenableFuture<T> submit(Runnable runnable, T t) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
@Override | ||
public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> collection) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
@Override | ||
public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> collection, long l, TimeUnit timeUnit) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
@Override | ||
public void shutdown() | ||
{ | ||
// No op, since it is already shutdown | ||
} | ||
|
||
@Override | ||
public List<Runnable> shutdownNow() | ||
{ | ||
return Collections.emptyList(); | ||
} | ||
|
||
@Override | ||
public boolean isShutdown() | ||
{ | ||
return true; | ||
} | ||
|
||
@Override | ||
public boolean isTerminated() | ||
{ | ||
return true; | ||
} | ||
|
||
@Override | ||
public boolean awaitTermination(long l, TimeUnit timeUnit) | ||
{ | ||
return true; | ||
} | ||
|
||
@Override | ||
public <T> T invokeAny(Collection<? extends Callable<T>> collection) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
@Override | ||
public <T> T invokeAny(Collection<? extends Callable<T>> collection, long l, TimeUnit timeUnit) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
@Override | ||
public void execute(Runnable runnable) | ||
{ | ||
throw unsupportedException(); | ||
} | ||
|
||
private DruidException unsupportedException() | ||
{ | ||
return DruidException.defensive("Unexpected call made to NoopQueryProcessingPool"); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.