You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zy...@apache.org on 2018/07/10 04:38:43 UTC

[21/50] [abbrv] hbase git commit: HBASE-20812 Add defaults to Table Interface so implementors don't have to Signed-off-by: Mike Drob

HBASE-20812 Add defaults to Table Interface so implementors don't have to
Signed-off-by: Mike Drob <md...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/632aaef8
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/632aaef8
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/632aaef8

Branch: refs/heads/HBASE-18477
Commit: 632aaef88b16cedaf30c213e4a5d3fd8bd58c3ce
Parents: 85b41f3
Author: Michael Stack <st...@apache.org>
Authored: Fri Jun 29 09:54:07 2018 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Fri Jun 29 11:05:26 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Table.java   | 303 ++++++++++++-------
 1 file changed, 193 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/632aaef8/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
index fab439c..191d359 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -43,7 +44,7 @@ import com.google.protobuf.ServiceException;
  * Used to communicate with a single HBase table.
  * Obtain an instance from a {@link Connection} and call {@link #close()} afterwards.
  *
- * <p>Table can be used to get, put, delete or scan data from a table.
+ * <p><code>Table</code> can be used to get, put, delete or scan data from a table.
  * @see ConnectionFactory
  * @see Connection
  * @see Admin
@@ -94,7 +95,9 @@ public interface Table extends Closeable {
    * @return true if the specified Get matches one or more keys, false if not
    * @throws IOException e
    */
-  boolean exists(Get get) throws IOException;
+  default boolean exists(Get get) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Test for the existence of columns in the table, as specified by the Gets.
@@ -111,7 +114,9 @@ public interface Table extends Closeable {
    * @return Array of boolean.  True if the specified Get matches one or more keys, false if not.
    * @throws IOException e
    */
-  boolean[] exists(List<Get> gets) throws IOException;
+  default boolean[] exists(List<Get> gets) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Test for the existence of columns in the table, as specified by the Gets.
@@ -145,46 +150,55 @@ public interface Table extends Closeable {
    * @throws IOException
    * @since 0.90.0
    */
-  void batch(final List<? extends Row> actions, final Object[] results) throws IOException,
-    InterruptedException;
+  default void batch(final List<? extends Row> actions, final Object[] results) throws IOException,
+    InterruptedException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Same as {@link #batch(List, Object[])}, but with a callback.
    * @since 0.96.0
    */
-  <R> void batchCallback(
-    final List<? extends Row> actions, final Object[] results, final Batch.Callback<R> callback
-  ) throws IOException, InterruptedException;
+  default <R> void batchCallback(
+    final List<? extends Row> actions, final Object[] results, final Batch.Callback<R> callback)
+      throws IOException, InterruptedException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Extracts certain cells from a given row.
    * @param get The object that specifies what data to fetch and from which row.
    * @return The data coming from the specified row, if it exists.  If the row
-   * specified doesn't exist, the {@link Result} instance returned won't
-   * contain any {@link org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}.
+   *   specified doesn't exist, the {@link Result} instance returned won't
+   *   contain any {@link org.apache.hadoop.hbase.KeyValue}, as indicated by
+   *   {@link Result#isEmpty()}.
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
    */
-  Result get(Get get) throws IOException;
+  default Result get(Get get) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Extracts specified cells from the given rows, as a batch.
    *
    * @param gets The objects that specify what data to fetch and from which rows.
    * @return The data coming from the specified rows, if it exists.  If the row specified doesn't
-   * exist, the {@link Result} instance returned won't contain any {@link
-   * org.apache.hadoop.hbase.Cell}s, as indicated by {@link Result#isEmpty()}. If there are any
-   * failures even after retries, there will be a <code>null</code> in the results' array for those
-   * Gets, AND an exception will be thrown. The ordering of the Result array corresponds to the order
-   * of the list of passed in Gets.
+   *   exist, the {@link Result} instance returned won't contain any
+   *   {@link org.apache.hadoop.hbase.Cell}s, as indicated by {@link Result#isEmpty()}. If there
+   *   are any failures even after retries, there will be a <code>null</code> in the results' array
+   *   for  those Gets, AND an exception will be thrown. The ordering of the Result array
+   *   corresponds to  the order of the list of passed in Gets.
    * @throws IOException if a remote or network exception occurs.
    * @since 0.90.0
    * @apiNote {@link #put(List)} runs pre-flight validations on the input list on client.
-   * Currently {@link #get(List)} doesn't run any validations on the client-side, currently there
-   * is no need, but this may change in the future. An
+   *   Currently {@link #get(List)} doesn't run any validations on the client-side, currently there
+   *   is no need, but this may change in the future. An
    * {@link IllegalArgumentException} will be thrown in this case.
    */
-  Result[] get(List<Get> gets) throws IOException;
+  default Result[] get(List<Get> gets) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Returns a scanner on the current table as specified by the {@link Scan}
@@ -197,7 +211,9 @@ public interface Table extends Closeable {
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
    */
-  ResultScanner getScanner(Scan scan) throws IOException;
+  default ResultScanner getScanner(Scan scan) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Gets a scanner on the current table for the given family.
@@ -207,7 +223,9 @@ public interface Table extends Closeable {
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
    */
-  ResultScanner getScanner(byte[] family) throws IOException;
+  default ResultScanner getScanner(byte[] family) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Gets a scanner on the current table for the given family and qualifier.
@@ -218,7 +236,9 @@ public interface Table extends Closeable {
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
    */
-  ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException;
+  default ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
 
   /**
@@ -228,7 +248,9 @@ public interface Table extends Closeable {
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
    */
-  void put(Put put) throws IOException;
+  default void put(Put put) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Batch puts the specified data into the table.
@@ -246,7 +268,9 @@ public interface Table extends Closeable {
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
    */
-  void put(List<Put> puts) throws IOException;
+  default void put(List<Put> puts) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected
@@ -263,8 +287,10 @@ public interface Table extends Closeable {
    * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
    */
   @Deprecated
-  boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-      byte[] value, Put put) throws IOException;
+  default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put)
+      throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected
@@ -287,8 +313,10 @@ public interface Table extends Closeable {
    * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
    */
   @Deprecated
-  boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
-      CompareFilter.CompareOp compareOp, byte[] value, Put put) throws IOException;
+  default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
+      CompareFilter.CompareOp compareOp, byte[] value, Put put) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected
@@ -311,8 +339,10 @@ public interface Table extends Closeable {
    * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
    */
   @Deprecated
-  boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
-      byte[] value, Put put) throws IOException;
+  default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
+      byte[] value, Put put) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Deletes the specified cells/row.
@@ -321,7 +351,9 @@ public interface Table extends Closeable {
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
    */
-  void delete(Delete delete) throws IOException;
+  default void delete(Delete delete) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Batch Deletes the specified cells/rows from the table.
@@ -346,7 +378,9 @@ public interface Table extends Closeable {
    * but this may change in the future. An * {@link IllegalArgumentException} will be thrown
    * in this case.
    */
-  void delete(List<Delete> deletes) throws IOException;
+  default void delete(List<Delete> deletes) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected
@@ -363,8 +397,10 @@ public interface Table extends Closeable {
    * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
    */
   @Deprecated
-  boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-    byte[] value, Delete delete) throws IOException;
+  default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+    byte[] value, Delete delete) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected
@@ -387,8 +423,10 @@ public interface Table extends Closeable {
    * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
    */
   @Deprecated
-  boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-    CompareFilter.CompareOp compareOp, byte[] value, Delete delete) throws IOException;
+  default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+    CompareFilter.CompareOp compareOp, byte[] value, Delete delete) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected
@@ -411,8 +449,10 @@ public interface Table extends Closeable {
    * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
    */
   @Deprecated
-  boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
-                         CompareOperator op, byte[] value, Delete delete) throws IOException;
+  default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
+                         CompareOperator op, byte[] value, Delete delete) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected value. If it does, it
@@ -427,7 +467,10 @@ public interface Table extends Closeable {
    * </code>
    * </pre>
    */
-  CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family);
+  default CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
+    throw new NotImplementedException("Add an implementation!");
+  }
+
   /**
    * A helper class for sending checkAndMutate request.
    */
@@ -478,7 +521,6 @@ public interface Table extends Closeable {
      * @return true if the new mutation was executed, false otherwise.
      */
     boolean thenMutate(RowMutations mutation) throws IOException;
-
   }
 
   /**
@@ -488,7 +530,9 @@ public interface Table extends Closeable {
    * @param rm object that specifies the set of mutations to perform atomically
    * @throws IOException
    */
-  void mutateRow(final RowMutations rm) throws IOException;
+  default void mutateRow(final RowMutations rm) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Appends values to one or more columns within a single row.
@@ -502,7 +546,9 @@ public interface Table extends Closeable {
    * @throws IOException e
    * @return values of columns after the append operation (maybe null)
    */
-  Result append(final Append append) throws IOException;
+  default Result append(final Append append) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Increments one or more columns within a single row.
@@ -516,7 +562,9 @@ public interface Table extends Closeable {
    * @throws IOException e
    * @return values of columns after the increment
    */
-  Result increment(final Increment increment) throws IOException;
+  default Result increment(final Increment increment) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
@@ -530,8 +578,10 @@ public interface Table extends Closeable {
    * @return The new value, post increment.
    * @throws IOException if a remote or network exception occurs.
    */
-  long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-    long amount) throws IOException;
+  default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount)
+      throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically increments a column value. If the column value already exists
@@ -550,8 +600,10 @@ public interface Table extends Closeable {
    * @return The new value, post increment.
    * @throws IOException if a remote or network exception occurs.
    */
-  long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-    long amount, Durability durability) throws IOException;
+  default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
+    long amount, Durability durability) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Releases any resources held or pending changes in internal buffers.
@@ -559,7 +611,9 @@ public interface Table extends Closeable {
    * @throws IOException if a remote or network exception occurs.
    */
   @Override
-  void close() throws IOException;
+  default void close() throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the
@@ -588,7 +642,9 @@ public interface Table extends Closeable {
    * @param row The row key used to identify the remote region location
    * @return A CoprocessorRpcChannel instance
    */
-  CoprocessorRpcChannel coprocessorService(byte[] row);
+  default CoprocessorRpcChannel coprocessorService(byte[] row) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
@@ -598,21 +654,23 @@ public interface Table extends Closeable {
    *
    * @param service the protocol buffer {@code Service} implementation to call
    * @param startKey start region selection with region containing this row.  If {@code null}, the
-   * selection will start with the first table region.
-   * @param endKey select regions up to and including the region containing this row. If {@code
-   * null}, selection will continue through the last table region.
-   * @param callable this instance's {@link org.apache.hadoop.hbase.client.coprocessor.Batch
-   * .Call#call}
-   * method will be invoked once per table region, using the {@link com.google.protobuf.Service}
-   * instance connected to that region.
+   *   selection will start with the first table region.
+   * @param endKey select regions up to and including the region containing this row. If
+   *   {@code null}, selection will continue through the last table region.
+   * @param callable this instance's
+   *   {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call}
+   *   method will be invoked once per table region, using the {@link com.google.protobuf.Service}
+   *   instance connected to that region.
    * @param <T> the {@link com.google.protobuf.Service} subclass to connect to
    * @param <R> Return type for the {@code callable} parameter's {@link
    * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
    * @return a map of result values keyed by region name
    */
-  <T extends Service, R> Map<byte[],R> coprocessorService(final Class<T> service,
+  default <T extends Service, R> Map<byte[],R> coprocessorService(final Class<T> service,
     byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable)
-    throws ServiceException, Throwable;
+    throws ServiceException, Throwable {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
@@ -620,27 +678,29 @@ public interface Table extends Closeable {
    * invokes the passed {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
    * with each {@link Service} instance.
    *
-   * <p> The given {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],
-   * byte[], Object)} method will be called with the return value from each region's {@link
-   * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} invocation. </p>
+   * <p> The given
+   * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}
+   * method will be called with the return value from each region's
+   * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} invocation. </p>
    *
    * @param service the protocol buffer {@code Service} implementation to call
    * @param startKey start region selection with region containing this row.  If {@code null}, the
-   * selection will start with the first table region.
-   * @param endKey select regions up to and including the region containing this row. If {@code
-   * null}, selection will continue through the last table region.
-   * @param callable this instance's {@link org.apache.hadoop.hbase.client.coprocessor.Batch
-   * .Call#call}
-   * method will be invoked once per table region, using the {@link Service} instance connected to
-   * that region.
-   * @param callback
+   *   selection will start with the first table region.
+   * @param endKey select regions up to and including the region containing this row. If
+   *   {@code null}, selection will continue through the last table region.
+   * @param callable this instance's
+   *   {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call}
+   *   method will be invoked once per table region, using the {@link Service} instance connected to
+   *   that region.
    * @param <T> the {@link Service} subclass to connect to
    * @param <R> Return type for the {@code callable} parameter's {@link
    * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
    */
-  <T extends Service, R> void coprocessorService(final Class<T> service,
+  default <T extends Service, R> void coprocessorService(final Class<T> service,
     byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable,
-    final Batch.Callback<R> callback) throws ServiceException, Throwable;
+    final Batch.Callback<R> callback) throws ServiceException, Throwable {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
@@ -662,13 +722,13 @@ public interface Table extends Closeable {
    *          the proto type of the response of the method in Service.
    * @param <R>
    *          the response type for the coprocessor Service method
-   * @throws ServiceException
-   * @throws Throwable
    * @return a map of result values keyed by region name
    */
-  <R extends Message> Map<byte[], R> batchCoprocessorService(
+  default <R extends Message> Map<byte[], R> batchCoprocessorService(
     Descriptors.MethodDescriptor methodDescriptor, Message request,
-    byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable;
+    byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
@@ -682,28 +742,23 @@ public interface Table extends Closeable {
    * method will be called with the return value from each region's invocation.
    * </p>
    *
-   * @param methodDescriptor
-   *          the descriptor for the protobuf service method to call.
-   * @param request
-   *          the method call parameters
-   * @param startKey
-   *          start region selection with region containing this row. If {@code null}, the
-   *          selection will start with the first table region.
-   * @param endKey
-   *          select regions up to and including the region containing this row. If {@code null},
-   *          selection will continue through the last table region.
-   * @param responsePrototype
-   *          the proto type of the response of the method in Service.
-   * @param callback
-   *          callback to invoke with the response for each region
+   * @param methodDescriptor the descriptor for the protobuf service method to call.
+   * @param request the method call parameters
+   * @param startKey start region selection with region containing this row.
+   *   If {@code null}, the selection will start with the first table region.
+   * @param endKey select regions up to and including the region containing this row.
+   *   If {@code null}, selection will continue through the last table region.
+   * @param responsePrototype the proto type of the response of the method in Service.
+   * @param callback callback to invoke with the response for each region
    * @param <R>
    *          the response type for the coprocessor Service method
-   * @throws ServiceException
-   * @throws Throwable
    */
-  <R extends Message> void batchCoprocessorService(Descriptors.MethodDescriptor methodDescriptor,
-    Message request, byte[] startKey, byte[] endKey, R responsePrototype,
-    Batch.Callback<R> callback) throws ServiceException, Throwable;
+  default <R extends Message> void batchCoprocessorService(
+      Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey,
+      byte[] endKey, R responsePrototype, Batch.Callback<R> callback)
+      throws ServiceException, Throwable {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected value.
@@ -726,8 +781,10 @@ public interface Table extends Closeable {
    * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
    */
   @Deprecated
-  boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
-      CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutation) throws IOException;
+  default boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
+      CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutation) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Atomically checks if a row/family/qualifier value matches the expected value.
@@ -750,8 +807,10 @@ public interface Table extends Closeable {
    * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
    */
   @Deprecated
-  boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
-                         byte[] value, RowMutations mutation) throws IOException;
+  default boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
+                         byte[] value, RowMutations mutation) throws IOException {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Get timeout of each rpc request in this Table instance. It will be overridden by a more
@@ -761,7 +820,9 @@ public interface Table extends Closeable {
    * @param unit the unit of time the timeout to be represented in
    * @return rpc timeout in the specified time unit
    */
-  long getRpcTimeout(TimeUnit unit);
+  default long getRpcTimeout(TimeUnit unit) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Get timeout (millisecond) of each rpc request in this Table instance.
@@ -771,7 +832,9 @@ public interface Table extends Closeable {
    *             {@link #getWriteRpcTimeout(TimeUnit)} instead
    */
   @Deprecated
-  int getRpcTimeout();
+  default int getRpcTimeout() {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Set timeout (millisecond) of each rpc request in operations of this Table instance, will
@@ -786,14 +849,18 @@ public interface Table extends Closeable {
    * @deprecated Use setReadRpcTimeout or setWriteRpcTimeout instead
    */
   @Deprecated
-  void setRpcTimeout(int rpcTimeout);
+  default void setRpcTimeout(int rpcTimeout) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Get timeout of each rpc read request in this Table instance.
    * @param unit the unit of time the timeout to be represented in
    * @return read rpc timeout in the specified time unit
    */
-  long getReadRpcTimeout(TimeUnit unit);
+  default long getReadRpcTimeout(TimeUnit unit) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Get timeout (millisecond) of each rpc read request in this Table instance.
@@ -801,7 +868,9 @@ public interface Table extends Closeable {
    *             use {@link #getReadRpcTimeout(TimeUnit)} instead
    */
   @Deprecated
-  int getReadRpcTimeout();
+  default int getReadRpcTimeout() {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Set timeout (millisecond) of each rpc read request in operations of this Table instance, will
@@ -813,14 +882,18 @@ public interface Table extends Closeable {
    * @deprecated since 2.0.0, use {@link TableBuilder#setReadRpcTimeout} instead
    */
   @Deprecated
-  void setReadRpcTimeout(int readRpcTimeout);
+  default void setReadRpcTimeout(int readRpcTimeout) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Get timeout of each rpc write request in this Table instance.
    * @param unit the unit of time the timeout to be represented in
    * @return write rpc timeout in the specified time unit
    */
-  long getWriteRpcTimeout(TimeUnit unit);
+  default long getWriteRpcTimeout(TimeUnit unit) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Get timeout (millisecond) of each rpc write request in this Table instance.
@@ -828,7 +901,9 @@ public interface Table extends Closeable {
    *             use {@link #getWriteRpcTimeout(TimeUnit)} instead
    */
   @Deprecated
-  int getWriteRpcTimeout();
+  default int getWriteRpcTimeout() {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Set timeout (millisecond) of each rpc write request in operations of this Table instance, will
@@ -840,14 +915,18 @@ public interface Table extends Closeable {
    * @deprecated since 2.0.0, use {@link TableBuilder#setWriteRpcTimeout} instead
    */
   @Deprecated
-  void setWriteRpcTimeout(int writeRpcTimeout);
+  default void setWriteRpcTimeout(int writeRpcTimeout) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Get timeout of each operation in Table instance.
    * @param unit the unit of time the timeout to be represented in
    * @return operation rpc timeout in the specified time unit
    */
-  long getOperationTimeout(TimeUnit unit);
+  default long getOperationTimeout(TimeUnit unit) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Get timeout (millisecond) of each operation for in Table instance.
@@ -855,7 +934,9 @@ public interface Table extends Closeable {
    *             use {@link #getOperationTimeout(TimeUnit)} instead
    */
   @Deprecated
-  int getOperationTimeout();
+  default int getOperationTimeout() {
+    throw new NotImplementedException("Add an implementation!");
+  }
 
   /**
    * Set timeout (millisecond) of each operation in this Table instance, will override the value
@@ -869,5 +950,7 @@ public interface Table extends Closeable {
    * @deprecated since 2.0.0, use {@link TableBuilder#setOperationTimeout} instead
    */
   @Deprecated
-  void setOperationTimeout(int operationTimeout);
+  default void setOperationTimeout(int operationTimeout) {
+    throw new NotImplementedException("Add an implementation!");
+  }
 }