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

svn commit: r619605 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/io/ src/test/org/apache/hadoop/hbase/

Author: stack
Date: Thu Feb  7 11:51:03 2008
New Revision: 619605

URL: http://svn.apache.org/viewvc?rev=619605&view=rev
Log:
HBASE-35 Make BatchUpdate public in the API

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMaster.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMerge.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegion.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInterface.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/io/BatchUpdate.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/HBaseTestCase.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/OOMERegionServer.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestDeleteAll.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGet2.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTimestamp.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Thu Feb  7 11:51:03 2008
@@ -11,6 +11,8 @@
   BUG FIXES
 
   IMPROVEMENTS
+   HBASE-415   Rewrite leases to use DelayedBlockingQueue instead of polling
+   HBASE-35    Make BatchUpdate public in the API
 
 Branch 0.1
 
@@ -38,7 +40,6 @@
    HADOOP-2555 Refactor the HTable#get and HTable#getRow methods to avoid
                repetition of retry-on-failure logic (thanks to Peter Dolan and
                Bryan Duxbury)
-   HBASE-415   Rewrite leases to use DelayedBlockingQueue instead of polling
 
 Release 0.16.0
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMaster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMaster.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMaster.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMaster.java Thu Feb  7 11:51:03 2008
@@ -399,10 +399,9 @@
             +" no longer has references to " + parent.toString());
       }
       
-      BatchUpdate b = new BatchUpdate(rand.nextLong());
-      long lockid = b.startUpdate(parent);
-      b.delete(lockid, splitColumn);
-      srvr.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
+      BatchUpdate b = new BatchUpdate(parent);
+      b.delete(splitColumn);
+      srvr.batchUpdate(metaRegionName, b);
         
       return result;
     }
@@ -2434,12 +2433,10 @@
           Writables.bytesToLong(this.startCode) + " and server "+
           serverAddress.toString());
         try {
-          BatchUpdate b = new BatchUpdate(rand.nextLong());
-          long lockid = b.startUpdate(regionInfo.getRegionName());
-          b.put(lockid, COL_SERVER,
-            Writables.stringToBytes(serverAddress.toString()));
-          b.put(lockid, COL_STARTCODE, startCode);
-          server.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
+          BatchUpdate b = new BatchUpdate(regionInfo.getRegionName());
+          b.put(COL_SERVER, Writables.stringToBytes(serverAddress.toString()));
+          b.put(COL_STARTCODE, startCode);
+          server.batchUpdate(metaRegionName, b);
           if (isMetaTable) {
             // It's a meta region.
             MetaRegion m = new MetaRegion(this.serverAddress,
@@ -2574,10 +2571,9 @@
           
       HRegionInfo info = region.getRegionInfo();
       Text regionName = region.getRegionName();
-      BatchUpdate b = new BatchUpdate(rand.nextLong());
-      long lockid = b.startUpdate(regionName);
-      b.put(lockid, COL_REGIONINFO, Writables.getBytes(info));
-      server.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
+      BatchUpdate b = new BatchUpdate(regionName);
+      b.put(COL_REGIONINFO, Writables.getBytes(info));
+      server.batchUpdate(metaRegionName, b);
 
       // 4. Close the new region to flush it to disk.  Close its log file too.
       
@@ -2823,12 +2819,11 @@
           LOG.debug("updating columns in row: " + i.getRegionName());
         }
 
-        BatchUpdate b = new BatchUpdate(rand.nextLong());
-        lockid = b.startUpdate(i.getRegionName());
+        BatchUpdate b = new BatchUpdate(i.getRegionName());
         updateRegionInfo(b, i);
-        b.delete(lockid, COL_SERVER);
-        b.delete(lockid, COL_STARTCODE);
-        server.batchUpdate(m.getRegionName(), System.currentTimeMillis(), b);
+        b.delete(COL_SERVER);
+        b.delete(COL_STARTCODE);
+        server.batchUpdate(m.getRegionName(), b);
         if (LOG.isDebugEnabled()) {
           LOG.debug("updated columns in row: " + i.getRegionName());
         }
@@ -2888,7 +2883,7 @@
       throws IOException {
       
       i.setOffline(!online);
-      b.put(lockid, COL_REGIONINFO, Writables.getBytes(i));
+      b.put(COL_REGIONINFO, Writables.getBytes(i));
     }
   }
 
@@ -2936,7 +2931,7 @@
         @SuppressWarnings("unused") HRegionInfo info) {
       for (int i = 0; i < ALL_META_COLUMNS.length; i++) {
         // Be sure to clean all cells
-        b.delete(lockid, ALL_META_COLUMNS[i]);
+        b.delete(ALL_META_COLUMNS[i]);
       }
     }
   }
@@ -2961,10 +2956,9 @@
     protected void updateRegionInfo(HRegionInterface server, Text regionName,
         HRegionInfo i) throws IOException {
 
-      BatchUpdate b = new BatchUpdate(rand.nextLong());
-      long lockid = b.startUpdate(i.getRegionName());
-      b.put(lockid, COL_REGIONINFO, Writables.getBytes(i));
-      server.batchUpdate(regionName, System.currentTimeMillis(), b);
+      BatchUpdate b = new BatchUpdate(i.getRegionName());
+      b.put(COL_REGIONINFO, Writables.getBytes(i));
+      server.batchUpdate(regionName, b);
       if (LOG.isDebugEnabled()) {
         LOG.debug("updated columns in row: " + i.getRegionName());
       }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMerge.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMerge.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMerge.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HMerge.java Thu Feb  7 11:51:03 2008
@@ -257,53 +257,30 @@
     }
 
     @Override
-    protected void updateMeta(Text oldRegion1, Text oldRegion2,
-        HRegion newRegion) throws IOException {
-      Text[] regionsToDelete = {
-          oldRegion1,
-          oldRegion2
-      };
+    protected void updateMeta(Text oldRegion1, Text oldRegion2, 
+      HRegion newRegion)
+    throws IOException {
+      Text[] regionsToDelete = {oldRegion1, oldRegion2};
       for(int r = 0; r < regionsToDelete.length; r++) {
         if(regionsToDelete[r].equals(latestRegion.getRegionName())) {
           latestRegion = null;
         }
-        long lockid = -1L;
-        try {
-          lockid = table.startUpdate(regionsToDelete[r]);
-          table.delete(lockid, COL_REGIONINFO);
-          table.delete(lockid, COL_SERVER);
-          table.delete(lockid, COL_STARTCODE);
-          table.delete(lockid, COL_SPLITA);
-          table.delete(lockid, COL_SPLITB);
-          table.commit(lockid);
-          lockid = -1L;
+        table.deleteAll(regionsToDelete[r]);
 
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("updated columns in row: " + regionsToDelete[r]);
-          }
-        } finally {
-          if(lockid != -1L) {
-            table.abort(lockid);
-          }
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("updated columns in row: " + regionsToDelete[r]);
         }
       }
       newRegion.getRegionInfo().setOffline(true);
-      long lockid = -1L;
-      try {
-        lockid = table.startUpdate(newRegion.getRegionName());
-        table.put(lockid, COL_REGIONINFO,
-            Writables.getBytes(newRegion.getRegionInfo()));
-        table.commit(lockid);
-        lockid = -1L;
 
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("updated columns in row: "
-              + newRegion.getRegionName());
-        }
-      } finally {
-        if(lockid != -1L) {
-          table.abort(lockid);
-        }
+      BatchUpdate update = new BatchUpdate(newRegion.getRegionName());
+      update.put(COL_REGIONINFO,
+        Writables.getBytes(newRegion.getRegionInfo()));
+      table.commit(update);
+
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("updated columns in row: "
+            + newRegion.getRegionName());
       }
     }
   }
@@ -372,15 +349,13 @@
       };
       for(int r = 0; r < regionsToDelete.length; r++) {
         long lockid = Math.abs(rand.nextLong());
-        BatchUpdate b = new BatchUpdate(lockid);
-        lockid = b.startUpdate(regionsToDelete[r]);
-        b.delete(lockid, COL_REGIONINFO);
-        b.delete(lockid, COL_SERVER);
-        b.delete(lockid, COL_STARTCODE);
-        b.delete(lockid, COL_SPLITA);
-        b.delete(lockid, COL_SPLITB);
-        root.batchUpdate(System.currentTimeMillis(), b);
-        lockid = -1L;
+        BatchUpdate b = new BatchUpdate(regionsToDelete[r]);
+        b.delete(COL_REGIONINFO);
+        b.delete(COL_SERVER);
+        b.delete(COL_STARTCODE);
+        b.delete(COL_SPLITA);
+        b.delete(COL_SPLITB);
+        root.batchUpdate(b);
 
         if(LOG.isDebugEnabled()) {
           LOG.debug("updated columns in row: " + regionsToDelete[r]);
@@ -388,11 +363,9 @@
       }
       HRegionInfo newInfo = newRegion.getRegionInfo();
       newInfo.setOffline(true);
-      long lockid = Math.abs(rand.nextLong());
-      BatchUpdate b = new BatchUpdate(lockid);
-      lockid = b.startUpdate(newRegion.getRegionName());
-      b.put(lockid, COL_REGIONINFO, Writables.getBytes(newInfo));
-      root.batchUpdate(System.currentTimeMillis(), b);
+      BatchUpdate b = new BatchUpdate(newRegion.getRegionName());
+      b.put(COL_REGIONINFO, Writables.getBytes(newInfo));
+      root.batchUpdate(b);
       if(LOG.isDebugEnabled()) {
         LOG.debug("updated columns in row: " + newRegion.getRegionName());
       }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegion.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegion.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegion.java Thu Feb  7 11:51:03 2008
@@ -1209,7 +1209,7 @@
    * @param b
    * @throws IOException
    */
-  public void batchUpdate(long timestamp, BatchUpdate b)
+  public void batchUpdate(BatchUpdate b)
     throws IOException {
     // Do a rough check that we have resources to accept a write.  The check is
     // 'rough' in that between the resource check and the call to obtain a 
@@ -1226,7 +1226,7 @@
     long lockid = obtainRowLock(row);
 
     long commitTime =
-      (timestamp == LATEST_TIMESTAMP) ? System.currentTimeMillis() : timestamp;
+      (b.getTimestamp() == LATEST_TIMESTAMP) ? System.currentTimeMillis() : b.getTimestamp();
       
     try {
       List<Text> deletes = null;
@@ -1239,7 +1239,7 @@
             throw new IOException("Cannot insert value: " + val);
           }
         } else {
-          if (timestamp == LATEST_TIMESTAMP) {
+          if (b.getTimestamp() == LATEST_TIMESTAMP) {
             // Save off these deletes
             if (deletes == null) {
               deletes = new ArrayList<Text>();
@@ -1841,14 +1841,9 @@
    * @see {@link #addRegionToMETA(HRegion, HRegion)}
    */
   static void removeRegionFromMETA(final HRegionInterface srvr,
-      final Text metaRegionName, final Text regionName)
+    final Text metaRegionName, final Text regionName)
   throws IOException {
-    BatchUpdate b = new BatchUpdate(rand.nextLong());
-    long lockid = b.startUpdate(regionName);
-    for (int i = 0; i < ALL_META_COLUMNS.length; i++) {
-      b.delete(lockid, ALL_META_COLUMNS[i]);
-    }
-    srvr.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
+    srvr.deleteAll(metaRegionName, regionName, HConstants.LATEST_TIMESTAMP);
   }
 
   /**
@@ -1861,17 +1856,16 @@
    * @see {@link #addRegionToMETA(HRegion, HRegion)}
    */
   static void offlineRegionInMETA(final HRegionInterface srvr,
-      final Text metaRegionName, final HRegionInfo info)
+    final Text metaRegionName, final HRegionInfo info)
   throws IOException {
-    BatchUpdate b = new BatchUpdate(rand.nextLong());
-    long lockid = b.startUpdate(info.getRegionName());
+    BatchUpdate b = new BatchUpdate(info.getRegionName());
     info.setOffline(true);
-    b.put(lockid, COL_REGIONINFO, Writables.getBytes(info));
-    b.delete(lockid, COL_SERVER);
-    b.delete(lockid, COL_STARTCODE);
+    b.put(COL_REGIONINFO, Writables.getBytes(info));
+    b.delete(COL_SERVER);
+    b.delete(COL_STARTCODE);
     // If carrying splits, they'll be in place when we show up on new
     // server.
-    srvr.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
+    srvr.batchUpdate(metaRegionName, b);
   }
 
   /**

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInterface.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInterface.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInterface.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionInterface.java Thu Feb  7 11:51:03 2008
@@ -144,7 +144,7 @@
    * @param b BatchUpdate
    * @throws IOException
    */
-  public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
+  public void batchUpdate(Text regionName, BatchUpdate b)
   throws IOException;
   
   /**

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HRegionServer.java Thu Feb  7 11:51:03 2008
@@ -318,22 +318,25 @@
 
       // Mark old region as offline and split in META.
       // NOTE: there is no need for retry logic here. HTable does it for us.
-      long lockid = t.startUpdate(oldRegionInfo.getRegionName());
       oldRegionInfo.setOffline(true);
       oldRegionInfo.setSplit(true);
-      t.put(lockid, COL_REGIONINFO, Writables.getBytes(oldRegionInfo));
-      t.put(lockid, COL_SPLITA, Writables.getBytes(
-        newRegions[0].getRegionInfo()));
-      t.put(lockid, COL_SPLITB, Writables.getBytes(
-        newRegions[1].getRegionInfo()));
-      t.commit(lockid);
+      BatchUpdate update = new BatchUpdate(oldRegionInfo.getRegionName());
+      update.put(COL_REGIONINFO, Writables.getBytes(oldRegionInfo));
+      update.put(COL_SPLITA, Writables.getBytes(newRegions[0].getRegionInfo()));
+      update.put(COL_SPLITB, Writables.getBytes(newRegions[1].getRegionInfo()));
+      t.commit(update);
       
       // Add new regions to META
       for (int i = 0; i < newRegions.length; i++) {
-        lockid = t.startUpdate(newRegions[i].getRegionName());
+        update = new BatchUpdate(newRegions[i].getRegionName());
+        update.put(COL_REGIONINFO, Writables.getBytes(
+          newRegions[i].getRegionInfo()));
+        t.commit(update);
+        
+/*        long lockid = t.startUpdate(newRegions[i].getRegionName());
         t.put(lockid, COL_REGIONINFO, Writables.getBytes(
           newRegions[i].getRegionInfo()));
-        t.commit(lockid);
+        t.commit(lockid);*/
       }
           
       // Now tell the master about the new regions
@@ -1424,13 +1427,13 @@
   }
 
   /** {@inheritDoc} */
-  public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
+  public void batchUpdate(Text regionName, BatchUpdate b)
     throws IOException {
     checkOpen();
     this.requestCount.incrementAndGet();
     HRegion region = getRegion(regionName);
     try {
-      region.batchUpdate(timestamp, b);
+      region.batchUpdate(b);
     } catch (IOException e) {
       checkFileSystem();
       throw e;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java Thu Feb  7 11:51:03 2008
@@ -564,11 +564,12 @@
    * @see #commit(long, long)
    * @see #abort(long)
    */
+  @Deprecated
   public synchronized long startUpdate(final Text row) {
     checkClosed();
     updateInProgress(false);
-    batch.set(new BatchUpdate(rand.nextLong()));
-    return batch.get().startUpdate(row);
+    batch.set(new BatchUpdate(row));
+    return 1;
   }
   
   /** 
@@ -579,13 +580,17 @@
    * @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[]) {
     checkClosed();
+    if (lockid != 1) {
+      throw new IllegalArgumentException("Invalid lock id!");
+    }
     if (val == null) {
       throw new IllegalArgumentException("value cannot be null");
     }
     updateInProgress(true);
-    batch.get().put(lockid, column, val);
+    batch.get().put(column, val);
   }
   
   /** 
@@ -598,6 +603,7 @@
    * @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));
   }
@@ -609,10 +615,14 @@
    * @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) {
     checkClosed();
+    if (lockid != 1) {
+      throw new IllegalArgumentException("Invalid lock id!");
+    }
     updateInProgress(true);
-    batch.get().delete(lockid, column);
+    batch.get().delete(column);
   }
   
   /** 
@@ -718,10 +728,11 @@
    *
    * @param lockid lock id returned from startUpdate
    */
+  @Deprecated
   public synchronized void abort(long lockid) {
     checkClosed();
-    if (batch.get() != null && batch.get().getLockid() != lockid) {
-      throw new IllegalArgumentException("invalid lock id " + lockid);
+    if (lockid != 1) {
+      throw new IllegalArgumentException("Invalid lock id!");
     }
     batch.set(null);
   }
@@ -740,6 +751,7 @@
    * @param lockid lock id returned from startUpdate
    * @throws IOException
    */
+  @Deprecated
   public void commit(long lockid) throws IOException {
     commit(lockid, LATEST_TIMESTAMP);
   }
@@ -751,27 +763,36 @@
    * @param timestamp time to associate with the change
    * @throws IOException
    */
-  public synchronized void commit(long lockid, final long timestamp)
+  @Deprecated
+  public void commit(long lockid, final long timestamp)
   throws IOException {
-    checkClosed();
     updateInProgress(true);
-    if (batch.get().getLockid() != lockid) {
-      throw new IllegalArgumentException("invalid lock id " + lockid);
+    if (lockid != 1) {
+      throw new IllegalArgumentException("Invalid lock id!");
     }
-    
     try {
-      getRegionServerWithRetries(
-        new ServerCallable<Boolean>(batch.get().getRow()){
-          public Boolean call() throws IOException {
-            server.batchUpdate(location.getRegionInfo().getRegionName(), 
-              timestamp, batch.get());
-            return null;
-          }
-        }
-      );
+      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 {
+    checkClosed();
+    getRegionServerWithRetries(
+      new ServerCallable<Boolean>(batchUpdate.getRow()){
+        public Boolean call() throws IOException {
+          server.batchUpdate(location.getRegionInfo().getRegionName(), 
+            batchUpdate);
+          return null;
+        }
+      }
+    );  
   }
   
   /**

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=619605&r1=619604&r2=619605&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 Feb  7 11:51:03 2008
@@ -27,6 +27,7 @@
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.hbase.HConstants;
 
 /**
  * A Writable object that contains a series of BatchOperations
@@ -39,73 +40,65 @@
   
   // the row being updated
   private Text row;
-  
-  // the lockid - not used on server side
-  private transient long lockid;
-  
+    
   // the batched operations
   private ArrayList<BatchOperation> operations;
   
+  private long timestamp;
+  
   /** Default constructor - used by Writable. */
   public BatchUpdate() {
-    this.row = new Text();
-    this.lockid = -1L;
-    this.operations = new ArrayList<BatchOperation>();
+    this(new Text());
   }
   
   /**
-   * Client side constructor. Clients need to provide the lockid by some means
-   * such as Random.nextLong()
+   * Initialize a BatchUpdate operation on a row. Timestamp is assumed to be
+   * now.
    * 
-   * @param lockid
+   * @param row
    */
-  public BatchUpdate(long lockid) {
-    this.row = new Text();
-    this.lockid = Math.abs(lockid);
+  public BatchUpdate(Text row) {
+    this(row, HConstants.LATEST_TIMESTAMP);
+  }
+  
+  /**
+   * Initialize a BatchUpdate operation on a row with a specific timestamp.
+   * 
+   * @param row
+   */
+  public BatchUpdate(Text row, long timestamp){
+    this.row = row;
+    this.timestamp = timestamp;
     this.operations = new ArrayList<BatchOperation>();
   }
 
-  /** @return the lock id */
-  public long getLockid() {
-    return lockid;
-  }
   
   /** @return the row */
   public Text getRow() {
     return row;
   }
-  
-  /** 
-   * Start a batch row insertion/update.
-   * 
-   * No changes are committed until the client commits the batch operation via
-   * HClient.batchCommit().
-   * 
-   * The entire batch update can be abandoned by calling HClient.batchAbort();
-   *
-   * Callers to this method are given a handle that corresponds to the row being
-   * changed. The handle must be supplied on subsequent put or delete calls.
-   * 
-   * @param row Name of row to start update against.
-   * @return Row lockid.
+
+  /**
+   * Return the timestamp this BatchUpdate will be committed with.
    */
-  public synchronized long startUpdate(final Text row) {
-    this.row = row;
-    return this.lockid;
+  public long getTimestamp() {
+    return timestamp;
+  }
+  
+  /**
+   * Set this BatchUpdate's timestamp.
+   */  
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
   }
   
   /** 
    * Change a value for the specified column
    *
-   * @param lid lock id returned from startUpdate
    * @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 long lid, final Text column,
-      final byte val[]) {
-    if(this.lockid != lid) {
-      throw new IllegalArgumentException("invalid lockid " + lid);
-    }
+  public synchronized void put(final Text column, final byte val[]) {
     if (val == null) {
       // If null, the PUT becomes a DELETE operation.
       throw new IllegalArgumentException("Passed value cannot be null");
@@ -117,13 +110,9 @@
    * Delete the value for a column
    * Deletes the cell whose row/column/commit-timestamp match those of the
    * delete.
-   * @param lid lock id returned from startUpdate
    * @param column name of column whose value is to be deleted
    */
-  public synchronized void delete(final long lid, final Text column) {
-    if(this.lockid != lid) {
-      throw new IllegalArgumentException("invalid lockid " + lid);
-    }
+  public synchronized void delete(final Text column) {
     operations.add(new BatchOperation(column));
   }
 
@@ -144,6 +133,7 @@
 
   public void readFields(final DataInput in) throws IOException {
     row.readFields(in);
+    timestamp = in.readLong();
     int nOps = in.readInt();
     for (int i = 0; i < nOps; i++) {
       BatchOperation op = new BatchOperation();
@@ -154,6 +144,7 @@
 
   public void write(final DataOutput out) throws IOException {
     row.write(out);
+    out.writeLong(timestamp);
     out.writeInt(operations.size());
     for (BatchOperation op: operations) {
       op.write(out);

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=619605&r1=619604&r2=619605&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  7 11:51:03 2008
@@ -100,7 +100,7 @@
         // flipping the switch in StopRowFilter stopping us returning all
         // of the rest of the other store content.
         if (i == 0) {
-          long id = inc.startBatchUpdate(new Text("bbb"));
+          long id = inc.startUpdate(new Text("bbb"));
           inc.put(id, families[0], "bbb".getBytes());
           inc.commit(id);
         }

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=619605&r1=619604&r2=619605&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  7 11:51:03 2008
@@ -272,7 +272,7 @@
             break EXIT;
           }
           try {
-            long lockid = updater.startBatchUpdate(t);
+            long lockid = updater.startUpdate(t);
             try {
               updater.put(lockid, new Text(column), bytes);
               if (ts == -1) {
@@ -329,6 +329,11 @@
   public static interface Incommon {
     /**
      * @param row
+     * @throws IOException
+     */
+    public long startUpdate(Text row) throws IOException;    
+    /**
+     * @param row
      * @param column
      * @return value for row/column pair
      * @throws IOException
@@ -353,12 +358,6 @@
     public byte [][] get(Text row, Text column, long ts, int versions)
     throws IOException;
     /**
-     * @param row
-     * @return batch update identifier
-     * @throws IOException
-     */
-    public long startBatchUpdate(final Text row) throws IOException;
-    /**
      * @param lockid
      * @param column
      * @param val
@@ -439,7 +438,8 @@
     throws IOException {
       checkBatch();
       try {
-        this.region.batchUpdate(ts, batch);
+        this.batch.setTimestamp(ts);
+        this.region.batchUpdate(batch);
       } finally {
         this.batch = null;
       }
@@ -447,21 +447,18 @@
     /** {@inheritDoc} */
     public void put(long lockid, Text column, byte[] val) {
       checkBatch();
-      this.batch.put(lockid, column, val);
+      this.batch.put(column, val);
     }
     /** {@inheritDoc} */
     public void delete(long lockid, Text column) {
       checkBatch();
-      this.batch.delete(lockid, column);
+      this.batch.delete(column);
     }
     /** {@inheritDoc} */
     public void deleteAll(Text row, Text column, long ts) throws IOException {
       this.region.deleteAll(row, column, ts);
     }
-    /** {@inheritDoc} */
-    public long startBatchUpdate(Text row) {
-      return startUpdate(row);
-    }
+
     /**
      * @param row
      * @return update id
@@ -470,9 +467,8 @@
       if (this.batch != null) {
         throw new IllegalStateException("Update already in progress");
       }
-      long lockid = Math.abs(rand.nextLong());
-      this.batch = new BatchUpdate(lockid);
-      return batch.startUpdate(row);
+      this.batch = new BatchUpdate(row);
+      return 1;
     }
     /** {@inheritDoc} */
     public HScannerInterface getScanner(Text [] columns, Text firstRow,
@@ -543,7 +539,7 @@
       this.table.deleteAll(row, column, ts);
     }
     /** {@inheritDoc} */
-    public long startBatchUpdate(Text row) {
+    public long startUpdate(Text row) {
       return this.table.startUpdate(row);
     }
     /** {@inheritDoc} */

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/OOMERegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/OOMERegionServer.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/OOMERegionServer.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/OOMERegionServer.java Thu Feb  7 11:51:03 2008
@@ -45,10 +45,9 @@
     super(address, conf);
   }
   
-  @Override
-  public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
-      throws IOException {
-    super.batchUpdate(regionName, timestamp, b);
+  public void batchUpdate(Text regionName, BatchUpdate b)
+  throws IOException {
+    super.batchUpdate(regionName, b);
     for (int i = 0; i < 30; i++) {
       // Add the batch update 30 times to bring on the OOME faster.
       this.retainer.add(b);

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestDeleteAll.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestDeleteAll.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestDeleteAll.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestDeleteAll.java Thu Feb  7 11:51:03 2008
@@ -86,7 +86,7 @@
     Text colB = new Text(COLUMNS[0].toString() + "b");
     Text colC = new Text(COLUMNS[0].toString() + "c");
     Text colD = new Text(COLUMNS[0].toString());
-          
+    
     long lock = region_incommon.startUpdate(row);
     region_incommon.put(lock, colA, cellData(0, flush).getBytes());
     region_incommon.put(lock, colB, cellData(0, flush).getBytes());

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGet2.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGet2.java?rev=619605&r1=619604&r2=619605&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGet2.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestGet2.java Thu Feb  7 11:51:03 2008
@@ -111,11 +111,11 @@
       long one_second_ago = right_now - 1000;
       
       Text t = new Text("test_row");
-      long lockid = region_incommon.startBatchUpdate(t);
+      long lockid = region_incommon.startUpdate(t);
       region_incommon.put(lockid, COLUMNS[0], "old text".getBytes());
       region_incommon.commit(lockid, one_second_ago);
  
-      lockid = region_incommon.startBatchUpdate(t);
+      lockid = region_incommon.startUpdate(t);
       region_incommon.put(lockid, COLUMNS[0], "new text".getBytes());
       region_incommon.commit(lockid, right_now);
 
@@ -158,19 +158,19 @@
       Text t30 = new Text("030");
       Text t40 = new Text("040");
       
-      long lockid = region_incommon.startBatchUpdate(t10);
+      long lockid = region_incommon.startUpdate(t10);
       region_incommon.put(lockid, COLUMNS[0], "t10 bytes".getBytes());
       region_incommon.commit(lockid);
       
-      lockid = region_incommon.startBatchUpdate(t20);
+      lockid = region_incommon.startUpdate(t20);
       region_incommon.put(lockid, COLUMNS[0], "t20 bytes".getBytes());
       region_incommon.commit(lockid);
       
-      lockid = region_incommon.startBatchUpdate(t30);
+      lockid = region_incommon.startUpdate(t30);
       region_incommon.put(lockid, COLUMNS[0], "t30 bytes".getBytes());
       region_incommon.commit(lockid);
       
-      lockid = region_incommon.startBatchUpdate(t40);
+      lockid = region_incommon.startUpdate(t40);
       region_incommon.put(lockid, COLUMNS[0], "t40 bytes".getBytes());
       region_incommon.commit(lockid);
       

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=619605&r1=619604&r2=619605&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  7 11:51:03 2008
@@ -298,7 +298,7 @@
   private void put(final Incommon loader, final byte [] bytes,
     final long ts)
   throws IOException {
-    long lockid = loader.startBatchUpdate(ROW);
+    long lockid = loader.startUpdate(ROW);
     loader.put(lockid, COLUMN, bytes);
     if (ts == HConstants.LATEST_TIMESTAMP) {
       loader.commit(lockid);
@@ -312,7 +312,7 @@
   }
 
   private void delete(final Incommon loader, final long ts) throws IOException {
-    long lockid = loader.startBatchUpdate(ROW);
+    long lockid = loader.startUpdate(ROW);
     loader.delete(lockid, COLUMN);
     if (ts == HConstants.LATEST_TIMESTAMP) {
       loader.commit(lockid);