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