1 /**
2 * Copyright 2010 The Apache Software Foundation
3 *
4 * Licensed to the Apache Software Foundation (ASF) under one
5 * or more contributor license agreements. See the NOTICE file
6 * distributed with this work for additional information
7 * regarding copyright ownership. The ASF licenses this file
8 * to you under the Apache License, Version 2.0 (the
9 * "License"); you may not use this file except in compliance
10 * with the License. You may obtain a copy of the License at
11 *
12 * http://www.apache.org/licenses/LICENSE-2.0
13 *
14 * Unless required by applicable law or agreed to in writing, software
15 * distributed under the License is distributed on an "AS IS" BASIS,
16 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17 * See the License for the specific language governing permissions and
18 * limitations under the License.
19 */
20 package org.apache.hadoop.hbase.master;
21
22 import java.lang.Thread.UncaughtExceptionHandler;
23 import java.util.concurrent.Executors;
24
25 import org.apache.hadoop.hbase.Server;
26
27 import com.google.common.util.concurrent.ThreadFactoryBuilder;
28
29 /**
30 * Base class used bulk assigning and unassigning regions.
31 * Encapsulates a fixed size thread pool of executors to run assignment/unassignment.
32 * Implement {@link #populatePool(java.util.concurrent.ExecutorService)} and
33 * {@link #waitUntilDone(long)}. The default implementation of
34 * the {@link #getUncaughtExceptionHandler()} is to abort the hosting
35 * Server.
36 */
37 public abstract class BulkAssigner {
38 final Server server;
39
40 /**
41 * @param server An instance of Server
42 */
43 public BulkAssigner(final Server server) {
44 this.server = server;
45 }
46
47 /**
48 * @return What to use for a thread prefix when executor runs.
49 */
50 protected String getThreadNamePrefix() {
51 return this.server.getServerName() + "-" + this.getClass().getName();
52 }
53
54 protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
55 return new UncaughtExceptionHandler() {
56 @Override
57 public void uncaughtException(Thread t, Throwable e) {
58 // Abort if exception of any kind.
59 server.abort("Uncaught exception in " + t.getName(), e);
60 }
61 };
62 }
63
64 protected int getThreadCount() {
65 return this.server.getConfiguration().
66 getInt("hbase.bulk.assignment.threadpool.size", 20);
67 }
68
69 protected long getTimeoutOnRIT() {
70 return this.server.getConfiguration().
71 getLong("hbase.bulk.assignment.waiton.empty.rit", 5 * 60 * 1000);
72 }
73
74 protected abstract void populatePool(final java.util.concurrent.ExecutorService pool);
75
76 public boolean bulkAssign() throws InterruptedException {
77 return bulkAssign(true);
78 }
79
80 /**
81 * Run the bulk assign.
82 * @param sync Whether to assign synchronously.
83 * @throws InterruptedException
84 * @return True if done.
85 */
86 public boolean bulkAssign(boolean sync) throws InterruptedException {
87 boolean result = false;
88 ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
89 builder.setDaemon(true);
90 builder.setNameFormat(getThreadNamePrefix() + "-%1$d");
91 builder.setUncaughtExceptionHandler(getUncaughtExceptionHandler());
92 int threadCount = getThreadCount();
93 java.util.concurrent.ExecutorService pool =
94 Executors.newFixedThreadPool(threadCount, builder.build());
95 try {
96 populatePool(pool);
97 // How long to wait on empty regions-in-transition. If we timeout, the
98 // RIT monitor should do fixup.
99 if (sync) result = waitUntilDone(getTimeoutOnRIT());
100 } finally {
101 // We're done with the pool. It'll exit when its done all in queue.
102 pool.shutdown();
103 }
104 return result;
105 }
106
107 /**
108 * Wait until bulk assign is done.
109 * @param timeout How long to wait.
110 * @throws InterruptedException
111 * @return True if the condition we were waiting on happened.
112 */
113 protected abstract boolean waitUntilDone(final long timeout)
114 throws InterruptedException;
115 }