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.ipc;
21  
22  import java.io.IOException;
23  import java.net.ConnectException;
24  import java.util.List;
25  
26  import org.apache.hadoop.hbase.Abortable;
27  import org.apache.hadoop.hbase.HRegionInfo;
28  import org.apache.hadoop.hbase.HServerInfo;
29  import org.apache.hadoop.hbase.NotServingRegionException;
30  import org.apache.hadoop.hbase.Stoppable;
31  import org.apache.hadoop.hbase.client.Delete;
32  import org.apache.hadoop.hbase.client.Get;
33  import org.apache.hadoop.hbase.client.Increment;
34  import org.apache.hadoop.hbase.client.MultiAction;
35  import org.apache.hadoop.hbase.client.MultiResponse;
36  import org.apache.hadoop.hbase.client.Put;
37  import org.apache.hadoop.hbase.client.Result;
38  import org.apache.hadoop.hbase.client.Scan;
39  import org.apache.hadoop.hbase.client.coprocessor.Exec;
40  import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
41  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
42  import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
43  import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
44  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
45  import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
46  import org.apache.hadoop.hbase.regionserver.wal.HLog;
47  import org.apache.hadoop.hbase.security.TokenInfo;
48  import org.apache.hadoop.hbase.security.KerberosInfo;
49  import org.apache.hadoop.hbase.util.Pair;
50  import org.apache.hadoop.ipc.RemoteException;
51  import org.apache.hadoop.hbase.ipc.VersionedProtocol;
52  
53  /**
54   * Clients interact with HRegionServers using a handle to the HRegionInterface.
55   *
56   * <p>NOTE: if you change the interface, you must change the RPC version
57   * number in HBaseRPCProtocolVersion
58   */
59  @KerberosInfo(
60      serverPrincipal = "hbase.regionserver.kerberos.principal")
61  @TokenInfo("HBASE_AUTH_TOKEN")
62  public interface HRegionInterface extends VersionedProtocol, Stoppable, Abortable {
63    /**
64     * This Interfaces' version. Version changes when the Interface changes.
65     */
66    // All HBase Interfaces used derive from HBaseRPCProtocolVersion.  It
67    // maintained a single global version number on all HBase Interfaces.  This
68    // meant all HBase RPC was broke though only one of the three RPC Interfaces
69    // had changed.  This has since been undone.
70    public static final long VERSION = 29L;
71  
72    /**
73     * Get metainfo about an HRegion
74     *
75     * @param regionName name of the region
76     * @return HRegionInfo object for region
77     * @throws NotServingRegionException
78     * @throws ConnectException
79     * @throws IOException This can manifest as an Hadoop ipc {@link RemoteException}
80     */
81    public HRegionInfo getRegionInfo(final byte [] regionName)
82    throws NotServingRegionException, ConnectException, IOException;
83  
84    /**
85     * Return all the data for the row that matches <i>row</i> exactly,
86     * or the one that immediately preceeds it.
87     *
88     * @param regionName region name
89     * @param row row key
90     * @param family Column family to look for row in.
91     * @return map of values
92     * @throws IOException e
93     */
94    public Result getClosestRowBefore(final byte [] regionName,
95      final byte [] row, final byte [] family)
96    throws IOException;
97  
98    /**
99     * Perform Get operation.
100    * @param regionName name of region to get from
101    * @param get Get operation
102    * @return Result
103    * @throws IOException e
104    */
105   public Result get(byte [] regionName, Get get) throws IOException;
106 
107   /**
108    * Perform exists operation.
109    * @param regionName name of region to get from
110    * @param get Get operation describing cell to test
111    * @return true if exists
112    * @throws IOException e
113    */
114   public boolean exists(byte [] regionName, Get get) throws IOException;
115 
116   /**
117    * Put data into the specified region
118    * @param regionName region name
119    * @param put the data to be put
120    * @throws IOException e
121    */
122   public void put(final byte [] regionName, final Put put)
123   throws IOException;
124 
125   /**
126    * Put an array of puts into the specified region
127    *
128    * @param regionName region name
129    * @param puts List of puts to execute
130    * @return The number of processed put's.  Returns -1 if all Puts
131    * processed successfully.
132    * @throws IOException e
133    */
134   public int put(final byte[] regionName, final List<Put> puts)
135   throws IOException;
136 
137   /**
138    * Deletes all the KeyValues that match those found in the Delete object,
139    * if their ts <= to the Delete. In case of a delete with a specific ts it
140    * only deletes that specific KeyValue.
141    * @param regionName region name
142    * @param delete delete object
143    * @throws IOException e
144    */
145   public void delete(final byte[] regionName, final Delete delete)
146   throws IOException;
147 
148   /**
149    * Put an array of deletes into the specified region
150    *
151    * @param regionName region name
152    * @param deletes delete List to execute
153    * @return The number of processed deletes.  Returns -1 if all Deletes
154    * processed successfully.
155    * @throws IOException e
156    */
157   public int delete(final byte[] regionName, final List<Delete> deletes)
158   throws IOException;
159 
160   /**
161    * Atomically checks if a row/family/qualifier value match the expectedValue.
162    * If it does, it adds the put. If passed expected value is null, then the
163    * check is for non-existance of the row/column.
164    *
165    * @param regionName region name
166    * @param row row to check
167    * @param family column family
168    * @param qualifier column qualifier
169    * @param value the expected value
170    * @param put data to put if check succeeds
171    * @throws IOException e
172    * @return true if the new put was execute, false otherwise
173    */
174   public boolean checkAndPut(final byte[] regionName, final byte [] row,
175       final byte [] family, final byte [] qualifier, final byte [] value,
176       final Put put)
177   throws IOException;
178 
179 
180   /**
181    * Atomically checks if a row/family/qualifier value match the expectedValue.
182    * If it does, it adds the delete. If passed expected value is null, then the
183    * check is for non-existance of the row/column.
184    *
185    * @param regionName region name
186    * @param row row to check
187    * @param family column family
188    * @param qualifier column qualifier
189    * @param value the expected value
190    * @param delete data to delete if check succeeds
191    * @throws IOException e
192    * @return true if the new delete was execute, false otherwise
193    */
194   public boolean checkAndDelete(final byte[] regionName, final byte [] row,
195       final byte [] family, final byte [] qualifier, final byte [] value,
196       final Delete delete)
197   throws IOException;
198 
199   /**
200    * Atomically increments a column value. If the column value isn't long-like,
201    * this could throw an exception. If passed expected value is null, then the
202    * check is for non-existance of the row/column.
203    *
204    * @param regionName region name
205    * @param row row to check
206    * @param family column family
207    * @param qualifier column qualifier
208    * @param amount long amount to increment
209    * @param writeToWAL whether to write the increment to the WAL
210    * @return new incremented column value
211    * @throws IOException e
212    */
213   public long incrementColumnValue(byte [] regionName, byte [] row,
214       byte [] family, byte [] qualifier, long amount, boolean writeToWAL)
215   throws IOException;
216 
217   /**
218    * Increments one or more columns values in a row.  Returns the
219    * updated keys after the increment.
220    * <p>
221    * This operation does not appear atomic to readers.  Increments are done
222    * under a row lock but readers do not take row locks.
223    * @param regionName region name
224    * @param increment increment operation
225    * @return incremented cells
226    */
227   public Result increment(byte[] regionName, Increment increment)
228   throws IOException;
229 
230   //
231   // remote scanner interface
232   //
233 
234   /**
235    * Opens a remote scanner with a RowFilter.
236    *
237    * @param regionName name of region to scan
238    * @param scan configured scan object
239    * @return scannerId scanner identifier used in other calls
240    * @throws IOException e
241    */
242   public long openScanner(final byte [] regionName, final Scan scan)
243   throws IOException;
244 
245   /**
246    * Get the next set of values
247    * @param scannerId clientId passed to openScanner
248    * @return map of values; returns null if no results.
249    * @throws IOException e
250    */
251   public Result next(long scannerId) throws IOException;
252 
253   /**
254    * Get the next set of values
255    * @param scannerId clientId passed to openScanner
256    * @param numberOfRows the number of rows to fetch
257    * @return Array of Results (map of values); array is empty if done with this
258    * region and null if we are NOT to go to the next region (happens when a
259    * filter rules that the scan is done).
260    * @throws IOException e
261    */
262   public Result [] next(long scannerId, int numberOfRows) throws IOException;
263 
264   /**
265    * Close a scanner
266    *
267    * @param scannerId the scanner id returned by openScanner
268    * @throws IOException e
269    */
270   public void close(long scannerId) throws IOException;
271 
272   /**
273    * Opens a remote row lock.
274    *
275    * @param regionName name of region
276    * @param row row to lock
277    * @return lockId lock identifier
278    * @throws IOException e
279    */
280   public long lockRow(final byte [] regionName, final byte [] row)
281   throws IOException;
282 
283   /**
284    * Releases a remote row lock.
285    *
286    * @param regionName region name
287    * @param lockId the lock id returned by lockRow
288    * @throws IOException e
289    */
290   public void unlockRow(final byte [] regionName, final long lockId)
291   throws IOException;
292 
293 
294   /**
295    * @return All regions online on this region server
296    * @throws IOException e
297    */
298   public List<HRegionInfo> getOnlineRegions() throws IOException;
299 
300   /**
301    * Method used when a master is taking the place of another failed one.
302    * @return This servers {@link HServerInfo}; it has RegionServer POV on the
303    * hostname which may not agree w/ how the Master sees this server.
304    * @throws IOException e
305    * @deprecated
306    */
307   public HServerInfo getHServerInfo() throws IOException;
308 
309   /**
310    * Method used for doing multiple actions(Deletes, Gets and Puts) in one call
311    * @param multi
312    * @return MultiResult
313    * @throws IOException
314    */
315   public <R> MultiResponse multi(MultiAction<R> multi) throws IOException;
316 
317   /**
318    * Atomically bulk load multiple HFiles (say from different column families)
319    * into an open region.
320    * 
321    * @param familyPaths List of (family, hfile path) pairs
322    * @param regionName name of region to load hfiles into
323    * @return true if successful, false if failed recoverably
324    * @throws IOException if fails unrecoverably
325    */
326   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths, byte[] regionName)
327   throws IOException;
328 
329   // Master methods
330 
331   /**
332    * Opens the specified region.
333    * 
334    * @param region
335    *          region to open
336    * @return RegionOpeningState 
337    *         OPENED         - if region open request was successful.
338    *         ALREADY_OPENED - if the region was already opened. 
339    *         FAILED_OPENING - if region opening failed.
340    *
341    * @throws IOException
342    */
343   public RegionOpeningState openRegion(final HRegionInfo region) throws IOException;
344 
345   /**
346    * Opens the specified region.
347    * @param region
348    *          region to open
349    * @param versionOfOfflineNode
350    *          the version of znode to compare when RS transitions the znode from
351    *          OFFLINE state.
352    * @return RegionOpeningState 
353    *         OPENED         - if region open request was successful.
354    *         ALREADY_OPENED - if the region was already opened. 
355    *         FAILED_OPENING - if region opening failed.
356    * @throws IOException
357    */
358   public RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode)
359       throws IOException;
360   
361   /**
362    * Opens the specified regions.
363    * @param regions regions to open
364    * @throws IOException
365    */
366   public void openRegions(final List<HRegionInfo> regions) throws IOException;
367 
368   /**
369    * Closes the specified region.
370    * @param region region to close
371    * @return true if closing region, false if not
372    * @throws IOException
373    */
374   public boolean closeRegion(final HRegionInfo region)
375   throws IOException;
376 
377   /**
378    * Closes the specified region and will use or not use ZK during the close
379    * according to the specified flag.
380    * @param region region to close
381    * @param zk true if transitions should be done in ZK, false if not
382    * @return true if closing region, false if not
383    * @throws IOException
384    */
385   public boolean closeRegion(final HRegionInfo region, final boolean zk)
386   throws IOException;
387   
388   /**
389    * Closes the region in the RS with the specified encoded regionName and will
390    * use or not use ZK during the close according to the specified flag. Note
391    * that the encoded region name is in byte format.
392    * 
393    * @param encodedRegionName
394    *          in bytes
395    * @param zk
396    *          true if to use zookeeper, false if need not.
397    * @return true if region is closed, false if not.
398    * @throws IOException
399    */
400   public boolean closeRegion(byte[] encodedRegionName, final boolean zk)
401       throws IOException;
402 
403   // Region administrative methods
404 
405   /**
406    * Flushes the MemStore of the specified region.
407    * <p>
408    * This method is synchronous.
409    * @param regionInfo region to flush
410    * @throws NotServingRegionException
411    * @throws IOException
412    */
413   void flushRegion(HRegionInfo regionInfo)
414   throws NotServingRegionException, IOException;
415 
416   /**
417    * Splits the specified region.
418    * <p>
419    * This method currently flushes the region and then forces a compaction which
420    * will then trigger a split.  The flush is done synchronously but the
421    * compaction is asynchronous.
422    * @param regionInfo region to split
423    * @throws NotServingRegionException
424    * @throws IOException
425    */
426   void splitRegion(HRegionInfo regionInfo)
427   throws NotServingRegionException, IOException;
428 
429   /**
430    * Splits the specified region.
431    * <p>
432    * This method currently flushes the region and then forces a compaction which
433    * will then trigger a split.  The flush is done synchronously but the
434    * compaction is asynchronous.
435    * @param regionInfo region to split
436    * @param splitPoint the explicit row to split on
437    * @throws NotServingRegionException
438    * @throws IOException
439    */
440   void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
441   throws NotServingRegionException, IOException;
442 
443   /**
444    * Compacts the specified region.  Performs a major compaction if specified.
445    * <p>
446    * This method is asynchronous.
447    * @param regionInfo region to compact
448    * @param major true to force major compaction
449    * @throws NotServingRegionException
450    * @throws IOException
451    */
452   void compactRegion(HRegionInfo regionInfo, boolean major)
453   throws NotServingRegionException, IOException;
454 
455   /**
456    * Replicates the given entries. The guarantee is that the given entries
457    * will be durable on the slave cluster if this method returns without
458    * any exception.
459    * hbase.replication has to be set to true for this to work.
460    *
461    * @param entries entries to replicate
462    * @throws IOException
463    */
464   public void replicateLogEntries(HLog.Entry[] entries) throws IOException;
465 
466   /**
467    * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
468    * method using the registered protocol handlers.
469    * {@link CoprocessorProtocol} implementations must be registered via the
470    * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
471    * method before they are available.
472    *
473    * @param regionName name of the region against which the invocation is executed
474    * @param call an {@code Exec} instance identifying the protocol, method name,
475    *     and parameters for the method invocation
476    * @return an {@code ExecResult} instance containing the region name of the
477    *     invocation and the return value
478    * @throws IOException if no registered protocol handler is found or an error
479    *     occurs during the invocation
480    * @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
481    */
482   ExecResult execCoprocessor(byte[] regionName, Exec call)
483       throws IOException;
484 
485   /**
486    * Atomically checks if a row/family/qualifier value match the expectedValue.
487    * If it does, it adds the put. If passed expected value is null, then the
488    * check is for non-existance of the row/column.
489    *
490    * @param regionName
491    * @param row
492    * @param family
493    * @param qualifier
494    * @param compareOp
495    * @param comparator
496    * @param put
497    * @throws IOException
498    * @return true if the new put was execute, false otherwise
499    */
500   public boolean checkAndPut(final byte[] regionName, final byte[] row,
501       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
502       final WritableByteArrayComparable comparator, final Put put)
503   throws IOException;
504 
505   /**
506    * Atomically checks if a row/family/qualifier value match the expectedValue.
507    * If it does, it adds the delete. If passed expected value is null, then the
508    * check is for non-existance of the row/column.
509    *
510    * @param regionName
511    * @param row
512    * @param family
513    * @param qualifier
514    * @param compareOp
515    * @param comparator
516    * @param delete
517    * @throws IOException
518    * @return true if the new put was execute, false otherwise
519    */
520   public boolean checkAndDelete(final byte[] regionName, final byte[] row,
521      final byte[] family, final byte[] qualifier, final CompareOp compareOp,
522      final WritableByteArrayComparable comparator, final Delete delete)
523      throws IOException;
524   
525   /**
526    * Performs a BlockCache summary and returns a List of BlockCacheColumnFamilySummary objects.
527    * This method could be fairly heavyweight in that it evaluates the entire HBase file-system
528    * against what is in the RegionServer BlockCache. 
529    * 
530    * @return BlockCacheColumnFamilySummary
531    * @throws IOException exception
532    */
533   public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries() throws IOException;
534   /**
535    * Roll the log writer. That is, start writing log messages to a new file.
536    * 
537    * @throws IOException
538    * @throws FailedLogCloseException
539    * @return If lots of logs, flush the returned regions so next time through
540    * we can clean logs. Returns null if nothing to flush.  Names are actual
541    * region names as returned by {@link HRegionInfo#getEncodedName()} 
542    */
543   public byte[][] rollHLogWriter() throws IOException, FailedLogCloseException;
544 }