001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.client;
020
021import java.io.Closeable;
022import java.io.IOException;
023import java.util.Collections;
024import java.util.List;
025import java.util.Map;
026import java.util.concurrent.TimeUnit;
027
028import org.apache.commons.lang3.NotImplementedException;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.hbase.Cell;
031import org.apache.hadoop.hbase.CompareOperator;
032import org.apache.hadoop.hbase.HTableDescriptor;
033import org.apache.hadoop.hbase.TableName;
034import org.apache.hadoop.hbase.io.TimeRange;
035import org.apache.yetus.audience.InterfaceAudience;
036
037import org.apache.hadoop.hbase.client.coprocessor.Batch;
038import org.apache.hadoop.hbase.filter.CompareFilter;
039import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
040import org.apache.hadoop.hbase.util.Bytes;
041import com.google.protobuf.Descriptors;
042import com.google.protobuf.Message;
043import com.google.protobuf.Service;
044import com.google.protobuf.ServiceException;
045
046/**
047 * Used to communicate with a single HBase table.
048 * Obtain an instance from a {@link Connection} and call {@link #close()} afterwards.
049 *
050 * <p><code>Table</code> can be used to get, put, delete or scan data from a table.
051 * @see ConnectionFactory
052 * @see Connection
053 * @see Admin
054 * @see RegionLocator
055 * @since 0.99.0
056 */
057@InterfaceAudience.Public
058public interface Table extends Closeable {
059  /**
060   * Gets the fully qualified table name instance of this table.
061   */
062  TableName getName();
063
064  /**
065   * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
066   * <p>
067   * The reference returned is not a copy, so any change made to it will
068   * affect this instance.
069   */
070  Configuration getConfiguration();
071
072  /**
073   * Gets the {@link org.apache.hadoop.hbase.HTableDescriptor table descriptor} for this table.
074   * @throws java.io.IOException if a remote or network exception occurs.
075   * @deprecated since 2.0 version and will be removed in 3.0 version.
076   *             use {@link #getDescriptor()}
077   */
078  @Deprecated
079  default HTableDescriptor getTableDescriptor() throws IOException {
080    TableDescriptor descriptor = getDescriptor();
081
082    if (descriptor instanceof HTableDescriptor) {
083      return (HTableDescriptor)descriptor;
084    } else {
085      return new HTableDescriptor(descriptor);
086    }
087  }
088
089  /**
090   * Gets the {@link org.apache.hadoop.hbase.client.TableDescriptor table descriptor} for this table.
091   * @throws java.io.IOException if a remote or network exception occurs.
092   */
093  TableDescriptor getDescriptor() throws IOException;
094
095  /**
096   * Test for the existence of columns in the table, as specified by the Get.
097   * <p>
098   *
099   * This will return true if the Get matches one or more keys, false if not.
100   * <p>
101   *
102   * This is a server-side call so it prevents any data from being transfered to
103   * the client.
104   *
105   * @param get the Get
106   * @return true if the specified Get matches one or more keys, false if not
107   * @throws IOException e
108   */
109  default boolean exists(Get get) throws IOException {
110    return exists(Collections.singletonList(get))[0];
111  }
112
113  /**
114   * Test for the existence of columns in the table, as specified by the Gets.
115   * <p>
116   *
117   * This will return an array of booleans. Each value will be true if the related Get matches
118   * one or more keys, false if not.
119   * <p>
120   *
121   * This is a server-side call so it prevents any data from being transferred to
122   * the client.
123   *
124   * @param gets the Gets
125   * @return Array of boolean.  True if the specified Get matches one or more keys, false if not.
126   * @throws IOException e
127   */
128  default boolean[] exists(List<Get> gets) throws IOException {
129    throw new NotImplementedException("Add an implementation!");
130  }
131
132  /**
133   * Test for the existence of columns in the table, as specified by the Gets.
134   * This will return an array of booleans. Each value will be true if the related Get matches
135   * one or more keys, false if not.
136   * This is a server-side call so it prevents any data from being transferred to
137   * the client.
138   *
139   * @param gets the Gets
140   * @return Array of boolean.  True if the specified Get matches one or more keys, false if not.
141   * @throws IOException e
142   * @deprecated since 2.0 version and will be removed in 3.0 version.
143   *             use {@link #exists(List)}
144   */
145  @Deprecated
146  default boolean[] existsAll(List<Get> gets) throws IOException {
147    return exists(gets);
148  }
149
150  /**
151   * Method that does a batch call on Deletes, Gets, Puts, Increments, Appends, RowMutations.
152   * The ordering of execution of the actions is not defined. Meaning if you do a Put and a
153   * Get in the same {@link #batch} call, you will not necessarily be
154   * guaranteed that the Get returns what the Put had put.
155   *
156   * @param actions list of Get, Put, Delete, Increment, Append, RowMutations.
157   * @param results Empty Object[], same size as actions. Provides access to partial
158   *                results, in case an exception is thrown. A null in the result array means that
159   *                the call for that action failed, even after retries. The order of the objects
160   *                in the results array corresponds to the order of actions in the request list.
161   * @throws IOException
162   * @since 0.90.0
163   */
164  default void batch(final List<? extends Row> actions, final Object[] results) throws IOException,
165    InterruptedException {
166    throw new NotImplementedException("Add an implementation!");
167  }
168
169  /**
170   * Same as {@link #batch(List, Object[])}, but with a callback.
171   * @since 0.96.0
172   */
173  default <R> void batchCallback(
174    final List<? extends Row> actions, final Object[] results, final Batch.Callback<R> callback)
175      throws IOException, InterruptedException {
176    throw new NotImplementedException("Add an implementation!");
177  }
178
179  /**
180   * Extracts certain cells from a given row.
181   * @param get The object that specifies what data to fetch and from which row.
182   * @return The data coming from the specified row, if it exists.  If the row
183   *   specified doesn't exist, the {@link Result} instance returned won't
184   *   contain any {@link org.apache.hadoop.hbase.KeyValue}, as indicated by
185   *   {@link Result#isEmpty()}.
186   * @throws IOException if a remote or network exception occurs.
187   * @since 0.20.0
188   */
189  default Result get(Get get) throws IOException {
190    return get(Collections.singletonList(get))[0];
191  }
192
193  /**
194   * Extracts specified cells from the given rows, as a batch.
195   *
196   * @param gets The objects that specify what data to fetch and from which rows.
197   * @return The data coming from the specified rows, if it exists.  If the row specified doesn't
198   *   exist, the {@link Result} instance returned won't contain any
199   *   {@link org.apache.hadoop.hbase.Cell}s, as indicated by {@link Result#isEmpty()}. If there
200   *   are any failures even after retries, there will be a <code>null</code> in the results' array
201   *   for  those Gets, AND an exception will be thrown. The ordering of the Result array
202   *   corresponds to  the order of the list of passed in Gets.
203   * @throws IOException if a remote or network exception occurs.
204   * @since 0.90.0
205   * @apiNote {@link #put(List)} runs pre-flight validations on the input list on client.
206   *          Currently {@link #get(List)} doesn't run any validations on the client-side,
207   *          currently there is no need, but this may change in the future. An
208   *          {@link IllegalArgumentException} will be thrown in this case.
209   */
210  default Result[] get(List<Get> gets) throws IOException {
211    throw new NotImplementedException("Add an implementation!");
212  }
213
214  /**
215   * Returns a scanner on the current table as specified by the {@link Scan}
216   * object.
217   * Note that the passed {@link Scan}'s start row and caching properties
218   * maybe changed.
219   *
220   * @param scan A configured {@link Scan} object.
221   * @return A scanner.
222   * @throws IOException if a remote or network exception occurs.
223   * @since 0.20.0
224   */
225  default ResultScanner getScanner(Scan scan) throws IOException {
226    throw new NotImplementedException("Add an implementation!");
227  }
228
229  /**
230   * Gets a scanner on the current table for the given family.
231   *
232   * @param family The column family to scan.
233   * @return A scanner.
234   * @throws IOException if a remote or network exception occurs.
235   * @since 0.20.0
236   */
237  default ResultScanner getScanner(byte[] family) throws IOException {
238    throw new NotImplementedException("Add an implementation!");
239  }
240
241  /**
242   * Gets a scanner on the current table for the given family and qualifier.
243   *
244   * @param family The column family to scan.
245   * @param qualifier The column qualifier to scan.
246   * @return A scanner.
247   * @throws IOException if a remote or network exception occurs.
248   * @since 0.20.0
249   */
250  default ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {
251    throw new NotImplementedException("Add an implementation!");
252  }
253
254  /**
255   * Puts some data in the table.
256   *
257   * @param put The data to put.
258   * @throws IOException if a remote or network exception occurs.
259   * @since 0.20.0
260   */
261  default void put(Put put) throws IOException {
262    put(Collections.singletonList(put));
263  }
264
265  /**
266   * Batch puts the specified data into the table.
267   * <p>
268   * This can be used for group commit, or for submitting user defined batches. Before sending
269   * a batch of mutations to the server, the client runs a few validations on the input list. If an
270   * error is found, for example, a mutation was supplied but was missing it's column an
271   * {@link IllegalArgumentException} will be thrown and no mutations will be applied. If there
272   * are any failures even after retries, a {@link RetriesExhaustedWithDetailsException} will be
273   * thrown. RetriesExhaustedWithDetailsException contains lists of failed mutations and
274   * corresponding remote exceptions. The ordering of mutations and exceptions in the
275   * encapsulating exception corresponds to the order of the input list of Put requests.
276   *
277   * @param puts The list of mutations to apply.
278   * @throws IOException if a remote or network exception occurs.
279   * @since 0.20.0
280   */
281  default void put(List<Put> puts) throws IOException {
282    throw new NotImplementedException("Add an implementation!");
283  }
284
285  /**
286   * Atomically checks if a row/family/qualifier value matches the expected
287   * value. If it does, it adds the put.  If the passed value is null, the check
288   * is for the lack of column (ie: non-existance)
289   *
290   * @param row to check
291   * @param family column family to check
292   * @param qualifier column qualifier to check
293   * @param value the expected value
294   * @param put data to put if check succeeds
295   * @throws IOException e
296   * @return true if the new put was executed, false otherwise
297   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
298   */
299  @Deprecated
300  default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put)
301      throws IOException {
302    return checkAndPut(row, family, qualifier, CompareOperator.EQUAL, value, put);
303  }
304
305  /**
306   * Atomically checks if a row/family/qualifier value matches the expected
307   * value. If it does, it adds the put.  If the passed value is null, the check
308   * is for the lack of column (ie: non-existence)
309   *
310   * The expected value argument of this call is on the left and the current
311   * value of the cell is on the right side of the comparison operator.
312   *
313   * Ie. eg. GREATER operator means expected value > existing <=> add the put.
314   *
315   * @param row to check
316   * @param family column family to check
317   * @param qualifier column qualifier to check
318   * @param compareOp comparison operator to use
319   * @param value the expected value
320   * @param put data to put if check succeeds
321   * @throws IOException e
322   * @return true if the new put was executed, false otherwise
323   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
324   */
325  @Deprecated
326  default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
327      CompareFilter.CompareOp compareOp, byte[] value, Put put) throws IOException {
328    RowMutations mutations = new RowMutations(put.getRow(), 1);
329    mutations.add(put);
330
331    return checkAndMutate(row, family, qualifier, compareOp, value, mutations);
332  }
333
334  /**
335   * Atomically checks if a row/family/qualifier value matches the expected
336   * value. If it does, it adds the put.  If the passed value is null, the check
337   * is for the lack of column (ie: non-existence)
338   *
339   * The expected value argument of this call is on the left and the current
340   * value of the cell is on the right side of the comparison operator.
341   *
342   * Ie. eg. GREATER operator means expected value > existing <=> add the put.
343   *
344   * @param row to check
345   * @param family column family to check
346   * @param qualifier column qualifier to check
347   * @param op comparison operator to use
348   * @param value the expected value
349   * @param put data to put if check succeeds
350   * @throws IOException e
351   * @return true if the new put was executed, false otherwise
352   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
353   */
354  @Deprecated
355  default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
356      byte[] value, Put put) throws IOException {
357    RowMutations mutations = new RowMutations(put.getRow(), 1);
358    mutations.add(put);
359
360    return checkAndMutate(row, family, qualifier, op, value, mutations);
361  }
362
363  /**
364   * Deletes the specified cells/row.
365   *
366   * @param delete The object that specifies what to delete.
367   * @throws IOException if a remote or network exception occurs.
368   * @since 0.20.0
369   */
370  default void delete(Delete delete) throws IOException {
371    throw new NotImplementedException("Add an implementation!");
372  }
373
374  /**
375   * Batch Deletes the specified cells/rows from the table.
376   * <p>
377   * If a specified row does not exist, {@link Delete} will report as though sucessful
378   * delete; no exception will be thrown. If there are any failures even after retries,
379   * a {@link RetriesExhaustedWithDetailsException} will be thrown.
380   * RetriesExhaustedWithDetailsException contains lists of failed {@link Delete}s and
381   * corresponding remote exceptions.
382   *
383   * @param deletes List of things to delete. The input list gets modified by this
384   * method. All successfully applied {@link Delete}s in the list are removed (in particular it
385   * gets re-ordered, so the order in which the elements are inserted in the list gives no
386   * guarantee as to the order in which the {@link Delete}s are executed).
387   * @throws IOException if a remote or network exception occurs. In that case
388   * the {@code deletes} argument will contain the {@link Delete} instances
389   * that have not be successfully applied.
390   * @since 0.20.1
391   * @apiNote In 3.0.0 version, the input list {@code deletes} will no longer be modified. Also,
392   *          {@link #put(List)} runs pre-flight validations on the input list on client. Currently
393   *          {@link #delete(List)} doesn't run validations on the client, there is no need
394   *          currently, but this may change in the future. An * {@link IllegalArgumentException}
395   *          will be thrown in this case.
396   */
397  default void delete(List<Delete> deletes) throws IOException {
398    throw new NotImplementedException("Add an implementation!");
399  }
400
401  /**
402   * Atomically checks if a row/family/qualifier value matches the expected
403   * value. If it does, it adds the delete.  If the passed value is null, the
404   * check is for the lack of column (ie: non-existance)
405   *
406   * @param row to check
407   * @param family column family to check
408   * @param qualifier column qualifier to check
409   * @param value the expected value
410   * @param delete data to delete if check succeeds
411   * @throws IOException e
412   * @return true if the new delete was executed, false otherwise
413   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
414   */
415  @Deprecated
416  default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
417    byte[] value, Delete delete) throws IOException {
418    return checkAndDelete(row, family, qualifier, CompareOperator.EQUAL, value, delete);
419  }
420
421  /**
422   * Atomically checks if a row/family/qualifier value matches the expected
423   * value. If it does, it adds the delete.  If the passed value is null, the
424   * check is for the lack of column (ie: non-existence)
425   *
426   * The expected value argument of this call is on the left and the current
427   * value of the cell is on the right side of the comparison operator.
428   *
429   * Ie. eg. GREATER operator means expected value > existing <=> add the delete.
430   *
431   * @param row to check
432   * @param family column family to check
433   * @param qualifier column qualifier to check
434   * @param compareOp comparison operator to use
435   * @param value the expected value
436   * @param delete data to delete if check succeeds
437   * @throws IOException e
438   * @return true if the new delete was executed, false otherwise
439   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
440   */
441  @Deprecated
442  default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
443    CompareFilter.CompareOp compareOp, byte[] value, Delete delete) throws IOException {
444    RowMutations mutations = new RowMutations(delete.getRow(), 1);
445    mutations.add(delete);
446
447    return checkAndMutate(row, family, qualifier, compareOp, value, mutations);
448  }
449
450  /**
451   * Atomically checks if a row/family/qualifier value matches the expected
452   * value. If it does, it adds the delete.  If the passed value is null, the
453   * check is for the lack of column (ie: non-existence)
454   *
455   * The expected value argument of this call is on the left and the current
456   * value of the cell is on the right side of the comparison operator.
457   *
458   * Ie. eg. GREATER operator means expected value > existing <=> add the delete.
459   *
460   * @param row to check
461   * @param family column family to check
462   * @param qualifier column qualifier to check
463   * @param op comparison operator to use
464   * @param value the expected value
465   * @param delete data to delete if check succeeds
466   * @throws IOException e
467   * @return true if the new delete was executed, false otherwise
468   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
469   */
470  @Deprecated
471  default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
472                         CompareOperator op, byte[] value, Delete delete) throws IOException {
473    RowMutations mutations = new RowMutations(delete.getRow(), 1);
474    mutations.add(delete);
475
476    return checkAndMutate(row, family, qualifier, op, value, mutations);
477  }
478
479  /**
480   * Atomically checks if a row/family/qualifier value matches the expected value. If it does, it
481   * adds the Put/Delete/RowMutations.
482   * <p>
483   * Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it.
484   * This is a fluent style API, the code is like:
485   *
486   * <pre>
487   * <code>
488   * table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put);
489   * </code>
490   * </pre>
491   */
492  default CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
493    throw new NotImplementedException("Add an implementation!");
494  }
495
496  /**
497   * A helper class for sending checkAndMutate request.
498   */
499  interface CheckAndMutateBuilder {
500
501    /**
502     * @param qualifier column qualifier to check.
503     */
504    CheckAndMutateBuilder qualifier(byte[] qualifier);
505
506    /**
507     * @param timeRange timeRange to check
508     */
509    CheckAndMutateBuilder timeRange(TimeRange timeRange);
510
511    /**
512     * Check for lack of column.
513     */
514    CheckAndMutateBuilder ifNotExists();
515
516    /**
517     * Check for equality.
518     * @param value the expected value
519     */
520    default CheckAndMutateBuilder ifEquals(byte[] value) {
521      return ifMatches(CompareOperator.EQUAL, value);
522    }
523
524    /**
525     * @param compareOp comparison operator to use
526     * @param value the expected value
527     */
528    CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
529
530    /**
531     * @param put data to put if check succeeds
532     * @return {@code true} if the new put was executed, {@code false} otherwise.
533     */
534    boolean thenPut(Put put) throws IOException;
535
536    /**
537     * @param delete data to delete if check succeeds
538     * @return {@code true} if the new delete was executed, {@code false} otherwise.
539     */
540    boolean thenDelete(Delete delete) throws IOException;
541    /**
542     * @param mutation mutations to perform if check succeeds
543     * @return true if the new mutation was executed, false otherwise.
544     */
545    boolean thenMutate(RowMutations mutation) throws IOException;
546  }
547
548  /**
549   * Performs multiple mutations atomically on a single row. Currently
550   * {@link Put} and {@link Delete} are supported.
551   *
552   * @param rm object that specifies the set of mutations to perform atomically
553   * @throws IOException
554   */
555  default void mutateRow(final RowMutations rm) throws IOException {
556    throw new NotImplementedException("Add an implementation!");
557  }
558
559  /**
560   * Appends values to one or more columns within a single row.
561   * <p>
562   * This operation guaranteed atomicity to readers. Appends are done
563   * under a single row lock, so write operations to a row are synchronized, and
564   * readers are guaranteed to see this operation fully completed.
565   *
566   * @param append object that specifies the columns and values to be appended
567   * @throws IOException e
568   * @return values of columns after the append operation (maybe null)
569   */
570  default Result append(final Append append) throws IOException {
571    throw new NotImplementedException("Add an implementation!");
572  }
573
574  /**
575   * Increments one or more columns within a single row.
576   * <p>
577   * This operation ensures atomicity to readers. Increments are done
578   * under a single row lock, so write operations to a row are synchronized, and
579   * readers are guaranteed to see this operation fully completed.
580   *
581   * @param increment object that specifies the columns and amounts to be used
582   *                  for the increment operations
583   * @throws IOException e
584   * @return values of columns after the increment
585   */
586  default Result increment(final Increment increment) throws IOException {
587    throw new NotImplementedException("Add an implementation!");
588  }
589
590  /**
591   * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
592   * <p>
593   * The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
594   * @param row The row that contains the cell to increment.
595   * @param family The column family of the cell to increment.
596   * @param qualifier The column qualifier of the cell to increment.
597   * @param amount The amount to increment the cell with (or decrement, if the
598   * amount is negative).
599   * @return The new value, post increment.
600   * @throws IOException if a remote or network exception occurs.
601   */
602  default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount)
603      throws IOException {
604    Increment increment = new Increment(row).addColumn(family, qualifier, amount);
605    Cell cell = increment(increment).getColumnLatestCell(family, qualifier);
606    return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
607  }
608
609  /**
610   * Atomically increments a column value. If the column value already exists
611   * and is not a big-endian long, this could throw an exception. If the column
612   * value does not yet exist it is initialized to <code>amount</code> and
613   * written to the specified column.
614   *
615   * <p>Setting durability to {@link Durability#SKIP_WAL} means that in a fail
616   * scenario you will lose any increments that have not been flushed.
617   * @param row The row that contains the cell to increment.
618   * @param family The column family of the cell to increment.
619   * @param qualifier The column qualifier of the cell to increment.
620   * @param amount The amount to increment the cell with (or decrement, if the
621   * amount is negative).
622   * @param durability The persistence guarantee for this increment.
623   * @return The new value, post increment.
624   * @throws IOException if a remote or network exception occurs.
625   */
626  default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
627    long amount, Durability durability) throws IOException {
628    Increment increment = new Increment(row)
629        .addColumn(family, qualifier, amount)
630        .setDurability(durability);
631    Cell cell = increment(increment).getColumnLatestCell(family, qualifier);
632    return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
633  }
634
635  /**
636   * Releases any resources held or pending changes in internal buffers.
637   *
638   * @throws IOException if a remote or network exception occurs.
639   */
640  @Override
641  default void close() throws IOException {
642    throw new NotImplementedException("Add an implementation!");
643  }
644
645  /**
646   * Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the
647   * table region containing the specified row.  The row given does not actually have
648   * to exist.  Whichever region would contain the row based on start and end keys will
649   * be used.  Note that the {@code row} parameter is also not passed to the
650   * coprocessor handler registered for this protocol, unless the {@code row}
651   * is separately passed as an argument in the service request.  The parameter
652   * here is only used to locate the region used to handle the call.
653   *
654   * <p>
655   * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
656   * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
657   * </p>
658   *
659   * <div style="background-color: #cccccc; padding: 2px">
660   * <blockquote><pre>
661   * CoprocessorRpcChannel channel = myTable.coprocessorService(rowkey);
662   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
663   * MyCallRequest request = MyCallRequest.newBuilder()
664   *     ...
665   *     .build();
666   * MyCallResponse response = service.myCall(null, request);
667   * </pre></blockquote></div>
668   *
669   * @param row The row key used to identify the remote region location
670   * @return A CoprocessorRpcChannel instance
671   */
672  default CoprocessorRpcChannel coprocessorService(byte[] row) {
673    throw new NotImplementedException("Add an implementation!");
674  }
675
676  /**
677   * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
678   * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), and
679   * invokes the passed {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
680   * with each {@link com.google.protobuf.Service} instance.
681   *
682   * @param service the protocol buffer {@code Service} implementation to call
683   * @param startKey start region selection with region containing this row.  If {@code null}, the
684   *   selection will start with the first table region.
685   * @param endKey select regions up to and including the region containing this row. If
686   *   {@code null}, selection will continue through the last table region.
687   * @param callable this instance's
688   *   {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call}
689   *   method will be invoked once per table region, using the {@link com.google.protobuf.Service}
690   *   instance connected to that region.
691   * @param <T> the {@link com.google.protobuf.Service} subclass to connect to
692   * @param <R> Return type for the {@code callable} parameter's {@link
693   * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
694   * @return a map of result values keyed by region name
695   */
696  default <T extends Service, R> Map<byte[],R> coprocessorService(final Class<T> service,
697    byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable)
698    throws ServiceException, Throwable {
699    throw new NotImplementedException("Add an implementation!");
700  }
701
702  /**
703   * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
704   * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), and
705   * invokes the passed {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
706   * with each {@link Service} instance.
707   *
708   * <p> The given
709   * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}
710   * method will be called with the return value from each region's
711   * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} invocation. </p>
712   *
713   * @param service the protocol buffer {@code Service} implementation to call
714   * @param startKey start region selection with region containing this row.  If {@code null}, the
715   *   selection will start with the first table region.
716   * @param endKey select regions up to and including the region containing this row. If
717   *   {@code null}, selection will continue through the last table region.
718   * @param callable this instance's
719   *   {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call}
720   *   method will be invoked once per table region, using the {@link Service} instance connected to
721   *   that region.
722   * @param <T> the {@link Service} subclass to connect to
723   * @param <R> Return type for the {@code callable} parameter's {@link
724   * org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
725   */
726  default <T extends Service, R> void coprocessorService(final Class<T> service,
727    byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable,
728    final Batch.Callback<R> callback) throws ServiceException, Throwable {
729    throw new NotImplementedException("Add an implementation!");
730  }
731
732  /**
733   * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
734   * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), all
735   * the invocations to the same region server will be batched into one call. The coprocessor
736   * service is invoked according to the service instance, method name and parameters.
737   *
738   * @param methodDescriptor
739   *          the descriptor for the protobuf service method to call.
740   * @param request
741   *          the method call parameters
742   * @param startKey
743   *          start region selection with region containing this row. If {@code null}, the
744   *          selection will start with the first table region.
745   * @param endKey
746   *          select regions up to and including the region containing this row. If {@code null},
747   *          selection will continue through the last table region.
748   * @param responsePrototype
749   *          the proto type of the response of the method in Service.
750   * @param <R>
751   *          the response type for the coprocessor Service method
752   * @return a map of result values keyed by region name
753   */
754  default <R extends Message> Map<byte[], R> batchCoprocessorService(
755    Descriptors.MethodDescriptor methodDescriptor, Message request,
756    byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable {
757    throw new NotImplementedException("Add an implementation!");
758  }
759
760  /**
761   * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table
762   * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), all
763   * the invocations to the same region server will be batched into one call. The coprocessor
764   * service is invoked according to the service instance, method name and parameters.
765   *
766   * <p>
767   * The given
768   * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}
769   * method will be called with the return value from each region's invocation.
770   * </p>
771   *
772   * @param methodDescriptor the descriptor for the protobuf service method to call.
773   * @param request the method call parameters
774   * @param startKey start region selection with region containing this row.
775   *   If {@code null}, the selection will start with the first table region.
776   * @param endKey select regions up to and including the region containing this row.
777   *   If {@code null}, selection will continue through the last table region.
778   * @param responsePrototype the proto type of the response of the method in Service.
779   * @param callback callback to invoke with the response for each region
780   * @param <R>
781   *          the response type for the coprocessor Service method
782   */
783  default <R extends Message> void batchCoprocessorService(
784      Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey,
785      byte[] endKey, R responsePrototype, Batch.Callback<R> callback)
786      throws ServiceException, Throwable {
787    throw new NotImplementedException("Add an implementation!");
788  }
789
790  /**
791   * Atomically checks if a row/family/qualifier value matches the expected value.
792   * If it does, it performs the row mutations.  If the passed value is null, the check
793   * is for the lack of column (ie: non-existence)
794   *
795   * The expected value argument of this call is on the left and the current
796   * value of the cell is on the right side of the comparison operator.
797   *
798   * Ie. eg. GREATER operator means expected value > existing <=> perform row mutations.
799   *
800   * @param row to check
801   * @param family column family to check
802   * @param qualifier column qualifier to check
803   * @param compareOp the comparison operator
804   * @param value the expected value
805   * @param mutation  mutations to perform if check succeeds
806   * @throws IOException e
807   * @return true if the new put was executed, false otherwise
808   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
809   */
810  @Deprecated
811  default boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier,
812      CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutation) throws IOException {
813    throw new NotImplementedException("Add an implementation!");
814  }
815
816  /**
817   * Atomically checks if a row/family/qualifier value matches the expected value.
818   * If it does, it performs the row mutations.  If the passed value is null, the check
819   * is for the lack of column (ie: non-existence)
820   *
821   * The expected value argument of this call is on the left and the current
822   * value of the cell is on the right side of the comparison operator.
823   *
824   * Ie. eg. GREATER operator means expected value > existing <=> perform row mutations.
825   *
826   * @param row to check
827   * @param family column family to check
828   * @param qualifier column qualifier to check
829   * @param op the comparison operator
830   * @param value the expected value
831   * @param mutation  mutations to perform if check succeeds
832   * @throws IOException e
833   * @return true if the new put was executed, false otherwise
834   * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #checkAndMutate(byte[], byte[])}
835   */
836  @Deprecated
837  default boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
838                         byte[] value, RowMutations mutation) throws IOException {
839    throw new NotImplementedException("Add an implementation!");
840  }
841
842  /**
843   * Get timeout of each rpc request in this Table instance. It will be overridden by a more
844   * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
845   * @see #getReadRpcTimeout(TimeUnit)
846   * @see #getWriteRpcTimeout(TimeUnit)
847   * @param unit the unit of time the timeout to be represented in
848   * @return rpc timeout in the specified time unit
849   */
850  default long getRpcTimeout(TimeUnit unit) {
851    throw new NotImplementedException("Add an implementation!");
852  }
853
854  /**
855   * Get timeout (millisecond) of each rpc request in this Table instance.
856   *
857   * @return Currently configured read timeout
858   * @deprecated use {@link #getReadRpcTimeout(TimeUnit)} or
859   *             {@link #getWriteRpcTimeout(TimeUnit)} instead
860   */
861  @Deprecated
862  default int getRpcTimeout() {
863    return (int)getRpcTimeout(TimeUnit.MILLISECONDS);
864  }
865
866  /**
867   * Set timeout (millisecond) of each rpc request in operations of this Table instance, will
868   * override the value of hbase.rpc.timeout in configuration.
869   * If a rpc request waiting too long, it will stop waiting and send a new request to retry until
870   * retries exhausted or operation timeout reached.
871   * <p>
872   * NOTE: This will set both the read and write timeout settings to the provided value.
873   *
874   * @param rpcTimeout the timeout of each rpc request in millisecond.
875   *
876   * @deprecated Use setReadRpcTimeout or setWriteRpcTimeout instead
877   */
878  @Deprecated
879  default void setRpcTimeout(int rpcTimeout) {
880    setReadRpcTimeout(rpcTimeout);
881    setWriteRpcTimeout(rpcTimeout);
882  }
883
884  /**
885   * Get timeout of each rpc read request in this Table instance.
886   * @param unit the unit of time the timeout to be represented in
887   * @return read rpc timeout in the specified time unit
888   */
889  default long getReadRpcTimeout(TimeUnit unit) {
890    throw new NotImplementedException("Add an implementation!");
891  }
892
893  /**
894   * Get timeout (millisecond) of each rpc read request in this Table instance.
895   * @deprecated since 2.0 and will be removed in 3.0 version
896   *             use {@link #getReadRpcTimeout(TimeUnit)} instead
897   */
898  @Deprecated
899  default int getReadRpcTimeout() {
900    return (int)getReadRpcTimeout(TimeUnit.MILLISECONDS);
901  }
902
903  /**
904   * Set timeout (millisecond) of each rpc read request in operations of this Table instance, will
905   * override the value of hbase.rpc.read.timeout in configuration.
906   * If a rpc read request waiting too long, it will stop waiting and send a new request to retry
907   * until retries exhausted or operation timeout reached.
908   *
909   * @param readRpcTimeout the timeout for read rpc request in milliseconds
910   * @deprecated since 2.0.0, use {@link TableBuilder#setReadRpcTimeout} instead
911   */
912  @Deprecated
913  default void setReadRpcTimeout(int readRpcTimeout) {
914    throw new NotImplementedException("Add an implementation!");
915  }
916
917  /**
918   * Get timeout of each rpc write request in this Table instance.
919   * @param unit the unit of time the timeout to be represented in
920   * @return write rpc timeout in the specified time unit
921   */
922  default long getWriteRpcTimeout(TimeUnit unit) {
923    throw new NotImplementedException("Add an implementation!");
924  }
925
926  /**
927   * Get timeout (millisecond) of each rpc write request in this Table instance.
928   * @deprecated since 2.0 and will be removed in 3.0 version
929   *             use {@link #getWriteRpcTimeout(TimeUnit)} instead
930   */
931  @Deprecated
932  default int getWriteRpcTimeout() {
933    return (int)getWriteRpcTimeout(TimeUnit.MILLISECONDS);
934  }
935
936  /**
937   * Set timeout (millisecond) of each rpc write request in operations of this Table instance, will
938   * override the value of hbase.rpc.write.timeout in configuration.
939   * If a rpc write request waiting too long, it will stop waiting and send a new request to retry
940   * until retries exhausted or operation timeout reached.
941   *
942   * @param writeRpcTimeout the timeout for write rpc request in milliseconds
943   * @deprecated since 2.0.0, use {@link TableBuilder#setWriteRpcTimeout} instead
944   */
945  @Deprecated
946  default void setWriteRpcTimeout(int writeRpcTimeout) {
947    throw new NotImplementedException("Add an implementation!");
948  }
949
950  /**
951   * Get timeout of each operation in Table instance.
952   * @param unit the unit of time the timeout to be represented in
953   * @return operation rpc timeout in the specified time unit
954   */
955  default long getOperationTimeout(TimeUnit unit) {
956    throw new NotImplementedException("Add an implementation!");
957  }
958
959  /**
960   * Get timeout (millisecond) of each operation for in Table instance.
961   * @deprecated since 2.0 and will be removed in 3.0 version
962   *             use {@link #getOperationTimeout(TimeUnit)} instead
963   */
964  @Deprecated
965  default int getOperationTimeout() {
966    return (int)getOperationTimeout(TimeUnit.MILLISECONDS);
967  }
968
969  /**
970   * Set timeout (millisecond) of each operation in this Table instance, will override the value
971   * of hbase.client.operation.timeout in configuration.
972   * Operation timeout is a top-level restriction that makes sure a blocking method will not be
973   * blocked more than this. In each operation, if rpc request fails because of timeout or
974   * other reason, it will retry until success or throw a RetriesExhaustedException. But if the
975   * total time being blocking reach the operation timeout before retries exhausted, it will break
976   * early and throw SocketTimeoutException.
977   * @param operationTimeout the total timeout of each operation in millisecond.
978   * @deprecated since 2.0.0, use {@link TableBuilder#setOperationTimeout} instead
979   */
980  @Deprecated
981  default void setOperationTimeout(int operationTimeout) {
982    throw new NotImplementedException("Add an implementation!");
983  }
984}