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