HBASE-3139 Server shutdown processor stuck because meta not online
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1026567 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5deb69715c
commit
1d6743ae7f
|
@ -609,6 +609,7 @@ Release 0.21.0 - Unreleased
|
|||
HBASE-2984 [shell] Altering a family shouldn't reset to default unchanged
|
||||
attributes
|
||||
HBASE-3143 Adding the tests' hbase-site.xml to the jar breaks some clients
|
||||
HBASE-3139 Server shutdown processor stuck because meta not online
|
||||
|
||||
|
||||
IMPROVEMENTS
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.PriorityBlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -41,7 +41,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
* threadpool, a queue to which {@link EventHandler.EventType}s can be submitted,
|
||||
* and a <code>Runnable</code> that handles the object that is added to the queue.
|
||||
*
|
||||
* <p>In order to create a new service, create an instance of this class and
|
||||
* <p>In order to create a new service, create an instance of this class and
|
||||
* then do: <code>instance.startExecutorService("myService");</code>. When done
|
||||
* call {@link #shutdown()}.
|
||||
*
|
||||
|
@ -243,16 +243,13 @@ public class ExecutorService {
|
|||
/**
|
||||
* Executor instance.
|
||||
*/
|
||||
private static class Executor {
|
||||
// default number of threads in the pool
|
||||
private int corePoolSize = 1;
|
||||
static class Executor {
|
||||
// how long to retain excess threads
|
||||
private long keepAliveTimeInMillis = 1000;
|
||||
final long keepAliveTimeInMillis = 1000;
|
||||
// the thread pool executor that services the requests
|
||||
private final ThreadPoolExecutor threadPoolExecutor;
|
||||
final ThreadPoolExecutor threadPoolExecutor;
|
||||
// work queue to use - unbounded queue
|
||||
BlockingQueue<Runnable> workQueue = new PriorityBlockingQueue<Runnable>();
|
||||
private final AtomicInteger threadid = new AtomicInteger(0);
|
||||
final BlockingQueue<Runnable> q = new LinkedBlockingQueue<Runnable>();
|
||||
private final String name;
|
||||
private final Map<EventHandler.EventType, EventHandlerListener> eventHandlerListeners;
|
||||
|
||||
|
@ -261,11 +258,11 @@ public class ExecutorService {
|
|||
this.name = name;
|
||||
this.eventHandlerListeners = eventHandlerListeners;
|
||||
// create the thread pool executor
|
||||
this.threadPoolExecutor = new ThreadPoolExecutor(corePoolSize, maxThreads,
|
||||
keepAliveTimeInMillis, TimeUnit.MILLISECONDS, workQueue);
|
||||
this.threadPoolExecutor = new ThreadPoolExecutor(maxThreads, maxThreads,
|
||||
keepAliveTimeInMillis, TimeUnit.MILLISECONDS, q);
|
||||
// name the threads for this threadpool
|
||||
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
|
||||
tfb.setNameFormat(this.name + "-" + this.threadid.incrementAndGet());
|
||||
tfb.setNameFormat(this.name + "-%d");
|
||||
this.threadPoolExecutor.setThreadFactory(tfb.build());
|
||||
}
|
||||
|
||||
|
|
|
@ -497,13 +497,13 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
try {
|
||||
// Start the executor service pools
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
|
||||
conf.getInt("hbase.master.executor.openregion.threads", 10));
|
||||
conf.getInt("hbase.master.executor.openregion.threads", 5));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
|
||||
conf.getInt("hbase.master.executor.closeregion.threads", 10));
|
||||
conf.getInt("hbase.master.executor.closeregion.threads", 5));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
|
||||
conf.getInt("hbase.master.executor.serverops.threads", 5));
|
||||
conf.getInt("hbase.master.executor.serverops.threads", 3));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS,
|
||||
conf.getInt("hbase.master.executor.tableops.threads", 5));
|
||||
conf.getInt("hbase.master.executor.tableops.threads", 3));
|
||||
|
||||
// Put up info server.
|
||||
int port = this.conf.getInt("hbase.master.info.port", 60010);
|
||||
|
|
|
@ -1143,13 +1143,13 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
// Start executor services
|
||||
this.service = new ExecutorService(getServerName());
|
||||
this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
|
||||
conf.getInt("hbase.regionserver.executor.openregion.threads", 5));
|
||||
conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
|
||||
this.service.startExecutorService(ExecutorType.RS_OPEN_ROOT,
|
||||
conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
|
||||
this.service.startExecutorService(ExecutorType.RS_OPEN_META,
|
||||
conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
|
||||
this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
|
||||
conf.getInt("hbase.regionserver.executor.closeregion.threads", 5));
|
||||
conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
|
||||
this.service.startExecutorService(ExecutorType.RS_CLOSE_ROOT,
|
||||
conf.getInt("hbase.regionserver.executor.closeroot.threads", 1));
|
||||
this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
|
||||
|
|
|
@ -0,0 +1,139 @@
|
|||
/**
|
||||
* Copyright 2010 The Apache Software Foundation
|
||||
*
|
||||
* 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.hadoop.hbase.executor;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService.Executor;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestExecutorService {
|
||||
private static final Log LOG = LogFactory.getLog(TestExecutorService.class);
|
||||
|
||||
@Test
|
||||
public void testExecutorService() throws Exception {
|
||||
int maxThreads = 5;
|
||||
int maxTries = 10;
|
||||
int sleepInterval = 10;
|
||||
|
||||
// Start an executor service pool with max 5 threads
|
||||
ExecutorService executorService = new ExecutorService("unit_test");
|
||||
executorService.startExecutorService(
|
||||
ExecutorType.MASTER_SERVER_OPERATIONS, maxThreads);
|
||||
|
||||
Executor executor =
|
||||
executorService.getExecutor(ExecutorType.MASTER_SERVER_OPERATIONS);
|
||||
ThreadPoolExecutor pool = executor.threadPoolExecutor;
|
||||
|
||||
// Assert no threads yet
|
||||
assertEquals(0, pool.getPoolSize());
|
||||
|
||||
AtomicBoolean lock = new AtomicBoolean(true);
|
||||
AtomicInteger counter = new AtomicInteger(0);
|
||||
|
||||
// Submit maxThreads executors.
|
||||
for (int i = 0; i < maxThreads; i++) {
|
||||
executorService.submit(
|
||||
new TestEventHandler(EventType.M_SERVER_SHUTDOWN, lock, counter));
|
||||
}
|
||||
|
||||
// The TestEventHandler will increment counter when it starts.
|
||||
int tries = 0;
|
||||
while (counter.get() < maxThreads && tries < maxTries) {
|
||||
LOG.info("Waiting for all event handlers to start...");
|
||||
Thread.sleep(sleepInterval);
|
||||
tries++;
|
||||
}
|
||||
|
||||
// Assert that pool is at max threads.
|
||||
assertEquals(maxThreads, counter.get());
|
||||
assertEquals(maxThreads, pool.getPoolSize());
|
||||
|
||||
// Now interrupt the running Executor
|
||||
synchronized (lock) {
|
||||
lock.set(false);
|
||||
lock.notifyAll();
|
||||
}
|
||||
|
||||
// Executor increments counter again on way out so.... test that happened.
|
||||
while (counter.get() < (maxThreads * 2) && tries < maxTries) {
|
||||
System.out.println("Waiting for all event handlers to finish...");
|
||||
Thread.sleep(sleepInterval);
|
||||
tries++;
|
||||
}
|
||||
|
||||
assertEquals(maxThreads * 2, counter.get());
|
||||
assertEquals(maxThreads, pool.getPoolSize());
|
||||
|
||||
// Add more than the number of threads items.
|
||||
// Make sure we don't get RejectedExecutionException.
|
||||
for (int i = 0; i < (2 * maxThreads); i++) {
|
||||
executorService.submit(
|
||||
new TestEventHandler(EventType.M_SERVER_SHUTDOWN, lock, counter));
|
||||
}
|
||||
// Now interrupt the running Executor
|
||||
synchronized (lock) {
|
||||
lock.set(false);
|
||||
lock.notifyAll();
|
||||
}
|
||||
|
||||
// Make sure threads are still around even after their timetolive expires.
|
||||
Thread.sleep(executor.keepAliveTimeInMillis * 2);
|
||||
assertEquals(maxThreads, pool.getPoolSize());
|
||||
}
|
||||
|
||||
public static class TestEventHandler extends EventHandler {
|
||||
private AtomicBoolean lock;
|
||||
private AtomicInteger counter;
|
||||
|
||||
public TestEventHandler(EventType eventType, AtomicBoolean lock,
|
||||
AtomicInteger counter) {
|
||||
super(null, eventType);
|
||||
this.lock = lock;
|
||||
this.counter = counter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
int num = counter.incrementAndGet();
|
||||
LOG.info("Running process #" + num + ", threadName=" +
|
||||
Thread.currentThread().getName());
|
||||
synchronized (lock) {
|
||||
while (lock.get()) {
|
||||
try {
|
||||
lock.wait();
|
||||
} catch (InterruptedException e) {
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
}
|
||||
counter.incrementAndGet();
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue