1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.procedure.flush;
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.Collection;
23 import java.util.List;
24 import java.util.concurrent.Callable;
25 import java.util.concurrent.ExecutionException;
26 import java.util.concurrent.ExecutorCompletionService;
27 import java.util.concurrent.Future;
28 import java.util.concurrent.LinkedBlockingQueue;
29 import java.util.concurrent.ThreadPoolExecutor;
30 import java.util.concurrent.TimeUnit;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.hbase.classification.InterfaceAudience;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.hbase.DaemonThreadFactory;
37 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
38 import org.apache.hadoop.hbase.TableName;
39 import org.apache.hadoop.hbase.errorhandling.ForeignException;
40 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
41 import org.apache.hadoop.hbase.procedure.ProcedureMember;
42 import org.apache.hadoop.hbase.procedure.ProcedureMemberRpcs;
43 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManager;
44 import org.apache.hadoop.hbase.procedure.Subprocedure;
45 import org.apache.hadoop.hbase.procedure.SubprocedureFactory;
46 import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
47 import org.apache.hadoop.hbase.regionserver.HRegion;
48 import org.apache.hadoop.hbase.regionserver.HRegionServer;
49 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
50 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
51 import org.apache.zookeeper.KeeperException;
52
53
54
55
56 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
57 public class RegionServerFlushTableProcedureManager extends RegionServerProcedureManager {
58 private static final Log LOG = LogFactory.getLog(RegionServerFlushTableProcedureManager.class);
59
60 private static final String CONCURENT_FLUSH_TASKS_KEY =
61 "hbase.flush.procedure.region.concurrentTasks";
62 private static final int DEFAULT_CONCURRENT_FLUSH_TASKS = 3;
63
64 public static final String FLUSH_REQUEST_THREADS_KEY =
65 "hbase.flush.procedure.region.pool.threads";
66 public static final int FLUSH_REQUEST_THREADS_DEFAULT = 10;
67
68 public static final String FLUSH_TIMEOUT_MILLIS_KEY =
69 "hbase.flush.procedure.region.timeout";
70 public static final long FLUSH_TIMEOUT_MILLIS_DEFAULT = 60000;
71
72 public static final String FLUSH_REQUEST_WAKE_MILLIS_KEY =
73 "hbase.flush.procedure.region.wakefrequency";
74 private static final long FLUSH_REQUEST_WAKE_MILLIS_DEFAULT = 500;
75
76 private RegionServerServices rss;
77 private ProcedureMemberRpcs memberRpcs;
78 private ProcedureMember member;
79
80
81
82
83
84
85
86
87 RegionServerFlushTableProcedureManager(Configuration conf, HRegionServer server,
88 ProcedureMemberRpcs memberRpc, ProcedureMember procMember) {
89 this.rss = server;
90 this.memberRpcs = memberRpc;
91 this.member = procMember;
92 }
93
94 public RegionServerFlushTableProcedureManager() {}
95
96
97
98
99 @Override
100 public void start() {
101 LOG.debug("Start region server flush procedure manager " + rss.getServerName().toString());
102 this.memberRpcs.start(rss.getServerName().toString(), member);
103 }
104
105
106
107
108
109
110 @Override
111 public void stop(boolean force) throws IOException {
112 String mode = force ? "abruptly" : "gracefully";
113 LOG.info("Stopping region server flush procedure manager " + mode + ".");
114
115 try {
116 this.member.close();
117 } finally {
118 this.memberRpcs.close();
119 }
120 }
121
122
123
124
125
126
127
128
129
130
131 public Subprocedure buildSubprocedure(String table) {
132
133
134 if (rss.isStopping() || rss.isStopped()) {
135 throw new IllegalStateException("Can't start flush region subprocedure on RS: "
136 + rss.getServerName() + ", because stopping/stopped!");
137 }
138
139
140 List<HRegion> involvedRegions;
141 try {
142 involvedRegions = getRegionsToFlush(table);
143 } catch (IOException e1) {
144 throw new IllegalStateException("Failed to figure out if there is region to flush.", e1);
145 }
146
147
148
149
150
151 LOG.debug("Launching subprocedure to flush regions for " + table);
152 ForeignExceptionDispatcher exnDispatcher = new ForeignExceptionDispatcher(table);
153 Configuration conf = rss.getConfiguration();
154 long timeoutMillis = conf.getLong(FLUSH_TIMEOUT_MILLIS_KEY,
155 FLUSH_TIMEOUT_MILLIS_DEFAULT);
156 long wakeMillis = conf.getLong(FLUSH_REQUEST_WAKE_MILLIS_KEY,
157 FLUSH_REQUEST_WAKE_MILLIS_DEFAULT);
158
159 FlushTableSubprocedurePool taskManager =
160 new FlushTableSubprocedurePool(rss.getServerName().toString(), conf);
161 return new FlushTableSubprocedure(member, exnDispatcher, wakeMillis,
162 timeoutMillis, involvedRegions, table, taskManager);
163 }
164
165
166
167
168
169
170
171
172
173
174
175 private List<HRegion> getRegionsToFlush(String table) throws IOException {
176 return rss.getOnlineRegions(TableName.valueOf(table));
177 }
178
179 public class FlushTableSubprocedureBuilder implements SubprocedureFactory {
180
181 @Override
182 public Subprocedure buildSubprocedure(String name, byte[] data) {
183
184 return RegionServerFlushTableProcedureManager.this.buildSubprocedure(name);
185 }
186
187 }
188
189
190
191
192
193
194
195
196
197 static class FlushTableSubprocedurePool {
198 private final ExecutorCompletionService<Void> taskPool;
199 private final ThreadPoolExecutor executor;
200 private volatile boolean stopped;
201 private final List<Future<Void>> futures = new ArrayList<Future<Void>>();
202 private final String name;
203
204 FlushTableSubprocedurePool(String name, Configuration conf) {
205
206 long keepAlive = conf.getLong(
207 RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_KEY,
208 RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_DEFAULT);
209 int threads = conf.getInt(CONCURENT_FLUSH_TASKS_KEY, DEFAULT_CONCURRENT_FLUSH_TASKS);
210 this.name = name;
211 executor = new ThreadPoolExecutor(1, threads, keepAlive, TimeUnit.MILLISECONDS,
212 new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("rs("
213 + name + ")-flush-proc-pool"));
214 taskPool = new ExecutorCompletionService<Void>(executor);
215 }
216
217 boolean hasTasks() {
218 return futures.size() != 0;
219 }
220
221
222
223
224
225
226 void submitTask(final Callable<Void> task) {
227 Future<Void> f = this.taskPool.submit(task);
228 futures.add(f);
229 }
230
231
232
233
234
235
236
237
238 boolean waitForOutstandingTasks() throws ForeignException, InterruptedException {
239 LOG.debug("Waiting for local region flush to finish.");
240
241 int sz = futures.size();
242 try {
243
244 for (int i = 0; i < sz; i++) {
245 Future<Void> f = taskPool.take();
246 f.get();
247 if (!futures.remove(f)) {
248 LOG.warn("unexpected future" + f);
249 }
250 LOG.debug("Completed " + (i+1) + "/" + sz + " local region flush tasks.");
251 }
252 LOG.debug("Completed " + sz + " local region flush tasks.");
253 return true;
254 } catch (InterruptedException e) {
255 LOG.warn("Got InterruptedException in FlushSubprocedurePool", e);
256 if (!stopped) {
257 Thread.currentThread().interrupt();
258 throw new ForeignException("FlushSubprocedurePool", e);
259 }
260
261 } catch (ExecutionException e) {
262 if (e.getCause() instanceof ForeignException) {
263 LOG.warn("Rethrowing ForeignException from FlushSubprocedurePool", e);
264 throw (ForeignException)e.getCause();
265 }
266 LOG.warn("Got Exception in FlushSubprocedurePool", e);
267 throw new ForeignException(name, e.getCause());
268 } finally {
269 cancelTasks();
270 }
271 return false;
272 }
273
274
275
276
277
278 void cancelTasks() throws InterruptedException {
279 Collection<Future<Void>> tasks = futures;
280 LOG.debug("cancelling " + tasks.size() + " flush region tasks " + name);
281 for (Future<Void> f: tasks) {
282 f.cancel(false);
283 }
284
285
286 futures.clear();
287 while (taskPool.poll() != null) {}
288 stop();
289 }
290
291
292
293
294 void stop() {
295 if (this.stopped) return;
296
297 this.stopped = true;
298 this.executor.shutdownNow();
299 }
300 }
301
302
303
304
305
306
307
308 @Override
309 public void initialize(RegionServerServices rss) throws KeeperException {
310 this.rss = rss;
311 ZooKeeperWatcher zkw = rss.getZooKeeper();
312 this.memberRpcs = new ZKProcedureMemberRpcs(zkw,
313 MasterFlushTableProcedureManager.FLUSH_TABLE_PROCEDURE_SIGNATURE);
314
315 Configuration conf = rss.getConfiguration();
316 long keepAlive = conf.getLong(FLUSH_TIMEOUT_MILLIS_KEY, FLUSH_TIMEOUT_MILLIS_DEFAULT);
317 int opThreads = conf.getInt(FLUSH_REQUEST_THREADS_KEY, FLUSH_REQUEST_THREADS_DEFAULT);
318
319
320 ThreadPoolExecutor pool = ProcedureMember.defaultPool(rss.getServerName().toString(),
321 opThreads, keepAlive);
322 this.member = new ProcedureMember(memberRpcs, pool, new FlushTableSubprocedureBuilder());
323 }
324
325 @Override
326 public String getProcedureSignature() {
327 return MasterFlushTableProcedureManager.FLUSH_TABLE_PROCEDURE_SIGNATURE;
328 }
329
330 }