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;
020
021import java.io.DataInputStream;
022import java.io.IOException;
023import java.util.ArrayList;
024import java.util.Arrays;
025import java.util.List;
026import java.util.stream.Collectors;
027
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.hbase.KeyValue.KVComparator;
030import org.apache.hadoop.hbase.client.RegionInfo;
031import org.apache.hadoop.hbase.client.RegionInfoBuilder;
032import org.apache.hadoop.hbase.client.RegionInfoDisplay;
033import org.apache.hadoop.hbase.exceptions.DeserializationException;
034import org.apache.hadoop.hbase.master.RegionState;
035import org.apache.hadoop.hbase.util.Bytes;
036import org.apache.hadoop.io.DataInputBuffer;
037import org.apache.yetus.audience.InterfaceAudience;
038import org.slf4j.Logger;
039import org.slf4j.LoggerFactory;
040import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
041import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
042
043/**
044 * Information about a region. A region is a range of keys in the whole keyspace of a table, an
045 * identifier (a timestamp) for differentiating between subset ranges (after region split)
046 * and a replicaId for differentiating the instance for the same range and some status information
047 * about the region.
048 *
049 * The region has a unique name which consists of the following fields:
050 * <ul>
051 * <li> tableName   : The name of the table </li>
052 * <li> startKey    : The startKey for the region. </li>
053 * <li> regionId    : A timestamp when the region is created. </li>
054 * <li> replicaId   : An id starting from 0 to differentiate replicas of the same region range
055 * but hosted in separated servers. The same region range can be hosted in multiple locations.</li>
056 * <li> encodedName : An MD5 encoded string for the region name.</li>
057 * </ul>
058 *
059 * <br> Other than the fields in the region name, region info contains:
060 * <ul>
061 * <li> endKey      : the endKey for the region (exclusive) </li>
062 * <li> split       : Whether the region is split </li>
063 * <li> offline     : Whether the region is offline </li>
064 * </ul>
065 *
066 * In 0.98 or before, a list of table's regions would fully cover the total keyspace, and at any
067 * point in time, a row key always belongs to a single region, which is hosted in a single server.
068 * In 0.99+, a region can have multiple instances (called replicas), and thus a range (or row) can
069 * correspond to multiple HRegionInfo's. These HRI's share the same fields however except the
070 * replicaId field. If the replicaId is not set, it defaults to 0, which is compatible with the
071 * previous behavior of a range corresponding to 1 region.
072 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
073 *             use {@link RegionInfoBuilder} to build {@link RegionInfo}.
074 */
075@Deprecated
076@InterfaceAudience.Public
077public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
078  private static final Logger LOG = LoggerFactory.getLogger(HRegionInfo.class);
079
080  /**
081   * The new format for a region name contains its encodedName at the end.
082   * The encoded name also serves as the directory name for the region
083   * in the filesystem.
084   *
085   * New region name format:
086   *    &lt;tablename>,,&lt;startkey>,&lt;regionIdTimestamp>.&lt;encodedName>.
087   * where,
088   *    &lt;encodedName> is a hex version of the MD5 hash of
089   *    &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
090   *
091   * The old region name format:
092   *    &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
093   * For region names in the old format, the encoded name is a 32-bit
094   * JenkinsHash integer value (in its decimal notation, string form).
095   *<p>
096   * **NOTE**
097   *
098   * The first hbase:meta region, and regions created by an older
099   * version of HBase (0.20 or prior) will continue to use the
100   * old region name format.
101   */
102
103  /** A non-capture group so that this can be embedded. */
104  public static final String ENCODED_REGION_NAME_REGEX = RegionInfoBuilder.ENCODED_REGION_NAME_REGEX;
105
106  private static final int MAX_REPLICA_ID = 0xFFFF;
107
108  /**
109   * @param regionName
110   * @return the encodedName
111   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
112   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#encodeRegionName(byte[])}.
113   */
114  @Deprecated
115  public static String encodeRegionName(final byte [] regionName) {
116    return RegionInfo.encodeRegionName(regionName);
117  }
118
119  /**
120   * @return Return a short, printable name for this region (usually encoded name) for us logging.
121   */
122  @Override
123  public String getShortNameToLog() {
124    return prettyPrint(this.getEncodedName());
125  }
126
127  /**
128   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
129   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(RegionInfo...)}.
130   */
131  @Deprecated
132  public static String getShortNameToLog(HRegionInfo...hris) {
133    return RegionInfo.getShortNameToLog(Arrays.asList(hris));
134  }
135
136  /**
137   * @return Return a String of short, printable names for <code>hris</code>
138   * (usually encoded name) for us logging.
139   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
140   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(List)})}.
141   */
142  @Deprecated
143  public static String getShortNameToLog(final List<HRegionInfo> hris) {
144    return RegionInfo.getShortNameToLog(hris.stream().collect(Collectors.toList()));
145  }
146
147  /**
148   * Use logging.
149   * @param encodedRegionName The encoded regionname.
150   * @return <code>hbase:meta</code> if passed <code>1028785192</code> else returns
151   * <code>encodedRegionName</code>
152   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
153   *             Use {@link RegionInfo#prettyPrint(String)}.
154   */
155  @Deprecated
156  @InterfaceAudience.Private
157  public static String prettyPrint(final String encodedRegionName) {
158    return RegionInfo.prettyPrint(encodedRegionName);
159  }
160
161  private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
162  // This flag is in the parent of a split while the parent is still referenced by daughter regions.
163  // We USED to set this flag when we disabled a table but now table state is kept up in zookeeper
164  // as of 0.90.0 HBase. And now in DisableTableProcedure, finally we will create bunch of
165  // UnassignProcedures and at the last of the procedure we will set the region state to CLOSED, and
166  // will not change the offLine flag.
167  private boolean offLine = false;
168  private long regionId = -1;
169  private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
170  private boolean split = false;
171  private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
172  private int hashCode = -1;
173  //TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
174  public static final String NO_HASH = null;
175  private String encodedName = null;
176  private byte [] encodedNameAsBytes = null;
177  private int replicaId = DEFAULT_REPLICA_ID;
178
179  // Current TableName
180  private TableName tableName = null;
181
182  // Duplicated over in RegionInfoDisplay
183  final static String DISPLAY_KEYS_KEY = RegionInfoDisplay.DISPLAY_KEYS_KEY;
184  public final static byte[] HIDDEN_END_KEY = RegionInfoDisplay.HIDDEN_END_KEY;
185  public final static byte[] HIDDEN_START_KEY = RegionInfoDisplay.HIDDEN_START_KEY;
186
187  /** HRegionInfo for first meta region */
188  // TODO: How come Meta regions still do not have encoded region names? Fix.
189  public static final HRegionInfo FIRST_META_REGIONINFO =
190      new HRegionInfo(1L, TableName.META_TABLE_NAME);
191
192  private void setHashCode() {
193    int result = Arrays.hashCode(this.regionName);
194    result = (int) (result ^ this.regionId);
195    result ^= Arrays.hashCode(this.startKey);
196    result ^= Arrays.hashCode(this.endKey);
197    result ^= Boolean.valueOf(this.offLine).hashCode();
198    result ^= Arrays.hashCode(this.tableName.getName());
199    result ^= this.replicaId;
200    this.hashCode = result;
201  }
202
203  /**
204   * Private constructor used constructing HRegionInfo for the
205   * first meta regions
206   */
207  private HRegionInfo(long regionId, TableName tableName) {
208    this(regionId, tableName, DEFAULT_REPLICA_ID);
209  }
210
211  public HRegionInfo(long regionId, TableName tableName, int replicaId) {
212    super();
213    this.regionId = regionId;
214    this.tableName = tableName;
215    this.replicaId = replicaId;
216    // Note: First Meta region replicas names are in old format
217    this.regionName = createRegionName(tableName, null, regionId, replicaId, false);
218    setHashCode();
219  }
220
221  public HRegionInfo(final TableName tableName) {
222    this(tableName, null, null);
223  }
224
225  /**
226   * Construct HRegionInfo with explicit parameters
227   *
228   * @param tableName the table name
229   * @param startKey first key in region
230   * @param endKey end of key range
231   * @throws IllegalArgumentException
232   */
233  public HRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey)
234  throws IllegalArgumentException {
235    this(tableName, startKey, endKey, false);
236  }
237
238  /**
239   * Construct HRegionInfo with explicit parameters
240   *
241   * @param tableName the table descriptor
242   * @param startKey first key in region
243   * @param endKey end of key range
244   * @param split true if this region has split and we have daughter regions
245   * regions that may or may not hold references to this region.
246   * @throws IllegalArgumentException
247   */
248  public HRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
249      final boolean split)
250  throws IllegalArgumentException {
251    this(tableName, startKey, endKey, split, System.currentTimeMillis());
252  }
253
254  /**
255   * Construct HRegionInfo with explicit parameters
256   *
257   * @param tableName the table descriptor
258   * @param startKey first key in region
259   * @param endKey end of key range
260   * @param split true if this region has split and we have daughter regions
261   * regions that may or may not hold references to this region.
262   * @param regionid Region id to use.
263   * @throws IllegalArgumentException
264   */
265  public HRegionInfo(final TableName tableName, final byte[] startKey,
266                     final byte[] endKey, final boolean split, final long regionid)
267  throws IllegalArgumentException {
268    this(tableName, startKey, endKey, split, regionid, DEFAULT_REPLICA_ID);
269  }
270
271  /**
272   * Construct HRegionInfo with explicit parameters
273   *
274   * @param tableName the table descriptor
275   * @param startKey first key in region
276   * @param endKey end of key range
277   * @param split true if this region has split and we have daughter regions
278   * regions that may or may not hold references to this region.
279   * @param regionid Region id to use.
280   * @param replicaId the replicaId to use
281   * @throws IllegalArgumentException
282   */
283  public HRegionInfo(final TableName tableName, final byte[] startKey,
284                     final byte[] endKey, final boolean split, final long regionid,
285                     final int replicaId)
286    throws IllegalArgumentException {
287    super();
288    if (tableName == null) {
289      throw new IllegalArgumentException("TableName cannot be null");
290    }
291    this.tableName = tableName;
292    this.offLine = false;
293    this.regionId = regionid;
294    this.replicaId = replicaId;
295    if (this.replicaId > MAX_REPLICA_ID) {
296      throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
297    }
298
299    this.regionName = createRegionName(this.tableName, startKey, regionId, replicaId, true);
300
301    this.split = split;
302    this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
303    this.startKey = startKey == null?
304      HConstants.EMPTY_START_ROW: startKey.clone();
305    this.tableName = tableName;
306    setHashCode();
307  }
308
309  /**
310   * Costruct a copy of another HRegionInfo
311   *
312   * @param other
313   */
314  public HRegionInfo(RegionInfo other) {
315    super();
316    this.endKey = other.getEndKey();
317    this.offLine = other.isOffline();
318    this.regionId = other.getRegionId();
319    this.regionName = other.getRegionName();
320    this.split = other.isSplit();
321    this.startKey = other.getStartKey();
322    this.hashCode = other.hashCode();
323    this.encodedName = other.getEncodedName();
324    this.tableName = other.getTable();
325    this.replicaId = other.getReplicaId();
326  }
327
328  public HRegionInfo(HRegionInfo other, int replicaId) {
329    this(other);
330    this.replicaId = replicaId;
331    this.setHashCode();
332  }
333
334  /**
335   * Make a region name of passed parameters.
336   * @param tableName
337   * @param startKey Can be null
338   * @param regionid Region id (Usually timestamp from when region was created).
339   * @param newFormat should we create the region name in the new format
340   *                  (such that it contains its encoded name?).
341   * @return Region name made of passed tableName, startKey and id
342   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
343   *             Use {@link RegionInfo#createRegionName(TableName, byte[], long, boolean)}.
344   */
345  @Deprecated
346  @InterfaceAudience.Private
347  public static byte [] createRegionName(final TableName tableName,
348      final byte [] startKey, final long regionid, boolean newFormat) {
349    return RegionInfo.createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
350  }
351
352  /**
353   * Make a region name of passed parameters.
354   * @param tableName
355   * @param startKey Can be null
356   * @param id Region id (Usually timestamp from when region was created).
357   * @param newFormat should we create the region name in the new format
358   *                  (such that it contains its encoded name?).
359   * @return Region name made of passed tableName, startKey and id
360   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
361   *             Use {@link RegionInfo#createRegionName(TableName, byte[], String, boolean)}.
362   */
363  @Deprecated
364  @InterfaceAudience.Private
365  public static byte [] createRegionName(final TableName tableName,
366      final byte [] startKey, final String id, boolean newFormat) {
367    return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
368  }
369
370  /**
371   * Make a region name of passed parameters.
372   * @param tableName
373   * @param startKey Can be null
374   * @param regionid Region id (Usually timestamp from when region was created).
375   * @param replicaId
376   * @param newFormat should we create the region name in the new format
377   *                  (such that it contains its encoded name?).
378   * @return Region name made of passed tableName, startKey, id and replicaId
379   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
380   *             Use {@link RegionInfo#createRegionName(TableName, byte[], long, int, boolean)}.
381   */
382  @Deprecated
383  @InterfaceAudience.Private
384  public static byte [] createRegionName(final TableName tableName,
385      final byte [] startKey, final long regionid, int replicaId, boolean newFormat) {
386    return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
387        replicaId, newFormat);
388  }
389
390  /**
391   * Make a region name of passed parameters.
392   * @param tableName
393   * @param startKey Can be null
394   * @param id Region id (Usually timestamp from when region was created).
395   * @param newFormat should we create the region name in the new format
396   *                  (such that it contains its encoded name?).
397   * @return Region name made of passed tableName, startKey and id
398   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
399   *             Use {@link RegionInfo#createRegionName(TableName, byte[], byte[], boolean)}.
400   */
401  @Deprecated
402  @InterfaceAudience.Private
403  public static byte [] createRegionName(final TableName tableName,
404      final byte [] startKey, final byte [] id, boolean newFormat) {
405    return RegionInfo.createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
406  }
407  /**
408   * Make a region name of passed parameters.
409   * @param tableName
410   * @param startKey Can be null
411   * @param id Region id (Usually timestamp from when region was created).
412   * @param replicaId
413   * @param newFormat should we create the region name in the new format
414   * @return Region name made of passed tableName, startKey, id and replicaId
415   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
416   *             Use {@link RegionInfo#createRegionName(TableName, byte[], byte[], int, boolean)}.
417   */
418  @Deprecated
419  @InterfaceAudience.Private
420  public static byte [] createRegionName(final TableName tableName,
421      final byte [] startKey, final byte [] id, final int replicaId, boolean newFormat) {
422    return RegionInfo.createRegionName(tableName, startKey, id, replicaId, newFormat);
423  }
424
425  /**
426   * Gets the table name from the specified region name.
427   * @param regionName to extract the table name from
428   * @return Table name
429   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
430   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getTable(byte[])}.
431   */
432  @Deprecated
433  public static TableName getTable(final byte [] regionName) {
434    return RegionInfo.getTable(regionName);
435  }
436
437  /**
438   * Gets the start key from the specified region name.
439   * @param regionName
440   * @return Start key.
441   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
442   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getStartKey(byte[])}.
443   */
444  @Deprecated
445  public static byte[] getStartKey(final byte[] regionName) throws IOException {
446    return RegionInfo.getStartKey(regionName);
447  }
448
449  /**
450   * Separate elements of a regionName.
451   * @param regionName
452   * @return Array of byte[] containing tableName, startKey and id
453   * @throws IOException
454   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
455   *             Use {@link RegionInfo#parseRegionName(byte[])}.
456   */
457  @Deprecated
458  @InterfaceAudience.Private
459  public static byte [][] parseRegionName(final byte [] regionName)
460  throws IOException {
461    return RegionInfo.parseRegionName(regionName);
462  }
463
464  /**
465   *
466   * @param regionName
467   * @return if region name is encoded.
468   * @throws IOException
469   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
470   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#isEncodedRegionName(byte[])}.
471   */
472  @Deprecated
473  public static boolean isEncodedRegionName(byte[] regionName) throws IOException {
474    return RegionInfo.isEncodedRegionName(regionName);
475  }
476
477  /** @return the regionId */
478  @Override
479  public long getRegionId(){
480    return regionId;
481  }
482
483  /**
484   * @return the regionName as an array of bytes.
485   * @see #getRegionNameAsString()
486   */
487  @Override
488  public byte [] getRegionName(){
489    return regionName;
490  }
491
492  /**
493   * @return Region name as a String for use in logging, etc.
494   */
495  @Override
496  public String getRegionNameAsString() {
497    if (RegionInfo.hasEncodedName(this.regionName)) {
498      // new format region names already have their encoded name.
499      return Bytes.toStringBinary(this.regionName);
500    }
501
502    // old format. regionNameStr doesn't have the region name.
503    //
504    //
505    return Bytes.toStringBinary(this.regionName) + "." + this.getEncodedName();
506  }
507
508  /**
509   * @return the encoded region name
510   */
511  @Override
512  public synchronized String getEncodedName() {
513    if (this.encodedName == null) {
514      this.encodedName = RegionInfo.encodeRegionName(this.regionName);
515    }
516    return this.encodedName;
517  }
518
519  @Override
520  public synchronized byte [] getEncodedNameAsBytes() {
521    if (this.encodedNameAsBytes == null) {
522      this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
523    }
524    return this.encodedNameAsBytes;
525  }
526
527  /**
528   * @return the startKey
529   */
530  @Override
531  public byte [] getStartKey(){
532    return startKey;
533  }
534
535  /**
536   * @return the endKey
537   */
538  @Override
539  public byte [] getEndKey(){
540    return endKey;
541  }
542
543  /**
544   * Get current table name of the region
545   * @return TableName
546   */
547  @Override
548  public TableName getTable() {
549    // This method name should be getTableName but there was already a method getTableName
550    // that returned a byte array.  It is unfortunate given everywhere else, getTableName returns
551    // a TableName instance.
552    if (tableName == null || tableName.getName().length == 0) {
553      tableName = getTable(getRegionName());
554    }
555    return this.tableName;
556  }
557
558  /**
559   * Returns true if the given inclusive range of rows is fully contained
560   * by this region. For example, if the region is foo,a,g and this is
561   * passed ["b","c"] or ["a","c"] it will return true, but if this is passed
562   * ["b","z"] it will return false.
563   * @throws IllegalArgumentException if the range passed is invalid (ie. end &lt; start)
564   */
565  @Override
566  public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
567    if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
568      throw new IllegalArgumentException(
569      "Invalid range: " + Bytes.toStringBinary(rangeStartKey) +
570      " > " + Bytes.toStringBinary(rangeEndKey));
571    }
572
573    boolean firstKeyInRange = Bytes.compareTo(rangeStartKey, startKey) >= 0;
574    boolean lastKeyInRange =
575      Bytes.compareTo(rangeEndKey, endKey) < 0 ||
576      Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY);
577    return firstKeyInRange && lastKeyInRange;
578  }
579
580  /**
581   * @return true if the given row falls in this region.
582   */
583  @Override
584  public boolean containsRow(byte[] row) {
585    return Bytes.compareTo(row, startKey) >= 0 &&
586      (Bytes.compareTo(row, endKey) < 0 ||
587       Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
588  }
589
590  /**
591   * @return true if this region is from hbase:meta
592   */
593  public boolean isMetaTable() {
594    return isMetaRegion();
595  }
596
597  /**
598   * @return true if this region is a meta region
599   */
600  @Override
601  public boolean isMetaRegion() {
602     return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTable());
603  }
604
605  /**
606   * @return true if this region is from a system table
607   */
608  public boolean isSystemTable() {
609    return tableName.isSystemTable();
610  }
611
612  /**
613   * @return true if has been split and has daughters.
614   */
615  @Override
616  public boolean isSplit() {
617    return this.split;
618  }
619
620  /**
621   * @param split set split status
622   */
623  public void setSplit(boolean split) {
624    this.split = split;
625  }
626
627  /**
628   * @return true if this region is offline.
629   */
630  @Override
631  public boolean isOffline() {
632    return this.offLine;
633  }
634
635  /**
636   * The parent of a region split is offline while split daughters hold
637   * references to the parent. Offlined regions are closed.
638   * @param offLine Set online/offline status.
639   */
640  public void setOffline(boolean offLine) {
641    this.offLine = offLine;
642  }
643
644  /**
645   * @return true if this is a split parent region.
646   */
647  @Override
648  public boolean isSplitParent() {
649    if (!isSplit()) return false;
650    if (!isOffline()) {
651      LOG.warn("Region is split but NOT offline: " + getRegionNameAsString());
652    }
653    return true;
654  }
655
656  /**
657   * Returns the region replica id
658   * @return returns region replica id
659   */
660  @Override
661  public int getReplicaId() {
662    return replicaId;
663  }
664
665  /**
666   * @see java.lang.Object#toString()
667   */
668  @Override
669  public String toString() {
670    return "{ENCODED => " + getEncodedName() + ", " +
671      HConstants.NAME + " => '" + Bytes.toStringBinary(this.regionName)
672      + "', STARTKEY => '" +
673      Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
674      Bytes.toStringBinary(this.endKey) + "'" +
675      (isOffline()? ", OFFLINE => true": "") +
676      (isSplit()? ", SPLIT => true": "") +
677      ((replicaId > 0)? ", REPLICA_ID => " + replicaId : "") + "}";
678  }
679
680  /**
681   * @see java.lang.Object#equals(java.lang.Object)
682   */
683  @Override
684  public boolean equals(Object o) {
685    if (this == o) {
686      return true;
687    }
688    if (o == null) {
689      return false;
690    }
691    if (!(o instanceof HRegionInfo)) {
692      return false;
693    }
694    return this.compareTo((HRegionInfo)o) == 0;
695  }
696
697  /**
698   * @see java.lang.Object#hashCode()
699   */
700  @Override
701  public int hashCode() {
702    return this.hashCode;
703  }
704
705  //
706  // Comparable
707  //
708
709  @Override
710  public int compareTo(HRegionInfo o) {
711    return RegionInfo.COMPARATOR.compare(this, o);
712  }
713
714  /**
715   * @return Comparator to use comparing {@link KeyValue}s.
716   * @deprecated Use Region#getCellComparator().  deprecated for hbase 2.0, remove for hbase 3.0
717   */
718  @Deprecated
719  public KVComparator getComparator() {
720    return isMetaRegion()?
721        KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
722  }
723
724  /**
725   * Convert a HRegionInfo to the protobuf RegionInfo
726   *
727   * @return the converted RegionInfo
728   */
729  HBaseProtos.RegionInfo convert() {
730    return convert(this);
731  }
732
733  /**
734   * Convert a HRegionInfo to a RegionInfo
735   *
736   * @param info the HRegionInfo to convert
737   * @return the converted RegionInfo
738   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
739   *             Use toRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
740   *             in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
741   */
742  @Deprecated
743  @InterfaceAudience.Private
744  public static HBaseProtos.RegionInfo convert(final HRegionInfo info) {
745    return ProtobufUtil.toRegionInfo(info);
746  }
747
748  /**
749   * Convert a RegionInfo to a HRegionInfo
750   *
751   * @param proto the RegionInfo to convert
752   * @return the converted HRegionInfo
753   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
754   *             Use toRegionInfo(HBaseProtos.RegionInfo)
755   *             in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
756   */
757  @Deprecated
758  @InterfaceAudience.Private
759  public static HRegionInfo convert(final HBaseProtos.RegionInfo proto) {
760    RegionInfo ri = ProtobufUtil.toRegionInfo(proto);
761    // This is hack of what is in RegionReplicaUtil but it is doing translation of
762    // RegionInfo into HRegionInfo which is what is wanted here.
763    HRegionInfo hri;
764    if (ri.isMetaRegion()) {
765      hri = ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID ?
766      HRegionInfo.FIRST_META_REGIONINFO :
767      new HRegionInfo(ri.getRegionId(), ri.getTable(), ri.getReplicaId());
768    } else {
769      hri = new HRegionInfo(
770        ri.getTable(),
771        ri.getStartKey(),
772        ri.getEndKey(),
773        ri.isSplit(),
774        ri.getRegionId(),
775        ri.getReplicaId());
776      if (proto.hasOffline()) {
777        hri.setOffline(proto.getOffline());
778      }
779    }
780    return hri;
781  }
782
783  /**
784   * @return This instance serialized as protobuf w/ a magic pb prefix.
785   * @see #parseFrom(byte[])
786   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
787   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#toByteArray(RegionInfo)}.
788   */
789  @Deprecated
790  public byte [] toByteArray() {
791    return RegionInfo.toByteArray(this);
792  }
793
794  /**
795   * @return A deserialized {@link HRegionInfo}
796   * or null if we failed deserialize or passed bytes null
797   * @see #toByteArray()
798   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
799   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[])}.
800   */
801  @Deprecated
802  public static HRegionInfo parseFromOrNull(final byte [] bytes) {
803    if (bytes == null) return null;
804    return parseFromOrNull(bytes, 0, bytes.length);
805  }
806
807  /**
808   * @return A deserialized {@link HRegionInfo} or null
809   *  if we failed deserialize or passed bytes null
810   * @see #toByteArray()
811   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
812   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[], int, int)}.
813   */
814  @Deprecated
815  public static HRegionInfo parseFromOrNull(final byte [] bytes, int offset, int len) {
816    if (bytes == null || len <= 0) return null;
817    try {
818      return parseFrom(bytes, offset, len);
819    } catch (DeserializationException e) {
820      return null;
821    }
822  }
823
824  /**
825   * @param bytes A pb RegionInfo serialized with a pb magic prefix.
826   * @return A deserialized {@link HRegionInfo}
827   * @throws DeserializationException
828   * @see #toByteArray()
829   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
830   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[])}.
831   */
832  public static HRegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
833    if (bytes == null) return null;
834    return parseFrom(bytes, 0, bytes.length);
835  }
836
837  /**
838   * @param bytes A pb RegionInfo serialized with a pb magic prefix.
839   * @param offset starting point in the byte array
840   * @param len length to read on the byte array
841   * @return A deserialized {@link HRegionInfo}
842   * @throws DeserializationException
843   * @see #toByteArray()
844   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
845   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[], int, int)}.
846   */
847  @Deprecated
848  public static HRegionInfo parseFrom(final byte [] bytes, int offset, int len)
849      throws DeserializationException {
850    if (ProtobufUtil.isPBMagicPrefix(bytes, offset, len)) {
851      int pblen = ProtobufUtil.lengthOfPBMagic();
852      try {
853        HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
854        ProtobufUtil.mergeFrom(builder, bytes, pblen + offset, len - pblen);
855        HBaseProtos.RegionInfo ri = builder.build();
856        return convert(ri);
857      } catch (IOException e) {
858        throw new DeserializationException(e);
859      }
860    } else {
861      throw new DeserializationException("PB encoded HRegionInfo expected");
862    }
863  }
864
865  /**
866   * Use this instead of {@link #toByteArray()} when writing to a stream and you want to use
867   * the pb mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want).
868   * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
869   * @throws IOException
870   * @see #toByteArray()
871   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
872   *             Use {@link RegionInfo#toDelimitedByteArray(RegionInfo)}.
873   */
874  @Deprecated
875  public byte [] toDelimitedByteArray() throws IOException {
876    return RegionInfo.toDelimitedByteArray(this);
877  }
878
879  /**
880   * Get the descriptive name as {@link RegionState} does it but with hidden
881   * startkey optionally
882   * @param state
883   * @param conf
884   * @return descriptive string
885   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
886   *             Use RegionInfoDisplay#getDescriptiveNameFromRegionStateForDisplay(RegionState, Configuration)
887   *             over in hbase-server module.
888   */
889  @Deprecated
890  @InterfaceAudience.Private
891  public static String getDescriptiveNameFromRegionStateForDisplay(RegionState state,
892      Configuration conf) {
893    return RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf);
894  }
895
896  /**
897   * Get the end key for display. Optionally hide the real end key.
898   * @param hri
899   * @param conf
900   * @return the endkey
901   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
902   *             Use RegionInfoDisplay#getEndKeyForDisplay(RegionInfo, Configuration)
903   *             over in hbase-server module.
904   */
905  @Deprecated
906  @InterfaceAudience.Private
907  public static byte[] getEndKeyForDisplay(HRegionInfo hri, Configuration conf) {
908    return RegionInfoDisplay.getEndKeyForDisplay(hri, conf);
909  }
910
911  /**
912   * Get the start key for display. Optionally hide the real start key.
913   * @param hri
914   * @param conf
915   * @return the startkey
916   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
917   *             Use RegionInfoDisplay#getStartKeyForDisplay(RegionInfo, Configuration)
918   *             over in hbase-server module.
919   */
920  @Deprecated
921  @InterfaceAudience.Private
922  public static byte[] getStartKeyForDisplay(HRegionInfo hri, Configuration conf) {
923    return RegionInfoDisplay.getStartKeyForDisplay(hri, conf);
924  }
925
926  /**
927   * Get the region name for display. Optionally hide the start key.
928   * @param hri
929   * @param conf
930   * @return region name as String
931   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
932   *             Use RegionInfoDisplay#getRegionNameAsStringForDisplay(RegionInfo, Configuration)
933   *             over in hbase-server module.
934   */
935  @Deprecated
936  @InterfaceAudience.Private
937  public static String getRegionNameAsStringForDisplay(HRegionInfo hri, Configuration conf) {
938    return RegionInfoDisplay.getRegionNameAsStringForDisplay(hri, conf);
939  }
940
941  /**
942   * Get the region name for display. Optionally hide the start key.
943   * @param hri
944   * @param conf
945   * @return region name bytes
946   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
947   *             Use RegionInfoDisplay#getRegionNameForDisplay(RegionInfo, Configuration)
948   *             over in hbase-server module.
949   */
950  @Deprecated
951  @InterfaceAudience.Private
952  public static byte[] getRegionNameForDisplay(HRegionInfo hri, Configuration conf) {
953    return RegionInfoDisplay.getRegionNameForDisplay(hri, conf);
954  }
955
956  /**
957   * Parses an HRegionInfo instance from the passed in stream.  Presumes the HRegionInfo was
958   * serialized to the stream with {@link #toDelimitedByteArray()}
959   * @param in
960   * @return An instance of HRegionInfo.
961   * @throws IOException
962   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
963   *             Use {@link RegionInfo#parseFrom(DataInputStream)}.
964   */
965  @Deprecated
966  @InterfaceAudience.Private
967  public static HRegionInfo parseFrom(final DataInputStream in) throws IOException {
968    // I need to be able to move back in the stream if this is not a pb serialization so I can
969    // do the Writable decoding instead.
970    int pblen = ProtobufUtil.lengthOfPBMagic();
971    byte [] pbuf = new byte[pblen];
972    if (in.markSupported()) { //read it with mark()
973      in.mark(pblen);
974    }
975
976    //assumption: if Writable serialization, it should be longer than pblen.
977    int read = in.read(pbuf);
978    if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
979    if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
980      return convert(HBaseProtos.RegionInfo.parseDelimitedFrom(in));
981    } else {
982      throw new IOException("PB encoded HRegionInfo expected");
983    }
984  }
985
986  /**
987   * Serializes given HRegionInfo's as a byte array. Use this instead of {@link #toByteArray()} when
988   * writing to a stream and you want to use the pb mergeDelimitedFrom (w/o the delimiter, pb reads
989   * to EOF which may not be what you want). {@link #parseDelimitedFrom(byte[], int, int)} can
990   * be used to read back the instances.
991   * @param infos HRegionInfo objects to serialize
992   * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
993   * @throws IOException
994   * @see #toByteArray()
995   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
996   *             Use {@link RegionInfo#toDelimitedByteArray(RegionInfo...)}.
997   */
998  @Deprecated
999  @InterfaceAudience.Private
1000  public static byte[] toDelimitedByteArray(HRegionInfo... infos) throws IOException {
1001    return RegionInfo.toDelimitedByteArray(infos);
1002  }
1003
1004  /**
1005   * Parses all the HRegionInfo instances from the passed in stream until EOF. Presumes the
1006   * HRegionInfo's were serialized to the stream with {@link #toDelimitedByteArray()}
1007   * @param bytes serialized bytes
1008   * @param offset the start offset into the byte[] buffer
1009   * @param length how far we should read into the byte[] buffer
1010   * @return All the hregioninfos that are in the byte array. Keeps reading till we hit the end.
1011   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
1012   *             Use {@link RegionInfo#parseDelimitedFrom(byte[], int, int)}.
1013   */
1014  @Deprecated
1015  public static List<HRegionInfo> parseDelimitedFrom(final byte[] bytes, final int offset,
1016      final int length) throws IOException {
1017    if (bytes == null) {
1018      throw new IllegalArgumentException("Can't build an object with empty bytes array");
1019    }
1020    DataInputBuffer in = new DataInputBuffer();
1021    List<HRegionInfo> hris = new ArrayList<>();
1022    try {
1023      in.reset(bytes, offset, length);
1024      while (in.available() > 0) {
1025        HRegionInfo hri = parseFrom(in);
1026        hris.add(hri);
1027      }
1028    } finally {
1029      in.close();
1030    }
1031    return hris;
1032  }
1033
1034  /**
1035   * Check whether two regions are adjacent
1036   * @param regionA
1037   * @param regionB
1038   * @return true if two regions are adjacent
1039   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
1040   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#areAdjacent(RegionInfo, RegionInfo)}.
1041   */
1042  @Deprecated
1043  public static boolean areAdjacent(HRegionInfo regionA, HRegionInfo regionB) {
1044    return RegionInfo.areAdjacent(regionA, regionB);
1045  }
1046}