Review Board 1.7.22


When HTable instance is discarded in putTable(), we should call tableFactory.releaseHTableInterface()

Review Request #573 - Created April 9, 2011 and updated

Ted Yu
trunk
HBASE-3750
Reviewers
hbase
larsgeorge
hbase
When HTable instance is discarded in putTable(), we should call tableFactory.releaseHTableInterface().
TestHTablePool passes.

Diff revision 2 (Latest)

1 2
1 2

  1. /src/main/java/org/apache/hadoop/hbase/client/HTablePool.java: Loading...
/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
Revision 1090659 New Change
[20] 16 lines
[+20]
17
 * See the License for the specific language governing permissions and
17
 * See the License for the specific language governing permissions and
18
 * limitations under the License.
18
 * limitations under the License.
19
 */
19
 */
20
package org.apache.hadoop.hbase.client;
20
package org.apache.hadoop.hbase.client;
21

    
   
21

   
22
import java.util.LinkedList;

   
23
import java.util.Queue;

   
24
import java.util.Map;
22
import java.util.Map;

    
   
23
import java.util.Queue;
25
import java.util.concurrent.ConcurrentHashMap;
24
import java.util.concurrent.ConcurrentHashMap;
26
import java.util.concurrent.ConcurrentMap;

   
27
import java.util.concurrent.ConcurrentLinkedQueue;
25
import java.util.concurrent.ConcurrentLinkedQueue;
28

    
   
26

   
29
import org.apache.hadoop.conf.Configuration;
27
import org.apache.hadoop.conf.Configuration;
30
import org.apache.hadoop.hbase.HBaseConfiguration;
28
import org.apache.hadoop.hbase.HBaseConfiguration;
31
import org.apache.hadoop.hbase.util.Bytes;
29
import org.apache.hadoop.hbase.util.Bytes;
[+20] [20] 84 lines
[+20] [+] public HTableInterface getTable(byte [] tableName) {
116
   * then nothing happens.
114
   * then nothing happens.
117
   * @param table table
115
   * @param table table
118
   */
116
   */
119
  public void putTable(HTableInterface table) {
117
  public void putTable(HTableInterface table) {
120
    Queue<HTableInterface> queue = tables.get(Bytes.toString(table.getTableName()));
118
    Queue<HTableInterface> queue = tables.get(Bytes.toString(table.getTableName()));
121
    if(queue.size() >= maxSize) return;
119
    if(queue.size() >= maxSize) {

    
   
120
      // release table instance since we're not reusing it

    
   
121
      this.tableFactory.releaseHTableInterface(table);

    
   
122
      return;

    
   
123
    }
122
    queue.add(table);
124
    queue.add(table);
123
  }
125
  }
124

    
   
126

   
125
  protected HTableInterface createHTable(String tableName) {
127
  protected HTableInterface createHTable(String tableName) {
126
    return this.tableFactory.createHTableInterface(config, Bytes.toBytes(tableName));
128
    return this.tableFactory.createHTableInterface(config, Bytes.toBytes(tableName));
[+20] [20] 37 lines
  1. /src/main/java/org/apache/hadoop/hbase/client/HTablePool.java: Loading...