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 1

This is not the most recent revision of the diff. The latest diff is revision 2. See what's changed.

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 1090500 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.io.IOException;
22
import java.util.LinkedList;
23
import java.util.LinkedList;
23
import java.util.Queue;
24
import java.util.Queue;
24
import java.util.Map;
25
import java.util.Map;
25
import java.util.concurrent.ConcurrentHashMap;
26
import java.util.concurrent.ConcurrentHashMap;
26
import java.util.concurrent.ConcurrentMap;
27
import java.util.concurrent.ConcurrentMap;
[+20] [20] 88 lines
[+20] [+] public HTableInterface getTable(byte [] tableName) {
115
   * If the pool already contains <i>maxSize</i> references to the table,
116
   * If the pool already contains <i>maxSize</i> references to the table,
116
   * then nothing happens.
117
   * then nothing happens.
117
   * @param table table
118
   * @param table table
118
   */
119
   */
119
  public void putTable(HTableInterface table) {
120
  public void putTable(HTableInterface table) {

    
   
121
    try {

    
   
122
      if (!table.isAutoFlush()) {

    
   
123
        table.flushCommits();

    
   
124
      }

    
   
125
    } catch (IOException ioe) {

    
   
126
      throw new RuntimeException(ioe);

    
   
127
    }
120
    Queue<HTableInterface> queue = tables.get(Bytes.toString(table.getTableName()));
128
    Queue<HTableInterface> queue = tables.get(Bytes.toString(table.getTableName()));
121
    if(queue.size() >= maxSize) return;
129
    if(queue.size() >= maxSize) {

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

    
   
131
      this.tableFactory.releaseHTableInterface(table);

    
   
132
      return;

    
   
133
    }
122
    queue.add(table);
134
    queue.add(table);
123
  }
135
  }
124

    
   
136

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