You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2009/08/18 14:46:53 UTC

svn commit: r805392 - in /hadoop/hbase/trunk: ./ src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/ src/java/org/apache/hadoop/hbase/client/ src/test/org/apache/hadoop/hbase/c...

Author: apurtell
Date: Tue Aug 18 12:46:53 2009
New Revision: 805392

URL: http://svn.apache.org/viewvc?rev=805392&view=rev
Log:
HBASE-1758 Extract interface out of HTable

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java
    hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java
    hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java
    hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java
    hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ScannerResultGenerator.java
    hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/SchemaResource.java
    hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/TableResource.java
    hadoop/hbase/trunk/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/TestTableResource.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTablePool.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTablePool.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Tue Aug 18 12:46:53 2009
@@ -15,6 +15,8 @@
                (Mathias via jgray)
    HBASE-1771  PE sequentialWrite is 7x slower because of
                MemStoreFlusher#checkStoreFileCount
+   HBASE-1758  Extract interface out of HTable (Vaibhav Puranik via Andrew
+               Purtell)
 
   OPTIMIZATIONS
 

Modified: hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java (original)
+++ hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RESTServlet.java Tue Aug 18 12:46:53 2009
@@ -28,6 +28,7 @@
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
 
 import com.sun.jersey.server.impl.container.servlet.ServletAdaptor;
@@ -36,8 +37,8 @@
  * Singleton class encapsulating global REST servlet state and functions.
  */
 public class RESTServlet extends ServletAdaptor {
-  
-  private static final long serialVersionUID = 1L;  
+
+  private static final long serialVersionUID = 1L;
   public static final int DEFAULT_MAX_AGE = 60 * 60 * 4;       // 4 hours
   public static final String VERSION_STRING = "0.0.1";
 
@@ -45,7 +46,7 @@
 
   private final HBaseConfiguration conf;
   private final HTablePool pool;
-  protected Map<String,Integer> maxAgeMap = 
+  protected Map<String,Integer> maxAgeMap =
     Collections.synchronizedMap(new HashMap<String,Integer>());
 
   /**
@@ -70,7 +71,7 @@
 
 
   /**
-   * Get a table pool for the given table. 
+   * Get a table pool for the given table.
    * @return the table pool
    */
   protected HTablePool getTablePool() {
@@ -87,7 +88,7 @@
   /**
    * @param tableName the table name
    * @return the maximum cache age suitable for use with this table, in
-   *  seconds 
+   *  seconds
    * @throws IOException
    */
   public int getMaxAge(String tableName) throws IOException {
@@ -95,10 +96,10 @@
     if (i != null) {
       return i.intValue();
     }
-    HTable table = pool.getTable(tableName);
+    HTableInterface table = pool.getTable(tableName);
     try {
       int maxAge = DEFAULT_MAX_AGE;
-      for (HColumnDescriptor family : 
+      for (HColumnDescriptor family :
           table.getTableDescriptor().getFamilies()) {
         int ttl = family.getTimeToLive();
         if (ttl < 0) {

Modified: hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java (original)
+++ hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RegionsResource.java Tue Aug 18 12:46:53 2009
@@ -1,99 +0,0 @@
-/*
- * Copyright 2009 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.stargate;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Map;
-
-import javax.ws.rs.GET;
-import javax.ws.rs.Produces;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.CacheControl;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.UriInfo;
-import javax.ws.rs.core.Response.ResponseBuilder;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HServerAddress;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HTablePool;
-import org.apache.hadoop.hbase.stargate.model.TableInfoModel;
-import org.apache.hadoop.hbase.stargate.model.TableRegionModel;
-
-public class RegionsResource implements Constants {
-  private static final Log LOG = LogFactory.getLog(RegionsResource.class);
-
-  private String table;
-  private CacheControl cacheControl;
-
-  public RegionsResource(String table) {
-    this.table = table;
-    cacheControl = new CacheControl();
-    cacheControl.setNoCache(true);
-    cacheControl.setNoTransform(false);
-  }
-
-  private Map<HRegionInfo,HServerAddress> getTableRegions()
-      throws IOException {
-    HTablePool pool = RESTServlet.getInstance().getTablePool();
-    HTable table = pool.getTable(this.table);
-    try {
-      return table.getRegionsInfo();
-    } finally {
-      pool.putTable(table);
-    }
-  }
-
-  @GET
-  @Produces({MIMETYPE_TEXT, MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_JAVASCRIPT,
-    MIMETYPE_PROTOBUF})
-  public Response get(@Context UriInfo uriInfo) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("GET " + uriInfo.getAbsolutePath());
-    }
-    try {
-      TableInfoModel model = new TableInfoModel(table);
-      Map<HRegionInfo,HServerAddress> regions = getTableRegions();
-      for (Map.Entry<HRegionInfo,HServerAddress> e: regions.entrySet()) {
-        HRegionInfo hri = e.getKey();
-        HServerAddress addr = e.getValue();
-        InetSocketAddress sa = addr.getInetSocketAddress();
-        model.add(
-          new TableRegionModel(table, hri.getRegionId(), hri.getStartKey(),
-                hri.getEndKey(),
-                sa.getHostName() + ":" + Integer.valueOf(sa.getPort())));
-      }
-      ResponseBuilder response = Response.ok(model);
-      response.cacheControl(cacheControl);
-      return response.build();
-    } catch (TableNotFoundException e) {
-      throw new WebApplicationException(Response.Status.NOT_FOUND);
-    } catch (IOException e) {
-      throw new WebApplicationException(e,
-                  Response.Status.SERVICE_UNAVAILABLE);
-    }
-  }
-}

Modified: hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java (original)
+++ hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResource.java Tue Aug 18 12:46:53 2009
@@ -41,7 +41,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.stargate.model.CellModel;
@@ -56,7 +56,7 @@
   private RowSpec rowspec;
   private CacheControl cacheControl;
 
-  public RowResource(String table, String rowspec, String versions) 
+  public RowResource(String table, String rowspec, String versions)
       throws IOException {
     this.table = table;
     this.rowspec = new RowSpec(rowspec);
@@ -137,10 +137,10 @@
     try {
       pool = RESTServlet.getInstance().getTablePool();
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
                   Response.Status.INTERNAL_SERVER_ERROR);
     }
-    HTable table = null;
+    HTableInterface table = null;
     try {
       table = pool.getTable(this.table);
       for (RowModel row: model.getRows()) {
@@ -171,16 +171,16 @@
     }
   }
 
-  private Response updateBinary(byte[] message, HttpHeaders headers, 
+  private Response updateBinary(byte[] message, HttpHeaders headers,
       boolean replace) {
     HTablePool pool;
     try {
       pool = RESTServlet.getInstance().getTablePool();
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
                   Response.Status.INTERNAL_SERVER_ERROR);
     }
-    HTable table = null;    
+    HTableInterface table = null;
     try {
       byte[] row = rowspec.getRow();
       byte[][] columns = rowspec.getColumns();
@@ -240,7 +240,7 @@
 
   @PUT
   @Consumes(MIMETYPE_BINARY)
-  public Response putBinary(byte[] message, @Context UriInfo uriInfo, 
+  public Response putBinary(byte[] message, @Context UriInfo uriInfo,
       @Context HttpHeaders headers)
   {
     if (LOG.isDebugEnabled()) {
@@ -261,7 +261,7 @@
 
   @POST
   @Consumes(MIMETYPE_BINARY)
-  public Response postBinary(byte[] message, @Context UriInfo uriInfo, 
+  public Response postBinary(byte[] message, @Context UriInfo uriInfo,
       @Context HttpHeaders headers)
   {
     if (LOG.isDebugEnabled()) {
@@ -281,17 +281,17 @@
       if (rowspec.hasTimestamp()) {
         delete.deleteColumns(split[0], split[1], rowspec.getTimestamp());
       } else {
-        delete.deleteColumns(split[0], split[1]);        
+        delete.deleteColumns(split[0], split[1]);
       }
     }
     HTablePool pool;
     try {
       pool = RESTServlet.getInstance().getTablePool();
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
                   Response.Status.INTERNAL_SERVER_ERROR);
     }
-    HTable table = null;
+    HTableInterface table = null;
     try {
       table = pool.getTable(this.table);
       table.delete(delete);
@@ -300,7 +300,7 @@
       }
       table.flushCommits();
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
                   Response.Status.SERVICE_UNAVAILABLE);
     } finally {
       if (table != null) {

Modified: hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java (original)
+++ hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/RowResultGenerator.java Tue Aug 18 12:46:53 2009
@@ -27,7 +27,7 @@
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.client.Result;
 
@@ -36,15 +36,15 @@
 
   public RowResultGenerator(String tableName, RowSpec rowspec)
       throws IllegalArgumentException, IOException {
-    HTablePool pool = RESTServlet.getInstance().getTablePool(); 
-    HTable table = pool.getTable(tableName);
+    HTablePool pool = RESTServlet.getInstance().getTablePool();
+    HTableInterface table = pool.getTable(tableName);
     try {
       Get get = new Get(rowspec.getRow());
       if (rowspec.hasColumns()) {
         get.addColumns(rowspec.getColumns());
       } else {
         // rowspec does not explicitly specify columns, return them all
-        for (HColumnDescriptor family: 
+        for (HColumnDescriptor family:
             table.getTableDescriptor().getFamilies()) {
           get.addFamily(family.getName());
         }

Modified: hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ScannerResultGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ScannerResultGenerator.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ScannerResultGenerator.java (original)
+++ hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/ScannerResultGenerator.java Tue Aug 18 12:46:53 2009
@@ -28,7 +28,7 @@
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.UnknownScannerException;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -38,7 +38,7 @@
 public class ScannerResultGenerator extends ResultGenerator {
   private static final Log LOG =
     LogFactory.getLog(ScannerResultGenerator.class);
-  
+
   private String id;
   private Iterator<KeyValue> rowI;
   private ResultScanner scanner;
@@ -46,8 +46,8 @@
 
   public ScannerResultGenerator(String tableName, RowSpec rowspec)
       throws IllegalArgumentException, IOException {
-    HTablePool pool = RESTServlet.getInstance().getTablePool(); 
-    HTable table = pool.getTable(tableName);
+    HTablePool pool = RESTServlet.getInstance().getTablePool();
+    HTableInterface table = pool.getTable(tableName);
     try {
       Scan scan;
       if (rowspec.hasEndRow()) {
@@ -58,12 +58,12 @@
       if (rowspec.hasColumns()) {
         scan.addColumns(rowspec.getColumns());
       } else {
-        for (HColumnDescriptor family: 
+        for (HColumnDescriptor family:
             table.getTableDescriptor().getFamilies()) {
           scan.addFamily(family.getName());
         }
       }
-      scan.setTimeRange(rowspec.getStartTime(), rowspec.getEndTime());          
+      scan.setTimeRange(rowspec.getStartTime(), rowspec.getEndTime());
       scan.setMaxVersions(rowspec.getMaxVersions());
       scanner = table.getScanner(scan);
       cached = null;

Modified: hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/SchemaResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/SchemaResource.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/SchemaResource.java (original)
+++ hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/SchemaResource.java Tue Aug 18 12:46:53 2009
@@ -44,7 +44,7 @@
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.stargate.model.ColumnSchemaModel;
@@ -67,7 +67,7 @@
   private HTableDescriptor getTableSchema() throws IOException,
       TableNotFoundException {
     HTablePool pool = RESTServlet.getInstance().getTablePool();
-    HTable table = pool.getTable(this.table);
+    HTableInterface table = pool.getTable(this.table);
     try {
       return table.getTableDescriptor();
     } finally {
@@ -88,7 +88,7 @@
       model.setName(htd.getNameAsString());
       for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
           htd.getValues().entrySet()) {
-        model.addAttribute(Bytes.toString(e.getKey().get()), 
+        model.addAttribute(Bytes.toString(e.getKey().get()),
             Bytes.toString(e.getValue().get()));
       }
       for (HColumnDescriptor hcd: htd.getFamilies()) {
@@ -96,7 +96,7 @@
         columnModel.setName(hcd.getNameAsString());
         for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
           hcd.getValues().entrySet()) {
-        columnModel.addAttribute(Bytes.toString(e.getKey().get()), 
+        columnModel.addAttribute(Bytes.toString(e.getKey().get()),
           Bytes.toString(e.getValue().get()));
       }
         model.addColumnFamily(columnModel);
@@ -138,10 +138,10 @@
       }
       return Response.created(uriInfo.getAbsolutePath()).build();
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
             Response.Status.SERVICE_UNAVAILABLE);
-    }      
-  } 
+    }
+  }
 
   private Response update(byte[] tableName, TableSchemaModel model,
       UriInfo uriInfo, HBaseAdmin admin) {
@@ -157,11 +157,11 @@
           if (htd.hasFamily(hcd.getName())) {
             admin.modifyColumn(tableName, hcd.getName(), hcd);
           } else {
-            admin.addColumn(model.getName(), hcd);            
+            admin.addColumn(model.getName(), hcd);
           }
         }
       } catch (IOException e) {
-        throw new WebApplicationException(e, 
+        throw new WebApplicationException(e,
             Response.Status.INTERNAL_SERVER_ERROR);
       } finally {
         admin.enableTable(tableName);
@@ -186,7 +186,7 @@
         return update(tableName, model, uriInfo, admin);
       }
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
             Response.Status.SERVICE_UNAVAILABLE);
     }
   }
@@ -212,7 +212,7 @@
   }
 
   @DELETE
-  public Response delete(@Context UriInfo uriInfo) {     
+  public Response delete(@Context UriInfo uriInfo) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("DELETE " + uriInfo.getAbsolutePath());
     }
@@ -225,7 +225,7 @@
     } catch (TableNotFoundException e) {
       throw new WebApplicationException(Response.Status.NOT_FOUND);
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
             Response.Status.SERVICE_UNAVAILABLE);
     }
   }

Modified: hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/TableResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/TableResource.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/TableResource.java (original)
+++ hadoop/hbase/trunk/src/contrib/stargate/src/java/org/apache/hadoop/hbase/stargate/TableResource.java Tue Aug 18 12:46:53 2009
@@ -85,27 +85,21 @@
       response.cacheControl(cacheControl);
       return response.build();
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
                   Response.Status.SERVICE_UNAVAILABLE);
     }
   }
 
-  @Path("{table}/regions")
-  public RegionsResource getRegionsResource(
-      @PathParam("table") String table) {
-    return new RegionsResource(table);    
-  }
-
   @Path("{table}/scanner")
   public ScannerResource getScannerResource(
       @PathParam("table") String table) {
-    return new ScannerResource(table);    
+    return new ScannerResource(table);
   }
 
   @Path("{table}/schema")
   public SchemaResource getSchemaResource(
       @PathParam("table") String table) {
-    return new SchemaResource(table);    
+    return new SchemaResource(table);
   }
 
   @Path("{table}/{rowspec: .+}")
@@ -116,7 +110,7 @@
     try {
       return new RowResource(table, rowspec, versions);
     } catch (IOException e) {
-      throw new WebApplicationException(e, 
+      throw new WebApplicationException(e,
                   Response.Status.INTERNAL_SERVER_ERROR);
     }
   }

Modified: hadoop/hbase/trunk/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/TestTableResource.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/TestTableResource.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/TestTableResource.java (original)
+++ hadoop/hbase/trunk/src/contrib/stargate/src/test/org/apache/hadoop/hbase/stargate/TestTableResource.java Tue Aug 18 12:46:53 2009
@@ -22,23 +22,15 @@
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.net.InetSocketAddress;
 import java.util.Iterator;
-import java.util.Map;
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.stargate.client.Client;
 import org.apache.hadoop.hbase.stargate.client.Cluster;
 import org.apache.hadoop.hbase.stargate.client.Response;
@@ -46,16 +38,10 @@
 import org.apache.hadoop.hbase.stargate.model.TableInfoModel;
 import org.apache.hadoop.hbase.stargate.model.TableListModel;
 import org.apache.hadoop.hbase.stargate.model.TableRegionModel;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.util.StringUtils;
 
 public class TestTableResource extends MiniClusterTestCase {
-  private static final Log LOG = 
-    LogFactory.getLog(TestTableResource.class);
-
   private static String TABLE = "TestTableResource";
   private static String COLUMN = "test:";
-  private static Map<HRegionInfo,HServerAddress> regionMap;
 
   private Client client;
   private JAXBContext context;
@@ -81,39 +67,7 @@
     HTableDescriptor htd = new HTableDescriptor(TABLE);
     htd.addFamily(new HColumnDescriptor(COLUMN));
     admin.createTable(htd);
-    HTable table = new HTable(conf, TABLE);
-    byte[] k = new byte[3];
-    byte [][] famAndQf = KeyValue.parseColumn(Bytes.toBytes(COLUMN));
-    for (byte b1 = 'a'; b1 < 'z'; b1++) {
-      for (byte b2 = 'a'; b2 < 'z'; b2++) {
-        for (byte b3 = 'a'; b3 < 'z'; b3++) {
-          k[0] = b1;
-          k[1] = b2;
-          k[2] = b3;
-          Put put = new Put(k);
-          put.add(famAndQf[0], famAndQf[1], k);
-          table.put(put);
-        }
-      }
-    }
-    table.flushCommits();
-    // get the initial layout (should just be one region)
-    Map<HRegionInfo,HServerAddress> m = table.getRegionsInfo();
-    assertEquals(m.size(), 1);
-    // tell the master to split the table
-    admin.split(TABLE);
-    // give some time for the split to happen
-    try {
-      Thread.sleep(15 * 1000);
-    } catch (InterruptedException e) {
-      LOG.warn(StringUtils.stringifyException(e));
-    }
-    // check again
-    m = table.getRegionsInfo();
-    // should have two regions now
-    assertEquals(m.size(), 2);
-    regionMap = m;
-    LOG.info("regions: " + regionMap);
+    new HTable(conf, TABLE);
   }
 
   @Override
@@ -162,59 +116,4 @@
     model.getObjectFromMessage(response.getBody());
     checkTableList(model);
   }
-
-  public void checkTableInfo(TableInfoModel model) {
-    assertEquals(model.getName(), TABLE);
-    Iterator<TableRegionModel> regions = model.getRegions().iterator();
-    assertTrue(regions.hasNext());
-    while (regions.hasNext()) {
-      TableRegionModel region = regions.next();
-      boolean found = false;
-      for (Map.Entry<HRegionInfo,HServerAddress> e: regionMap.entrySet()) {
-        HRegionInfo hri = e.getKey();
-        if (hri.getRegionNameAsString().equals(region.getName())) {
-          found = true;
-          byte[] startKey = hri.getStartKey();
-          byte[] endKey = hri.getEndKey();
-          InetSocketAddress sa = e.getValue().getInetSocketAddress();
-          String location = sa.getHostName() + ":" +
-            Integer.valueOf(sa.getPort());
-          assertEquals(hri.getRegionId(), region.getId());
-          assertTrue(Bytes.equals(startKey, region.getStartKey()));
-          assertTrue(Bytes.equals(endKey, region.getEndKey()));
-          assertEquals(location, region.getLocation());
-          break;
-        }
-      }
-      assertTrue(found);
-    }
-  }
-
-  public void testTableInfoText() throws IOException {
-    Response response = client.get("/" + TABLE + "/regions", MIMETYPE_PLAIN);
-    assertEquals(response.getCode(), 200);
-  }
-
-  public void testTableInfoXML() throws IOException, JAXBException {
-    Response response = client.get("/" + TABLE + "/regions", MIMETYPE_XML);
-    assertEquals(response.getCode(), 200);
-    TableInfoModel model = (TableInfoModel)
-      context.createUnmarshaller()
-        .unmarshal(new ByteArrayInputStream(response.getBody()));
-    checkTableInfo(model);
-  }
-
-  public void testTableInfoJSON() throws IOException {
-    Response response = client.get("/" + TABLE + "/regions", MIMETYPE_JSON);
-    assertEquals(response.getCode(), 200);
-  }
-
-  public void testTableInfoPB() throws IOException, JAXBException {
-    Response response = 
-      client.get("/" + TABLE + "/regions", MIMETYPE_PROTOBUF);
-    assertEquals(response.getCode(), 200);
-    TableInfoModel model = new TableInfoModel();
-    model.getObjectFromMessage(response.getBody());
-    checkTableInfo(model);
-  }
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java Tue Aug 18 12:46:53 2009
@@ -56,7 +56,7 @@
 /**
  * Used to communicate with a single HBase table
  */
-public class HTable {
+public class HTable implements HTableInterface {
   private final HConnection connection;
   private final byte [] tableName;
   protected final int scannerTimeout;
@@ -91,7 +91,7 @@
 
   /**
    * Creates an object to access a HBase table
-   * 
+   *
    * @param conf configuration object
    * @param tableName name of the table
    * @throws IOException
@@ -103,7 +103,7 @@
 
   /**
    * Creates an object to access a HBase table.
-   * 
+   *
    * @param conf configuration object
    * @param tableName name of the table
    * @throws IOException
@@ -144,7 +144,7 @@
   public static boolean isTableEnabled(byte[] tableName) throws IOException {
     return isTableEnabled(new HBaseConfiguration(), tableName);
   }
-  
+
   /**
    * @param conf HBaseConfiguration object
    * @param tableName name of table to check
@@ -202,7 +202,7 @@
   public HConnection getConnection() {
     return this.connection;
   }
-  
+
   /**
    * Get the number of rows for caching that will be passed to scanners
    * @return the number of rows for caching
@@ -220,7 +220,7 @@
   }
 
   /**
-   * @return table metadata 
+   * @return table metadata
    * @throws IOException
    */
   public HTableDescriptor getTableDescriptor() throws IOException {
@@ -230,7 +230,7 @@
 
   /**
    * Gets the starting row key for every region in the currently open table
-   * 
+   *
    * @return Array of region starting row keys
    * @throws IOException
    */
@@ -240,7 +240,7 @@
 
   /**
    * Gets the ending row key for every region in the currently open table
-   * 
+   *
    * @return Array of region ending row keys
    * @throws IOException
    */
@@ -251,7 +251,7 @@
   /**
    * Gets the starting and ending row keys for every region in the currently
    * open table
-   * 
+   *
    * @return Pair of arrays of region starting and ending row keys
    * @throws IOException
    */
@@ -262,7 +262,7 @@
     MetaScannerVisitor visitor = new MetaScannerVisitor() {
       public boolean processRow(Result rowResult) throws IOException {
         HRegionInfo info = Writables.getHRegionInfo(
-            rowResult.getValue(HConstants.CATALOG_FAMILY, 
+            rowResult.getValue(HConstants.CATALOG_FAMILY,
                 HConstants.REGIONINFO_QUALIFIER));
         if (Bytes.equals(info.getTableDesc().getName(), getTableName())) {
           if (!(info.isOffline() || info.isSplit())) {
@@ -280,7 +280,7 @@
 
   /**
    * Get all the regions and their address for this table
-   * 
+   *
    * @return A map of HRegionInfo with it's server address
    * @throws IOException
    */
@@ -291,21 +291,21 @@
     MetaScannerVisitor visitor = new MetaScannerVisitor() {
       public boolean processRow(Result rowResult) throws IOException {
         HRegionInfo info = Writables.getHRegionInfo(
-            rowResult.getValue(HConstants.CATALOG_FAMILY, 
+            rowResult.getValue(HConstants.CATALOG_FAMILY,
                 HConstants.REGIONINFO_QUALIFIER));
-        
+
         if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) {
           return false;
         }
 
         HServerAddress server = new HServerAddress();
-        byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY, 
+        byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
             HConstants.SERVER_QUALIFIER);
         if (value != null && value.length > 0) {
           String address = Bytes.toString(value);
           server = new HServerAddress(address);
         }
-        
+
         if (!(info.isOffline() || info.isSplit())) {
           regionMap.put(new UnmodifyableHRegionInfo(info), server);
         }
@@ -318,9 +318,9 @@
   }
 
   /**
-   * Return the row that matches <i>row</i> exactly, 
+   * Return the row that matches <i>row</i> exactly,
    * or the one that immediately preceeds it.
-   * 
+   *
    * @param row row key
    * @param family Column family to look for row in.
    * @return map of values
@@ -339,9 +339,9 @@
    }
 
   /**
-  * Return the row that matches <i>row</i> exactly, 
+  * Return the row that matches <i>row</i> exactly,
   * or the one that immediately preceeds it.
-  * 
+  *
   * @param row row key
   * @param family Column family to look for row in.
   * @return map of values
@@ -354,7 +354,7 @@
     return r == null || r.isEmpty()? null: r.getRowResult();
   }
 
-  /** 
+  /**
    * Get a scanner on the current table as specified by the {@link Scan} object
    *
    * @param scan a configured {@link Scan} object
@@ -369,7 +369,7 @@
   }
   /**
    * Get a scanner on the current table as specified by the {@link Scan} object
-   * 
+   *
    * @param family  The column family to scan.
    * @return The scanner.
    * @throws IOException
@@ -380,10 +380,10 @@
     scan.addFamily(family);
     return getScanner(scan);
   }
-  
+
   /**
    * Get a scanner on the current table as specified by the {@link Scan} object
-   * 
+   *
    * @param family  The column family to scan.
    * @param qualifier  The column qualifier to scan.
    * @return The scanner.
@@ -413,10 +413,10 @@
         }
     );
   }
-  
+
   /**
-   * 
-   * @param delete 
+   *
+   * @param delete
    * @throws IOException
    * @since 0.20.0
    */
@@ -431,7 +431,7 @@
         }
     );
   }
-  
+
   /**
    * Commit a Put to the table.
    * <p>
@@ -448,7 +448,7 @@
       flushCommits();
     }
   }
-  
+
   /**
    * Commit a List of Puts to the table.
    * <p>
@@ -467,11 +467,11 @@
       flushCommits();
     }
   }
-  
+
   /**
    * Atomically increments a column value. If the column value already exists
    * and is not a big-endian long, this could throw an exception.<p>
-   * 
+   *
    * @param row
    * @param family
    * @param qualifier
@@ -479,7 +479,7 @@
    * @return The new value.
    * @throws IOException
    */
-  public long incrementColumnValue(final byte [] row, final byte [] family, 
+  public long incrementColumnValue(final byte [] row, final byte [] family,
       final byte [] qualifier, final long amount)
   throws IOException {
     return incrementColumnValue(row, family, qualifier, amount, true);
@@ -488,8 +488,8 @@
   /**
    * Atomically increments a column value. If the column value already exists
    * and is not a big-endian long, this could throw an exception.<p>
-   * 
-   * Setting writeToWAL to false means that in a fail scenario, you will lose 
+   *
+   * Setting writeToWAL to false means that in a fail scenario, you will lose
    * any increments that have not been flushed.
    * @param row
    * @param family
@@ -499,7 +499,7 @@
    * @return The new value.
    * @throws IOException
    */
-  public long incrementColumnValue(final byte [] row, final byte [] family, 
+  public long incrementColumnValue(final byte [] row, final byte [] family,
       final byte [] qualifier, final long amount, final boolean writeToWAL)
   throws IOException {
     NullPointerException npe = null;
@@ -517,7 +517,7 @@
         new ServerCallable<Long>(connection, tableName, row) {
           public Long call() throws IOException {
             return server.incrementColumnValue(
-                location.getRegionInfo().getRegionName(), row, family, 
+                location.getRegionInfo().getRegionName(), row, family,
                 qualifier, amount, writeToWAL);
           }
         }
@@ -527,7 +527,7 @@
   /**
    * Atomically checks if a row/family/qualifier value match the expectedValue.
    * If it does, it adds the put.
-   * 
+   *
    * @param row
    * @param family
    * @param qualifier
@@ -536,8 +536,8 @@
    * @throws IOException
    * @return true if the new put was execute, false otherwise
    */
-  public synchronized boolean checkAndPut(final byte [] row, 
-      final byte [] family, final byte [] qualifier, final byte [] value, 
+  public synchronized boolean checkAndPut(final byte [] row,
+      final byte [] family, final byte [] qualifier, final byte [] value,
       final Put put)
   throws IOException {
     return connection.getRegionServerWithRetries(
@@ -549,12 +549,12 @@
         }
       ).booleanValue();
   }
-  
+
   /**
    * Test for the existence of columns in the table, as specified in the Get.<p>
-   * 
+   *
    * This will return true if the Get matches one or more keys, false if not.<p>
-   * 
+   *
    * This is a server-side call so it prevents any data from being transfered
    * to the client.
    * @param get
@@ -571,7 +571,7 @@
       }
     ).booleanValue();
   }
-  
+
   /**
    * Commit to the table the buffer of BatchUpdate.
    * Called automatically in the commit methods when autoFlush is true.
@@ -585,19 +585,19 @@
       writeBuffer.clear();
     }
   }
-   
+
   /**
    * Release held resources
-   * 
+   *
    * @throws IOException
   */
   public void close() throws IOException{
     flushCommits();
   }
-  
+
   /**
    * Utility method that verifies Put is well formed.
-   * 
+   *
    * @param put
    * @throws IllegalArgumentException
    */
@@ -606,7 +606,7 @@
       throw new IllegalArgumentException("No columns to insert");
     }
   }
-  
+
   /**
    * Obtain a row lock
    * @param row The row to lock
@@ -644,7 +644,7 @@
       }
     );
   }
-  
+
   /**
    * Get the value of autoFlush. If true, updates will not be buffered
    * @return value of autoFlush
@@ -684,7 +684,7 @@
   }
 
   /**
-   * Get the write buffer 
+   * Get the write buffer
    * @return the current write buffer
    */
   public ArrayList<Put> getWriteBuffer() {
@@ -692,10 +692,10 @@
   }
 
   // Old API. Pre-hbase-880, hbase-1304.
-  
+
   /**
    * Get a single value for the specified row and column
-   * 
+   *
    * @param row row key
    * @param column column name
    * @return value for specified row/column
@@ -707,7 +707,7 @@
     return get(Bytes.toBytes(row), Bytes.toBytes(column));
   }
 
-  /** 
+  /**
    * Get a single value for the specified row and column
    *
    * @param row row key
@@ -722,7 +722,7 @@
     return get(Bytes.toBytes(row), Bytes.toBytes(column), numVersions);
   }
 
-  /** 
+  /**
    * Get a single value for the specified row and column
    *
    * @param row row key
@@ -740,7 +740,7 @@
     return r == null || r.size() <= 0? null: r.getCellValue();
   }
 
-  /** 
+  /**
    * Get the specified number of versions of the specified row and column
    * @param row row key
    * @param column column name
@@ -755,7 +755,7 @@
     return get(row, column, HConstants.LATEST_TIMESTAMP, numVersions);
   }
 
-  /** 
+  /**
    * Get the specified number of versions of the specified row and column with
    * the specified timestamp.
    *
@@ -773,7 +773,7 @@
     return get(Bytes.toBytes(row), Bytes.toBytes(column), timestamp, numVersions);
   }
 
-  /** 
+  /**
    * Get the specified number of versions of the specified row and column with
    * the specified timestamp.
    *
@@ -803,9 +803,9 @@
     return r == null || r.size() <= 0? null: r.getCellValues();
   }
 
-  /** 
+  /**
    * Get all the data for the specified row at the latest timestamp
-   * 
+   *
    * @param row row key
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
@@ -815,9 +815,9 @@
     return getRow(Bytes.toBytes(row));
   }
 
-  /** 
+  /**
    * Get all the data for the specified row at the latest timestamp
-   * 
+   *
    * @param row row key
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
@@ -826,10 +826,10 @@
   public RowResult getRow(final byte [] row) throws IOException {
     return getRow(row, HConstants.LATEST_TIMESTAMP);
   }
- 
-  /** 
+
+  /**
    * Get more than one version of all columns for the specified row
-   * 
+   *
    * @param row row key
    * @param numVersions number of versions to return
    * @return RowResult is <code>null</code> if row does not exist.
@@ -838,13 +838,13 @@
    */
   public RowResult getRow(final String row, final int numVersions)
   throws IOException {
-    return getRow(Bytes.toBytes(row), null, 
+    return getRow(Bytes.toBytes(row), null,
                   HConstants.LATEST_TIMESTAMP, numVersions, null);
   }
 
-  /** 
+  /**
    * Get more than one version of all columns for the specified row
-   * 
+   *
    * @param row row key
    * @param numVersions number of versions to return
    * @return RowResult is <code>null</code> if row does not exist.
@@ -856,38 +856,38 @@
     return getRow(row, null, HConstants.LATEST_TIMESTAMP, numVersions, null);
   }
 
-  /** 
+  /**
    * Get all the data for the specified row at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param ts timestamp
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final String row, final long ts) 
+  public RowResult getRow(final String row, final long ts)
   throws IOException {
     return getRow(Bytes.toBytes(row), ts);
   }
 
-  /** 
+  /**
    * Get all the data for the specified row at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param ts timestamp
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final byte [] row, final long ts) 
+  public RowResult getRow(final byte [] row, final long ts)
   throws IOException {
     return getRow(row,null,ts);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of all columns for the specified row
    * at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param ts timestamp
    * @param numVersions number of versions to return
@@ -899,11 +899,11 @@
       final int numVersions) throws IOException {
     return getRow(Bytes.toBytes(row), null, ts, numVersions, null);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of all columns for the specified row
    * at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param timestamp timestamp
    * @param numVersions number of versions to return
@@ -916,37 +916,37 @@
     return getRow(row, null, timestamp, numVersions, null);
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at the latest timestamp
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final String row, final String [] columns) 
+  public RowResult getRow(final String row, final String [] columns)
   throws IOException {
     return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns));
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at the latest timestamp
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final byte [] row, final byte [][] columns) 
+  public RowResult getRow(final byte [] row, final byte [][] columns)
   throws IOException {
     return getRow(row, columns, HConstants.LATEST_TIMESTAMP);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of selected columns for the specified row
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param numVersions number of versions to return
@@ -959,10 +959,10 @@
     return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns),
                   HConstants.LATEST_TIMESTAMP, numVersions, null);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of selected columns for the specified row
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param numVersions number of versions to return
@@ -975,9 +975,9 @@
     return getRow(row, columns, HConstants.LATEST_TIMESTAMP, numVersions, null);
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param ts timestamp
@@ -985,15 +985,15 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final String row, final String [] columns, 
-    final long ts) 
-  throws IOException {  
+  public RowResult getRow(final String row, final String [] columns,
+    final long ts)
+  throws IOException {
     return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), ts);
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param ts timestamp
@@ -1001,16 +1001,16 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final byte [] row, final byte [][] columns, 
-    final long ts) 
-  throws IOException {       
+  public RowResult getRow(final byte [] row, final byte [][] columns,
+    final long ts)
+  throws IOException {
     return getRow(row,columns,ts,1,null);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of selected columns for the specified row,
    * using an existing row lock.
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param numVersions number of versions to return
@@ -1026,21 +1026,21 @@
                   numVersions, rowLock);
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at a specified timestamp
    * using existing row lock.
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param ts timestamp
-   * @param numVersions 
+   * @param numVersions
    * @param rl row lock
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final byte [] row, final byte [][] columns, 
-    final long ts, final int numVersions, final RowLock rl) 
+  public RowResult getRow(final byte [] row, final byte [][] columns,
+    final long ts, final int numVersions, final RowLock rl)
   throws IOException {
     Get g = rl != null? new Get(row, rl): new Get(row);
     if (columns != null) {
@@ -1061,7 +1061,7 @@
     return r == null || r.size() <= 0? null: r.getRowResult();
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at first row.
    * Return the specified columns.
    *
@@ -1079,7 +1079,7 @@
     return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1098,7 +1098,7 @@
     return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow));
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at first row.
    * Return the specified columns.
    *
@@ -1117,7 +1117,7 @@
       HConstants.LATEST_TIMESTAMP, null);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1135,8 +1135,8 @@
   throws IOException {
     return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, null);
   }
-  
-  /** 
+
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1156,8 +1156,8 @@
   throws IOException {
     return getScanner(columns, startRow, timestamp, null);
   }
-  
-  /** 
+
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1174,11 +1174,11 @@
    */
   public Scanner getScanner(final byte[][] columns, final byte [] startRow,
     RowFilterInterface filter)
-  throws IOException { 
+  throws IOException {
     return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, filter);
   }
-  
-  /** 
+
+  /**
    * Get a scanner on the current table starting at the specified row and
    * ending just before <code>stopRow<code>.
    * Return the specified columns.
@@ -1202,7 +1202,7 @@
     return getScanner(columns, startRow, stopRow, HConstants.LATEST_TIMESTAMP);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row and
    * ending just before <code>stopRow<code>.
    * Return the specified columns.
@@ -1228,7 +1228,7 @@
       Bytes.toBytes(stopRow), timestamp);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row and
    * ending just before <code>stopRow<code>.
    * Return the specified columns.
@@ -1254,7 +1254,7 @@
       new WhileMatchRowFilter(new StopRowFilter(stopRow)));
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1277,7 +1277,7 @@
       timestamp, filter);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1336,7 +1336,7 @@
   public void deleteAll(final String row) throws IOException {
     deleteAll(row, null);
   }
-  
+
   /**
    * Completely delete the row's cells.
    *
@@ -1376,11 +1376,11 @@
     deleteAll(row, null, ts);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column.
    * @param row Row to update
    * @param column name of column whose value is to be deleted
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAll(final String row, final String column)
@@ -1388,13 +1388,13 @@
     deleteAll(row, column, HConstants.LATEST_TIMESTAMP);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp.
    * @param row Row to update
    * @param column name of column whose value is to be deleted
    * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAll(final String row, final String column, final long ts)
@@ -1403,13 +1403,13 @@
       column != null? Bytes.toBytes(column): null, ts);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp.
    * @param row Row to update
    * @param column name of column whose value is to be deleted
    * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAll(final byte [] row, final byte [] column, final long ts)
@@ -1417,7 +1417,7 @@
     deleteAll(row,column,ts,null);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp, using an
    * existing row lock.
@@ -1425,7 +1425,7 @@
    * @param column name of column whose value is to be deleted
    * @param ts Delete all cells of the same timestamp or older.
    * @param rl Existing row lock
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAll(final byte [] row, final byte [] column, final long ts,
@@ -1437,12 +1437,12 @@
     }
     delete(d);
   }
-  
-  /** 
+
+  /**
    * Delete all cells that match the passed row and column.
    * @param row Row to update
    * @param colRegex column regex expression
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAllByRegex(final String row, final String colRegex)
@@ -1450,35 +1450,35 @@
     deleteAllByRegex(row, colRegex, HConstants.LATEST_TIMESTAMP);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp.
    * @param row Row to update
    * @param colRegex Column Regex expression
    * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteAllByRegex(final String row, final String colRegex, 
+  public void deleteAllByRegex(final String row, final String colRegex,
       final long ts) throws IOException {
     deleteAllByRegex(Bytes.toBytes(row), colRegex, ts);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp.
    * @param row Row to update
    * @param colRegex Column Regex expression
    * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteAllByRegex(final byte [] row, final String colRegex, 
+  public void deleteAllByRegex(final byte [] row, final String colRegex,
       final long ts) throws IOException {
     deleteAllByRegex(row, colRegex, ts, null);
   }
-  
-  /** 
+
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp, using an
    * existing row lock.
@@ -1486,10 +1486,10 @@
    * @param colRegex Column regex expression
    * @param ts Delete all cells of the same timestamp or older.
    * @param rl Existing row lock
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteAllByRegex(final byte [] row, final String colRegex, 
+  public void deleteAllByRegex(final byte [] row, final String colRegex,
       final long ts, final RowLock rl)
   throws IOException {
     throw new UnsupportedOperationException("TODO: Not yet implemented");
@@ -1503,7 +1503,7 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamily(final String row, final String family) 
+  public void deleteFamily(final String row, final String family)
   throws IOException {
     deleteFamily(row, family, HConstants.LATEST_TIMESTAMP);
   }
@@ -1516,7 +1516,7 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamily(final byte[] row, final byte[] family) 
+  public void deleteFamily(final byte[] row, final byte[] family)
   throws IOException {
     deleteFamily(row, family, HConstants.LATEST_TIMESTAMP);
   }
@@ -1530,7 +1530,7 @@
    * @param timestamp Timestamp to match
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
-   */  
+   */
   public void deleteFamily(final String row, final String family,
       final long timestamp)
   throws IOException{
@@ -1547,7 +1547,7 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamily(final byte [] row, final byte [] family, 
+  public void deleteFamily(final byte [] row, final byte [] family,
     final long timestamp)
   throws IOException {
     deleteFamily(row,family,timestamp,null);
@@ -1564,16 +1564,16 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamily(final byte [] row, final byte [] family, 
+  public void deleteFamily(final byte [] row, final byte [] family,
     final long timestamp, final RowLock rl)
   throws IOException {
     Delete d = new Delete(row, HConstants.LATEST_TIMESTAMP, rl);
     d.deleteFamily(stripColon(family), timestamp);
     delete(d);
   }
-  
+
   /**
-   * Delete all cells for a row with matching column family regex 
+   * Delete all cells for a row with matching column family regex
    * at all timestamps.
    *
    * @param row The row to operate on
@@ -1581,13 +1581,13 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamilyByRegex(final String row, final String familyRegex) 
+  public void deleteFamilyByRegex(final String row, final String familyRegex)
   throws IOException {
     deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP);
   }
 
   /**
-   * Delete all cells for a row with matching column family regex 
+   * Delete all cells for a row with matching column family regex
    * at all timestamps.
    *
    * @param row The row to operate on
@@ -1595,7 +1595,7 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamilyByRegex(final byte[] row, final String familyRegex) 
+  public void deleteFamilyByRegex(final byte[] row, final String familyRegex)
   throws IOException {
     deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP);
   }
@@ -1609,7 +1609,7 @@
    * @param timestamp Timestamp to match
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
-   */  
+   */
   public void deleteFamilyByRegex(final String row, final String familyRegex,
       final long timestamp)
   throws IOException{
@@ -1626,17 +1626,17 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamilyByRegex(final byte [] row, final String familyRegex, 
+  public void deleteFamilyByRegex(final byte [] row, final String familyRegex,
     final long timestamp)
   throws IOException {
     deleteFamilyByRegex(row,familyRegex,timestamp,null);
   }
-  
+
   /**
    * Delete all cells for a row with matching column family regex with
    * timestamps less than or equal to <i>timestamp</i>, using existing
    * row lock.
-   * 
+   *
    * @param row The row to operate on
    * @param familyRegex Column Family Regex
    * @param timestamp Timestamp to match
@@ -1652,7 +1652,7 @@
 
   /**
    * Test for the existence of a row in the table.
-   * 
+   *
    * @param row The row
    * @return true if the row exists, false otherwise
    * @throws IOException
@@ -1664,7 +1664,7 @@
 
   /**
    * Test for the existence of a row and column in the table.
-   * 
+   *
    * @param row The row
    * @param column The column
    * @return true if the row exists, false otherwise
@@ -1678,7 +1678,7 @@
 
   /**
    * Test for the existence of a coordinate in the table.
-   * 
+   *
    * @param row The row
    * @param column The column
    * @param timestamp The timestamp
@@ -1693,7 +1693,7 @@
 
   /**
    * Test for the existence of a coordinate in the table.
-   * 
+   *
    * @param row The row
    * @param column The column
    * @param timestamp The timestamp
@@ -1717,12 +1717,12 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} or
    * {@link #put(Put)}
-   */ 
-  public synchronized void commit(final BatchUpdate batchUpdate) 
+   */
+  public synchronized void commit(final BatchUpdate batchUpdate)
   throws IOException {
     commit(batchUpdate, null);
   }
-  
+
   /**
    * Commit a BatchUpdate to the table using existing row lock.
    * If autoFlush is false, the update is buffered
@@ -1731,9 +1731,9 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} or
    * {@link #put(Put)}
-   */ 
+   */
   public synchronized void commit(final BatchUpdate batchUpdate,
-      final RowLock rl) 
+      final RowLock rl)
   throws IOException {
     for (BatchOperation bo: batchUpdate) {
       if (!bo.isPut()) throw new IOException("Only Puts in BU as of 0.20.0");
@@ -1750,7 +1750,7 @@
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} or
    * {@link #put(List)}
-   */ 
+   */
   public synchronized void commit(final List<BatchUpdate> batchUpdates)
       throws IOException {
     // Am I breaking something here in old API by doing this?
@@ -1758,11 +1758,11 @@
       commit(bu);
     }
   }
-  
+
   /**
-   * Atomically checks if a row's values match the expectedValues. 
+   * Atomically checks if a row's values match the expectedValues.
    * If it does, it uses the batchUpdate to update the row.<p>
-   * 
+   *
    * This operation is not currently supported, use {@link #checkAndPut}
    * @param batchUpdate batchupdate to apply if check is successful
    * @param expectedValues values to check
@@ -1795,11 +1795,11 @@
     private long lastNext;
     // Keep lastResult returned successfully in case we have to reset scanner.
     private Result lastResult = null;
-    
+
     protected ClientScanner(final Scan scan) {
       if (CLIENT_LOG.isDebugEnabled()) {
-        CLIENT_LOG.debug("Creating scanner over " 
-            + Bytes.toString(getTableName()) 
+        CLIENT_LOG.debug("Creating scanner over "
+            + Bytes.toString(getTableName())
             + " starting at key '" + Bytes.toStringBinary(scan.getStartRow()) + "'");
       }
       this.scan = scan;
@@ -1827,7 +1827,7 @@
     protected Scan getScan() {
       return scan;
     }
-    
+
     protected long getTimestamp() {
       return lastNext;
     }
@@ -1845,10 +1845,10 @@
         getConnection().getRegionServerWithRetries(callable);
         this.callable = null;
       }
-      
+
       // Where to start the next scanner
       byte [] localStartKey = null;
-      
+
       // if we're at the end of the table, then close and return false
       // to stop iterating
       if (this.currentRegion != null) {
@@ -1870,10 +1870,10 @@
       if (CLIENT_LOG.isDebugEnabled()) {
         CLIENT_LOG.debug("Advancing internal scanner to startKey at '" +
           Bytes.toStringBinary(localStartKey) + "'");
-      }            
+      }
       try {
         callable = getScannerCallable(localStartKey, nbRows);
-        // Open a scanner on the region server starting at the 
+        // Open a scanner on the region server starting at the
         // beginning of the region
         getConnection().getRegionServerWithRetries(callable);
         this.currentRegion = callable.getHRegionInfo();
@@ -1883,11 +1883,11 @@
       }
       return true;
     }
-    
+
     protected ScannerCallable getScannerCallable(byte [] localStartKey,
         int nbRows) {
       scan.setStartRow(localStartKey);
-      ScannerCallable s = new ScannerCallable(getConnection(), 
+      ScannerCallable s = new ScannerCallable(getConnection(),
         getTableName(), scan);
       s.setCaching(nbRows);
       return s;
@@ -1936,7 +1936,7 @@
       if (cache.size() == 0) {
         Result [] values = null;
         int countdown = this.caching;
-        // We need to reset it if it's a new callable that was created 
+        // We need to reset it if it's a new callable that was created
         // with a countdown in nextScanner
         callable.setCaching(this.caching);
         // This flag is set when we want to skip the result returned.  We do
@@ -2029,7 +2029,7 @@
       return new Iterator<Result>() {
         // The next RowResult, possibly pre-read
         Result next = null;
-        
+
         // return true if there is another item pending, false if there isn't.
         // this method is where the actual advancing takes place, but you need
         // to call next() to consume it. hasNext() will only advance if there
@@ -2054,7 +2054,7 @@
           if (!hasNext()) {
             return null;
           }
-          
+
           // if we get to here, then hasNext() has given us an item to return.
           // we want to return the item and then null out the next pointer, so
           // we use a temporary variable.
@@ -2075,7 +2075,7 @@
    */
   protected class OldClientScanner implements Scanner {
     private final ClientScanner cs;
- 
+
     OldClientScanner(final ClientScanner cs) {
       this.cs = cs;
     }
@@ -2111,7 +2111,7 @@
       return new Iterator<RowResult>() {
         // The next RowResult, possibly pre-read
         RowResult next = null;
-        
+
         // return true if there is another item pending, false if there isn't.
         // this method is where the actual advancing takes place, but you need
         // to call next() to consume it. hasNext() will only advance if there
@@ -2123,7 +2123,7 @@
               return next != null;
             } catch (IOException e) {
               throw new RuntimeException(e);
-            }            
+            }
           }
           return true;
         }
@@ -2136,7 +2136,7 @@
           if (!hasNext()) {
             return null;
           }
-          
+
           // if we get to here, then hasNext() has given us an item to return.
           // we want to return the item and then null out the next pointer, so
           // we use a temporary variable.
@@ -2151,7 +2151,7 @@
       };
     }
   }
-  
+
   private static byte [] stripColon(final byte [] n) {
     byte col = n[n.length-1];
     if (col == ':') {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTablePool.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTablePool.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTablePool.java Tue Aug 18 12:46:53 2009
@@ -19,7 +19,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -30,20 +29,21 @@
 
 /**
  * A simple pool of HTable instances.<p>
- * 
+ *
  * Each HTablePool acts as a pool for all tables.  To use, instantiate an
  * HTablePool and use {@link #getTable(String)} to get an HTable from the pool.
  * Once you are done with it, return it to the pool with {@link #putTable(HTable)}.<p>
- * 
+ *
  * A pool can be created with a <i>maxSize</i> which defines the most HTable
  * references that will ever be retained for each table.  Otherwise the default
  * is {@link Integer#MAX_VALUE}.<p>
  */
 public class HTablePool {
-  private final Map<String, LinkedList<HTable>> tables = 
-      Collections.synchronizedMap(new HashMap<String, LinkedList<HTable>>());
+  private final Map<String, LinkedList<HTableInterface>> tables =
+      Collections.synchronizedMap(new HashMap<String, LinkedList<HTableInterface>>());
   private final HBaseConfiguration config;
   private final int maxSize;
+  private HTableInterfaceFactory tableFactory = new HTableFactory();
 
   /**
    * Default Constructor.  Default HBaseConfiguration and no limit on pool size.
@@ -62,63 +62,65 @@
     this.maxSize = maxSize;
   }
 
+  public HTablePool(HBaseConfiguration config, int maxSize, HTableInterfaceFactory tableFactory) {
+    this.config = config;
+    this.maxSize = maxSize;
+    this.tableFactory = tableFactory;
+  }
+
   /**
    * Get a reference to the specified table from the pool.<p>
-   * 
+   *
    * Create a new one if one is not available.
    * @param tableName
    * @return a reference to the specified table
    * @throws RuntimeException if there is a problem instantiating the HTable
    */
-  public HTable getTable(String tableName) {
-    LinkedList<HTable> queue = tables.get(tableName);
+  public HTableInterface getTable(String tableName) {
+    LinkedList<HTableInterface> queue = tables.get(tableName);
     if(queue == null) {
-      queue = new LinkedList<HTable>();
+      queue = new LinkedList<HTableInterface>();
       tables.put(tableName, queue);
-      return newHTable(tableName);
+      return createHTable(tableName);
     }
-    HTable table;
+    HTableInterface table;
     synchronized(queue) {
       table = queue.poll();
     }
     if(table == null) {
-      return newHTable(tableName);
+      return createHTable(tableName);
     }
     return table;
   }
 
   /**
    * Get a reference to the specified table from the pool.<p>
-   * 
+   *
    * Create a new one if one is not available.
    * @param tableName
    * @return a reference to the specified table
    * @throws RuntimeException if there is a problem instantiating the HTable
    */
-  public HTable getTable(byte [] tableName) {
+  public HTableInterface getTable(byte [] tableName) {
     return getTable(Bytes.toString(tableName));
   }
 
   /**
    * Puts the specified HTable back into the pool.<p>
-   * 
+   *
    * If the pool already contains <i>maxSize</i> references to the table,
    * then nothing happens.
    * @param table
    */
-  public void putTable(HTable table) {
-    LinkedList<HTable> queue = tables.get(Bytes.toString(table.getTableName()));
+  public void putTable(HTableInterface table) {
+    LinkedList<HTableInterface> queue = tables.get(Bytes.toString(table.getTableName()));
     synchronized(queue) {
       if(queue.size() >= maxSize) return;
       queue.add(table);
     }
   }
 
-  private HTable newHTable(String tableName) {
-    try {
-      return new HTable(config, Bytes.toBytes(tableName));
-    } catch(IOException ioe) {
-      throw new RuntimeException(ioe);
-    }
+  private HTableInterface createHTable(String tableName) {
+    return this.tableFactory.createHTableInterface(config, Bytes.toBytes(tableName));
   }
 }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTablePool.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTablePool.java?rev=805392&r1=805391&r2=805392&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTablePool.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTablePool.java Tue Aug 18 12:46:53 2009
@@ -33,14 +33,14 @@
     String tableName = "testTable";
 
     // Request a table from an empty pool
-    HTable table = pool.getTable(tableName);
+    HTableInterface table = pool.getTable(tableName);
     assertNotNull(table);
-  
+
     // Return the table to the pool
     pool.putTable(table);
 
     // Request a table of the same name
-    HTable sameTable = pool.getTable(tableName);
+    HTableInterface sameTable = pool.getTable(tableName);
     assertSame(table, sameTable);
   }
 
@@ -49,36 +49,36 @@
     byte[] tableName = Bytes.toBytes("testTable");
 
     // Request a table from an empty pool
-    HTable table = pool.getTable(tableName);
+    HTableInterface table = pool.getTable(tableName);
     assertNotNull(table);
-  
+
     // Return the table to the pool
     pool.putTable(table);
 
     // Request a table of the same name
-    HTable sameTable = pool.getTable(tableName);
+    HTableInterface sameTable = pool.getTable(tableName);
     assertSame(table, sameTable);
   }
 
   public void testTableWithMaxSize() {
     HTablePool pool = new HTablePool((HBaseConfiguration)null, 2);
     String tableName = "testTable";
-    
+
     // Request tables from an empty pool
-    HTable table1 = pool.getTable(tableName);
-    HTable table2 = pool.getTable(tableName);
-    HTable table3 = pool.getTable(tableName);
-    
+    HTableInterface table1 = pool.getTable(tableName);
+    HTableInterface table2 = pool.getTable(tableName);
+    HTableInterface table3 = pool.getTable(tableName);
+
     // Return the tables to the pool
     pool.putTable(table1);
     pool.putTable(table2);
     // The pool should reject this one since it is already full
     pool.putTable(table3);
-    
+
     // Request tables of the same name
-    HTable sameTable1 = pool.getTable(tableName);
-    HTable sameTable2 = pool.getTable(tableName);
-    HTable sameTable3 = pool.getTable(tableName);
+    HTableInterface sameTable1 = pool.getTable(tableName);
+    HTableInterface sameTable2 = pool.getTable(tableName);
+    HTableInterface sameTable3 = pool.getTable(tableName);
     assertSame(table1, sameTable1);
     assertSame(table2, sameTable2);
     assertNotSame(table3, sameTable3);
@@ -90,17 +90,17 @@
     String tableName2 = "testTable2";
 
     // Request a table from an empty pool
-    HTable table1 = pool.getTable(tableName1);
-    HTable table2 = pool.getTable(tableName2);
+    HTableInterface table1 = pool.getTable(tableName1);
+    HTableInterface table2 = pool.getTable(tableName2);
     assertNotNull(table2);
-  
+
     // Return the tables to the pool
     pool.putTable(table1);
     pool.putTable(table2);
 
     // Request tables of the same names
-    HTable sameTable1 = pool.getTable(tableName1);
-    HTable sameTable2 = pool.getTable(tableName2);
+    HTableInterface sameTable1 = pool.getTable(tableName1);
+    HTableInterface sameTable2 = pool.getTable(tableName2);
     assertSame(table1, sameTable1);
     assertSame(table2, sameTable2);
   }