You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ji...@apache.org on 2008/08/28 21:27:45 UTC

svn commit: r689938 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/client/transactional/ src/java/org/apache/hadoop/hbase/io/ src/java/org/apache/hadoop/hbase/mas...

Author: jimk
Date: Thu Aug 28 12:27:44 2008
New Revision: 689938

URL: http://svn.apache.org/viewvc?rev=689938&view=rev
Log:
HBASE-842  Remove methods that have Text as a parameter and were deprecated in 0.2.1 (Jean-Daniel Cryans via Jim Kellerman)

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HTable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Bytes.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Thu Aug 28 12:27:44 2008
@@ -63,6 +63,8 @@
               compilation problems that were introduced by patch.
    HBASE-669  MultiRegion transactions with Optimistic Concurrency Control
               (Clint Morgan via Stack)
+   HBASE-842  Remove methods that have Text as a parameter and were deprecated
+              in 0.2.1 (Jean-Daniel Cryans via Jim Kellerman)
 
   OPTIMIZATIONS
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java Thu Aug 28 12:27:44 2008
@@ -136,16 +136,6 @@
   public HColumnDescriptor(final String columnName) {
     this(Bytes.toBytes(columnName));
   }
-
-  /**
-   * Construct a column descriptor specifying only the family name 
-   * The other attributes are defaulted.
-   * 
-   * @param columnName - column family name
-   */
-  public HColumnDescriptor(final Text columnName) {
-    this(columnName.getBytes());
-  }
   
   /**
    * Construct a column descriptor specifying only the family name 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMsg.java Thu Aug 28 12:27:44 2008
@@ -24,7 +24,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -97,7 +97,7 @@
 
   private Type type = null;
   private HRegionInfo info = null;
-  private Text message = null;
+  private byte[] message = null;
 
   // Some useful statics.  Use these rather than create a new HMsg each time.
   public static final HMsg REPORT_EXITING = new HMsg(Type.MSG_REPORT_EXITING);
@@ -141,7 +141,7 @@
    * null.  If no info associated, used other Constructor.
    * @param msg Optional message (Stringified exception, etc.)
    */
-  public HMsg(final HMsg.Type type, final HRegionInfo hri, final Text msg) {
+  public HMsg(final HMsg.Type type, final HRegionInfo hri, final byte[] msg) {
     if (type == null) {
       throw new NullPointerException("Message type cannot be null");
     }
@@ -172,7 +172,7 @@
     return this.type.equals(other);
   }
   
-  public Text getMessage() {
+  public byte[] getMessage() {
     return this.message;
   }
 
@@ -188,7 +188,7 @@
       sb.append(": ");
       sb.append(this.info.getRegionNameAsString());
     }
-    if (this.message != null && this.message.getLength() > 0) {
+    if (this.message != null && this.message.length > 0) {
       sb.append(": " + this.message);
     }
     return sb.toString();
@@ -221,11 +221,11 @@
   public void write(DataOutput out) throws IOException {
      out.writeInt(this.type.ordinal());
      this.info.write(out);
-     if (this.message == null || this.message.getLength() == 0) {
+     if (this.message == null || this.message.length == 0) {
        out.writeBoolean(false);
      } else {
        out.writeBoolean(true);
-       this.message.write(out);
+       Bytes.writeByteArray(out, this.message);
      }
    }
 
@@ -238,10 +238,7 @@
      this.info.readFields(in);
      boolean hasMessage = in.readBoolean();
      if (hasMessage) {
-       if (this.message == null) {
-         this.message = new Text();
-       }
-       this.message.readFields(in);
+       this.message = Bytes.readByteArray(in);
      }
    }
-}
\ No newline at end of file
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Thu Aug 28 12:27:44 2008
@@ -40,7 +40,6 @@
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
 
 /**
@@ -86,16 +85,6 @@
    * @return True if table exists already.
    * @throws MasterNotRunningException
    */
-  public boolean tableExists(final Text tableName)
-  throws MasterNotRunningException {
-    return tableExists(tableName.getBytes());
-  }
-
-  /**
-   * @param tableName Table to check.
-   * @return True if table exists already.
-   * @throws MasterNotRunningException
-   */
   public boolean tableExists(final String tableName)
   throws MasterNotRunningException {
     return tableExists(Bytes.toBytes(tableName));
@@ -194,16 +183,6 @@
       throw RemoteExceptionHandler.decodeRemoteException(e);
     }
   }
-  
-  /**
-   * Deletes a table
-   * 
-   * @param tableName name of table to delete
-   * @throws IOException
-   */
-  public void deleteTable(final Text tableName) throws IOException {
-    deleteTable(tableName.getBytes());
-  }
 
   /**
    * Deletes a table
@@ -298,16 +277,6 @@
    * @param tableName name of the table
    * @throws IOException
    */
-  public void enableTable(final Text tableName) throws IOException {
-    enableTable(tableName.getBytes());
-  }
-
-  /**
-   * Brings a table on-line (enables it)
-   * 
-   * @param tableName name of the table
-   * @throws IOException
-   */
   public void enableTable(final String tableName) throws IOException {
     enableTable(Bytes.toBytes(tableName));
   }
@@ -360,17 +329,6 @@
    * @param tableName name of table
    * @throws IOException
    */
-  public void disableTable(final Text tableName) throws IOException {
-    disableTable(tableName.getBytes());
-  }
-
-  /**
-   * Disables a table (takes it off-line) If it is being served, the master
-   * will tell the servers to stop serving it.
-   * 
-   * @param tableName name of table
-   * @throws IOException
-   */
   public void disableTable(final String tableName) throws IOException {
     disableTable(Bytes.toBytes(tableName));
   }
@@ -421,14 +379,6 @@
    * @return true if table is on-line
    * @throws IOException
    */
-  public boolean isTableEnabled(Text tableName) throws IOException {
-    return isTableEnabled(tableName.getBytes());
-  }
-  /**
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException
-   */
   public boolean isTableEnabled(String tableName) throws IOException {
     return isTableEnabled(Bytes.toBytes(tableName));
   }
@@ -440,18 +390,6 @@
   public boolean isTableEnabled(byte[] tableName) throws IOException {
     return connection.isTableEnabled(tableName);
   }
-  
-  /**
-   * Add a column to an existing table
-   * 
-   * @param tableName name of the table to add column to
-   * @param column column descriptor of column to be added
-   * @throws IOException
-   */
-  public void addColumn(final Text tableName, HColumnDescriptor column)
-  throws IOException {
-    addColumn(tableName.getBytes(), column);
-  }
 
   /**
    * Add a column to an existing table
@@ -492,18 +430,6 @@
    * @param columnName name of column to be deleted
    * @throws IOException
    */
-  public void deleteColumn(final Text tableName, final Text columnName)
-  throws IOException {
-    deleteColumn(tableName.getBytes(), columnName.getBytes());
-  }
-
-  /**
-   * Delete a column from a table
-   * 
-   * @param tableName name of table
-   * @param columnName name of column to be deleted
-   * @throws IOException
-   */
   public void deleteColumn(final String tableName, final String columnName)
   throws IOException {
     deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName));
@@ -537,20 +463,6 @@
    * @param descriptor new column descriptor to use
    * @throws IOException
    */
-  public void modifyColumn(final Text tableName, final Text columnName, 
-      HColumnDescriptor descriptor)
-  throws IOException {
-    modifyColumn(tableName.getBytes(), columnName.getBytes(), descriptor);
-  }
-
-  /**
-   * Modify an existing column family on a table
-   * 
-   * @param tableName name of table
-   * @param columnName name of column to be modified
-   * @param descriptor new column descriptor to use
-   * @throws IOException
-   */
   public void modifyColumn(final String tableName, final String columnName, 
       HColumnDescriptor descriptor)
   throws IOException {

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=689938&r1=689937&r2=689938&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 Thu Aug 28 12:27:44 2008
@@ -43,7 +43,6 @@
 import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.Text;
 
 /**
  * Used to communicate with a single HBase table
@@ -58,18 +57,6 @@
    *
    * @param tableName name of the table
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public HTable(final Text tableName)
-  throws IOException {
-    this(new HBaseConfiguration(), tableName.getBytes());
-  }
-
-  /**
-   * Creates an object to access a HBase table
-   *
-   * @param tableName name of the table
-   * @throws IOException
    */
   public HTable(final String tableName)
   throws IOException {
@@ -93,19 +80,6 @@
    * @param conf configuration object
    * @param tableName name of the table
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public HTable(HBaseConfiguration conf, final Text tableName)
-  throws IOException {
-    this(conf, tableName.getBytes());
-  }
-
-  /**
-   * Creates an object to access a HBase table
-   * 
-   * @param conf configuration object
-   * @param tableName name of the table
-   * @throws IOException
    */
   public HTable(HBaseConfiguration conf, final String tableName)
   throws IOException {
@@ -131,15 +105,6 @@
    * @param tableName name of table to check
    * @return true if table is on-line
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public static boolean isTableEnabled(Text tableName) throws IOException {
-    return isTableEnabled(tableName.getBytes());
-  }
-  /**
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException
    */
   public static boolean isTableEnabled(String tableName) throws IOException {
     return isTableEnabled(Bytes.toBytes(tableName));
@@ -158,18 +123,6 @@
    * @param tableName name of table to check
    * @return true if table is on-line
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public static boolean isTableEnabled(HBaseConfiguration conf, Text tableName)
-  throws IOException {
-    return isTableEnabled(conf, tableName.getBytes());
-  }
-  
-  /**
-   * @param conf HBaseConfiguration object
-   * @param tableName name of table to check
-   * @return true if table is on-line
-   * @throws IOException
    */
   public static boolean isTableEnabled(HBaseConfiguration conf, String tableName)
   throws IOException {
@@ -186,18 +139,6 @@
   throws IOException {
     return HConnectionManager.getConnection(conf).isTableEnabled(tableName);
   }
-  
-  /**
-   * Find region location hosting passed row using cached info
-   * @param row Row to find.
-   * @return Location of row.
-   * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public HRegionLocation getRegionLocation(final Text row)
-  throws IOException {
-    return connection.getRegionLocation(tableName, row.getBytes(), false);
-  }
 
   /**
    * Find region location hosting passed row using cached info
@@ -239,15 +180,6 @@
    * @return table metadata 
    * @throws IOException
    */
-  @Deprecated
-  public HTableDescriptor getMetadata() throws IOException {
-    return getTableDescriptor();
-  }
-
-  /**
-   * @return table metadata 
-   * @throws IOException
-   */
   public HTableDescriptor getTableDescriptor() throws IOException {
     return new UnmodifyableHTableDescriptor(
       this.connection.getHTableDescriptor(this.tableName));
@@ -316,35 +248,6 @@
     MetaScanner.metaScan(configuration, visitor, tableName);
     return regionMap;
   }
-  
-  /**
-   * 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
-   * @deprecated Use String or byte [] overload instead
-   */
-  public Cell get(final Text row, final Text column)
-  throws IOException {
-    return get(row.getBytes(), column.getBytes());
-  }
-
-  /** 
-   * Get a single value for the specified row and column
-   *
-   * @param row row key
-   * @param column column name
-   * @param numVersions - number of versions to retrieve
-   * @return value for specified row/column
-   * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public Cell[] get(final Text row, final Text column, int numVersions)
-  throws IOException {
-    return get(row.getBytes(), column.getBytes(), numVersions);
-  }
 
   /**
    * Get a single value for the specified row and column
@@ -425,24 +328,6 @@
    * @param numVersions - number of versions to retrieve
    * @return            - array of values that match the above criteria
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public Cell[] get(final Text row, final Text column,
-    final long timestamp, final int numVersions)
-  throws IOException {
-    return get(row.getBytes(), column.getBytes(), timestamp, numVersions);
-  }
-
-  /** 
-   * 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 Cell[] get(final String row, final String column,
     final long timestamp, final int numVersions)
@@ -490,18 +375,6 @@
    * @param row row key
    * @return RowResult is empty if row does not exist.
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public RowResult getRow(final Text row) throws IOException {
-    return getRow(row.getBytes());
-  }
-
-  /** 
-   * Get all the data for the specified row at the latest timestamp
-   * 
-   * @param row row key
-   * @return RowResult is empty if row does not exist.
-   * @throws IOException
    */
   public RowResult getRow(final String row) throws IOException {
     return getRow(Bytes.toBytes(row));
@@ -525,20 +398,6 @@
    * @param ts timestamp
    * @return RowResult is empty if row does not exist.
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public RowResult getRow(final Text row, final long ts) 
-  throws IOException {
-    return getRow(row.getBytes(), ts);
-  }
-
-  /** 
-   * Get all the data for the specified row at a specified timestamp
-   * 
-   * @param row row key
-   * @param ts timestamp
-   * @return RowResult is empty if row does not exist.
-   * @throws IOException
    */
   public RowResult getRow(final String row, final long ts) 
   throws IOException {
@@ -565,20 +424,6 @@
    * @param columns Array of column names and families you want to retrieve.
    * @return RowResult is empty if row does not exist.
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public RowResult getRow(final Text row, final Text[] columns) 
-  throws IOException {
-    return getRow(row.getBytes(), 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 empty if row does not exist.
-   * @throws IOException
    */
   public RowResult getRow(final String row, final String [] columns) 
   throws IOException {
@@ -606,22 +451,6 @@
    * @param ts timestamp
    * @return RowResult is empty if row does not exist.
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public RowResult getRow(final Text row, final Text [] columns, 
-    final long ts) 
-  throws IOException {  
-    return getRow(row.getBytes(), 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
-   * @return RowResult is empty if row does not exist.
-   * @throws IOException
    */
   public RowResult getRow(final String row, final String [] columns, 
     final long ts) 
@@ -683,24 +512,6 @@
    * <code>\+|^&*$[]]}{)(</code>.
    * @return scanner
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public Scanner getScanner(final Text [] columns)
-  throws IOException {
-    return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW);
-  }
-
-  /** 
-   * Get a scanner on the current table starting at first 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>.
-   * @return scanner
-   * @throws IOException
    */
   public Scanner getScanner(final String [] columns)
   throws IOException {
@@ -719,25 +530,6 @@
    * @param startRow starting row in table to scan
    * @return scanner
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public Scanner getScanner(final Text [] columns, final Text startRow)
-  throws IOException {
-    return getScanner(Bytes.toByteArrays(columns), startRow.getBytes());
-  }
-
-  /** 
-   * 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 Scanner getScanner(final String [] columns, final String startRow)
   throws IOException {
@@ -861,32 +653,6 @@
    * @param timestamp only return results whose timestamp <= this value
    * @return scanner
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public Scanner getScanner(final Text[] columns,
-    final Text startRow, final Text stopRow, final long timestamp)
-  throws IOException {
-    return getScanner(Bytes.toByteArrays(columns), startRow.getBytes(),
-      stopRow.getBytes(), 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 Scanner getScanner(final String [] columns,
     final String startRow, final String stopRow, final long timestamp)
@@ -918,29 +684,6 @@
   throws IOException {
     return getScanner(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
-   * @deprecated Use String or byte [] overload instead
-   */
-  public Scanner getScanner(Text[] columns,
-    Text startRow, long timestamp, RowFilterInterface filter)
-  throws IOException {
-    return getScanner(Bytes.toByteArrays(columns), startRow.getBytes(),
-      timestamp, filter);
   }
 
   /** 
@@ -997,18 +740,7 @@
   public void deleteAll(final byte [] row) throws IOException {
     deleteAll(row, null);
   }
-  
-  /**
-   * Completely delete the row's cells.
-   *
-   * @param row Key of the row you want to completely delete.
-   * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public void deleteAll(final Text row) throws IOException {
-    deleteAll(row, null);
-  }
-  
+
   /**
    * Completely delete the row's cells.
    *
@@ -1055,30 +787,6 @@
     deleteAll(row, null, ts);
   }
 
-  /**
-   * Completely delete the row's cells.
-   *
-   * @param row Key of the row you want to completely delete.
-   * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */
-  public void deleteAll(final Text row, final long ts)
-  throws IOException {
-    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 
-   * @deprecated Use String or byte [] overload instead
-   */
-  public void deleteAll(final Text row, final Text column) throws IOException {
-    deleteAll(row, column, HConstants.LATEST_TIMESTAMP);
-  }
-
   /** 
    * Delete all cells that match the passed row and column.
    * @param row Row to update
@@ -1097,20 +805,6 @@
    * @param column name of column whose value is to be deleted
    * @param ts Delete all cells of the same timestamp or older.
    * @throws IOException 
-   * @deprecated Use String or byte [] overload instead
-   */
-  public void deleteAll(final Text row, final Text column, final long ts)
-  throws IOException {
-    deleteAll(row.getBytes(), column.getBytes(), 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 
    */
   public void deleteAll(final String row, final String column, final long ts)
   throws IOException {
@@ -1169,14 +863,11 @@
    *
    * @param row The row to operate on
    * @param family The column family to match
-   * @param timestamp Timestamp to match
    * @throws IOException
-   * @deprecated Use String or byte [] overload instead
-   */  
-  public void deleteFamily(final Text row, final Text family,
-      final long timestamp)
-  throws IOException{
-    deleteFamily(row.getBytes(), family.getBytes(), timestamp);
+   */
+  public void deleteFamily(final String row, final String family) 
+  throws IOException {
+    deleteFamily(row, family, HConstants.LATEST_TIMESTAMP);
   }
 
   /**
@@ -1184,6 +875,19 @@
    *
    * @param row The row to operate on
    * @param family The column family to match
+   * @throws IOException
+   */
+  public void deleteFamily(final byte[] row, final byte[] family) 
+  throws IOException {
+    deleteFamily(row, family, 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
    */  
@@ -1336,12 +1040,6 @@
     private HRegionInfo currentRegion = null;
     private ScannerCallable callable = null;
     protected RowFilterInterface filter;
-    
-    protected ClientScanner(final Text [] columns, final Text startRow,
-        long timestamp, RowFilterInterface filter) {
-      this(Bytes.toByteArrays(columns), startRow.getBytes(), timestamp,
-        filter);
-    }
 
     protected ClientScanner(final byte[][] columns, final byte [] startRow,
         final long timestamp, final RowFilterInterface filter) {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Scanner.java Thu Aug 28 12:27:44 2008
@@ -30,7 +30,7 @@
 public interface Scanner extends Closeable, Iterable<RowResult> {
   /**
    * Grab the next row's worth of values. The scanner will return a RowResult
-   * that contains both the row's key and a map of Text column names to Cell 
+   * that contains both the row's key and a map of byte[] column names to Cell 
    * value objects. The data returned will only contain the most recent data 
    * value for each row that is not newer than the target time passed when the
    * scanner was created.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/transactional/TransactionalTable.java Thu Aug 28 12:27:44 2008
@@ -33,7 +33,6 @@
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.ipc.TransactionalRegionInterface;
-import org.apache.hadoop.io.Text;
 
 /**
  * Table with transactional support.
@@ -46,11 +45,6 @@
     super(conf, tableName);
   }
 
-  public TransactionalTable(final HBaseConfiguration conf, final Text tableName)
-      throws IOException {
-    super(conf, tableName);
-  }
-
   public TransactionalTable(final HBaseConfiguration conf,
       final byte[] tableName) throws IOException {
     super(conf, tableName);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java Thu Aug 28 12:27:44 2008
@@ -27,7 +27,6 @@
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -61,16 +60,6 @@
    * 
    * @param row
    */
-  public BatchUpdate(final Text row) {
-    this(row, HConstants.LATEST_TIMESTAMP);
-  }
-  
-  /**
-   * Initialize a BatchUpdate operation on a row. Timestamp is assumed to be
-   * now.
-   * 
-   * @param row
-   */
   public BatchUpdate(final String row) {
     this(Bytes.toBytes(row), HConstants.LATEST_TIMESTAMP);
   }
@@ -99,15 +88,6 @@
    * 
    * @param row
    */
-  public BatchUpdate(final Text row, long timestamp){
-    this(row.getBytes(), timestamp);
-  }
-
-  /**
-   * Initialize a BatchUpdate operation on a row with a specific timestamp.
-   * 
-   * @param row
-   */
   public BatchUpdate(final byte [] row, long timestamp){
     this.row = row;
     this.timestamp = timestamp;
@@ -139,16 +119,6 @@
    * @param column column whose value is being set
    * @param val new value for column.  Cannot be null (can be empty).
    */
-  public synchronized void put(final Text column, final byte val[]) {
-    put(column.getBytes(), val);
-  }
-  
-  /** 
-   * Change a value for the specified column
-   *
-   * @param column column whose value is being set
-   * @param val new value for column.  Cannot be null (can be empty).
-   */
   public synchronized void put(final String column, final byte val[]) {
     put(Bytes.toBytes(column), val);
   }
@@ -173,16 +143,6 @@
    * delete.
    * @param column name of column whose value is to be deleted
    */
-  public void delete(final Text column) {
-    delete(column.getBytes());
-  }
- 
-  /** 
-   * Delete the value for a column
-   * Deletes the cell whose row/column/commit-timestamp match those of the
-   * delete.
-   * @param column name of column whose value is to be deleted
-   */
   public void delete(final String column) {
     delete(Bytes.toBytes(column));
   }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/RegionManager.java Thu Aug 28 12:27:44 2008
@@ -50,7 +50,6 @@
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.Text;
 
 /**
  * Class to manage assigning regions to servers, state of root and meta, etc.
@@ -74,7 +73,7 @@
     Collections.synchronizedSortedMap(new TreeMap<byte [],
       MetaRegion>(Bytes.BYTES_COMPARATOR));
 
-  private static final Text OVERLOADED = new Text("Overloaded");
+  private static final byte[] OVERLOADED = Bytes.toBytes("Overloaded");
 
   /**
    * The 'unassignedRegions' table maps from a HRegionInfo to a timestamp that

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/master/ServerManager.java Thu Aug 28 12:27:44 2008
@@ -43,7 +43,6 @@
 import org.apache.hadoop.hbase.Leases;
 import org.apache.hadoop.hbase.LeaseListener;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.io.Text;
 
 /**
  * The ServerManager class manages info about region servers - HServerInfo, 
@@ -437,7 +436,7 @@
       // Otherwise the HMaster will think the Region was closed on purpose, 
       // and then try to reopen it elsewhere; that's not what we want.
       returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_CLOSE_WITHOUT_REPORT,
-        region, new Text("Duplicate assignment")));
+        region, "Duplicate assignment".getBytes()));
     } else {
       // it was assigned, and it's not a duplicate assignment, so take it out 
       // of the unassigned list.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Aug 28 12:27:44 2008
@@ -86,7 +86,6 @@
 import org.apache.hadoop.hbase.util.Sleeper;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.io.MapWritable;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.util.Progressable;
@@ -742,10 +741,9 @@
   }
 
   /* Add to the outbound message buffer */
-  private void reportClose(final HRegionInfo region, final Text message) {
+  private void reportClose(final HRegionInfo region, final byte[] message) {
     outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_CLOSE, region, message));
   }
-
   
   /**
    * Add to the outbound message buffer
@@ -761,9 +759,9 @@
       HRegionInfo newRegionB) {
 
     outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_SPLIT, oldRegion,
-      new Text(oldRegion.getRegionNameAsString() + " split; daughters: " +
+      (oldRegion.getRegionNameAsString() + " split; daughters: " +
         newRegionA.getRegionNameAsString() + ", " +
-        newRegionB.getRegionNameAsString())));
+        newRegionB.getRegionNameAsString()).getBytes()));
     outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionA));
     outboundMsgs.add(new HMsg(HMsg.Type.MSG_REPORT_OPEN, newRegionB));
   }
@@ -884,7 +882,7 @@
         // TODO: add an extra field in HRegionInfo to indicate that there is
         // an error. We can't do that now because that would be an incompatible
         // change that would require a migration
-        reportClose(regionInfo, new Text(StringUtils.stringifyException(e)));
+        reportClose(regionInfo, StringUtils.stringifyException(e).getBytes());
         return;
       }
       this.lock.writeLock().lock();

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Bytes.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Bytes.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Bytes.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Bytes.java Thu Aug 28 12:27:44 2008
@@ -9,7 +9,6 @@
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableUtils;
 
@@ -239,18 +238,6 @@
     return result;
   }
   
-  
-  /**
-   * @param t
-   * @return Array of byte arrays made from passed array of Text
-   */
-  public static byte [][] toByteArrays(final Text [] t) {
-    byte [][] result = new byte[t.length][];
-    for (int i = 0; i < t.length; i++) {
-      result[i] = t[i].getBytes();
-    }
-    return result;
-  }
 
   /**
    * @param t

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=689938&r1=689937&r2=689938&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 Aug 28 12:27:44 2008
@@ -35,13 +35,11 @@
 import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Text;
 
 /** test the scanner API at all levels */
 public class TestScannerAPI extends HBaseClusterTestCase {
-  private final byte [][] columns = Bytes.toByteArrays(new Text[] {
-    new Text("a:"),
-    new Text("b:")
+  private final byte [][] columns = Bytes.toByteArrays(new String[] {
+    "a:", "b:"
   });
   private final byte [] startRow = Bytes.toBytes("0");
 

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java Thu Aug 28 12:27:44 2008
@@ -27,7 +27,6 @@
 import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.Text;
 
 /**
  * Test HBase Writables serializations
@@ -50,7 +49,7 @@
     m = new HMsg(HMsg.Type.MSG_REGIONSERVER_QUIESCE,
       new HRegionInfo(new HTableDescriptor(getName()),
         HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY),
-        new Text("Some message"));
+        "Some message".getBytes());
     mb = Writables.getBytes(m);
     deserializedHMsg = (HMsg)Writables.getWritable(mb, new HMsg());
     assertTrue(m.equals(deserializedHMsg));

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java Thu Aug 28 12:27:44 2008
@@ -26,7 +26,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.HBaseTestCase;
 
 import org.apache.hadoop.hbase.util.Bytes;
@@ -112,8 +111,8 @@
       batchUpdate.put(HConstants.COL_SERVER, 
         Bytes.toBytes(new HServerAddress(SERVER_ADDRESS).toString()));
       batchUpdate.put(HConstants.COL_STARTCODE, Bytes.toBytes(12345));
-      batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) +
-        "region"), Bytes.toBytes("region"));
+      batchUpdate.put(Bytes.toString(HConstants.COLUMN_FAMILY) +
+        "region", Bytes.toBytes("region"));
       r.commit(batchUpdate);
       
       // Verify that get works the same from memcache as when reading from disk
@@ -134,14 +133,12 @@
       // Update one family member and add a new one
       
       batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis());
-      batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) +
-        "region"),
+      batchUpdate.put(Bytes.toString(HConstants.COLUMN_FAMILY) + "region",
         "region2".getBytes(HConstants.UTF8_ENCODING));
       String otherServerName = "bar.foo.com:4321";
       batchUpdate.put(HConstants.COL_SERVER, 
         Bytes.toBytes(new HServerAddress(otherServerName).toString()));
-      batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) +
-        "junk"),
+      batchUpdate.put(Bytes.toString(HConstants.COLUMN_FAMILY) + "junk",
         "junk".getBytes(HConstants.UTF8_ENCODING));
       r.commit(batchUpdate);
 

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java Thu Aug 28 12:27:44 2008
@@ -24,7 +24,6 @@
 
 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;
 import org.apache.hadoop.hbase.HBaseClusterTestCase;
@@ -115,7 +114,7 @@
 
     for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
       BatchUpdate b =
-        new BatchUpdate(new Text("row" + String.format("%1$04d", i)));
+        new BatchUpdate("row" + String.format("%1$04d", i));
       b.put(HConstants.COLUMN_FAMILY, value);
       table.commit(b);
 

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java?rev=689938&r1=689937&r2=689938&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestMergeTool.java Thu Aug 28 12:27:44 2008
@@ -35,7 +35,6 @@
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.HLog;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.ToolRunner;
 
 /** Test stand alone merge tool that can merge arbitrary regions */
@@ -90,11 +89,11 @@
      * Now create some row keys
      */
     this.rows = new byte [5][][];
-    this.rows[0] = Bytes.toByteArrays(new Text[] { new Text("row_0210"), new Text("row_0280") });
-    this.rows[1] = Bytes.toByteArrays(new Text[] { new Text("row_0260"), new Text("row_0350") });
-    this.rows[2] = Bytes.toByteArrays(new Text[] { new Text("row_0110"), new Text("row_0175") });
-    this.rows[3] = Bytes.toByteArrays(new Text[] { new Text("row_0525"), new Text("row_0560") });
-    this.rows[4] = Bytes.toByteArrays(new Text[] { new Text("row_0050"), new Text("row_1000") });
+    this.rows[0] = Bytes.toByteArrays(new String[] { "row_0210", "row_0280" });
+    this.rows[1] = Bytes.toByteArrays(new String[] { "row_0260", "row_0350" });
+    this.rows[2] = Bytes.toByteArrays(new String[] { "row_0110", "row_0175" });
+    this.rows[3] = Bytes.toByteArrays(new String[] { "row_0525", "row_0560" });
+    this.rows[4] = Bytes.toByteArrays(new String[] { "row_0050", "row_1000" });
     
     // Start up dfs
     this.dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);