View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.client;
21  
22  import java.io.IOException;
23  import java.util.List;
24  
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.hbase.HTableDescriptor;
27  import org.apache.hadoop.hbase.KeyValue;
28  import org.apache.hadoop.hbase.client.coprocessor.Batch;
29  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
30  
31  import java.util.Map;
32  
33  /**
34   * Used to communicate with a single HBase table.
35   *
36   * @since 0.21.0
37   */
38  public interface HTableInterface {
39  
40    /**
41     * Gets the name of this table.
42     *
43     * @return the table name.
44     */
45    byte[] getTableName();
46  
47    /**
48     * Returns the {@link Configuration} object used by this instance.
49     * <p>
50     * The reference returned is not a copy, so any change made to it will
51     * affect this instance.
52     */
53    Configuration getConfiguration();
54  
55    /**
56     * Gets the {@link HTableDescriptor table descriptor} for this table.
57     * @throws IOException if a remote or network exception occurs.
58     */
59    HTableDescriptor getTableDescriptor() throws IOException;
60  
61    /**
62     * Test for the existence of columns in the table, as specified in the Get.
63     * <p>
64     *
65     * This will return true if the Get matches one or more keys, false if not.
66     * <p>
67     *
68     * This is a server-side call so it prevents any data from being transfered to
69     * the client.
70     *
71     * @param get the Get
72     * @return true if the specified Get matches one or more keys, false if not
73     * @throws IOException e
74     */
75    boolean exists(Get get) throws IOException;
76  
77    /**
78     * Method that does a batch call on Deletes, Gets and Puts. The ordering of
79     * execution of the actions is not defined. Meaning if you do a Put and a
80     * Get in the same {@link #batch} call, you will not necessarily be
81     * guaranteed that the Get returns what the Put had put.
82     *
83     * @param actions list of Get, Put, Delete objects
84     * @param results Empty Object[], same size as actions. Provides access to partial
85     *                results, in case an exception is thrown. A null in the result array means that
86     *                the call for that action failed, even after retries
87     * @throws IOException
88     * @since 0.90.0
89     */
90    void batch(final List<Row> actions, final Object[] results) throws IOException, InterruptedException;
91  
92    /**
93     * Same as {@link #batch(List, Object[])}, but returns an array of
94     * results instead of using a results parameter reference.
95     *
96     * @param actions list of Get, Put, Delete objects
97     * @return the results from the actions. A null in the return array means that
98     *         the call for that action failed, even after retries
99     * @throws IOException
100    * @since 0.90.0
101    */
102   Object[] batch(final List<Row> actions) throws IOException, InterruptedException;
103 
104   /**
105    * Extracts certain cells from a given row.
106    * @param get The object that specifies what data to fetch and from which row.
107    * @return The data coming from the specified row, if it exists.  If the row
108    * specified doesn't exist, the {@link Result} instance returned won't
109    * contain any {@link KeyValue}, as indicated by {@link Result#isEmpty()}.
110    * @throws IOException if a remote or network exception occurs.
111    * @since 0.20.0
112    */
113   Result get(Get get) throws IOException;
114 
115   /**
116    * Extracts certain cells from the given rows, in batch.
117    *
118    * @param gets The objects that specify what data to fetch and from which rows.
119    *
120    * @return The data coming from the specified rows, if it exists.  If the row
121    *         specified doesn't exist, the {@link Result} instance returned won't
122    *         contain any {@link KeyValue}, as indicated by {@link Result#isEmpty()}.
123    *         If there are any failures even after retries, there will be a null in
124    *         the results array for those Gets, AND an exception will be thrown.
125    * @throws IOException if a remote or network exception occurs.
126    *
127    * @since 0.90.0
128    */
129   Result[] get(List<Get> gets) throws IOException;
130 
131   /**
132    * Return the row that matches <i>row</i> exactly,
133    * or the one that immediately precedes it.
134    *
135    * @param row A row key.
136    * @param family Column family to include in the {@link Result}.
137    * @throws IOException if a remote or network exception occurs.
138    * @since 0.20.0
139    * 
140    * @deprecated As of version 0.92 this method is deprecated without
141    * replacement.   
142    * getRowOrBefore is used internally to find entries in .META. and makes
143    * various assumptions about the table (which are true for .META. but not
144    * in general) to be efficient.
145    */
146   Result getRowOrBefore(byte[] row, byte[] family) throws IOException;
147 
148   /**
149    * Returns a scanner on the current table as specified by the {@link Scan}
150    * object.
151    *
152    * @param scan A configured {@link Scan} object.
153    * @return A scanner.
154    * @throws IOException if a remote or network exception occurs.
155    * @since 0.20.0
156    */
157   ResultScanner getScanner(Scan scan) throws IOException;
158 
159   /**
160    * Gets a scanner on the current table for the given family.
161    *
162    * @param family The column family to scan.
163    * @return A scanner.
164    * @throws IOException if a remote or network exception occurs.
165    * @since 0.20.0
166    */
167   ResultScanner getScanner(byte[] family) throws IOException;
168 
169   /**
170    * Gets a scanner on the current table for the given family and qualifier.
171    *
172    * @param family The column family to scan.
173    * @param qualifier The column qualifier to scan.
174    * @return A scanner.
175    * @throws IOException if a remote or network exception occurs.
176    * @since 0.20.0
177    */
178   ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException;
179 
180 
181   /**
182    * Puts some data in the table.
183    * <p>
184    * If {@link #isAutoFlush isAutoFlush} is false, the update is buffered
185    * until the internal buffer is full.
186    * @param put The data to put.
187    * @throws IOException if a remote or network exception occurs.
188    * @since 0.20.0
189    */
190   void put(Put put) throws IOException;
191 
192   /**
193    * Puts some data in the table, in batch.
194    * <p>
195    * If {@link #isAutoFlush isAutoFlush} is false, the update is buffered
196    * until the internal buffer is full.
197    * <p>
198    * This can be used for group commit, or for submitting user defined
199    * batches.  The writeBuffer will be periodically inspected while the List
200    * is processed, so depending on the List size the writeBuffer may flush
201    * not at all, or more than once.
202    * @param puts The list of mutations to apply. The batch put is done by
203    * aggregating the iteration of the Puts over the write buffer
204    * at the client-side for a single RPC call.
205    * @throws IOException if a remote or network exception occurs.
206    * @since 0.20.0
207    */
208   void put(List<Put> puts) throws IOException;
209 
210   /**
211    * Atomically checks if a row/family/qualifier value matches the expected
212    * value. If it does, it adds the put.  If the passed value is null, the check
213    * is for the lack of column (ie: non-existance)
214    *
215    * @param row to check
216    * @param family column family to check
217    * @param qualifier column qualifier to check
218    * @param value the expected value
219    * @param put data to put if check succeeds
220    * @throws IOException e
221    * @return true if the new put was executed, false otherwise
222    */
223   boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
224       byte[] value, Put put) throws IOException;
225 
226   /**
227    * Deletes the specified cells/row.
228    *
229    * @param delete The object that specifies what to delete.
230    * @throws IOException if a remote or network exception occurs.
231    * @since 0.20.0
232    */
233   void delete(Delete delete) throws IOException;
234 
235   /**
236    * Deletes the specified cells/rows in bulk.
237    * @param deletes List of things to delete.  List gets modified by this
238    * method (in particular it gets re-ordered, so the order in which the elements
239    * are inserted in the list gives no guarantee as to the order in which the
240    * {@link Delete}s are executed).
241    * @throws IOException if a remote or network exception occurs. In that case
242    * the {@code deletes} argument will contain the {@link Delete} instances
243    * that have not be successfully applied.
244    * @since 0.20.1
245    */
246   void delete(List<Delete> deletes) throws IOException;
247 
248   /**
249    * Atomically checks if a row/family/qualifier value matches the expected
250    * value. If it does, it adds the delete.  If the passed value is null, the
251    * check is for the lack of column (ie: non-existance)
252    *
253    * @param row to check
254    * @param family column family to check
255    * @param qualifier column qualifier to check
256    * @param value the expected value
257    * @param delete data to delete if check succeeds
258    * @throws IOException e
259    * @return true if the new delete was executed, false otherwise
260    */
261   boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
262       byte[] value, Delete delete) throws IOException;
263 
264   /**
265    * Increments one or more columns within a single row.
266    * <p>
267    * This operation does not appear atomic to readers.  Increments are done
268    * under a single row lock, so write operations to a row are synchronized, but
269    * readers do not take row locks so get and scan operations can see this
270    * operation partially completed.
271    *
272    * @param increment object that specifies the columns and amounts to be used
273    *                  for the increment operations
274    * @throws IOException e
275    * @return values of columns after the increment
276    */
277   public Result increment(final Increment increment) throws IOException;
278 
279   /**
280    * Atomically increments a column value.
281    * <p>
282    * Equivalent to {@link #incrementColumnValue(byte[], byte[], byte[],
283    * long, boolean) incrementColumnValue}(row, family, qualifier, amount,
284    * <b>true</b>)}
285    * @param row The row that contains the cell to increment.
286    * @param family The column family of the cell to increment.
287    * @param qualifier The column qualifier of the cell to increment.
288    * @param amount The amount to increment the cell with (or decrement, if the
289    * amount is negative).
290    * @return The new value, post increment.
291    * @throws IOException if a remote or network exception occurs.
292    */
293   long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
294       long amount) throws IOException;
295 
296   /**
297    * Atomically increments a column value. If the column value already exists
298    * and is not a big-endian long, this could throw an exception. If the column
299    * value does not yet exist it is initialized to <code>amount</code> and
300    * written to the specified column.
301    *
302    * <p>Setting writeToWAL to false means that in a fail scenario, you will lose
303    * any increments that have not been flushed.
304    * @param row The row that contains the cell to increment.
305    * @param family The column family of the cell to increment.
306    * @param qualifier The column qualifier of the cell to increment.
307    * @param amount The amount to increment the cell with (or decrement, if the
308    * amount is negative).
309    * @param writeToWAL if {@code true}, the operation will be applied to the
310    * Write Ahead Log (WAL).  This makes the operation slower but safer, as if
311    * the call returns successfully, it is guaranteed that the increment will
312    * be safely persisted.  When set to {@code false}, the call may return
313    * successfully before the increment is safely persisted, so it's possible
314    * that the increment be lost in the event of a failure happening before the
315    * operation gets persisted.
316    * @return The new value, post increment.
317    * @throws IOException if a remote or network exception occurs.
318    */
319   long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
320       long amount, boolean writeToWAL) throws IOException;
321 
322   /**
323    * Tells whether or not 'auto-flush' is turned on.
324    *
325    * @return {@code true} if 'auto-flush' is enabled (default), meaning
326    * {@link Put} operations don't get buffered/delayed and are immediately
327    * executed.
328    */
329   boolean isAutoFlush();
330 
331   /**
332    * Executes all the buffered {@link Put} operations.
333    * <p>
334    * This method gets called once automatically for every {@link Put} or batch
335    * of {@link Put}s (when <code>put(List<Put>)</code> is used) when
336    * {@link #isAutoFlush} is {@code true}.
337    * @throws IOException if a remote or network exception occurs.
338    */
339   void flushCommits() throws IOException;
340 
341   /**
342    * Releases any resources help or pending changes in internal buffers.
343    *
344    * @throws IOException if a remote or network exception occurs.
345    */
346   void close() throws IOException;
347 
348   /**
349    * Obtains a lock on a row.
350    *
351    * @param row The row to lock.
352    * @return A {@link RowLock} containing the row and lock id.
353    * @throws IOException if a remote or network exception occurs.
354    * @see RowLock
355    * @see #unlockRow
356    */
357   RowLock lockRow(byte[] row) throws IOException;
358 
359   /**
360    * Releases a row lock.
361    *
362    * @param rl The row lock to release.
363    * @throws IOException if a remote or network exception occurs.
364    * @see RowLock
365    * @see #unlockRow
366    */
367   void unlockRow(RowLock rl) throws IOException;
368 
369   /**
370    * Creates and returns a proxy to the CoprocessorProtocol instance running in the
371    * region containing the specified row.  The row given does not actually have
372    * to exist.  Whichever region would contain the row based on start and end keys will
373    * be used.  Note that the {@code row} parameter is also not passed to the
374    * coprocessor handler registered for this protocol, unless the {@code row}
375    * is separately passed as an argument in a proxy method call.  The parameter
376    * here is just used to locate the region used to handle the call.
377    *
378    * @param protocol The class or interface defining the remote protocol
379    * @param row The row key used to identify the remote region location
380    * @return A CoprocessorProtocol instance
381    */
382   <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol, byte[] row);
383 
384   /**
385    * Invoke the passed
386    * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call} against
387    * the {@link CoprocessorProtocol} instances running in the selected regions.
388    * All regions beginning with the region containing the <code>startKey</code>
389    * row, through to the region containing the <code>endKey</code> row (inclusive)
390    * will be used.  If <code>startKey</code> or <code>endKey</code> is
391    * <code>null</code>, the first and last regions in the table, respectively,
392    * will be used in the range selection.
393    *
394    * @param protocol the CoprocessorProtocol implementation to call
395    * @param startKey start region selection with region containing this row
396    * @param endKey select regions up to and including the region containing
397    * this row
398    * @param callable wraps the CoprocessorProtocol implementation method calls
399    * made per-region
400    * @param <T> CoprocessorProtocol subclass for the remote invocation
401    * @param <R> Return type for the
402    * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)}
403    * method
404    * @return a <code>Map</code> of region names to
405    * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)} return values
406    */
407   <T extends CoprocessorProtocol, R> Map<byte[],R> coprocessorExec(
408       Class<T> protocol, byte[] startKey, byte[] endKey, Batch.Call<T,R> callable)
409       throws IOException, Throwable;
410 
411   /**
412    * Invoke the passed
413    * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call} against
414    * the {@link CoprocessorProtocol} instances running in the selected regions.
415    * All regions beginning with the region containing the <code>startKey</code>
416    * row, through to the region containing the <code>endKey</code> row
417    * (inclusive)
418    * will be used.  If <code>startKey</code> or <code>endKey</code> is
419    * <code>null</code>, the first and last regions in the table, respectively,
420    * will be used in the range selection.
421    *
422    * <p>
423    * For each result, the given
424    * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[], byte[], Object)}
425    * method will be called.
426    *</p>
427    *
428    * @param protocol the CoprocessorProtocol implementation to call
429    * @param startKey start region selection with region containing this row
430    * @param endKey select regions up to and including the region containing
431    * this row
432    * @param callable wraps the CoprocessorProtocol implementation method calls
433    * made per-region
434    * @param callback an instance upon which
435    * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[], byte[], Object)} with the
436    * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)}
437    * return value for each region
438    * @param <T> CoprocessorProtocol subclass for the remote invocation
439    * @param <R> Return type for the
440    * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)}
441    * method
442    */
443   <T extends CoprocessorProtocol, R> void coprocessorExec(
444       Class<T> protocol, byte[] startKey, byte[] endKey,
445       Batch.Call<T,R> callable, Batch.Callback<R> callback)
446       throws IOException, Throwable;
447 }