mirror of https://github.com/apache/druid.git
review comments
This commit is contained in:
parent
fb819abd6f
commit
3db33c3268
|
@ -60,7 +60,7 @@ public class Execs
|
|||
* @param capacity maximum capacity after which the executorService will block on accepting new tasks
|
||||
* @return ExecutorService which blocks accepting new tasks when the capacity reached
|
||||
*/
|
||||
public static ExecutorService blockingSingleThreaded(String nameFormat, int capacity)
|
||||
public static ExecutorService newBlockingSingleThreaded(String nameFormat, int capacity)
|
||||
{
|
||||
return new ThreadPoolExecutor(
|
||||
1, 1,
|
||||
|
|
|
@ -1,3 +1,22 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.concurrent;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
|
@ -5,7 +24,9 @@ import org.junit.Assert;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public class ExecsTest
|
||||
|
@ -14,17 +35,20 @@ public class ExecsTest
|
|||
public void testBlockingExecutorService() throws Exception
|
||||
{
|
||||
final int capacity = 3;
|
||||
final ExecutorService executorService = Execs.blockingSingleThreaded("test%d", capacity);
|
||||
final ExecutorService executorService = Execs.newBlockingSingleThreaded("test%d", capacity);
|
||||
final AtomicInteger producedCount = new AtomicInteger();
|
||||
final AtomicInteger consumedCount = new AtomicInteger();
|
||||
final CyclicBarrier barrier = new CyclicBarrier(2);
|
||||
Thread producer = new Thread("producer")
|
||||
ExecutorService producer = Executors.newCachedThreadPool();
|
||||
|
||||
producer.submit(
|
||||
new Runnable()
|
||||
{
|
||||
public void run()
|
||||
{
|
||||
for (int i = 0; i < 2 * capacity; i++) {
|
||||
final int taskID = i;
|
||||
System.out.println("Produced task"+ taskID);
|
||||
System.out.println("Produced task" + taskID);
|
||||
executorService.submit(
|
||||
new Runnable()
|
||||
{
|
||||
|
@ -33,12 +57,13 @@ public class ExecsTest
|
|||
{
|
||||
System.out.println("Starting task" + taskID);
|
||||
try {
|
||||
barrier.await();
|
||||
consumedCount.incrementAndGet();
|
||||
barrier.await(); //1
|
||||
barrier.await(); //2
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
consumedCount.incrementAndGet();
|
||||
System.out.println("Completed task" + taskID);
|
||||
}
|
||||
}
|
||||
|
@ -46,18 +71,23 @@ public class ExecsTest
|
|||
producedCount.incrementAndGet();
|
||||
}
|
||||
}
|
||||
};
|
||||
producer.start();
|
||||
}
|
||||
);
|
||||
while(producedCount.get() < capacity ){
|
||||
Thread.sleep(5);
|
||||
}
|
||||
|
||||
for(int i=0;i<capacity;i++){
|
||||
Thread.sleep(500);
|
||||
// total completed tasks +1 running task+ capacity = total produced tasks
|
||||
Assert.assertEquals(consumedCount.intValue()+1+capacity,producedCount.intValue());
|
||||
barrier.await();
|
||||
barrier.await(); //1
|
||||
// total consumed tasks + capacity = total produced tasks
|
||||
Assert.assertEquals(consumedCount.intValue() + capacity, producedCount.intValue());
|
||||
barrier.await(); //2
|
||||
|
||||
}
|
||||
for(int i=0;i<capacity;i++){
|
||||
barrier.await();
|
||||
}
|
||||
producer.shutdownNow();
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -479,7 +479,7 @@ public class RealtimePlumberSchool implements PlumberSchool
|
|||
{
|
||||
if (persistExecutor == null) {
|
||||
// use a blocking single threaded executor to throttle the firehose when write to disk is slow
|
||||
persistExecutor = Execs.blockingSingleThreaded(
|
||||
persistExecutor = Execs.newBlockingSingleThreaded(
|
||||
"plumber_persist_%d", maxPendingPersistBatches
|
||||
);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue