View Javadoc

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.client;
21  
22  import java.io.Closeable;
23  import java.io.IOException;
24  import java.util.Collection;
25  import java.util.List;
26  import java.util.Map;
27  
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.client.coprocessor.Batch;
32  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
33  import org.apache.hadoop.hbase.util.Bytes;
34  import org.apache.hadoop.hbase.util.PoolMap;
35  import org.apache.hadoop.hbase.util.PoolMap.PoolType;
36  
37  /**
38   * A simple pool of HTable instances.
39   * 
40   * Each HTablePool acts as a pool for all tables. To use, instantiate an
41   * HTablePool and use {@link #getTable(String)} to get an HTable from the pool.
42   *
43     * This method is not needed anymore, clients should call
44     * HTableInterface.close() rather than returning the tables to the pool
45     *
46   * Once you are done with it, close your instance of {@link HTableInterface}
47   * by calling {@link HTableInterface#close()} rather than returning the tables
48   * to the pool with (deprecated) {@link #putTable(HTableInterface)}.
49   * 
50   * <p>
51   * A pool can be created with a <i>maxSize</i> which defines the most HTable
52   * references that will ever be retained for each table. Otherwise the default
53   * is {@link Integer#MAX_VALUE}.
54   * 
55   * <p>
56   * Pool will manage its own connections to the cluster. See
57   * {@link HConnectionManager}.
58   */
59  public class HTablePool implements Closeable {
60    private final PoolMap<String, HTableInterface> tables;
61    private final int maxSize;
62    private final PoolType poolType;
63    private final Configuration config;
64    private final HTableInterfaceFactory tableFactory;
65  
66    /**
67     * Default Constructor. Default HBaseConfiguration and no limit on pool size.
68     */
69    public HTablePool() {
70      this(HBaseConfiguration.create(), Integer.MAX_VALUE);
71    }
72  
73    /**
74     * Constructor to set maximum versions and use the specified configuration.
75     * 
76     * @param config
77     *          configuration
78     * @param maxSize
79     *          maximum number of references to keep for each table
80     */
81    public HTablePool(final Configuration config, final int maxSize) {
82      this(config, maxSize, null, null);
83    }
84  
85    /**
86     * Constructor to set maximum versions and use the specified configuration and
87     * table factory.
88     * 
89     * @param config
90     *          configuration
91     * @param maxSize
92     *          maximum number of references to keep for each table
93     * @param tableFactory
94     *          table factory
95     */
96    public HTablePool(final Configuration config, final int maxSize,
97        final HTableInterfaceFactory tableFactory) {
98      this(config, maxSize, null, PoolType.Reusable);
99    }
100 
101   /**
102    * Constructor to set maximum versions and use the specified configuration and
103    * pool type.
104    * 
105    * @param config
106    *          configuration
107    * @param maxSize
108    *          maximum number of references to keep for each table
109    * @param poolType
110    *          pool type which is one of {@link PoolType#Reusable} or
111    *          {@link PoolType#ThreadLocal}
112    */
113   public HTablePool(final Configuration config, final int maxSize,
114       final PoolType poolType) {
115     this(config, maxSize, null, poolType);
116   }
117 
118   /**
119    * Constructor to set maximum versions and use the specified configuration,
120    * table factory and pool type. The HTablePool supports the
121    * {@link PoolType#Reusable} and {@link PoolType#ThreadLocal}. If the pool
122    * type is null or not one of those two values, then it will default to
123    * {@link PoolType#Reusable}.
124    * 
125    * @param config
126    *          configuration
127    * @param maxSize
128    *          maximum number of references to keep for each table
129    * @param tableFactory
130    *          table factory
131    * @param poolType
132    *          pool type which is one of {@link PoolType#Reusable} or
133    *          {@link PoolType#ThreadLocal}
134    */
135   public HTablePool(final Configuration config, final int maxSize,
136       final HTableInterfaceFactory tableFactory, PoolType poolType) {
137     // Make a new configuration instance so I can safely cleanup when
138     // done with the pool.
139     this.config = config == null ? new Configuration() : config;
140     this.maxSize = maxSize;
141     this.tableFactory = tableFactory == null ? new HTableFactory()
142         : tableFactory;
143     if (poolType == null) {
144       this.poolType = PoolType.Reusable;
145     } else {
146       switch (poolType) {
147       case Reusable:
148       case ThreadLocal:
149         this.poolType = poolType;
150         break;
151       default:
152         this.poolType = PoolType.Reusable;
153         break;
154       }
155     }
156     this.tables = new PoolMap<String, HTableInterface>(this.poolType,
157         this.maxSize);
158   }
159 
160   /**
161    * Get a reference to the specified table from the pool.
162    * <p>
163    * <p/>
164    * 
165    * @param tableName
166    *          table name
167    * @return a reference to the specified table
168    * @throws RuntimeException
169    *           if there is a problem instantiating the HTable
170    */
171   public HTableInterface getTable(String tableName) {
172     // call the old getTable implementation renamed to findOrCreateTable
173     HTableInterface table = findOrCreateTable(tableName);
174     // return a proxy table so when user closes the proxy, the actual table
175     // will be returned to the pool
176     try {
177       return new PooledHTable(table);
178     } catch (IOException ioe) {
179       throw new RuntimeException(ioe);
180     }
181   }
182 
183   /**
184    * Get a reference to the specified table from the pool.
185    * <p>
186    * 
187    * Create a new one if one is not available.
188    * 
189    * @param tableName
190    *          table name
191    * @return a reference to the specified table
192    * @throws RuntimeException
193    *           if there is a problem instantiating the HTable
194    */
195   private HTableInterface findOrCreateTable(String tableName) {
196     HTableInterface table = tables.get(tableName);
197     if (table == null) {
198       table = createHTable(tableName);
199     }
200     return table;
201   }
202 
203   /**
204    * Get a reference to the specified table from the pool.
205    * <p>
206    * 
207    * Create a new one if one is not available.
208    * 
209    * @param tableName
210    *          table name
211    * @return a reference to the specified table
212    * @throws RuntimeException
213    *           if there is a problem instantiating the HTable
214    */
215   public HTableInterface getTable(byte[] tableName) {
216     return getTable(Bytes.toString(tableName));
217   }
218 
219   /**
220    * This method is not needed anymore, clients should call
221    * HTableInterface.close() rather than returning the tables to the pool
222    * 
223    * @param table
224    *          the proxy table user got from pool
225    * @deprecated
226    */
227   public void putTable(HTableInterface table) throws IOException {
228     // we need to be sure nobody puts a proxy implementation in the pool
229     // but if the client code is not updated
230     // and it will continue to call putTable() instead of calling close()
231     // then we need to return the wrapped table to the pool instead of the
232     // proxy
233     // table
234     if (table instanceof PooledHTable) {
235       returnTable(((PooledHTable) table).getWrappedTable());
236     } else {
237       // normally this should not happen if clients pass back the same
238       // table
239       // object they got from the pool
240       // but if it happens then it's better to reject it
241       throw new IllegalArgumentException("not a pooled table: " + table);
242     }
243   }
244 
245   /**
246    * Puts the specified HTable back into the pool.
247    * <p>
248    * 
249    * If the pool already contains <i>maxSize</i> references to the table, then
250    * the table instance gets closed after flushing buffered edits.
251    * 
252    * @param table
253    *          table
254    */
255   private void returnTable(HTableInterface table) throws IOException {
256     // this is the old putTable method renamed and made private
257     String tableName = Bytes.toString(table.getTableName());
258     if (tables.size(tableName) >= maxSize) {
259       // release table instance since we're not reusing it
260       this.tables.remove(tableName, table);
261       this.tableFactory.releaseHTableInterface(table);
262       return;
263     }
264     tables.put(tableName, table);
265   }
266 
267   protected HTableInterface createHTable(String tableName) {
268     return this.tableFactory.createHTableInterface(config,
269         Bytes.toBytes(tableName));
270   }
271 
272   /**
273    * Closes all the HTable instances , belonging to the given table, in the
274    * table pool.
275    * <p>
276    * Note: this is a 'shutdown' of the given table pool and different from
277    * {@link #putTable(HTableInterface)}, that is used to return the table
278    * instance to the pool for future re-use.
279    * 
280    * @param tableName
281    */
282   public void closeTablePool(final String tableName) throws IOException {
283     Collection<HTableInterface> tables = this.tables.values(tableName);
284     if (tables != null) {
285       for (HTableInterface table : tables) {
286         this.tableFactory.releaseHTableInterface(table);
287       }
288     }
289     this.tables.remove(tableName);
290   }
291 
292   /**
293    * See {@link #closeTablePool(String)}.
294    * 
295    * @param tableName
296    */
297   public void closeTablePool(final byte[] tableName) throws IOException {
298     closeTablePool(Bytes.toString(tableName));
299   }
300 
301   /**
302    * Closes all the HTable instances , belonging to all tables in the table
303    * pool.
304    * <p>
305    * Note: this is a 'shutdown' of all the table pools.
306    */
307   public void close() throws IOException {
308     for (String tableName : tables.keySet()) {
309       closeTablePool(tableName);
310     }
311     this.tables.clear();
312   }
313 
314   int getCurrentPoolSize(String tableName) {
315     return tables.size(tableName);
316   }
317 
318   /**
319    * A proxy class that implements HTableInterface.close method to return the
320    * wrapped table back to the table pool
321    * 
322    */
323   class PooledHTable extends HTable {
324 
325     private HTableInterface table; // actual table implementation
326 
327     public PooledHTable(HTableInterface table) throws IOException {
328       super(table.getConfiguration(), table.getTableName());
329       this.table = table;
330     }
331 
332     @Override
333     public byte[] getTableName() {
334       return table.getTableName();
335     }
336 
337     @Override
338     public Configuration getConfiguration() {
339       return table.getConfiguration();
340     }
341 
342     @Override
343     public HTableDescriptor getTableDescriptor() throws IOException {
344       return table.getTableDescriptor();
345     }
346 
347     @Override
348     public boolean exists(Get get) throws IOException {
349       return table.exists(get);
350     }
351 
352     @Override
353     public void batch(List<Row> actions, Object[] results) throws IOException,
354         InterruptedException {
355       table.batch(actions, results);
356     }
357 
358     @Override
359     public Object[] batch(List<Row> actions) throws IOException,
360         InterruptedException {
361       return table.batch(actions);
362     }
363 
364     @Override
365     public Result get(Get get) throws IOException {
366       return table.get(get);
367     }
368 
369     @Override
370     public Result[] get(List<Get> gets) throws IOException {
371       return table.get(gets);
372     }
373 
374     @Override
375     public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
376       return table.getRowOrBefore(row, family);
377     }
378 
379     @Override
380     public ResultScanner getScanner(Scan scan) throws IOException {
381       return table.getScanner(scan);
382     }
383 
384     @Override
385     public ResultScanner getScanner(byte[] family) throws IOException {
386       return table.getScanner(family);
387     }
388 
389     @Override
390     public ResultScanner getScanner(byte[] family, byte[] qualifier)
391         throws IOException {
392       return table.getScanner(family, qualifier);
393     }
394 
395     @Override
396     public void put(Put put) throws IOException {
397       table.put(put);
398     }
399 
400     @Override
401     public void put(List<Put> puts) throws IOException {
402       table.put(puts);
403     }
404 
405     @Override
406     public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
407         byte[] value, Put put) throws IOException {
408       return table.checkAndPut(row, family, qualifier, value, put);
409     }
410 
411     @Override
412     public void delete(Delete delete) throws IOException {
413       table.delete(delete);
414     }
415 
416     @Override
417     public void delete(List<Delete> deletes) throws IOException {
418       table.delete(deletes);
419     }
420 
421     @Override
422     public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
423         byte[] value, Delete delete) throws IOException {
424       return table.checkAndDelete(row, family, qualifier, value, delete);
425     }
426 
427     @Override
428     public Result increment(Increment increment) throws IOException {
429       return table.increment(increment);
430     }
431 
432     @Override
433     public long incrementColumnValue(byte[] row, byte[] family,
434         byte[] qualifier, long amount) throws IOException {
435       return table.incrementColumnValue(row, family, qualifier, amount);
436     }
437 
438     @Override
439     public long incrementColumnValue(byte[] row, byte[] family,
440         byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
441       return table.incrementColumnValue(row, family, qualifier, amount,
442           writeToWAL);
443     }
444 
445     @Override
446     public boolean isAutoFlush() {
447       return table.isAutoFlush();
448     }
449 
450     @Override
451     public void flushCommits() throws IOException {
452       table.flushCommits();
453     }
454 
455     /**
456      * Returns the actual table back to the pool
457      * 
458      * @throws IOException
459      */
460     public void close() throws IOException {
461       returnTable(table);
462     }
463 
464     @Override
465     public RowLock lockRow(byte[] row) throws IOException {
466       return table.lockRow(row);
467     }
468 
469     @Override
470     public void unlockRow(RowLock rl) throws IOException {
471       table.unlockRow(rl);
472     }
473 
474     @Override
475     public <T extends CoprocessorProtocol> T coprocessorProxy(
476         Class<T> protocol, byte[] row) {
477       return table.coprocessorProxy(protocol, row);
478     }
479 
480     @Override
481     public <T extends CoprocessorProtocol, R> Map<byte[], R> coprocessorExec(
482         Class<T> protocol, byte[] startKey, byte[] endKey,
483         Batch.Call<T, R> callable) throws IOException, Throwable {
484       return table.coprocessorExec(protocol, startKey, endKey, callable);
485     }
486 
487     @Override
488     public <T extends CoprocessorProtocol, R> void coprocessorExec(
489         Class<T> protocol, byte[] startKey, byte[] endKey,
490         Batch.Call<T, R> callable, Batch.Callback<R> callback)
491         throws IOException, Throwable {
492       table.coprocessorExec(protocol, startKey, endKey, callable, callback);
493     }
494 
495     @Override
496     public String toString() {
497       return "PooledHTable{" + ", table=" + table + '}';
498     }
499 
500     /**
501      * Expose the wrapped HTable to tests in the same package
502      * 
503      * @return wrapped htable
504      */
505     HTableInterface getWrappedTable() {
506       return table;
507     }
508   }
509 }