You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jx...@apache.org on 2012/08/15 17:44:47 UTC

svn commit: r1373481 - in /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase: client/HTableInterface.java coprocessor/CoprocessorHost.java rest/client/RemoteHTable.java

Author: jxiang
Date: Wed Aug 15 15:44:46 2012
New Revision: 1373481

URL: http://svn.apache.org/viewvc?rev=1373481&view=rev
Log:
HBASE-5728 Methods Missing in HTableInterface

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java?rev=1373481&r1=1373480&r2=1373481&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java Wed Aug 15 15:44:46 2012
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.client;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -31,8 +32,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
 
-import java.util.Map;
-
 /**
  * Used to communicate with a single HBase table.
  *
@@ -493,4 +492,60 @@ public interface HTableInterface extends
       Class<T> protocol, byte[] startKey, byte[] endKey,
       Batch.Call<T,R> callable, Batch.Callback<R> callback)
       throws IOException, Throwable;
+
+  /**
+   * See {@link #setAutoFlush(boolean, boolean)}
+   *
+   * @param autoFlush
+   *          Whether or not to enable 'auto-flush'.
+   */
+  public void setAutoFlush(boolean autoFlush);
+
+  /**
+   * Turns 'auto-flush' on or off.
+   * <p>
+   * When enabled (default), {@link Put} operations don't get buffered/delayed
+   * and are immediately executed. Failed operations are not retried. This is
+   * slower but safer.
+   * <p>
+   * Turning off {@link #autoFlush} means that multiple {@link Put}s will be
+   * accepted before any RPC is actually sent to do the write operations. If the
+   * application dies before pending writes get flushed to HBase, data will be
+   * lost.
+   * <p>
+   * When you turn {@link #autoFlush} off, you should also consider the
+   * {@link #clearBufferOnFail} option. By default, asynchronous {@link Put}
+   * requests will be retried on failure until successful. However, this can
+   * pollute the writeBuffer and slow down batching performance. Additionally,
+   * you may want to issue a number of Put requests and call
+   * {@link #flushCommits()} as a barrier. In both use cases, consider setting
+   * clearBufferOnFail to true to erase the buffer after {@link #flushCommits()}
+   * has been called, regardless of success.
+   *
+   * @param autoFlush
+   *          Whether or not to enable 'auto-flush'.
+   * @param clearBufferOnFail
+   *          Whether to keep Put failures in the writeBuffer
+   * @see #flushCommits
+   */
+  public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail);
+
+  /**
+   * Returns the maximum size in bytes of the write buffer for this HTable.
+   * <p>
+   * The default value comes from the configuration parameter
+   * {@code hbase.client.write.buffer}.
+   * @return The size of the write buffer in bytes.
+   */
+  public long getWriteBufferSize();
+
+  /**
+   * Sets the size of the buffer in bytes.
+   * <p>
+   * If the new size is less than the current amount of data in the
+   * write buffer, the buffer gets flushed.
+   * @param writeBufferSize The new write buffer size, in bytes.
+   * @throws IOException if a remote or network exception occurs.
+   */
+  public void setWriteBufferSize(long writeBufferSize) throws IOException;
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java?rev=1373481&r1=1373480&r2=1373481&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java Wed Aug 15 15:44:46 2012
@@ -541,6 +541,26 @@ public abstract class CoprocessorHost<E 
       public void mutateRow(RowMutations rm) throws IOException {
         table.mutateRow(rm);
       }
+
+      @Override
+      public void setAutoFlush(boolean autoFlush) {
+        table.setAutoFlush(autoFlush);
+      }
+
+      @Override
+      public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
+        table.setAutoFlush(autoFlush, clearBufferOnFail);
+      }
+
+      @Override
+      public long getWriteBufferSize() {
+         return table.getWriteBufferSize();
+      }
+
+      @Override
+      public void setWriteBufferSize(long writeBufferSize) throws IOException {
+        table.setWriteBufferSize(writeBufferSize);
+      }
     }
 
     /** The coprocessor */

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java?rev=1373481&r1=1373480&r2=1373481&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java Wed Aug 15 15:44:46 2012
@@ -737,4 +737,24 @@ public class RemoteHTable implements HTa
   public void mutateRow(RowMutations rm) throws IOException {
     throw new IOException("atomicMutation not supported");
   }
+
+  @Override
+  public void setAutoFlush(boolean autoFlush) {
+    throw new UnsupportedOperationException("setAutoFlush not implemented");
+  }
+
+  @Override
+  public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
+    throw new UnsupportedOperationException("setAutoFlush not implemented");
+  }
+
+  @Override
+  public long getWriteBufferSize() {
+    throw new UnsupportedOperationException("getWriteBufferSize not implemented");
+  }
+
+  @Override
+  public void setWriteBufferSize(long writeBufferSize) throws IOException {
+    throw new IOException("setWriteBufferSize not supported");
+  }
 }