Skip to content

Commit

Permalink
Merge pull request #17017 from jasontedor/generic-thread-pool
Browse files Browse the repository at this point in the history
Actually bound the generic thread pool
  • Loading branch information
jasontedor committed Apr 26, 2016
2 parents e3126df + b89a935 commit efeec4d
Show file tree
Hide file tree
Showing 9 changed files with 492 additions and 182 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,16 +26,12 @@
import java.util.Arrays;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedTransferQueue;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;

/**
*
*/
public class EsExecutors {

/**
Expand All @@ -62,16 +58,11 @@ public static PrioritizedEsThreadPoolExecutor newSinglePrioritizing(String name,

public static EsThreadPoolExecutor newScaling(String name, int min, int max, long keepAliveTime, TimeUnit unit, ThreadFactory threadFactory, ThreadContext contextHolder) {
ExecutorScalingQueue<Runnable> queue = new ExecutorScalingQueue<>();
// we force the execution, since we might run into concurrency issues in offer for ScalingBlockingQueue
EsThreadPoolExecutor executor = new EsThreadPoolExecutor(name, min, max, keepAliveTime, unit, queue, threadFactory, new ForceQueuePolicy(), contextHolder);
queue.executor = executor;
return executor;
}

public static EsThreadPoolExecutor newCached(String name, long keepAliveTime, TimeUnit unit, ThreadFactory threadFactory, ThreadContext contextHolder) {
return new EsThreadPoolExecutor(name, 0, Integer.MAX_VALUE, keepAliveTime, unit, new SynchronousQueue<Runnable>(), threadFactory, new EsAbortPolicy(), contextHolder);
}

public static EsThreadPoolExecutor newFixed(String name, int size, int queueCapacity, ThreadFactory threadFactory, ThreadContext contextHolder) {
BlockingQueue<Runnable> queue;
if (queueCapacity < 0) {
Expand Down Expand Up @@ -114,6 +105,7 @@ public static ThreadFactory daemonThreadFactory(String namePrefix) {
}

static class EsThreadFactory implements ThreadFactory {

final ThreadGroup group;
final AtomicInteger threadNumber = new AtomicInteger(1);
final String namePrefix;
Expand All @@ -133,6 +125,7 @@ public Thread newThread(Runnable r) {
t.setDaemon(true);
return t;
}

}

/**
Expand All @@ -141,7 +134,6 @@ public Thread newThread(Runnable r) {
private EsExecutors() {
}


static class ExecutorScalingQueue<E> extends LinkedTransferQueue<E> {

ThreadPoolExecutor executor;
Expand All @@ -151,9 +143,17 @@ public ExecutorScalingQueue() {

@Override
public boolean offer(E e) {
// first try to transfer to a waiting worker thread
if (!tryTransfer(e)) {
// check if there might be spare capacity in the thread
// pool executor
int left = executor.getMaximumPoolSize() - executor.getCorePoolSize();
if (left > 0) {
// reject queuing the task to force the thread pool
// executor to add a worker if it can; combined
// with ForceQueuePolicy, this causes the thread
// pool to always scale up to max pool size and we
// only queue when there is no spare capacity
return false;
} else {
return super.offer(e);
Expand All @@ -162,6 +162,7 @@ public boolean offer(E e) {
return true;
}
}

}

/**
Expand All @@ -184,4 +185,5 @@ public long rejected() {
return 0;
}
}

}
180 changes: 108 additions & 72 deletions core/src/main/java/org/elasticsearch/threadpool/ThreadPool.java

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -50,8 +50,7 @@ protected Settings nodeSettings(int nodeOrdinal) {
//Have very low pool and queue sizes to overwhelm internal pools easily
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put("threadpool.generic.size", 1)
.put("threadpool.generic.queue_size", 1)
.put("threadpool.generic.max", 4)
// don't mess with this one! It's quite sensitive to a low queue size
// (see also ThreadedActionListener which is happily spawning threads even when we already got rejected)
//.put("threadpool.listener.queue_size", 1)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.threadpool;

import org.elasticsearch.test.ESTestCase;

import java.util.Map;
import java.util.stream.Collectors;

public abstract class ESThreadPoolTestCase extends ESTestCase {

protected final ThreadPool.Info info(final ThreadPool threadPool, final String name) {
for (final ThreadPool.Info info : threadPool.info()) {
if (info.getName().equals(name)) {
return info;
}
}
throw new IllegalArgumentException(name);
}

protected final ThreadPoolStats.Stats stats(final ThreadPool threadPool, final String name) {
for (final ThreadPoolStats.Stats stats : threadPool.stats()) {
if (name.equals(stats.getName())) {
return stats;
}
}
throw new IllegalArgumentException(name);
}

protected final void terminateThreadPoolIfNeeded(final ThreadPool threadPool) throws InterruptedException {
if (threadPool != null) {
terminate(threadPool);
}
}

static String randomThreadPool(final ThreadPool.ThreadPoolType type) {
return randomFrom(
ThreadPool.THREAD_POOL_TYPES
.entrySet().stream()
.filter(t -> t.getValue().equals(type))
.map(Map.Entry::getKey)
.collect(Collectors.toList()));
}

}
Loading

0 comments on commit efeec4d

Please sign in to comment.