You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by br...@apache.org on 2008/02/15 01:29:08 UTC

svn commit: r627918 [2/3] - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/generated/master/ src/java/org/apache/hadoop/hbase/hql/ src/java/org/apache/hadoop/hbase/...

Added: 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=627918&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java Thu Feb 14 16:29:04 2008
@@ -0,0 +1,1032 @@
+/**
+ * Copyright 2007 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.client;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.filter.StopRowFilter;
+import org.apache.hadoop.hbase.filter.WhileMatchRowFilter;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.HbaseMapWritable;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HScannerInterface;
+import org.apache.hadoop.hbase.HRegionInterface;
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.RemoteExceptionHandler;
+import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.HRegionInterface;
+import org.apache.hadoop.hbase.HStoreKey;
+
+/**
+ * Used to communicate with a single HBase table
+ */
+public class HTable implements HConstants {
+  protected final Log LOG = LogFactory.getLog(this.getClass().getName());
+
+  protected final HConnection connection;
+  protected final Text tableName;
+  protected final long pause;
+  protected final int numRetries;
+  protected Random rand;
+  protected AtomicReference<BatchUpdate> batch;
+
+  protected volatile boolean tableDoesNotExist;
+  
+  // For row mutation operations
+  
+  /**
+   * Creates an object to access a HBase table
+   * 
+   * @param conf configuration object
+   * @param tableName name of the table
+   * @throws IOException
+   */
+  public HTable(HBaseConfiguration conf, Text tableName) throws IOException {
+    this.connection = HConnectionManager.getConnection(conf);
+    this.tableName = tableName;
+    this.pause = conf.getLong("hbase.client.pause", 10 * 1000);
+    this.numRetries = conf.getInt("hbase.client.retries.number", 5);
+    this.rand = new Random();
+    this.batch = new AtomicReference<BatchUpdate>();
+    this.connection.locateRegion(tableName, EMPTY_START_ROW);
+  }
+
+  /**
+   * Find region location hosting passed row using cached info
+   * @param row Row to find.
+   * @return Location of row.
+   */
+  public HRegionLocation getRegionLocation(Text row) throws IOException {
+    return this.connection.locateRegion(this.tableName, row);
+  }
+
+  /**
+   * Find region location hosting passed row
+   * @param row Row to find.
+   * @param reload If true do not use cache, otherwise bypass.
+   * @return Location of row.
+   */
+  HRegionLocation getRegionLocation(Text row, boolean reload) throws IOException {
+    return reload?
+      this.connection.relocateRegion(this.tableName, row):
+      this.connection.locateRegion(tableName, row);
+  }
+
+
+  /** @return the connection */
+  public HConnection getConnection() {
+    return connection;
+  }
+  
+  /**
+   * Verifies that no update is in progress
+   */
+  public synchronized void checkUpdateInProgress() {
+    updateInProgress(false);
+  }
+  
+  /*
+   * Checks to see if an update is in progress
+   * 
+   * @param updateMustBeInProgress
+   *    If true, an update must be in progress. An IllegalStateException will be
+   *    thrown if not.
+   *    
+   *    If false, an update must not be in progress. An IllegalStateException
+   *    will be thrown if an update is in progress.
+   */
+  private void updateInProgress(boolean updateMustBeInProgress) {
+    if (updateMustBeInProgress) {
+      if (batch.get() == null) {
+        throw new IllegalStateException("no update in progress");
+      }
+    } else {
+      if (batch.get() != null) {
+        throw new IllegalStateException("update in progress");
+      }
+    }
+  }
+  
+
+  /** @return the table name */
+  public Text getTableName() {
+    return this.tableName;
+  }
+
+  /**
+   * @return table metadata 
+   * @throws IOException
+   */
+  public HTableDescriptor getMetadata() throws IOException {
+    HTableDescriptor [] metas = this.connection.listTables();
+    HTableDescriptor result = null;
+    for (int i = 0; i < metas.length; i++) {
+      if (metas[i].getName().equals(this.tableName)) {
+        result = metas[i];
+        break;
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Gets the starting row key for every region in the currently open table
+   * @return Array of region starting row keys
+   * @throws IOException
+   */
+  @SuppressWarnings("null")
+  public Text[] getStartKeys() throws IOException {
+    List<Text> keyList = new ArrayList<Text>();
+
+    long scannerId = -1L;
+
+    Text startRow = new Text(tableName.toString() + ",,999999999999999");
+    HRegionLocation metaLocation = null;
+    HRegionInterface server;
+    
+    // scan over the each meta region
+    do {
+      try{
+        // turn the start row into a location
+        metaLocation = 
+          connection.locateRegion(META_TABLE_NAME, startRow);
+
+        // connect to the server hosting the .META. region
+        server = 
+          connection.getHRegionConnection(metaLocation.getServerAddress());
+
+        // open a scanner over the meta region
+        scannerId = server.openScanner(
+          metaLocation.getRegionInfo().getRegionName(),
+          COLUMN_FAMILY_ARRAY, tableName, LATEST_TIMESTAMP,
+          null);
+        
+        // iterate through the scanner, accumulating unique table names
+        SCANNER_LOOP: while (true) {
+          HbaseMapWritable values = server.next(scannerId);
+          if (values == null || values.size() == 0) {
+            break;
+          }
+          for (Map.Entry<Writable, Writable> e: values.entrySet()) {
+            HStoreKey key = (HStoreKey) e.getKey();
+            if (key.getColumn().equals(COL_REGIONINFO)) {
+              HRegionInfo info = new HRegionInfo();
+              info = (HRegionInfo) Writables.getWritable(
+                  ((ImmutableBytesWritable) e.getValue()).get(), info);
+
+              if (!info.getTableDesc().getName().equals(this.tableName)) {
+                break SCANNER_LOOP;
+              }
+
+              if (info.isOffline()) {
+                continue SCANNER_LOOP;
+              }
+
+              if (info.isSplit()) {
+                continue SCANNER_LOOP;
+              }
+
+              keyList.add(info.getStartKey());
+            }
+          }
+        }
+        
+        // close that remote scanner
+        server.close(scannerId);
+          
+        // advance the startRow to the end key of the current region
+        startRow = metaLocation.getRegionInfo().getEndKey();          
+      } catch (IOException e) {
+        // need retry logic?
+        throw e;
+      }
+    } while (startRow.compareTo(EMPTY_START_ROW) != 0);
+
+    Text[] arr = new Text[keyList.size()];
+    for (int i = 0; i < keyList.size(); i++ ){
+      arr[i] = keyList.get(i);
+    }
+    
+    return arr;
+  }
+  
+  /** 
+   * Get a single value for the specified row and column
+   *
+   * @param row row key
+   * @param column column name
+   * @return value for specified row/column
+   * @throws IOException
+   */
+   public byte[] get(Text row, final Text column) throws IOException {
+     return getRegionServerWithRetries(new ServerCallable<byte[]>(row){
+       public byte[] call() throws IOException {
+         return server.get(location.getRegionInfo().getRegionName(), row, column);
+       }
+     });
+   }
+ 
+  /** 
+   * Get the specified number of versions of the specified row and column
+   * 
+   * @param row         - row key
+   * @param column      - column name
+   * @param numVersions - number of versions to retrieve
+   * @return            - array byte values
+   * @throws IOException
+   */
+  public byte[][] get(final Text row, final Text column, final int numVersions) 
+  throws IOException {
+    byte [][] values = null;
+
+    values = getRegionServerWithRetries(new ServerCallable<byte[][]>(row) {
+      public byte [][] call() throws IOException {
+        return server.get(location.getRegionInfo().getRegionName(), row, 
+          column, numVersions);
+      }
+    });
+
+    if (values != null) {
+      ArrayList<byte[]> bytes = new ArrayList<byte[]>();
+      for (int i = 0 ; i < values.length; i++) {
+        bytes.add(values[i]);
+      }
+      return bytes.toArray(new byte[values.length][]);
+    }
+    return null;
+  }
+  
+  /** 
+   * Get the specified number of versions of the specified row and column with
+   * the specified timestamp.
+   *
+   * @param row         - row key
+   * @param column      - column name
+   * @param timestamp   - timestamp
+   * @param numVersions - number of versions to retrieve
+   * @return            - array of values that match the above criteria
+   * @throws IOException
+   */
+  public byte[][] get(final Text row, final Text column, final long timestamp, 
+    final int numVersions)
+  throws IOException {
+    byte [][] values = null;
+
+    values = getRegionServerWithRetries(new ServerCallable<byte[][]>(row) {
+      public byte [][] call() throws IOException {
+        return server.get(location.getRegionInfo().getRegionName(), row, 
+          column, timestamp, numVersions);
+      }
+    });
+
+    if (values != null) {
+      ArrayList<byte[]> bytes = new ArrayList<byte[]>();
+      for (int i = 0 ; i < values.length; i++) {
+        bytes.add(values[i]);
+      }
+      return bytes.toArray(new byte[values.length][]);
+    }
+    return null;
+  }
+    
+  /** 
+   * Get all the data for the specified row at the latest timestamp
+   * 
+   * @param row row key
+   * @return Map of columns to values.  Map is empty if row does not exist.
+   * @throws IOException
+   */
+  public SortedMap<Text, byte[]> getRow(Text row) throws IOException {
+    return getRow(row, HConstants.LATEST_TIMESTAMP);
+  }
+
+  /** 
+   * Get all the data for the specified row at a specified timestamp
+   * 
+   * @param row row key
+   * @param ts timestamp
+   * @return Map of columns to values.  Map is empty if row does not exist.
+   * @throws IOException
+   */
+  public SortedMap<Text, byte[]> getRow(final Text row, final long ts) 
+  throws IOException {
+    HbaseMapWritable value = null;
+         
+    value = getRegionServerWithRetries(new ServerCallable<HbaseMapWritable>(row) {
+      public HbaseMapWritable call() throws IOException {
+        return server.getRow(location.getRegionInfo().getRegionName(), row, ts);
+      }
+    });
+    
+    SortedMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
+    if (value != null && value.size() != 0) {
+      for (Map.Entry<Writable, Writable> e: value.entrySet()) {
+        HStoreKey key = (HStoreKey) e.getKey();
+        results.put(key.getColumn(),
+            ((ImmutableBytesWritable) e.getValue()).get());
+      }
+    }
+    return results;
+  }
+
+
+  /** 
+   * Get a scanner on the current table starting at the specified row.
+   * Return the specified columns.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
+   * @param startRow starting row in table to scan
+   * @return scanner
+   * @throws IOException
+   */
+  public HScannerInterface obtainScanner(Text[] columns, Text startRow)
+  throws IOException {
+    return obtainScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, null);
+  }
+  
+  /** 
+   * Get a scanner on the current table starting at the specified row.
+   * Return the specified columns.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
+   * @param startRow starting row in table to scan
+   * @param timestamp only return results whose timestamp <= this value
+   * @return scanner
+   * @throws IOException
+   */
+  public HScannerInterface obtainScanner(Text[] columns, Text startRow,
+    long timestamp)
+  throws IOException {
+    return obtainScanner(columns, startRow, timestamp, null);
+  }
+  
+  /** 
+   * Get a scanner on the current table starting at the specified row.
+   * Return the specified columns.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
+   * @param startRow starting row in table to scan
+   * @param filter a row filter using row-key regexp and/or column data filter.
+   * @return scanner
+   * @throws IOException
+   */
+  public HScannerInterface obtainScanner(Text[] columns, Text startRow,
+    RowFilterInterface filter)
+  throws IOException { 
+    return obtainScanner(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.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
+   * @param startRow starting row in table to scan
+   * @param stopRow Row to stop scanning on. Once we hit this row we stop
+   * returning values; i.e. we return the row before this one but not the
+   * <code>stopRow</code> itself.
+   * @return scanner
+   * @throws IOException
+   */
+  public HScannerInterface obtainScanner(final Text[] columns,
+    final Text startRow, final Text stopRow)
+  throws IOException {
+    return obtainScanner(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.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
+   * @param startRow starting row in table to scan
+   * @param stopRow Row to stop scanning on. Once we hit this row we stop
+   * returning values; i.e. we return the row before this one but not the
+   * <code>stopRow</code> itself.
+   * @param timestamp only return results whose timestamp <= this value
+   * @return scanner
+   * @throws IOException
+   */
+  public HScannerInterface obtainScanner(final Text[] columns,
+    final Text startRow, final Text stopRow, final long timestamp)
+  throws IOException {
+    return obtainScanner(columns, startRow, timestamp,
+      new WhileMatchRowFilter(new StopRowFilter(stopRow)));
+  }
+  
+  /** 
+   * Get a scanner on the current table starting at the specified row.
+   * Return the specified columns.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier. A column qualifier is judged to
+   * be a regex if it contains at least one of the following characters:
+   * <code>\+|^&*$[]]}{)(</code>.
+   * @param startRow starting row in table to scan
+   * @param timestamp only return results whose timestamp <= this value
+   * @param filter a row filter using row-key regexp and/or column data filter.
+   * @return scanner
+   * @throws IOException
+   */
+  public HScannerInterface obtainScanner(Text[] columns,
+    Text startRow, long timestamp, RowFilterInterface filter)
+  throws IOException {
+    return new ClientScanner(columns, startRow, timestamp, filter);
+  }
+
+  /** 
+   * Start an atomic row insertion/update.  No changes are committed until the 
+   * call to commit() returns. A call to abort() will abandon any updates in
+   * progress.
+   * 
+   * <p>
+   * Example:
+   * <br>
+   * <pre><span style="font-family: monospace;">
+   * long lockid = table.startUpdate(new Text(article.getName()));
+   * for (File articleInfo: article.listFiles(new NonDirectories())) {
+   *   String article = null;
+   *   try {
+   *     DataInputStream in = new DataInputStream(new FileInputStream(articleInfo));
+   *     article = in.readUTF();
+   *   } catch (IOException e) {
+   *     // Input error - abandon update
+   *     table.abort(lockid);
+   *     throw e;
+   *   }
+   *   try {
+   *     table.put(lockid, columnName(articleInfo.getName()), article.getBytes());
+   *   } catch (RuntimeException e) {
+   *     // Put failed - abandon update
+   *     table.abort(lockid);
+   *     throw e;
+   *   }
+   * }
+   * table.commit(lockid);
+   * </span></pre>
+   *
+   * 
+   * @param row Name of row to start update against.  Note, choose row names
+   * with care.  Rows are sorted lexicographically (comparison is done
+   * using {@link Text#compareTo(Object)}.  If your keys are numeric,
+   * lexicographic sorting means that 46 sorts AFTER 450 (If you want to use
+   * numerics for keys, zero-pad).
+   * @return Row lock id..
+   * @see #commit(long)
+   * @see #commit(long, long)
+   * @see #abort(long)
+   */
+  @Deprecated
+  public synchronized long startUpdate(final Text row) {
+    updateInProgress(false);
+    batch.set(new BatchUpdate(row));
+    return 1;
+  }
+  
+  /** 
+   * Update a value for the specified column.
+   * Runs {@link #abort(long)} if exception thrown.
+   *
+   * @param lockid lock id returned from startUpdate
+   * @param column column whose value is being set
+   * @param val new value for column.  Cannot be null.
+   */
+  @Deprecated
+  public void put(long lockid, Text column, byte val[]) {
+    if (lockid != 1) {
+      throw new IllegalArgumentException("Invalid lock id!");
+    }
+    if (val == null) {
+      throw new IllegalArgumentException("value cannot be null");
+    }
+    updateInProgress(true);
+    batch.get().put(column, val);
+  }
+  
+  /** 
+   * Update a value for the specified column.
+   * Runs {@link #abort(long)} if exception thrown.
+   *
+   * @param lockid lock id returned from startUpdate
+   * @param column column whose value is being set
+   * @param val new value for column.  Cannot be null.
+   * @throws IOException throws this if the writable can't be
+   * converted into a byte array 
+   */
+  @Deprecated
+  public void put(long lockid, Text column, Writable val) throws IOException {    
+    put(lockid, column, Writables.getBytes(val));
+  }
+  
+  /** 
+   * Delete the value for a column.
+   * Deletes the cell whose row/column/commit-timestamp match those of the
+   * delete.
+   * @param lockid lock id returned from startUpdate
+   * @param column name of column whose value is to be deleted
+   */
+  @Deprecated
+  public void delete(long lockid, Text column) {
+    if (lockid != 1) {
+      throw new IllegalArgumentException("Invalid lock id!");
+    }
+    updateInProgress(true);
+    batch.get().delete(column);
+  }
+  
+  /** 
+   * 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 
+   */
+  public void deleteAll(final Text row, final Text column) throws IOException {
+    deleteAll(row, column, 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 
+   */
+  public void deleteAll(final Text row, final Text column, final long ts)
+  throws IOException {
+    getRegionServerWithRetries(new ServerCallable<Boolean>(row) {
+      public Boolean call() throws IOException {
+        server.deleteAll(location.getRegionInfo().getRegionName(), row, 
+          column, ts);
+        return null;
+      }
+    });
+  }
+  
+  /**
+   * Completely delete the row's cells of the same timestamp or older.
+   *
+   * @param row Key of the row you want to completely delete.
+   * @param ts Timestamp of cells to delete
+   * @throws IOException
+   */
+  public void deleteAll(final Text row, final long ts) throws IOException {
+    getRegionServerWithRetries(new ServerCallable<Boolean>(row){
+      public Boolean call() throws IOException {
+        server.deleteAll(location.getRegionInfo().getRegionName(), row, ts);
+        return null;
+      }
+    });
+  }
+      
+  /**
+   * Completely delete the row's cells.
+   *
+   * @param row Key of the row you want to completely delete.
+   * @throws IOException
+   */
+  public void deleteAll(final Text row) throws IOException {
+    deleteAll(row, HConstants.LATEST_TIMESTAMP);
+  }
+  
+  /**
+   * Delete all cells for a row with matching column family with timestamps
+   * less than or equal to <i>timestamp</i>.
+   *
+   * @param row The row to operate on
+   * @param family The column family to match
+   * @param timestamp Timestamp to match
+   * @throws IOException
+   */
+  public void deleteFamily(final Text row, final Text family, 
+    final long timestamp)
+  throws IOException {
+    getRegionServerWithRetries(new ServerCallable<Boolean>(row){
+      public Boolean call() throws IOException {
+        server.deleteFamily(location.getRegionInfo().getRegionName(), row, 
+          family, timestamp);
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Delete all cells for a row with matching column family at all timestamps.
+   *
+   * @param row The row to operate on
+   * @param family The column family to match
+   * @throws IOException
+   */  
+  public void deleteFamily(final Text row, final Text family) throws IOException{
+    deleteFamily(row, family, HConstants.LATEST_TIMESTAMP);
+  }
+  
+  /** 
+   * Abort a row mutation.
+   * 
+   * This method should be called only when an update has been started and it
+   * is determined that the update should not be committed.
+   * 
+   * Releases resources being held by the update in progress.
+   *
+   * @param lockid lock id returned from startUpdate
+   */
+  @Deprecated
+  public synchronized void abort(long lockid) {
+    if (lockid != 1) {
+      throw new IllegalArgumentException("Invalid lock id!");
+    }
+    batch.set(null);
+  }
+  
+  /** 
+   * Finalize a row mutation.
+   * 
+   * When this method is specified, we pass the server a value that says use
+   * the 'latest' timestamp.  If we are doing a put, on the server-side, cells
+   * will be given the servers's current timestamp.  If the we are commiting
+   * deletes, then delete removes the most recently modified cell of stipulated
+   * column.
+   * 
+   * @see #commit(long, long)
+   * 
+   * @param lockid lock id returned from startUpdate
+   * @throws IOException
+   */
+  @Deprecated
+  public void commit(long lockid) throws IOException {
+    commit(lockid, LATEST_TIMESTAMP);
+  }
+
+  /** 
+   * Finalize a row mutation and release any resources associated with the update.
+   * 
+   * @param lockid lock id returned from startUpdate
+   * @param timestamp time to associate with the change
+   * @throws IOException
+   */
+  @Deprecated
+  public void commit(long lockid, final long timestamp)
+  throws IOException {
+    updateInProgress(true);
+    if (lockid != 1) {
+      throw new IllegalArgumentException("Invalid lock id!");
+    }
+    try {
+      batch.get().setTimestamp(timestamp);
+      commit(batch.get());
+    } finally {
+      batch.set(null);
+    }
+  }
+  
+  /**
+   * Commit a BatchUpdate to the table.
+   */ 
+  public synchronized void commit(final BatchUpdate batchUpdate) 
+  throws IOException {
+    getRegionServerWithRetries(
+      new ServerCallable<Boolean>(batchUpdate.getRow()){
+        public Boolean call() throws IOException {
+          server.batchUpdate(location.getRegionInfo().getRegionName(), 
+            batchUpdate);
+          return null;
+        }
+      }
+    );  
+  }
+  
+  /**
+   * Implements the scanner interface for the HBase client.
+   * If there are multiple regions in a table, this scanner will iterate
+   * through them all.
+   */
+  protected class ClientScanner implements HScannerInterface {
+    private final Text EMPTY_COLUMN = new Text();
+    private Text[] columns;
+    private Text startRow;
+    private long scanTime;
+    @SuppressWarnings("hiding")
+    private boolean closed;
+    private HRegionLocation currentRegionLocation;
+    private HRegionInterface server;
+    private long scannerId;
+    private RowFilterInterface filter;
+    
+    protected ClientScanner(Text[] columns, Text startRow, long timestamp,
+      RowFilterInterface filter) 
+    throws IOException {
+
+      LOG.info("Creating scanner over " + tableName + " starting at key " + startRow);
+
+      // defaults
+      this.closed = false;
+      this.server = null;
+      this.scannerId = -1L;
+    
+      // save off the simple parameters
+      this.columns = columns;
+      this.startRow = startRow;
+      this.scanTime = timestamp;
+      
+      // save the filter, and make sure that the filter applies to the data
+      // we're expecting to pull back
+      this.filter = filter;
+      if (filter != null) {
+        filter.validate(columns);
+      }
+
+      nextScanner();
+    }
+        
+    /*
+     * Gets a scanner for the next region.
+     * Returns false if there are no more scanners.
+     */
+    private boolean nextScanner() throws IOException {
+      // close the previous scanner if it's open
+      if (this.scannerId != -1L) {
+        this.server.close(this.scannerId);
+        this.scannerId = -1L;
+      }
+
+      // if we're at the end of the table, then close and return false
+      // to stop iterating
+      if (this.currentRegionLocation != null){
+        LOG.debug("Advancing forward from region " 
+          + this.currentRegionLocation.getRegionInfo());
+        
+        if (this.currentRegionLocation.getRegionInfo().getEndKey() == null
+          || this.currentRegionLocation.getRegionInfo().getEndKey().equals(EMPTY_TEXT)) {
+            LOG.debug("We're at the end of the region, returning.");
+            close();
+            return false;
+        }
+      } 
+      
+      HRegionLocation oldLocation = this.currentRegionLocation;
+      
+      Text localStartKey = oldLocation == null ? 
+        startRow : oldLocation.getRegionInfo().getEndKey();
+
+      // advance to the region that starts with the current region's end key
+      LOG.debug("Advancing internal scanner to startKey " + localStartKey);
+      this.currentRegionLocation = getRegionLocation(localStartKey);
+      
+      LOG.debug("New region: " + this.currentRegionLocation);
+      
+      try {
+        for (int tries = 0; tries < numRetries; tries++) {
+          // connect to the server
+          server = connection.getHRegionConnection(
+            this.currentRegionLocation.getServerAddress());
+          
+          try {
+            // open a scanner on the region server starting at the 
+            // beginning of the region
+            scannerId = server.openScanner(
+              this.currentRegionLocation.getRegionInfo().getRegionName(),
+              this.columns, localStartKey, scanTime, filter);
+              
+            break;
+          } catch (IOException e) {
+            if (e instanceof RemoteException) {
+              e = RemoteExceptionHandler.decodeRemoteException(
+                  (RemoteException) e);
+            }
+            if (tries == numRetries - 1) {
+              // No more tries
+              throw e;
+            }
+            try {
+              Thread.sleep(pause);
+            } catch (InterruptedException ie) {
+              // continue
+            }
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("reloading table servers because: " + e.getMessage());
+            }
+            currentRegionLocation = getRegionLocation(localStartKey, true);
+          }
+        }
+      } catch (IOException e) {
+        close();
+        if (e instanceof RemoteException) {
+          e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
+        }
+        throw e;
+      }
+      return true;
+    }
+
+    /** {@inheritDoc} */
+    public boolean next(HStoreKey key, SortedMap<Text, byte[]> results)
+    throws IOException {
+      if (this.closed) {
+        return false;
+      }
+      HbaseMapWritable values = null;
+      // Clear the results so we don't inherit any values from any previous
+      // calls to next.
+      results.clear();
+      do {
+        values = server.next(scannerId);
+      } while (values != null && values.size() == 0 && nextScanner());
+
+      if (values != null && values.size() != 0) {
+        for (Map.Entry<Writable, Writable> e: values.entrySet()) {
+          HStoreKey k = (HStoreKey) e.getKey();
+          key.setRow(k.getRow());
+          key.setVersion(k.getTimestamp());
+          key.setColumn(EMPTY_COLUMN);
+          results.put(k.getColumn(),
+            ((ImmutableBytesWritable) e.getValue()).get());
+        }
+      }
+      return values == null ? false : values.size() != 0;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public void close() throws IOException {
+      if (scannerId != -1L) {
+        try {
+          server.close(scannerId);
+          
+        } catch (IOException e) {
+          if (e instanceof RemoteException) {
+            e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
+          }
+          if (!(e instanceof NotServingRegionException)) {
+            throw e;
+          }
+        }
+        scannerId = -1L;
+      }
+      server = null;
+      closed = true;
+    }
+
+    /** {@inheritDoc} */
+    public Iterator<Entry<HStoreKey, SortedMap<Text, byte[]>>> iterator() {
+      return new Iterator<Entry<HStoreKey, SortedMap<Text, byte[]>>>() {
+        HStoreKey key = null;
+        SortedMap<Text, byte []> value = null;
+        
+        public boolean hasNext() {
+          boolean hasNext = false;
+          try {
+            this.key = new HStoreKey();
+            this.value = new TreeMap<Text, byte[]>();
+            hasNext = ClientScanner.this.next(key, value);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+          return hasNext;
+        }
+
+        public Entry<HStoreKey, SortedMap<Text, byte[]>> next() {
+          return new Map.Entry<HStoreKey, SortedMap<Text, byte[]>>() {
+            public HStoreKey getKey() {
+              return key;
+            }
+
+            public SortedMap<Text, byte[]> getValue() {
+              return value;
+            }
+
+            public SortedMap<Text, byte[]> setValue(@SuppressWarnings("unused")
+            SortedMap<Text, byte[]> value) {
+              throw new UnsupportedOperationException();
+            }
+          };
+        }
+
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+  }
+  
+  /**
+   * Inherits from Callable, used to define the particular actions you would
+   * like to take with retry logic.
+   */
+  protected abstract class ServerCallable<T> implements Callable<T> {
+    HRegionLocation location;
+    HRegionInterface server;
+    Text row;
+  
+    protected ServerCallable(Text row) {
+      this.row = row;
+    }
+  
+    void instantiateServer(boolean reload) throws IOException {
+      this.location = getRegionLocation(row, reload);
+      this.server = connection.getHRegionConnection(location.getServerAddress());
+    }    
+  }
+  
+  /**
+   * Pass in a ServerCallable with your particular bit of logic defined and 
+   * this method will manage the process of doing retries with timed waits 
+   * and refinds of missing regions.
+   */
+  protected <T> T getRegionServerWithRetries(ServerCallable<T> callable) 
+  throws IOException, RuntimeException {
+    for(int tries = 0; tries < numRetries; tries++) {
+      try {
+        callable.instantiateServer(tries != 0);
+        return callable.call();
+      } catch (IOException e) {
+        if (e instanceof RemoteException) {
+          e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
+        }
+        if (tries == numRetries - 1) {
+          throw e;
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("reloading table servers because: " + e.getMessage());
+        }
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+      try {
+        Thread.sleep(pause);
+      } catch (InterruptedException e) {
+        // continue
+      }
+    }
+    return null;    
+  }
+  
+  @Deprecated
+  public void close() {
+    // do nothing...
+  }
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/generated/master/master_jsp.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/generated/master/master_jsp.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/generated/master/master_jsp.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/generated/master/master_jsp.java Thu Feb 14 16:29:04 2008
@@ -9,10 +9,9 @@
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.master.MetaRegion;
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.HServerAddress;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.hql.ShowCommand;
 import org.apache.hadoop.hbase.hql.TableFormatter;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/AlterCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/AlterCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/AlterCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/AlterCommand.java Thu Feb 14 16:29:04 2008
@@ -25,12 +25,12 @@
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.io.Text;
 
 import org.apache.hadoop.hbase.BloomFilterDescriptor;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/CommandFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/CommandFactory.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/CommandFactory.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/CommandFactory.java Thu Feb 14 16:29:04 2008
@@ -24,4 +24,4 @@
  */
 public interface CommandFactory {
   Command getCommand();
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/CreateCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/CreateCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/CreateCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/CreateCommand.java Thu Feb 14 16:29:04 2008
@@ -24,11 +24,11 @@
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.io.Text;
 
@@ -45,12 +45,11 @@
 
   public ReturnMsg execute(HBaseConfiguration conf) {
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
-      if (conn.tableExists(tableName)) {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      if (admin.tableExists(tableName)) {
         return new ReturnMsg(0, "'" + tableName + "' table already exist.");
       }
 
-      HBaseAdmin admin = new HBaseAdmin(conf);
       HTableDescriptor tableDesc = new HTableDescriptor(tableName.toString());
       HColumnDescriptor columnDesc = null;
       Set<String> columns = columnSpecMap.keySet();

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DeleteCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DeleteCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DeleteCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DeleteCommand.java Thu Feb 14 16:29:04 2008
@@ -24,12 +24,10 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
-import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
 
 /**
  * Deletes values from tables.
@@ -48,12 +46,12 @@
       throw new IllegalArgumentException("Column list is null");
     }
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
-      if (!conn.tableExists(tableName)) {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+
+      if (!admin.tableExists(tableName)) {
         return new ReturnMsg(0, "'" + tableName + "'" + TABLE_NOT_FOUND);
       }
 
-      HBaseAdmin admin = new HBaseAdmin(conf);
       HTable hTable = new HTable(conf, tableName);
 
       if (rowKey != null) {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DescCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DescCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DescCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DescCommand.java Thu Feb 14 16:29:04 2008
@@ -24,8 +24,7 @@
 
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.io.Text;
 
@@ -52,11 +51,11 @@
     if (tableName == null)
       return new ReturnMsg(0, "Syntax error : Please check 'Describe' syntax.");
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
-      if (!conn.tableExists(tableName)) {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      if (!admin.tableExists(tableName)) {
         return new ReturnMsg(0, "Table not found.");
       }
-      HTableDescriptor[] tables = conn.listTables();
+      HTableDescriptor[] tables = admin.listTables();
       HColumnDescriptor[] columns = null;
       for (int i = 0; i < tables.length; i++) {
         if (tables[i].getName().equals(tableName)) {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DisableCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DisableCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DisableCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DisableCommand.java Thu Feb 14 16:29:04 2008
@@ -22,10 +22,8 @@
 import java.io.IOException;
 import java.io.Writer;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -42,12 +40,12 @@
     assert tableName != null;
 
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
-      if (!conn.tableExists(new Text(tableName))) {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+
+      if (!admin.tableExists(new Text(tableName))) {
         return new ReturnMsg(0, "'" + tableName + "'" + TABLE_NOT_FOUND);
       }
 
-      HBaseAdmin admin = new HBaseAdmin(conf);
       admin.disableTable(new Text(tableName));
 
       return new ReturnMsg(1, "Table disabled successfully.");

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DropCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DropCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DropCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/DropCommand.java Thu Feb 14 16:29:04 2008
@@ -23,10 +23,8 @@
 import java.io.Writer;
 import java.util.List;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -46,11 +44,10 @@
 
     try {
       HBaseAdmin admin = new HBaseAdmin(conf);
-      HConnection conn = HConnectionManager.getConnection(conf);
 
       int count = 0;
       for (String table : tableList) {
-        if (!conn.tableExists(new Text(table))) {
+        if (!admin.tableExists(new Text(table))) {
           println("'" + table + "' table not found.");
         } else {
           println("Dropping " + table + "... Please wait.");

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/EnableCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/EnableCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/EnableCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/EnableCommand.java Thu Feb 14 16:29:04 2008
@@ -22,10 +22,8 @@
 import java.io.IOException;
 import java.io.Writer;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -41,12 +39,10 @@
   public ReturnMsg execute(HBaseConfiguration conf) {
     assert tableName != null;
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
-      if (!conn.tableExists(new Text(tableName))) {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      if (!admin.tableExists(new Text(tableName))) {
         return new ReturnMsg(0, "'" + tableName + "'" + TABLE_NOT_FOUND);
       }
-
-      HBaseAdmin admin = new HBaseAdmin(conf);
       admin.enableTable(new Text(tableName));
       return new ReturnMsg(1, "Table enabled successfully.");
     } catch (IOException e) {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/InsertCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/InsertCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/InsertCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/InsertCommand.java Thu Feb 14 16:29:04 2008
@@ -24,10 +24,10 @@
 import java.util.List;
 
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
-import org.apache.hadoop.hbase.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.MasterNotRunningException;
 
 /**
  * Inserts values into tables.
@@ -46,38 +46,42 @@
   public ReturnMsg execute(HBaseConfiguration conf) {
     if (tableName == null || values == null || rowKey == null)
       return new ReturnMsg(0, "Syntax error : Please check 'Insert' syntax.");
-
-    HConnection conn = HConnectionManager.getConnection(conf);
-    if (!conn.tableExists(tableName)) {
-      return new ReturnMsg(0, "'" + tableName + "'" + TABLE_NOT_FOUND);
-    }
-
-    if (columnfamilies.size() != values.size())
-      return new ReturnMsg(0,
-          "Mismatch between values list and columnfamilies list.");
-
+    
     try {
-      HTable table = new HTable(conf, tableName);
-      long lockId = table.startUpdate(getRow());
-
-      for (int i = 0; i < values.size(); i++) {
-        Text column = null;
-        if (getColumn(i).toString().contains(":"))
-          column = getColumn(i);
-        else
-          column = new Text(getColumn(i) + ":");
-        table.put(lockId, column, getValue(i));
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      if (!admin.tableExists(tableName)) {
+        return new ReturnMsg(0, "'" + tableName + "'" + TABLE_NOT_FOUND);
       }
       
-      if(timestamp != null) 
-        table.commit(lockId, Long.parseLong(timestamp));
-      else
-        table.commit(lockId);
-
-      return new ReturnMsg(1, "1 row inserted successfully.");
-    } catch (IOException e) {
-      String[] msg = e.getMessage().split("[\n]");
-      return new ReturnMsg(0, msg[0]);
+      if (columnfamilies.size() != values.size())
+        return new ReturnMsg(0,
+            "Mismatch between values list and columnfamilies list.");
+
+      try {
+        HTable table = new HTable(conf, tableName);
+        long lockId = table.startUpdate(getRow());
+
+        for (int i = 0; i < values.size(); i++) {
+          Text column = null;
+          if (getColumn(i).toString().contains(":"))
+            column = getColumn(i);
+          else
+            column = new Text(getColumn(i) + ":");
+          table.put(lockId, column, getValue(i));
+        }
+
+        if(timestamp != null) 
+          table.commit(lockId, Long.parseLong(timestamp));
+        else
+          table.commit(lockId);
+
+        return new ReturnMsg(1, "1 row inserted successfully.");
+      } catch (IOException e) {
+        String[] msg = e.getMessage().split("[\n]");
+        return new ReturnMsg(0, msg[0]);
+      } 
+    } catch (MasterNotRunningException e) {
+      return new ReturnMsg(0, "Master is not running!");
     }
   }
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/SelectCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/SelectCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/SelectCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/SelectCommand.java Thu Feb 14 16:29:04 2008
@@ -28,14 +28,10 @@
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HScannerInterface;
 import org.apache.hadoop.hbase.HStoreKey;
-import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Shell;
 import org.apache.hadoop.hbase.filter.RowFilterInterface;
@@ -44,6 +40,8 @@
 import org.apache.hadoop.hbase.hql.generated.HQLParser;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /**
  * Selects values from tables.
@@ -82,13 +80,12 @@
       return new ReturnMsg(0, "Syntax error : Please check 'Select' syntax.");
     }
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
-      if (!conn.tableExists(tableName) && !isMetaTable()) {
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      if (!admin.tableExists(tableName) && !isMetaTable()) {
         return new ReturnMsg(0, "'" + tableName + "'" + TABLE_NOT_FOUND);
       }
 
       HTable table = new HTable(conf, tableName);
-      HBaseAdmin admin = new HBaseAdmin(conf);
       int count = 0;
       if (whereClause) {
         if (countFunction) {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/ShowCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/ShowCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/ShowCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/ShowCommand.java Thu Feb 14 16:29:04 2008
@@ -22,7 +22,7 @@
 import java.io.IOException;
 import java.io.Writer;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/TruncateCommand.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/TruncateCommand.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/TruncateCommand.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/TruncateCommand.java Thu Feb 14 16:29:04 2008
@@ -22,11 +22,9 @@
 import java.io.IOException;
 import java.io.Writer;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.io.Text;
 
@@ -45,14 +43,12 @@
       return new ReturnMsg(0, "Syntax error : Please check 'Truncate' syntax.");
 
     try {
-      HConnection conn = HConnectionManager.getConnection(conf);
       HBaseAdmin admin = new HBaseAdmin(conf);
-
-      if (!conn.tableExists(tableName)) {
+      if (!admin.tableExists(tableName)) {
         return new ReturnMsg(0, "Table not found.");
       }
 
-      HTableDescriptor[] tables = conn.listTables();
+      HTableDescriptor[] tables = admin.listTables();
       HColumnDescriptor[] columns = null;
       for (int i = 0; i < tables.length; i++) {
         if (tables[i].getName().equals(tableName)) {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/formatter/AsciiTableFormatter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/formatter/AsciiTableFormatter.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/formatter/AsciiTableFormatter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/hql/formatter/AsciiTableFormatter.java Thu Feb 14 16:29:04 2008
@@ -165,4 +165,4 @@
   public void setNoFormatting(boolean noFormatting) {
     this.noFormatting = noFormatting;
   }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/LuceneDocumentWrapper.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/LuceneDocumentWrapper.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/LuceneDocumentWrapper.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/LuceneDocumentWrapper.java Thu Feb 14 16:29:04 2008
@@ -48,4 +48,4 @@
   public void write(DataOutput out) throws IOException {
     // intentionally left blank
   }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java Thu Feb 14 16:29:04 2008
@@ -37,10 +37,10 @@
 import org.apache.hadoop.mapred.Reporter;
 
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.hbase.HScannerInterface;
 import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.client.HTable;
 
 import org.apache.log4j.Logger;
 
@@ -207,4 +207,4 @@
       throw new IOException("expecting at least one column");
     }
   }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java Thu Feb 14 16:29:04 2008
@@ -35,8 +35,8 @@
 import org.apache.hadoop.util.Progressable;
 
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.client.HTable;
 
 import org.apache.log4j.Logger;
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/HMaster.java Thu Feb 14 16:29:04 2008
@@ -62,8 +62,8 @@
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.Leases;
 import org.apache.hadoop.hbase.HServerAddress;
-import org.apache.hadoop.hbase.HConnection;
-import org.apache.hadoop.hbase.HConnectionManager;
+import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HRegionInterface;
@@ -72,7 +72,7 @@
 import org.apache.hadoop.hbase.HRegion;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.HMsg;
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.HServerInfo;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/Dispatcher.java Thu Feb 14 16:29:04 2008
@@ -28,7 +28,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.util.InfoServer;
 import org.apache.hadoop.mapred.StatusHttpServer;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/GenericHandler.java Thu Feb 14 16:29:04 2008
@@ -29,16 +29,15 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.io.Text;
 import org.mortbay.servlet.MultiPartResponse;
 import org.znerd.xmlenc.LineBreak;
 import org.znerd.xmlenc.XMLOutputter;
 
-
+import org.apache.hadoop.hbase.client.HTable;
 /**
  * GenericHandler contains some basic common stuff that all the individual
  * REST handler types take advantage of.
@@ -259,4 +258,4 @@
   protected HTable getTable(final String tableName) throws IOException {
     return new HTable(this.conf, new Text(tableName));
   }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/MetaHandler.java Thu Feb 14 16:29:04 2008
@@ -26,7 +26,7 @@
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.znerd.xmlenc.XMLOutputter;
@@ -104,4 +104,4 @@
         doNotAcceptable(response);
     }
   }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/ScannerHandler.java Thu Feb 14 16:29:04 2008
@@ -30,8 +30,8 @@
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
-import org.apache.hadoop.hbase.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HScannerInterface;
@@ -326,4 +326,4 @@
       response.getOutputStream().close();
     }
   }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/rest/TableHandler.java Thu Feb 14 16:29:04 2008
@@ -33,13 +33,14 @@
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
+
 import org.mortbay.servlet.MultiPartResponse;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
@@ -488,4 +489,4 @@
       response.setStatus(202);
     }
   } 
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java Thu Feb 14 16:29:04 2008
@@ -30,13 +30,12 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HScannerInterface;
 import org.apache.hadoop.hbase.HStoreKey;
-import org.apache.hadoop.hbase.HTable;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
@@ -49,6 +48,7 @@
 import org.apache.hadoop.hbase.thrift.generated.RegionDescriptor;
 import org.apache.hadoop.hbase.thrift.generated.ScanEntry;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
 
 import com.facebook.thrift.TException;
 import com.facebook.thrift.protocol.TBinaryProtocol;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Migrate.java Thu Feb 14 16:29:04 2008
@@ -51,7 +51,7 @@
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HLog;

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DFSAbort.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DFSAbort.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DFSAbort.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DFSAbort.java Thu Feb 14 16:29:04 2008
@@ -21,6 +21,7 @@
 
 import junit.framework.TestSuite;
 import junit.textui.TestRunner;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /**
  * Test ability of HBase to handle DFS failure
@@ -70,4 +71,4 @@
   public static void main(@SuppressWarnings("unused") String[] args) {
     TestRunner.run(new TestSuite(DFSAbort.class));
   }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java Thu Feb 14 16:29:04 2008
@@ -43,6 +43,8 @@
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /**
  * Additional scanner tests.

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseClusterTestCase.java Thu Feb 14 16:29:04 2008
@@ -24,6 +24,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.hbase.client.HConnectionManager;
 
 /**
  * Abstract base class for HBase cluster junit tests.  Spins up an hbase

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseTestCase.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseTestCase.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseTestCase.java Thu Feb 14 16:29:04 2008
@@ -32,6 +32,7 @@
 import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
 
 /**
  * Abstract base class for test cases. Performs all static initialization

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/MultiRegionTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/MultiRegionTable.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/MultiRegionTable.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/MultiRegionTable.java Thu Feb 14 16:29:04 2008
@@ -31,6 +31,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
 
 /**
  * Utility class to build a table of multiple regions.

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/PerformanceEvaluation.java Thu Feb 14 16:29:04 2008
@@ -46,6 +46,8 @@
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.log4j.Logger;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /**
  * Script used evaluating HBase performance and scalability.  Runs a HBase

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestBloomFilters.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestBloomFilters.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestBloomFilters.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestBloomFilters.java Thu Feb 14 16:29:04 2008
@@ -23,6 +23,8 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /** Tests per-column bloom filters */
 public class TestBloomFilters extends HBaseClusterTestCase {

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java Thu Feb 14 16:29:04 2008
@@ -25,6 +25,8 @@
 import java.util.TreeMap;
 
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 
 /**
  * Test HBase Master and Region servers, client API 

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestInfoServers.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestInfoServers.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestInfoServers.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestInfoServers.java Thu Feb 14 16:29:04 2008
@@ -26,6 +26,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /**
  * Testing, info servers are disabled.  This test enables then and checks that

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java Thu Feb 14 16:29:04 2008
@@ -26,6 +26,8 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /**
  * Test log deletion as logs are rolled.

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java Thu Feb 14 16:29:04 2008
@@ -22,6 +22,8 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 
 /** tests administrative functions */
 public class TestMasterAdmin extends HBaseClusterTestCase {
@@ -85,4 +87,4 @@
     admin.deleteColumn(testDesc.getName(), new Text("col2:"));
     admin.deleteTable(testDesc.getName());
   }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionServerExit.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionServerExit.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionServerExit.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionServerExit.java Thu Feb 14 16:29:04 2008
@@ -29,6 +29,8 @@
 import org.apache.hadoop.io.Text;
 
 import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /**
  * Tests region server failover when a region server exits both cleanly and

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java Thu Feb 14 16:29:04 2008
@@ -28,6 +28,8 @@
 import java.util.TreeMap;
 
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 
 /** test the scanner API at all levels */
 public class TestScannerAPI extends HBaseClusterTestCase {

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTable.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTable.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTable.java Thu Feb 14 16:29:04 2008
@@ -22,6 +22,8 @@
 
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
 
 /** Tests table creation restrictions*/
 public class TestTable extends HBaseClusterTestCase {

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTimestamp.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTimestamp.java?rev=627918&r1=627917&r2=627918&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTimestamp.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTimestamp.java Thu Feb 14 16:29:04 2008
@@ -26,6 +26,8 @@
 import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 /**
  * Tests user specifiable time stamps putting, getting and scanning.  Also

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java?rev=627918&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java Thu Feb 14 16:29:04 2008
@@ -0,0 +1,110 @@
+/**
+ * Copyright 2007 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.client;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.Map;
+import java.util.TreeMap;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.HBaseClusterTestCase;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HScannerInterface;
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+
+/**
+ * Test batch updates
+ */
+public class TestBatchUpdate extends HBaseClusterTestCase {
+  private static final String CONTENTS_STR = "contents:";
+  private static final Text CONTENTS = new Text(CONTENTS_STR);
+  private byte[] value;
+
+  private HTableDescriptor desc = null;
+  private HTable table = null;
+
+  /**
+   * @throws UnsupportedEncodingException
+   */
+  public TestBatchUpdate() throws UnsupportedEncodingException {
+    super();
+    value = "abcd".getBytes(HConstants.UTF8_ENCODING);
+  }
+  
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    this.desc = new HTableDescriptor("test");
+    desc.addFamily(new HColumnDescriptor(CONTENTS_STR));
+    HBaseAdmin admin = new HBaseAdmin(conf);
+    admin.createTable(desc);
+    table = new HTable(conf, desc.getName());
+  }
+
+  /**
+   * @throws IOException
+   */
+  public void testBatchUpdate() throws IOException {
+    try {
+      table.commit(-1L);
+      
+    } catch (IllegalStateException e) {
+      // expected
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+
+    long lockid = table.startUpdate(new Text("row1"));
+    
+    try {
+      @SuppressWarnings("unused")
+      long dummy = table.startUpdate(new Text("row2"));
+    } catch (IllegalStateException e) {
+      // expected
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+    table.put(lockid, CONTENTS, value);
+    table.delete(lockid, CONTENTS);
+    table.commit(lockid);
+
+    lockid = table.startUpdate(new Text("row2"));
+    table.put(lockid, CONTENTS, value);
+    table.commit(lockid);
+
+    Text[] columns = { CONTENTS };
+    HScannerInterface scanner = table.obtainScanner(columns, new Text());
+    HStoreKey key = new HStoreKey();
+    TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
+    while(scanner.next(key, results)) {
+      for(Map.Entry<Text, byte[]> e: results.entrySet()) {
+        System.out.println(key + ": row: " + e.getKey() + " value: " + 
+            new String(e.getValue(), HConstants.UTF8_ENCODING));
+      }
+    }
+  }
+}