Skip to content

Commit

Permalink
Rename the bulk thread pool to write thread pool (#29593)
Browse files Browse the repository at this point in the history
This commit renames the bulk thread pool to the write thread pool. This
is to better reflect the fact that the underlying thread pool is used to
execute any document write request (single-document index/delete/update
requests, and bulk requests).

With this change, we add support for fallback settings
thread_pool.bulk.* which will be supported until 7.0.0.

We also add a system property so that the display name of the thread
pool remains as "bulk" if needed to avoid breaking users.
  • Loading branch information
jasontedor committed Apr 19, 2018
1 parent fa10520 commit c12c2a6
Show file tree
Hide file tree
Showing 23 changed files with 162 additions and 81 deletions.
4 changes: 2 additions & 2 deletions docs/reference/cat.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -93,8 +93,8 @@ Responds with:
// TESTRESPONSE[s/9300 27 sLBaIGK/\\d+ \\d+ .+/ _cat]

You can also request multiple columns using simple wildcards like
`/_cat/thread_pool?h=ip,bulk.*` to get all headers (or aliases) starting
with `bulk.`.
`/_cat/thread_pool?h=ip,queue*` to get all headers (or aliases) starting
with `queue`.

[float]
[[numeric-formats]]
Expand Down
4 changes: 2 additions & 2 deletions docs/reference/cat/thread_pool.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ Which looks like:
[source,txt]
--------------------------------------------------
node-0 analyze 0 0 0
node-0 bulk 0 0 0
node-0 fetch_shard_started 0 0 0
node-0 fetch_shard_store 0 0 0
node-0 flush 0 0 0
Expand All @@ -28,6 +27,7 @@ node-0 refresh 0 0 0
node-0 search 0 0 0
node-0 snapshot 0 0 0
node-0 warmer 0 0 0
node-0 write 0 0 0
--------------------------------------------------
// TESTRESPONSE[s/\d+/\\d+/ _cat]

Expand All @@ -44,7 +44,6 @@ The second column is the thread pool name
--------------------------------------------------
name
analyze
bulk
fetch_shard_started
fetch_shard_store
flush
Expand All @@ -57,6 +56,7 @@ refresh
search
snapshot
warmer
write
--------------------------------------------------


Expand Down
9 changes: 4 additions & 5 deletions docs/reference/modules/threadpool.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,10 @@ There are several thread pools, but the important ones include:
`analyze`::
For analyze requests. Thread pool type is `fixed` with a size of 1, queue size of 16.

`bulk`::
For bulk operations. Thread pool type is `fixed`
with a size of `# of available processors`,
queue_size of `200`. The maximum size for this pool
is `1 + # of available processors`.
`write`::
For single-document index/delete/update and bulk requests. Thread pool type
is `fixed` with a size of `# of available processors`, queue_size of `200`.
The maximum size for this pool is `1 + # of available processors`.

`snapshot`::
For snapshot/restore operations. Thread pool type is `scaling` with a
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -158,10 +158,10 @@ private void testCase(

final Settings nodeSettings = Settings.builder()
// use pools of size 1 so we can block them
.put("thread_pool.bulk.size", 1)
.put("thread_pool.write.size", 1)
.put("thread_pool.search.size", 1)
// use queues of size 1 because size 0 is broken and because search requests need the queue to function
.put("thread_pool.bulk.queue_size", 1)
.put("thread_pool.write.queue_size", 1)
.put("thread_pool.search.queue_size", 1)
.put("node.attr.color", "blue")
.build();
Expand Down Expand Up @@ -203,7 +203,7 @@ private void testCase(
assertBusy(() -> assertThat(taskStatus(action).getSearchRetries(), greaterThan(0L)));

logger.info("Blocking bulk and unblocking search so we start to get bulk rejections");
CyclicBarrier bulkBlock = blockExecutor(ThreadPool.Names.BULK, node);
CyclicBarrier bulkBlock = blockExecutor(ThreadPool.Names.WRITE, node);
initialSearchBlock.await();

logger.info("Waiting for bulk rejections");
Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,5 @@
---
"Test cat thread_pool output":

- skip:
version: " - 6.99.99"
reason: this API was changed in a backwards-incompatible fashion in 7.0.0 so we need to skip in a mixed cluster
Expand Down Expand Up @@ -33,29 +32,29 @@
- do:
cat.thread_pool:
thread_pool_patterns: bulk,management,flush,generic,force_merge
thread_pool_patterns: write,management,flush,generic,force_merge
h: id,name,active
v: true

- match:
$body: |
/^ id \s+ name \s+ active \n
(\S+\s+ bulk \s+ \d+ \n
\S+\s+ flush \s+ \d+ \n
(\S+\s+ flush \s+ \d+ \n
\S+\s+ force_merge \s+ \d+ \n
\S+\s+ generic \s+ \d+ \n
\S+\s+ management \s+ \d+ \n)+ $/
\S+\s+ management \s+ \d+ \n
\S+\s+ write \s+ \d+ \n)+ $/
- do:
cat.thread_pool:
thread_pool_patterns: bulk
h: id,name,type,active,pool_size,queue,queue_size,rejected,largest,completed,core,max,size,keep_alive
thread_pool_patterns: write
h: id,name,type,active,size,queue,queue_size,rejected,largest,completed,min,max,keep_alive
v: true

- match:
$body: |
/^ id \s+ name \s+ type \s+ active \s+ pool_size \s+ queue \s+ queue_size \s+ rejected \s+ largest \s+ completed \s+ core \s+ max \s+ size \s+ keep_alive \n
(\S+ \s+ bulk \s+ fixed \s+ \d+ \s+ \d+ \s+ \d+ \s+ (-1|\d+) \s+ \d+ \s+ \d+ \s+ \d+ \s+ \d* \s+ \d* \s+ \d* \s+ \S* \n)+ $/
/^ id \s+ name \s+ type \s+ active \s+ size \s+ queue \s+ queue_size \s+ rejected \s+ largest \s+ completed \s+ max \s+ keep_alive \n
(\S+ \s+ write \s+ fixed \s+ \d+ \s+ \d+ \s+ \d+ \s+ (-1|\d+) \s+ \d+ \s+ \d+ \s+ \d+ \s+ \d* \s+ \S* \n)+ $/
- do:
cat.thread_pool:
Expand All @@ -71,11 +70,11 @@
- do:
cat.thread_pool:
thread_pool_patterns: bulk,search
thread_pool_patterns: write,search
size: ""

- match:
$body: |
/ #node_name name active queue rejected
^ (\S+ \s+ bulk \s+ \d+ \s+ \d+ \s+ \d+ \n
\S+ \s+ search \s+ \d+ \s+ \d+ \s+ \d+ \n)+ $/
^ (\S+ \s+ search \s+ \d+ \s+ \d+ \s+ \d+ \n
\S+ \s+ write \s+ \d+ \s+ \d+ \s+ \d+ \n)+ $/
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ public TransportShardBulkAction(Settings settings, TransportService transportSer
MappingUpdatedAction mappingUpdatedAction, UpdateHelper updateHelper, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
indexNameExpressionResolver, BulkShardRequest::new, BulkShardRequest::new, ThreadPool.Names.BULK);
indexNameExpressionResolver, BulkShardRequest::new, BulkShardRequest::new, ThreadPool.Names.WRITE);
this.updateHelper = updateHelper;
this.mappingUpdatedAction = mappingUpdatedAction;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public TransportDeleteAction(Settings settings, TransportService transportServic
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
TransportBulkAction bulkAction, TransportShardBulkAction shardBulkAction) {
super(settings, DeleteAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction,
actionFilters, indexNameExpressionResolver, DeleteRequest::new, DeleteRequest::new, ThreadPool.Names.BULK,
actionFilters, indexNameExpressionResolver, DeleteRequest::new, DeleteRequest::new, ThreadPool.Names.WRITE,
bulkAction, shardBulkAction);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ public TransportIndexAction(Settings settings, TransportService transportService
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
TransportBulkAction bulkAction, TransportShardBulkAction shardBulkAction) {
super(settings, IndexAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction,
actionFilters, indexNameExpressionResolver, IndexRequest::new, IndexRequest::new, ThreadPool.Names.BULK,
actionFilters, indexNameExpressionResolver, IndexRequest::new, IndexRequest::new, ThreadPool.Names.WRITE,
bulkAction, shardBulkAction);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ public TransportResyncReplicationAction(Settings settings, TransportService tran
ShardStateAction shardStateAction, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
indexNameExpressionResolver, ResyncReplicationRequest::new, ResyncReplicationRequest::new, ThreadPool.Names.BULK);
indexNameExpressionResolver, ResyncReplicationRequest::new, ResyncReplicationRequest::new, ThreadPool.Names.WRITE);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ public TransportUpdateAction(Settings settings, ThreadPool threadPool, ClusterSe

@Override
protected String executor() {
return ThreadPool.Names.BULK;
return ThreadPool.Names.WRITE;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public PipelineExecutionService(PipelineStore store, ThreadPool threadPool) {
public void executeBulkRequest(Iterable<DocWriteRequest> actionRequests,
BiConsumer<IndexRequest, Exception> itemFailureHandler,
Consumer<Exception> completionHandler) {
threadPool.executor(ThreadPool.Names.BULK).execute(new AbstractRunnable() {
threadPool.executor(ThreadPool.Names.WRITE).execute(new AbstractRunnable() {

@Override
public void onFailure(Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ protected static String settingsKey(final String prefix, final String key) {
}

protected int applyHardSizeLimit(final Settings settings, final String name) {
if (name.equals(ThreadPool.Names.BULK)) {
if (name.equals("bulk") || name.equals(ThreadPool.Names.WRITE)) {
return 1 + EsExecutors.numberOfProcessors(settings);
} else {
return Integer.MAX_VALUE;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.elasticsearch.threadpool;

import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.SizeValue;
Expand All @@ -38,7 +39,9 @@
public final class FixedExecutorBuilder extends ExecutorBuilder<FixedExecutorBuilder.FixedExecutorSettings> {

private final Setting<Integer> sizeSetting;
private final Setting<Integer> fallbackSizeSetting;
private final Setting<Integer> queueSizeSetting;
private final Setting<Integer> fallbackQueueSizeSetting;

/**
* Construct a fixed executor builder; the settings will have the key prefix "thread_pool." followed by the executor name.
Expand All @@ -52,6 +55,19 @@ public final class FixedExecutorBuilder extends ExecutorBuilder<FixedExecutorBui
this(settings, name, size, queueSize, "thread_pool." + name);
}

/**
* Construct a fixed executor builder; the settings will have the key prefix "thread_pool." followed by the executor name.
*
* @param settings the node-level settings
* @param name the name of the executor
* @param fallbackName the fallback name of the executor (used for transitioning the name of a setting)
* @param size the fixed number of threads
* @param queueSize the size of the backing queue, -1 for unbounded
*/
FixedExecutorBuilder(final Settings settings, final String name, final String fallbackName, final int size, final int queueSize) {
this(settings, name, fallbackName, size, queueSize, "thread_pool." + name, "thread_pool." + fallbackName);
}

/**
* Construct a fixed executor builder.
*
Expand All @@ -62,21 +78,81 @@ public final class FixedExecutorBuilder extends ExecutorBuilder<FixedExecutorBui
* @param prefix the prefix for the settings keys
*/
public FixedExecutorBuilder(final Settings settings, final String name, final int size, final int queueSize, final String prefix) {
this(settings, name, null, size, queueSize, prefix, null);
}

/**
* Construct a fixed executor builder.
*
* @param settings the node-level settings
* @param name the name of the executor
* @param size the fixed number of threads
* @param queueSize the size of the backing queue, -1 for unbounded
* @param prefix the prefix for the settings keys
*/
private FixedExecutorBuilder(
final Settings settings,
final String name,
final String fallbackName,
final int size,
final int queueSize,
final String prefix,
final String fallbackPrefix) {
super(name);
final String sizeKey = settingsKey(prefix, "size");
this.sizeSetting =
new Setting<>(
sizeKey,
s -> Integer.toString(size),
s -> Setting.parseInt(s, 1, applyHardSizeLimit(settings, name), sizeKey),
Setting.Property.NodeScope);
final String queueSizeKey = settingsKey(prefix, "queue_size");
this.queueSizeSetting = Setting.intSetting(queueSizeKey, queueSize, Setting.Property.NodeScope);
if (fallbackName == null) {
assert fallbackPrefix == null;
final Setting.Property[] properties = {Setting.Property.NodeScope};
this.sizeSetting = sizeSetting(settings, name, size, prefix, properties);
this.fallbackSizeSetting = null;
this.queueSizeSetting = queueSizeSetting(prefix, queueSize, properties);
this.fallbackQueueSizeSetting = null;
} else {
assert fallbackPrefix != null;
final Setting.Property[] properties = { Setting.Property.NodeScope };
final Setting.Property[] fallbackProperties = { Setting.Property.NodeScope, Setting.Property.Deprecated };
final Setting<Integer> fallbackSizeSetting = sizeSetting(settings, fallbackName, size, fallbackPrefix, fallbackProperties);
this.sizeSetting =
new Setting<>(
new Setting.SimpleKey(sizeKey),
fallbackSizeSetting,
s -> Setting.parseInt(s, 1, applyHardSizeLimit(settings, name), sizeKey),
properties);
this.fallbackSizeSetting = fallbackSizeSetting;
final Setting<Integer> fallbackQueueSizeSetting = queueSizeSetting(fallbackPrefix, queueSize, fallbackProperties);
this.queueSizeSetting =
new Setting<>(
new Setting.SimpleKey(queueSizeKey),
fallbackQueueSizeSetting,
s -> Setting.parseInt(s, Integer.MIN_VALUE, queueSizeKey),
properties);
this.fallbackQueueSizeSetting = fallbackQueueSizeSetting;
}
}

private Setting<Integer> sizeSetting(
final Settings settings, final String name, final int size, final String prefix, final Setting.Property[] properties) {
final String sizeKey = settingsKey(prefix, "size");
return new Setting<>(
sizeKey,
s -> Integer.toString(size),
s -> Setting.parseInt(s, 1, applyHardSizeLimit(settings, name), sizeKey),
properties);
}

private Setting<Integer> queueSizeSetting(final String prefix, final int queueSize, final Setting.Property[] properties) {
return Setting.intSetting(settingsKey(prefix, "queue_size"), queueSize, properties);
}

@Override
public List<Setting<?>> getRegisteredSettings() {
return Arrays.asList(sizeSetting, queueSizeSetting);
if (fallbackSizeSetting == null && fallbackQueueSizeSetting == null) {
return Arrays.asList(sizeSetting, queueSizeSetting);
} else {
assert fallbackSizeSetting != null && fallbackQueueSizeSetting != null;
return Arrays.asList(sizeSetting, fallbackSizeSetting, queueSizeSetting, fallbackQueueSizeSetting);
}
}

@Override
Expand All @@ -94,8 +170,14 @@ ThreadPool.ExecutorHolder build(final FixedExecutorSettings settings, final Thre
final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(EsExecutors.threadName(settings.nodeName, name()));
final ExecutorService executor =
EsExecutors.newFixed(settings.nodeName + "/" + name(), size, queueSize, threadFactory, threadContext);
final String name;
if ("write".equals(name()) && Booleans.parseBoolean(System.getProperty("es.thread_pool.write.use_bulk_as_display_name", "false"))) {
name = "bulk";
} else {
name = name();
}
final ThreadPool.Info info =
new ThreadPool.Info(name(), ThreadPool.ThreadPoolType.FIXED, size, size, null, queueSize < 0 ? null : new SizeValue(queueSize));
new ThreadPool.Info(name, ThreadPool.ThreadPoolType.FIXED, size, size, null, queueSize < 0 ? null : new SizeValue(queueSize));
return new ThreadPool.ExecutorHolder(executor, info);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ public static class Names {
public static final String LISTENER = "listener";
public static final String GET = "get";
public static final String ANALYZE = "analyze";
public static final String BULK = "bulk";
public static final String WRITE = "write";
public static final String SEARCH = "search";
public static final String MANAGEMENT = "management";
public static final String FLUSH = "flush";
Expand Down Expand Up @@ -125,7 +125,7 @@ public static ThreadPoolType fromType(String type) {
map.put(Names.LISTENER, ThreadPoolType.FIXED);
map.put(Names.GET, ThreadPoolType.FIXED);
map.put(Names.ANALYZE, ThreadPoolType.FIXED);
map.put(Names.BULK, ThreadPoolType.FIXED);
map.put(Names.WRITE, ThreadPoolType.FIXED);
map.put(Names.SEARCH, ThreadPoolType.FIXED_AUTO_QUEUE_SIZE);
map.put(Names.MANAGEMENT, ThreadPoolType.SCALING);
map.put(Names.FLUSH, ThreadPoolType.SCALING);
Expand Down Expand Up @@ -170,7 +170,7 @@ public ThreadPool(final Settings settings, final ExecutorBuilder<?>... customBui
final int halfProcMaxAt10 = halfNumberOfProcessorsMaxTen(availableProcessors);
final int genericThreadPoolMax = boundedBy(4 * availableProcessors, 128, 512);
builders.put(Names.GENERIC, new ScalingExecutorBuilder(Names.GENERIC, 4, genericThreadPoolMax, TimeValue.timeValueSeconds(30)));
builders.put(Names.BULK, new FixedExecutorBuilder(settings, Names.BULK, availableProcessors, 200)); // now that we reuse bulk for index/delete ops
builders.put(Names.WRITE, new FixedExecutorBuilder(settings, Names.WRITE, "bulk", availableProcessors, 200));
builders.put(Names.GET, new FixedExecutorBuilder(settings, Names.GET, availableProcessors, 1000));
builders.put(Names.ANALYZE, new FixedExecutorBuilder(settings, Names.ANALYZE, 1, 16));
builders.put(Names.SEARCH, new AutoQueueAdjustingExecutorBuilder(settings,
Expand Down Expand Up @@ -264,7 +264,7 @@ public Info info(String name) {
public ThreadPoolStats stats() {
List<ThreadPoolStats.Stats> stats = new ArrayList<>();
for (ExecutorHolder holder : executors.values()) {
String name = holder.info.getName();
final String name = holder.info.getName();
// no need to have info on "same" thread pool
if ("same".equals(name)) {
continue;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,8 +54,8 @@ protected Settings nodeSettings(int nodeOrdinal) {
// (see also ThreadedActionListener which is happily spawning threads even when we already got rejected)
//.put("thread_pool.listener.queue_size", 1)
.put("thread_pool.get.queue_size", 1)
// default is 50
.put("thread_pool.bulk.queue_size", 30)
// default is 200
.put("thread_pool.write.queue_size", 30)
.build();
}

Expand Down
Loading

0 comments on commit c12c2a6

Please sign in to comment.