1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import java.io.Closeable;
22 import java.io.IOException;
23 import java.util.Collection;
24 import java.util.List;
25 import java.util.Map;
26
27 import org.apache.hadoop.classification.InterfaceAudience;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.HBaseConfiguration;
30 import org.apache.hadoop.hbase.HTableDescriptor;
31 import org.apache.hadoop.hbase.TableName;
32 import org.apache.hadoop.hbase.client.coprocessor.Batch;
33 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
34 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
35 import org.apache.hadoop.hbase.util.Bytes;
36 import org.apache.hadoop.hbase.util.PoolMap;
37 import org.apache.hadoop.hbase.util.PoolMap.PoolType;
38
39 import com.google.protobuf.Service;
40 import com.google.protobuf.ServiceException;
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65 @InterfaceAudience.Private
66 public class HTablePool implements Closeable {
67 private final PoolMap<String, HTableInterface> tables;
68 private final int maxSize;
69 private final PoolType poolType;
70 private final Configuration config;
71 private final HTableInterfaceFactory tableFactory;
72
73
74
75
76 public HTablePool() {
77 this(HBaseConfiguration.create(), Integer.MAX_VALUE);
78 }
79
80
81
82
83
84
85
86
87
88 public HTablePool(final Configuration config, final int maxSize) {
89 this(config, maxSize, null, null);
90 }
91
92
93
94
95
96
97
98
99
100
101
102
103 public HTablePool(final Configuration config, final int maxSize,
104 final HTableInterfaceFactory tableFactory) {
105 this(config, maxSize, tableFactory, PoolType.Reusable);
106 }
107
108
109
110
111
112
113
114
115
116
117
118
119
120 public HTablePool(final Configuration config, final int maxSize,
121 final PoolType poolType) {
122 this(config, maxSize, null, poolType);
123 }
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142 public HTablePool(final Configuration config, final int maxSize,
143 final HTableInterfaceFactory tableFactory, PoolType poolType) {
144
145
146 this.config = config == null ? HBaseConfiguration.create() : config;
147 this.maxSize = maxSize;
148 this.tableFactory = tableFactory == null ? new HTableFactory()
149 : tableFactory;
150 if (poolType == null) {
151 this.poolType = PoolType.Reusable;
152 } else {
153 switch (poolType) {
154 case Reusable:
155 case ThreadLocal:
156 this.poolType = poolType;
157 break;
158 default:
159 this.poolType = PoolType.Reusable;
160 break;
161 }
162 }
163 this.tables = new PoolMap<String, HTableInterface>(this.poolType,
164 this.maxSize);
165 }
166
167
168
169
170
171
172
173
174
175
176
177
178 public HTableInterface getTable(String tableName) {
179
180 HTableInterface table = findOrCreateTable(tableName);
181
182
183 return new PooledHTable(table);
184 }
185
186
187
188
189
190
191
192
193
194
195
196
197
198 private HTableInterface findOrCreateTable(String tableName) {
199 HTableInterface table = tables.get(tableName);
200 if (table == null) {
201 table = createHTable(tableName);
202 }
203 return table;
204 }
205
206
207
208
209
210
211
212
213
214
215
216
217
218 public HTableInterface getTable(byte[] tableName) {
219 return getTable(Bytes.toString(tableName));
220 }
221
222
223
224
225
226
227
228
229
230 public void putTable(HTableInterface table) throws IOException {
231
232
233
234
235
236
237 if (table instanceof PooledHTable) {
238 returnTable(((PooledHTable) table).getWrappedTable());
239 } else {
240
241
242
243
244 throw new IllegalArgumentException("not a pooled table: " + table);
245 }
246 }
247
248
249
250
251
252
253
254
255
256
257
258 private void returnTable(HTableInterface table) throws IOException {
259
260 String tableName = Bytes.toString(table.getTableName());
261 if (tables.size(tableName) >= maxSize) {
262
263 this.tables.removeValue(tableName, table);
264 this.tableFactory.releaseHTableInterface(table);
265 return;
266 }
267 tables.put(tableName, table);
268 }
269
270 protected HTableInterface createHTable(String tableName) {
271 return this.tableFactory.createHTableInterface(config,
272 Bytes.toBytes(tableName));
273 }
274
275
276
277
278
279
280
281
282
283
284
285 public void closeTablePool(final String tableName) throws IOException {
286 Collection<HTableInterface> tables = this.tables.values(tableName);
287 if (tables != null) {
288 for (HTableInterface table : tables) {
289 this.tableFactory.releaseHTableInterface(table);
290 }
291 }
292 this.tables.remove(tableName);
293 }
294
295
296
297
298
299
300 public void closeTablePool(final byte[] tableName) throws IOException {
301 closeTablePool(Bytes.toString(tableName));
302 }
303
304
305
306
307
308
309
310 public void close() throws IOException {
311 for (String tableName : tables.keySet()) {
312 closeTablePool(tableName);
313 }
314 this.tables.clear();
315 }
316
317 public int getCurrentPoolSize(String tableName) {
318 return tables.size(tableName);
319 }
320
321
322
323
324
325
326 class PooledHTable implements HTableInterface {
327
328 private boolean open = false;
329
330 private HTableInterface table;
331
332 public PooledHTable(HTableInterface table) {
333 this.table = table;
334 this.open = true;
335 }
336
337 @Override
338 public byte[] getTableName() {
339 checkState();
340 return table.getTableName();
341 }
342
343 @Override
344 public TableName getName() {
345 return table.getName();
346 }
347
348 @Override
349 public Configuration getConfiguration() {
350 checkState();
351 return table.getConfiguration();
352 }
353
354 @Override
355 public HTableDescriptor getTableDescriptor() throws IOException {
356 checkState();
357 return table.getTableDescriptor();
358 }
359
360 @Override
361 public boolean exists(Get get) throws IOException {
362 checkState();
363 return table.exists(get);
364 }
365
366 @Override
367 public Boolean[] exists(List<Get> gets) throws IOException {
368 checkState();
369 return table.exists(gets);
370 }
371
372 @Override
373 public void batch(List<? extends Row> actions, Object[] results) throws IOException,
374 InterruptedException {
375 checkState();
376 table.batch(actions, results);
377 }
378
379
380
381
382
383
384 @Override
385 public Object[] batch(List<? extends Row> actions) throws IOException,
386 InterruptedException {
387 checkState();
388 return table.batch(actions);
389 }
390
391 @Override
392 public Result get(Get get) throws IOException {
393 checkState();
394 return table.get(get);
395 }
396
397 @Override
398 public Result[] get(List<Get> gets) throws IOException {
399 checkState();
400 return table.get(gets);
401 }
402
403 @Override
404 @SuppressWarnings("deprecation")
405 public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
406 checkState();
407 return table.getRowOrBefore(row, family);
408 }
409
410 @Override
411 public ResultScanner getScanner(Scan scan) throws IOException {
412 checkState();
413 return table.getScanner(scan);
414 }
415
416 @Override
417 public ResultScanner getScanner(byte[] family) throws IOException {
418 checkState();
419 return table.getScanner(family);
420 }
421
422 @Override
423 public ResultScanner getScanner(byte[] family, byte[] qualifier)
424 throws IOException {
425 checkState();
426 return table.getScanner(family, qualifier);
427 }
428
429 @Override
430 public void put(Put put) throws IOException {
431 checkState();
432 table.put(put);
433 }
434
435 @Override
436 public void put(List<Put> puts) throws IOException {
437 checkState();
438 table.put(puts);
439 }
440
441 @Override
442 public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
443 byte[] value, Put put) throws IOException {
444 checkState();
445 return table.checkAndPut(row, family, qualifier, value, put);
446 }
447
448 @Override
449 public void delete(Delete delete) throws IOException {
450 checkState();
451 table.delete(delete);
452 }
453
454 @Override
455 public void delete(List<Delete> deletes) throws IOException {
456 checkState();
457 table.delete(deletes);
458 }
459
460 @Override
461 public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
462 byte[] value, Delete delete) throws IOException {
463 checkState();
464 return table.checkAndDelete(row, family, qualifier, value, delete);
465 }
466
467 @Override
468 public Result increment(Increment increment) throws IOException {
469 checkState();
470 return table.increment(increment);
471 }
472
473 @Override
474 public long incrementColumnValue(byte[] row, byte[] family,
475 byte[] qualifier, long amount) throws IOException {
476 checkState();
477 return table.incrementColumnValue(row, family, qualifier, amount);
478 }
479
480 @Override
481 public long incrementColumnValue(byte[] row, byte[] family,
482 byte[] qualifier, long amount, Durability durability) throws IOException {
483 checkState();
484 return table.incrementColumnValue(row, family, qualifier, amount,
485 durability);
486 }
487
488 @Override
489 public boolean isAutoFlush() {
490 checkState();
491 return table.isAutoFlush();
492 }
493
494 @Override
495 public void flushCommits() throws IOException {
496 checkState();
497 table.flushCommits();
498 }
499
500
501
502
503
504
505 public void close() throws IOException {
506 checkState();
507 open = false;
508 returnTable(table);
509 }
510
511 @Override
512 public CoprocessorRpcChannel coprocessorService(byte[] row) {
513 checkState();
514 return table.coprocessorService(row);
515 }
516
517 @Override
518 public <T extends Service, R> Map<byte[], R> coprocessorService(Class<T> service,
519 byte[] startKey, byte[] endKey, Batch.Call<T, R> callable)
520 throws ServiceException, Throwable {
521 checkState();
522 return table.coprocessorService(service, startKey, endKey, callable);
523 }
524
525 @Override
526 public <T extends Service, R> void coprocessorService(Class<T> service,
527 byte[] startKey, byte[] endKey, Batch.Call<T, R> callable, Callback<R> callback)
528 throws ServiceException, Throwable {
529 checkState();
530 table.coprocessorService(service, startKey, endKey, callable, callback);
531 }
532
533 @Override
534 public String toString() {
535 return "PooledHTable{" + ", table=" + table + '}';
536 }
537
538
539
540
541
542
543 HTableInterface getWrappedTable() {
544 return table;
545 }
546
547 @Override
548 public <R> void batchCallback(List<? extends Row> actions,
549 Object[] results, Callback<R> callback) throws IOException,
550 InterruptedException {
551 checkState();
552 table.batchCallback(actions, results, callback);
553 }
554
555
556
557
558
559
560
561
562 @Override
563 public <R> Object[] batchCallback(List<? extends Row> actions,
564 Callback<R> callback) throws IOException, InterruptedException {
565 checkState();
566 return table.batchCallback(actions, callback);
567 }
568
569 @Override
570 public void mutateRow(RowMutations rm) throws IOException {
571 checkState();
572 table.mutateRow(rm);
573 }
574
575 @Override
576 public Result append(Append append) throws IOException {
577 checkState();
578 return table.append(append);
579 }
580
581 @Override
582 public void setAutoFlush(boolean autoFlush) {
583 checkState();
584 table.setAutoFlush(autoFlush, autoFlush);
585 }
586
587 @Override
588 public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
589 checkState();
590 table.setAutoFlush(autoFlush, clearBufferOnFail);
591 }
592
593 @Override
594 public void setAutoFlushTo(boolean autoFlush) {
595 table.setAutoFlushTo(autoFlush);
596 }
597
598 @Override
599 public long getWriteBufferSize() {
600 checkState();
601 return table.getWriteBufferSize();
602 }
603
604 @Override
605 public void setWriteBufferSize(long writeBufferSize) throws IOException {
606 checkState();
607 table.setWriteBufferSize(writeBufferSize);
608 }
609
610 boolean isOpen() {
611 return open;
612 }
613
614 private void checkState() {
615 if (!isOpen()) {
616 throw new IllegalStateException("Table=" + new String(table.getTableName()) + " already closed");
617 }
618 }
619
620 @Override
621 public long incrementColumnValue(byte[] row, byte[] family,
622 byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
623 return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
624 }
625 }
626 }