You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2007/09/06 00:35:15 UTC

svn commit: r573088 - in /lucene/hadoop/trunk/src/contrib/hbase: CHANGES.txt src/java/org/apache/hadoop/hbase/HRegionInterface.java src/java/org/apache/hadoop/hbase/HRegionServer.java

Author: jimk
Date: Wed Sep  5 15:35:13 2007
New Revision: 573088

URL: http://svn.apache.org/viewvc?rev=573088&view=rev
Log:
HADOOP-1794 Remove deprecated APIs

Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
    lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java

Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?rev=573088&r1=573087&r2=573088&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Wed Sep  5 15:35:13 2007
@@ -39,6 +39,7 @@
                 filter types
     HADOOP-1760 Use new MapWritable and SortedMapWritable classes from
                 org.apache.hadoop.io
+    HADOOP-1794 Remove deprecated APIs
     HADOOP-1802 Startup scripts should wait until hdfs as cleared 'safe mode'
     HADOOP-1835 Updated Documentation for HBase setup/installation
            (Izaak Rubin via Stack)

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java?rev=573088&r1=573087&r2=573088&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java Wed Sep  5 15:35:13 2007
@@ -28,10 +28,9 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.VersionedProtocol;
 
-/*******************************************************************************
- * Clients interact with HRegionServers using
- * a handle to the HRegionInterface.
- ******************************************************************************/
+/**
+ * Clients interact with HRegionServers using a handle to the HRegionInterface.
+ */
 public interface HRegionInterface extends VersionedProtocol {
   /** initial version */
   public static final long versionID = 1L;
@@ -100,120 +99,19 @@
   public MapWritable getRow(final Text regionName, final Text row)
   throws IOException;
 
-  //////////////////////////////////////////////////////////////////////////////
-  // 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.
-  //
-  // Callers to this method are given a lease for each unique lockid; before the
-  // lease expires, either abort() or commit() must be called. If it is not 
-  // called, the system will automatically call abort() on the client's behalf.
-  //
-  // The client can gain extra time with a call to renewLease().
-  //////////////////////////////////////////////////////////////////////////////
-
-  /** 
-   * 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.
-   *
-   * Callers to this method are given a lease for each unique lockid; before the
-   * lease expires, either abort() or commit() must be called. If it is not 
-   * called, the system will automatically call abort() on the client's behalf.
-   *
-   * The client can gain extra time with a call to renewLease().
-   * Start an atomic row insertion or update
-   * 
-   * @param regionName region name
-   * @param clientid a unique value to identify the client
-   * @param row Name of row to start update against.
-   * @return Row lockid.
-   * @throws IOException
-   * 
-   * Deprecated. Use @see {@link #batchUpdate(Text, long, BatchUpdate)} instead.
-   */
-  @Deprecated
-  public long startUpdate(final Text regionName, final long clientid,
-      final Text row)
-  throws IOException;
-  
-  /** 
-   * Change a value for the specified column
-   *
-   * @param regionName region name
-   * @param clientid a unique value to identify the client
-   * @param lockid lock id returned from startUpdate
-   * @param column column whose value is being set
-   * @param val new value for column
-   * @throws IOException
-   * 
-   * Deprecated. Use @see {@link #batchUpdate(Text, long, BatchUpdate)} instead.
-   */
-  @Deprecated
-  public void put(final Text regionName, final long clientid, final long lockid,
-      final Text column, final byte [] val)
-  throws IOException;
-  
-  /** 
-   * Delete the value for a column
-   *
-   * @param regionName region name
-   * @param clientid a unique value to identify the client
-   * @param lockid lock id returned from startUpdate
-   * @param column name of column whose value is to be deleted
-   * @throws IOException
-   * 
-   * Deprecated. Use @see {@link #batchUpdate(Text, long, BatchUpdate)} instead.
-   */
-  @Deprecated
-  public void delete(final Text regionName, final long clientid,
-      final long lockid, final Text column)
-  throws IOException;
-  
-  /** 
-   * Abort a row mutation
-   *
-   * @param regionName region name
-   * @param clientid a unique value to identify the client
-   * @param lockid lock id returned from startUpdate
-   * @throws IOException
-   * 
-   * Deprecated. Use @see {@link #batchUpdate(Text, long, BatchUpdate)} instead.
-   */
-  @Deprecated
-  public void abort(final Text regionName, final long clientid, 
-      final long lockid)
-  throws IOException;
-  
-  /** 
-   * Finalize a row mutation
-   *
-   * @param regionName region name
-   * @param clientid a unique value to identify the client
-   * @param lockid lock id returned from startUpdate
-   * @param timestamp the time (in milliseconds to associate with this change)
-   * @throws IOException
-   * 
-   * Deprecated. Use @see {@link #batchUpdate(Text, long, BatchUpdate)} instead.
-   */
-  @Deprecated
-  public void commit(final Text regionName, final long clientid,
-      final long lockid, final long timestamp)
-  throws IOException;
-  
   /**
-   * Renew lease on update
+   * Applies a batch of updates via one RPC
    * 
-   * @param lockid lock id returned from startUpdate
-   * @param clientid a unique value to identify the client
+   * @param regionName name of the region to update
+   * @param timestamp the time to be associated with the changes
+   * @param b BatchUpdate
    * @throws IOException
-   * 
-   * Deprecated. Use @see {@link #batchUpdate(Text, long, BatchUpdate)} instead.
    */
-  @Deprecated
-  public void renewLease(long lockid, long clientid) throws IOException;
-
-  //////////////////////////////////////////////////////////////////////////////
+  public void batchUpdate(Text regionName, long timestamp, BatchUpdate b) throws IOException;
+  
+  //
   // remote scanner interface
-  //////////////////////////////////////////////////////////////////////////////
+  //
 
   /**
    * Opens a remote scanner with a RowFilter.
@@ -231,16 +129,6 @@
       long timestamp, RowFilterInterface filter)
   throws IOException;
 
-  /**
-   * Applies a batch of updates via one RPC
-   * 
-   * @param regionName name of the region to update
-   * @param timestamp the time to be associated with the changes
-   * @param b BatchUpdate
-   * @throws IOException
-   */
-  public void batchUpdate(Text regionName, long timestamp, BatchUpdate b) throws IOException;
-  
   /**
    * Get the next set of values
    * 

Modified: lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?rev=573088&r1=573087&r2=573088&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java Wed Sep  5 15:35:13 2007
@@ -985,26 +985,6 @@
   }
 
   /** {@inheritDoc} */
-  public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
-  throws IOException {
-    requestCount.incrementAndGet();
-    long clientid = rand.nextLong();
-    long lockid = startUpdate(regionName, clientid, b.getRow());
-    for(BatchOperation op: b) {
-      switch(op.getOp()) {
-      case BatchOperation.PUT_OP:
-        put(regionName, clientid, lockid, op.getColumn(), op.getValue());
-        break;
-
-      case BatchOperation.DELETE_OP:
-        delete(regionName, clientid, lockid, op.getColumn());
-        break;
-      }
-    }
-    commit(regionName, clientid, lockid, timestamp);
-  }
-  
-  /** {@inheritDoc} */
   public byte [] get(final Text regionName, final Text row,
       final Text column)
   throws IOException {
@@ -1083,102 +1063,51 @@
     return values;
   }
 
-  /*
-   * NOTE: When startUpdate, put, delete, abort, commit and renewLease are
-   * removed from HRegionInterface, these methods (with the exception of
-   * renewLease) must remain, as they are called by batchUpdate (renewLease
-   * can just be removed)
-   * 
-   * However, the remaining methods can become protected instead of public
-   * at that point.
-   */
-
-  /** Create a lease for an update. If it times out, the update is aborted */
-  private static class RegionListener implements LeaseListener {
-    private HRegion localRegion;
-    private long localLockId;
-    
-    RegionListener(HRegion region, long lockId) {
-      this.localRegion = region;
-      this.localLockId = lockId;
-    }
-    
-    /** {@inheritDoc} */
-    public void leaseExpired() {
-      try {
-        localRegion.abort(localLockId);
-      } catch (IOException iex) {
-        if (iex instanceof RemoteException) {
-          try {
-            iex = RemoteExceptionHandler.decodeRemoteException((RemoteException) iex);
-            
-          } catch (IOException x) {
-            iex = x;
-          }
-        }
-        LOG.error("", iex);
+  /** {@inheritDoc} */
+  public void batchUpdate(Text regionName, long timestamp, BatchUpdate b)
+  throws IOException {
+    requestCount.incrementAndGet();
+    long lockid = startUpdate(regionName, b.getRow());
+    for(BatchOperation op: b) {
+      switch(op.getOp()) {
+      case BatchOperation.PUT_OP:
+        put(regionName, lockid, op.getColumn(), op.getValue());
+        break;
+
+      case BatchOperation.DELETE_OP:
+        delete(regionName, lockid, op.getColumn());
+        break;
       }
     }
+    commit(regionName, lockid, timestamp);
   }
   
-  /** {@inheritDoc} */
-  @SuppressWarnings("deprecation")
-  public long startUpdate(Text regionName, long clientid, Text row) 
+  protected long startUpdate(Text regionName, Text row) 
       throws IOException {
-    requestCount.incrementAndGet();
+    
     HRegion region = getRegion(regionName);
-    long lockid = region.startUpdate(row);
-    this.leases.createLease(clientid, lockid,
-      new RegionListener(region, lockid));
-    return lockid;
+    return region.startUpdate(row);
   }
 
-  /** {@inheritDoc} */
-  @SuppressWarnings("deprecation")
-  public void put(final Text regionName, final long clientid,
-      final long lockid, final Text column, final byte [] val)
-  throws IOException {
-    requestCount.incrementAndGet();
+  protected void put(final Text regionName, final long lockid,
+      final Text column, final byte [] val) throws IOException {
+
     HRegion region = getRegion(regionName, true);
-    leases.renewLease(clientid, lockid);
     region.put(lockid, column, val);
   }
 
-  /** {@inheritDoc} */
-  @SuppressWarnings("deprecation")
-  public void delete(Text regionName, long clientid, long lockid, Text column) 
+  protected void delete(Text regionName, long lockid, Text column) 
   throws IOException {
-    requestCount.incrementAndGet();
+
     HRegion region = getRegion(regionName);
-    leases.renewLease(clientid, lockid);
     region.delete(lockid, column);
   }
 
-  /** {@inheritDoc} */
-  @SuppressWarnings("deprecation")
-  public void abort(Text regionName, long clientid, long lockid) 
-  throws IOException {
-    requestCount.incrementAndGet();
-    HRegion region = getRegion(regionName, true);
-    leases.cancelLease(clientid, lockid);
-    region.abort(lockid);
-  }
-
-  /** {@inheritDoc} */
-  @SuppressWarnings("deprecation")
-  public void commit(Text regionName, final long clientid, final long lockid,
+  protected void commit(Text regionName, final long lockid,
       final long timestamp) throws IOException {
-    requestCount.incrementAndGet();
+
     HRegion region = getRegion(regionName, true);
-    leases.cancelLease(clientid, lockid);
     region.commit(lockid, timestamp);
-  }
-
-  /** {@inheritDoc} */
-  @SuppressWarnings("deprecation")
-  public void renewLease(long lockid, long clientid) throws IOException {
-    requestCount.incrementAndGet();
-    leases.renewLease(clientid, lockid);
   }
 
   /**