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 org.apache.hadoop.conf.Configuration;
23 import org.apache.hadoop.hbase.HBaseConfiguration;
24 import org.apache.hadoop.hbase.util.Bytes;
25
26 import java.util.Collections;
27 import java.util.HashMap;
28 import java.util.LinkedList;
29 import java.util.Map;
30 import java.util.Queue;
31
32 /**
33 * A simple pool of HTable instances.<p>
34 *
35 * Each HTablePool acts as a pool for all tables. To use, instantiate an
36 * HTablePool and use {@link #getTable(String)} to get an HTable from the pool.
37 * Once you are done with it, return it to the pool with {@link #putTable(HTableInterface)}.<p>
38 *
39 * A pool can be created with a <i>maxSize</i> which defines the most HTable
40 * references that will ever be retained for each table. Otherwise the default
41 * is {@link Integer#MAX_VALUE}.<p>
42 */
43 public class HTablePool {
44 private final Map<String, LinkedList<HTableInterface>> tables =
45 Collections.synchronizedMap(new HashMap<String, LinkedList<HTableInterface>>());
46 private final Configuration config;
47 private final int maxSize;
48 private HTableInterfaceFactory tableFactory = new HTableFactory();
49
50 /**
51 * Default Constructor. Default HBaseConfiguration and no limit on pool size.
52 */
53 public HTablePool() {
54 this(HBaseConfiguration.create(), Integer.MAX_VALUE);
55 }
56
57 /**
58 * Constructor to set maximum versions and use the specified configuration.
59 * @param config configuration
60 * @param maxSize maximum number of references to keep for each table
61 */
62 public HTablePool(Configuration config, int maxSize) {
63 this.config = config;
64 this.maxSize = maxSize;
65 }
66
67 public HTablePool(Configuration config, int maxSize, HTableInterfaceFactory tableFactory) {
68 this.config = config;
69 this.maxSize = maxSize;
70 this.tableFactory = tableFactory;
71 }
72
73 /**
74 * Get a reference to the specified table from the pool.<p>
75 *
76 * Create a new one if one is not available.
77 * @param tableName table name
78 * @return a reference to the specified table
79 * @throws RuntimeException if there is a problem instantiating the HTable
80 */
81 public HTableInterface getTable(String tableName) {
82 LinkedList<HTableInterface> queue = tables.get(tableName);
83 if(queue == null) {
84 queue = new LinkedList<HTableInterface>();
85 tables.put(tableName, queue);
86 return createHTable(tableName);
87 }
88 HTableInterface table;
89 synchronized(queue) {
90 table = queue.poll();
91 }
92 if(table == null) {
93 return createHTable(tableName);
94 }
95 return table;
96 }
97
98 /**
99 * Get a reference to the specified table from the pool.<p>
100 *
101 * Create a new one if one is not available.
102 * @param tableName table name
103 * @return a reference to the specified table
104 * @throws RuntimeException if there is a problem instantiating the HTable
105 */
106 public HTableInterface getTable(byte [] tableName) {
107 return getTable(Bytes.toString(tableName));
108 }
109
110 /**
111 * Puts the specified HTable back into the pool.<p>
112 *
113 * If the pool already contains <i>maxSize</i> references to the table,
114 * then nothing happens.
115 * @param table table
116 */
117 public void putTable(HTableInterface table) {
118 LinkedList<HTableInterface> queue = tables.get(Bytes.toString(table.getTableName()));
119 synchronized(queue) {
120 if(queue.size() >= maxSize) return;
121 queue.add(table);
122 }
123 }
124
125 protected HTableInterface createHTable(String tableName) {
126 return this.tableFactory.createHTableInterface(config, Bytes.toBytes(tableName));
127 }
128
129 /**
130 * Closes all the HTable instances , belonging to the given table, in the table pool.
131 * <p>
132 * Note: this is a 'shutdown' of the given table pool and different from
133 * {@link #putTable(HTableInterface)}, that is used to return the table
134 * instance to the pool for future re-use.
135 *
136 * @param tableName
137 */
138 public void closeTablePool(final String tableName) {
139 Queue<HTableInterface> queue = tables.get(tableName);
140 synchronized (queue) {
141 HTableInterface table = queue.poll();
142 while (table != null) {
143 this.tableFactory.releaseHTableInterface(table);
144 table = queue.poll();
145 }
146 }
147
148 }
149
150 /**
151 * See {@link #closeTablePool(String)}.
152 *
153 * @param tableName
154 */
155 public void closeTablePool(final byte[] tableName) {
156 closeTablePool(Bytes.toString(tableName));
157 }
158
159 int getCurrentPoolSize(String tableName) {
160 Queue<HTableInterface> queue = tables.get(tableName);
161 synchronized(queue) {
162 return queue.size();
163 }
164 }
165 }