You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2017/09/14 21:25:15 UTC

[1/2] hbase git commit: HBASE-17980 Any HRegionInfo we give out should be immutable

Repository: hbase
Updated Branches:
  refs/heads/master a4277f37e -> 58988cb53


http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
new file mode 100644
index 0000000..a76767d
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
@@ -0,0 +1,624 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.util.Arrays;
+
+@InterfaceAudience.Private
+public class RegionInfoBuilder {
+  private static final Log LOG = LogFactory.getLog(RegionInfoBuilder.class);
+
+  /** A non-capture group so that this can be embedded. */
+  public static final String ENCODED_REGION_NAME_REGEX = "(?:[a-f0-9]+)";
+
+  private static final int MAX_REPLICA_ID = 0xFFFF;
+
+  //TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
+  public static final String NO_HASH = null;
+
+  /**
+   * RegionInfo for first meta region
+   * You cannot use this builder to make an instance of the {@link #FIRST_META_REGIONINFO}.
+   * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+   * just RI so the mutable methods are not available (unless you go casting); it appears
+   * as immutable (I tried adding Immutable type but it just makes a mess).
+   */
+  // TODO: How come Meta regions still do not have encoded region names? Fix.
+  // hbase:meta,,1.1588230740 should be the hbase:meta first region name.
+  public static final RegionInfo FIRST_META_REGIONINFO =
+    new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID);
+
+  private MutableRegionInfo content = null;
+
+  public static RegionInfoBuilder newBuilder(TableName tableName) {
+    return new RegionInfoBuilder(tableName);
+  }
+
+  public static RegionInfoBuilder newBuilder(RegionInfo regionInfo) {
+    return new RegionInfoBuilder(regionInfo);
+  }
+
+  private RegionInfoBuilder(TableName tableName) {
+    this.content = new MutableRegionInfo(tableName);
+  }
+
+  private RegionInfoBuilder(RegionInfo regionInfo) {
+    this.content = new MutableRegionInfo(regionInfo);
+  }
+
+  public RegionInfoBuilder setStartKey(byte[] startKey) {
+    content.setStartKey(startKey);
+    return this;
+  }
+
+  public RegionInfoBuilder setEndKey(byte[] endKey) {
+    content.setEndKey(endKey);
+    return this;
+  }
+
+  public RegionInfoBuilder setRegionId(long regionId) {
+    content.setRegionId(regionId);
+    return this;
+  }
+
+  public RegionInfoBuilder setReplicaId(int replicaId) {
+    content.setReplicaId(replicaId);
+    return this;
+  }
+
+  public RegionInfoBuilder setSplit(boolean isSplit) {
+    content.setSplit(isSplit);
+    return this;
+  }
+
+  public RegionInfoBuilder setOffline(boolean isOffline) {
+    content.setOffline(isOffline);
+    return this;
+  }
+
+  public RegionInfo build() {
+    RegionInfo ri = new MutableRegionInfo(content);
+    // Run a late check that we are not creating default meta region.
+    if (ri.getTable().equals(TableName.META_TABLE_NAME) &&
+        ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
+      throw new IllegalArgumentException("Cannot create the default meta region; " +
+        "use static define FIRST_META_REGIONINFO");
+    }
+    return new MutableRegionInfo(content);
+  }
+
+  /**
+   * An implementation of RegionInfo that adds mutable methods so can build a RegionInfo instance.
+   */
+  @InterfaceAudience.Private
+  static class MutableRegionInfo implements RegionInfo, Comparable<RegionInfo> {
+    /**
+     * The new format for a region name contains its encodedName at the end.
+     * The encoded name also serves as the directory name for the region
+     * in the filesystem.
+     *
+     * New region name format:
+     *    &lt;tablename>,,&lt;startkey>,&lt;regionIdTimestamp>.&lt;encodedName>.
+     * where,
+     *    &lt;encodedName> is a hex version of the MD5 hash of
+     *    &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
+     *
+     * The old region name format:
+     *    &lt;tablename>,&lt;startkey>,&lt;regionIdTimestamp>
+     * For region names in the old format, the encoded name is a 32-bit
+     * JenkinsHash integer value (in its decimal notation, string form).
+     *<p>
+     * **NOTE**
+     *
+     * The first hbase:meta region, and regions created by an older
+     * version of HBase (0.20 or prior) will continue to use the
+     * old region name format.
+     */
+
+    // This flag is in the parent of a split while the parent is still referenced
+    // by daughter regions.  We USED to set this flag when we disabled a table
+    // but now table state is kept up in zookeeper as of 0.90.0 HBase.
+    private boolean offLine = false;
+    private boolean split = false;
+    private long regionId = -1;
+    private int replicaId = RegionInfo.DEFAULT_REPLICA_ID;
+    private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
+    private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
+    private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
+    private int hashCode = -1;
+    private String encodedName;
+    private byte [] encodedNameAsBytes;
+    // Current TableName
+    private TableName tableName;
+
+    private void setHashCode() {
+      int result = Arrays.hashCode(this.regionName);
+      result ^= this.regionId;
+      result ^= Arrays.hashCode(this.startKey);
+      result ^= Arrays.hashCode(this.endKey);
+      result ^= Boolean.valueOf(this.offLine).hashCode();
+      result ^= Arrays.hashCode(this.tableName.getName());
+      result ^= this.replicaId;
+      this.hashCode = result;
+    }
+
+    /**
+     * Private constructor used constructing MutableRegionInfo for the
+     * first meta regions
+     */
+    private MutableRegionInfo(long regionId, TableName tableName, int replicaId) {
+      // This constructor is currently private for making hbase:meta region only.
+      super();
+      this.regionId = regionId;
+      this.tableName = tableName;
+      this.replicaId = replicaId;
+      // Note: First Meta region replicas names are in old format so we pass false here.
+      this.regionName =
+        RegionInfo.createRegionName(tableName, null, regionId, replicaId, false);
+      setHashCode();
+    }
+
+    MutableRegionInfo(final TableName tableName) {
+      this(tableName, null, null);
+    }
+
+    /**
+     * Construct MutableRegionInfo with explicit parameters
+     *
+     * @param tableName the table name
+     * @param startKey first key in region
+     * @param endKey end of key range
+     * @throws IllegalArgumentException
+     */
+    MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey)
+    throws IllegalArgumentException {
+      this(tableName, startKey, endKey, false);
+    }
+
+    /**
+     * Construct MutableRegionInfo with explicit parameters
+     *
+     * @param tableName the table descriptor
+     * @param startKey first key in region
+     * @param endKey end of key range
+     * @param split true if this region has split and we have daughter regions
+     * regions that may or may not hold references to this region.
+     * @throws IllegalArgumentException
+     */
+    MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
+        final boolean split)
+    throws IllegalArgumentException {
+      this(tableName, startKey, endKey, split, System.currentTimeMillis());
+    }
+
+    /**
+     * Construct MutableRegionInfo with explicit parameters
+     *
+     * @param tableName the table descriptor
+     * @param startKey first key in region
+     * @param endKey end of key range
+     * @param split true if this region has split and we have daughter regions
+     * regions that may or may not hold references to this region.
+     * @param regionid Region id to use.
+     * @throws IllegalArgumentException
+     */
+    MutableRegionInfo(final TableName tableName, final byte[] startKey,
+                       final byte[] endKey, final boolean split, final long regionid)
+    throws IllegalArgumentException {
+      this(tableName, startKey, endKey, split, regionid, RegionInfo.DEFAULT_REPLICA_ID);
+    }
+
+    /**
+     * Construct MutableRegionInfo with explicit parameters
+     *
+     * @param tableName the table descriptor
+     * @param startKey first key in region
+     * @param endKey end of key range
+     * @param split true if this region has split and we have daughter regions
+     * regions that may or may not hold references to this region.
+     * @param regionid Region id to use.
+     * @param replicaId the replicaId to use
+     * @throws IllegalArgumentException
+     */
+    MutableRegionInfo(final TableName tableName, final byte[] startKey,
+                       final byte[] endKey, final boolean split, final long regionid,
+                       final int replicaId)
+      throws IllegalArgumentException {
+      super();
+      if (tableName == null) {
+        throw new IllegalArgumentException("TableName cannot be null");
+      }
+      this.tableName = tableName;
+      this.offLine = false;
+      this.regionId = regionid;
+      this.replicaId = replicaId;
+      if (this.replicaId > MAX_REPLICA_ID) {
+        throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
+      }
+
+      this.regionName = RegionInfo.createRegionName(this.tableName, startKey, regionId, replicaId,
+        !this.tableName.equals(TableName.META_TABLE_NAME));
+
+      this.split = split;
+      this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
+      this.startKey = startKey == null?
+        HConstants.EMPTY_START_ROW: startKey.clone();
+      this.tableName = tableName;
+      setHashCode();
+    }
+
+    /**
+     * Construct MutableRegionInfo.
+     * Only for RegionInfoBuilder to use.
+     * @param other
+     */
+    MutableRegionInfo(MutableRegionInfo other, boolean isMetaRegion) {
+      super();
+      if (other.getTable() == null) {
+        throw new IllegalArgumentException("TableName cannot be null");
+      }
+      this.tableName = other.getTable();
+      this.offLine = other.isOffline();
+      this.regionId = other.getRegionId();
+      this.replicaId = other.getReplicaId();
+      if (this.replicaId > MAX_REPLICA_ID) {
+        throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
+      }
+
+      if(isMetaRegion) {
+        // Note: First Meta region replicas names are in old format
+        this.regionName = RegionInfo.createRegionName(
+                other.getTable(), null, other.getRegionId(),
+                other.getReplicaId(), false);
+      } else {
+        this.regionName = RegionInfo.createRegionName(
+                other.getTable(), other.getStartKey(), other.getRegionId(),
+                other.getReplicaId(), true);
+      }
+
+      this.split = other.isSplit();
+      this.endKey = other.getEndKey() == null? HConstants.EMPTY_END_ROW: other.getEndKey().clone();
+      this.startKey = other.getStartKey() == null?
+        HConstants.EMPTY_START_ROW: other.getStartKey().clone();
+      this.tableName = other.getTable();
+      setHashCode();
+    }
+
+    /**
+     * Construct a copy of RegionInfo as MutableRegionInfo.
+     * Only for RegionInfoBuilder to use.
+     * @param regionInfo
+     */
+    MutableRegionInfo(RegionInfo regionInfo) {
+      super();
+      this.endKey = regionInfo.getEndKey();
+      this.offLine = regionInfo.isOffline();
+      this.regionId = regionInfo.getRegionId();
+      this.regionName = regionInfo.getRegionName();
+      this.split = regionInfo.isSplit();
+      this.startKey = regionInfo.getStartKey();
+      this.hashCode = regionInfo.hashCode();
+      this.encodedName = regionInfo.getEncodedName();
+      this.tableName = regionInfo.getTable();
+      this.replicaId = regionInfo.getReplicaId();
+    }
+
+    /**
+     * @return Return a short, printable name for this region
+     * (usually encoded name) for us logging.
+     */
+    @Override
+    public String getShortNameToLog() {
+      return RegionInfo.prettyPrint(this.getEncodedName());
+    }
+
+    /** @return the regionId */
+    @Override
+    public long getRegionId(){
+      return regionId;
+    }
+
+    /**
+     * set region id.
+     * @param regionId
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setRegionId(long regionId) {
+      this.regionId = regionId;
+      return this;
+    }
+
+    /**
+     * @return the regionName as an array of bytes.
+     * @see #getRegionNameAsString()
+     */
+    @Override
+    public byte [] getRegionName(){
+      return regionName;
+    }
+
+    /**
+     * set region name.
+     * @param regionName
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setRegionName(byte[] regionName) {
+      this.regionName = regionName;
+      return this;
+    }
+
+    /**
+     * @return Region name as a String for use in logging, etc.
+     */
+    @Override
+    public String getRegionNameAsString() {
+      if (RegionInfo.hasEncodedName(this.regionName)) {
+        // new format region names already have their encoded name.
+        return Bytes.toStringBinary(this.regionName);
+      }
+
+      // old format. regionNameStr doesn't have the region name.
+      //
+      //
+      return Bytes.toStringBinary(this.regionName) + "." + this.getEncodedName();
+    }
+
+    /** @return the encoded region name */
+    @Override
+    public synchronized String getEncodedName() {
+      if (this.encodedName == null) {
+        this.encodedName = RegionInfo.encodeRegionName(this.regionName);
+      }
+      return this.encodedName;
+    }
+
+    @Override
+    public synchronized byte [] getEncodedNameAsBytes() {
+      if (this.encodedNameAsBytes == null) {
+        this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
+      }
+      return this.encodedNameAsBytes;
+    }
+
+    /** @return the startKey */
+    @Override
+    public byte [] getStartKey(){
+      return startKey;
+    }
+
+    /**
+     * @param startKey
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setStartKey(byte[] startKey) {
+      this.startKey = startKey;
+      return this;
+    }
+
+    /** @return the endKey */
+    @Override
+    public byte [] getEndKey(){
+      return endKey;
+    }
+
+    /**
+     * @param endKey
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setEndKey(byte[] endKey) {
+      this.endKey = endKey;
+      return this;
+    }
+
+    /**
+     * Get current table name of the region
+     * @return TableName
+     */
+    @Override
+    public TableName getTable() {
+      // This method name should be getTableName but there was already a method getTableName
+      // that returned a byte array.  It is unfortunate given everywhere else, getTableName returns
+      // a TableName instance.
+      if (tableName == null || tableName.getName().length == 0) {
+        tableName = RegionInfo.getTable(getRegionName());
+      }
+      return this.tableName;
+    }
+
+    /**
+     * Returns true if the given inclusive range of rows is fully contained
+     * by this region. For example, if the region is foo,a,g and this is
+     * passed ["b","c"] or ["a","c"] it will return true, but if this is passed
+     * ["b","z"] it will return false.
+     * @throws IllegalArgumentException if the range passed is invalid (ie. end &lt; start)
+     */
+    @Override
+    public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
+      if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
+        throw new IllegalArgumentException(
+        "Invalid range: " + Bytes.toStringBinary(rangeStartKey) +
+        " > " + Bytes.toStringBinary(rangeEndKey));
+      }
+
+      boolean firstKeyInRange = Bytes.compareTo(rangeStartKey, startKey) >= 0;
+      boolean lastKeyInRange =
+        Bytes.compareTo(rangeEndKey, endKey) < 0 ||
+        Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY);
+      return firstKeyInRange && lastKeyInRange;
+    }
+
+    /**
+     * Return true if the given row falls in this region.
+     */
+    @Override
+    public boolean containsRow(byte[] row) {
+      return Bytes.compareTo(row, startKey) >= 0 &&
+        (Bytes.compareTo(row, endKey) < 0 ||
+         Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
+    }
+
+    /**
+     * @return true if this region is from hbase:meta
+     */
+    @Override
+    public boolean isMetaTable() {
+      return isMetaRegion();
+    }
+
+    /** @return true if this region is a meta region */
+    @Override
+    public boolean isMetaRegion() {
+       return tableName.equals(FIRST_META_REGIONINFO.getTable());
+    }
+
+    /**
+     * @return true if this region is from a system table
+     */
+    @Override
+    public boolean isSystemTable() {
+      return tableName.isSystemTable();
+    }
+
+    /**
+     * @return True if has been split and has daughters.
+     */
+    @Override
+    public boolean isSplit() {
+      return this.split;
+    }
+
+    /**
+     * @param split set split status
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setSplit(boolean split) {
+      this.split = split;
+      return this;
+    }
+
+    /**
+     * @return True if this region is offline.
+     */
+    @Override
+    public boolean isOffline() {
+      return this.offLine;
+    }
+
+    /**
+     * The parent of a region split is offline while split daughters hold
+     * references to the parent. Offlined regions are closed.
+     * @param offLine Set online/offline status.
+     * @return MutableRegionInfo
+     */
+    public MutableRegionInfo setOffline(boolean offLine) {
+      this.offLine = offLine;
+      return this;
+    }
+
+    /**
+     * @return True if this is a split parent region.
+     */
+    @Override
+    public boolean isSplitParent() {
+      if (!isSplit()) return false;
+      if (!isOffline()) {
+        LOG.warn("Region is split but NOT offline: " + getRegionNameAsString());
+      }
+      return true;
+    }
+
+    /**
+     * Returns the region replica id
+     * @return returns region replica id
+     */
+    @Override
+    public int getReplicaId() {
+      return replicaId;
+    }
+
+    public MutableRegionInfo setReplicaId(int replicaId) {
+      this.replicaId = replicaId;
+      return this;
+    }
+
+    /**
+     * @see java.lang.Object#toString()
+     */
+    @Override
+    public String toString() {
+      return "{ENCODED => " + getEncodedName() + ", " +
+        HConstants.NAME + " => '" + Bytes.toStringBinary(this.regionName)
+        + "', STARTKEY => '" +
+        Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
+        Bytes.toStringBinary(this.endKey) + "'" +
+        (isOffline()? ", OFFLINE => true": "") +
+        (isSplit()? ", SPLIT => true": "") +
+        ((replicaId > 0)? ", REPLICA_ID => " + replicaId : "") + "}";
+    }
+
+    /**
+     * @param o
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null) {
+        return false;
+      }
+      if (!(o instanceof RegionInfo)) {
+        return false;
+      }
+      return this.compareTo((RegionInfo)o) == 0;
+    }
+
+    /**
+     * @see java.lang.Object#hashCode()
+     */
+    @Override
+    public int hashCode() {
+      return this.hashCode;
+    }
+
+    @Override
+    public int compareTo(RegionInfo other) {
+      return RegionInfo.COMPARATOR.compare(this, other);
+    }
+
+    /**
+     * @return Comparator to use comparing {@link KeyValue}s.
+     * @deprecated Use Region#getCellComparator().  deprecated for hbase 2.0, remove for hbase 3.0
+     */
+    @Deprecated
+    public KeyValue.KVComparator getComparator() {
+      return isMetaRegion()?
+          KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java
new file mode 100644
index 0000000..7ced1b3
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java
@@ -0,0 +1,135 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+
+/**
+ * Utility used composing RegionInfo for 'display'; e.g. on the web UI
+ */
+@InterfaceAudience.Private
+public class RegionInfoDisplay {
+  public final static String DISPLAY_KEYS_KEY = "hbase.display.keys";
+  public final static byte[] HIDDEN_END_KEY = Bytes.toBytes("hidden-end-key");
+  public final static byte[] HIDDEN_START_KEY = Bytes.toBytes("hidden-start-key");
+
+  /**
+   * Get the descriptive name as {@link RegionState} does it but with hidden
+   * startkey optionally
+   * @return descriptive string
+   */
+  public static String getDescriptiveNameFromRegionStateForDisplay(RegionState state,
+                                                                   Configuration conf) {
+    if (conf.getBoolean(DISPLAY_KEYS_KEY, true)) return state.toDescriptiveString();
+    String descriptiveStringFromState = state.toDescriptiveString();
+    int idx = descriptiveStringFromState.lastIndexOf(" state=");
+    String regionName = getRegionNameAsStringForDisplay(
+    RegionInfoBuilder.newBuilder(state.getRegion()).build(), conf);
+    return regionName + descriptiveStringFromState.substring(idx);
+  }
+
+  /**
+   * Get the end key for display. Optionally hide the real end key.
+   * @return the endkey
+   */
+  public static byte[] getEndKeyForDisplay(RegionInfo ri, Configuration conf) {
+    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
+    if (displayKey) return ri.getEndKey();
+    return HIDDEN_END_KEY;
+  }
+
+  /**
+   * Get the start key for display. Optionally hide the real start key.
+   * @param ri
+   * @param conf
+   * @return the startkey
+   */
+  public static byte[] getStartKeyForDisplay(RegionInfo ri, Configuration conf) {
+    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
+    if (displayKey) return ri.getStartKey();
+    return HIDDEN_START_KEY;
+  }
+
+  /**
+   * Get the region name for display. Optionally hide the start key.
+   * @param ri
+   * @param conf
+   * @return region name as String
+   */
+  public static String getRegionNameAsStringForDisplay(RegionInfo ri, Configuration conf) {
+    return Bytes.toStringBinary(getRegionNameForDisplay(ri, conf));
+  }
+
+  /**
+   * Get the region name for display. Optionally hide the start key.
+   * @param ri
+   * @param conf
+   * @return region name bytes
+   */
+  public static byte[] getRegionNameForDisplay(RegionInfo ri, Configuration conf) {
+    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
+    if (displayKey || ri.getTable().equals(TableName.META_TABLE_NAME)) {
+      return ri.getRegionName();
+    } else {
+      // create a modified regionname with the startkey replaced but preserving
+      // the other parts including the encodedname.
+      try {
+        byte[][]regionNameParts = RegionInfo.parseRegionName(ri.getRegionName());
+        regionNameParts[1] = HIDDEN_START_KEY; //replace the real startkey
+        int len = 0;
+        // get the total length
+        for (byte[] b : regionNameParts) {
+          len += b.length;
+        }
+        byte[] encodedRegionName =
+        Bytes.toBytes(RegionInfo.encodeRegionName(ri.getRegionName()));
+        len += encodedRegionName.length;
+        //allocate some extra bytes for the delimiters and the last '.'
+        byte[] modifiedName = new byte[len + regionNameParts.length + 1];
+        int lengthSoFar = 0;
+        int loopCount = 0;
+        for (byte[] b : regionNameParts) {
+          System.arraycopy(b, 0, modifiedName, lengthSoFar, b.length);
+          lengthSoFar += b.length;
+          if (loopCount++ == 2) modifiedName[lengthSoFar++] = RegionInfo.REPLICA_ID_DELIMITER;
+          else  modifiedName[lengthSoFar++] = HConstants.DELIMITER;
+        }
+        // replace the last comma with '.'
+        modifiedName[lengthSoFar - 1] = RegionInfo.ENC_SEPARATOR;
+        System.arraycopy(encodedRegionName, 0, modifiedName, lengthSoFar,
+        encodedRegionName.length);
+        lengthSoFar += encodedRegionName.length;
+        modifiedName[lengthSoFar] = RegionInfo.ENC_SEPARATOR;
+        return modifiedName;
+      } catch (IOException e) {
+        //LOG.warn("Encountered exception " + e);
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
index 502e0a5..2c1d478 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionReplicaUtil.java
@@ -57,6 +57,7 @@ public class RegionReplicaUtil {
    * @return an HRegionInfo object corresponding to the same range (table, start and
    * end key), but for the given replicaId.
    */
+  @Deprecated // Deprecated for HBase-2.0.0, use #getRegionInfoForReplica
   public static HRegionInfo getRegionInfoForReplica(HRegionInfo regionInfo, int replicaId) {
     if (regionInfo.getReplicaId() == replicaId) {
       return regionInfo;
@@ -73,6 +74,37 @@ public class RegionReplicaUtil {
   }
 
   /**
+   * Returns the RegionInfo for the given replicaId.
+   * RegionInfo's correspond to a range of a table, but more than one
+   * "instance" of the same range can be deployed which are differentiated by
+   * the replicaId.
+   * @param regionInfo
+   * @param replicaId the replicaId to use
+   * @return an RegionInfo object corresponding to the same range (table, start and
+   * end key), but for the given replicaId.
+   */
+  public static RegionInfo getRegionInfoForReplica(RegionInfo regionInfo, int replicaId) {
+    if (regionInfo.getReplicaId() == replicaId) {
+      return regionInfo;
+    }
+    RegionInfoBuilder replicaInfo;
+    RegionInfo ri;
+    if (regionInfo.isMetaRegion()) {
+      ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+    } else {
+      replicaInfo = RegionInfoBuilder.newBuilder(regionInfo.getTable())
+              .setStartKey(regionInfo.getStartKey())
+              .setEndKey(regionInfo.getEndKey())
+              .setSplit(regionInfo.isSplit())
+              .setRegionId(regionInfo.getRegionId())
+              .setReplicaId(replicaId);
+      replicaInfo.setOffline(regionInfo.isOffline());
+      ri = replicaInfo.build();
+    }
+    return ri;
+  }
+
+  /**
    * Returns the HRegionInfo for the default replicaId (0). HRegionInfo's correspond to
    * a range of a table, but more than one "instance" of the same range can be
    * deployed which are differentiated by the replicaId.

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
index 89900d4..dfe04e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHRegionInfo.java
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 
-@InterfaceAudience.Public
+@InterfaceAudience.Private
 class UnmodifyableHRegionInfo extends HRegionInfo {
   /*
    * Creates an unmodifyable copy of an HRegionInfo

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 711315f..3c9738e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -79,7 +79,9 @@ import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLoadStats;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
@@ -3306,4 +3308,66 @@ public final class ProtobufUtil {
     }
     return lockedResourceJsons.toString();
   }
+
+  /**
+   * Convert a RegionInfo to a Proto RegionInfo
+   *
+   * @param info the RegionInfo to convert
+   * @return the converted Proto RegionInfo
+   */
+  public static HBaseProtos.RegionInfo toProtoRegionInfo(final org.apache.hadoop.hbase.client.RegionInfo info) {
+    if (info == null) return null;
+    HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
+    builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
+    builder.setRegionId(info.getRegionId());
+    if (info.getStartKey() != null) {
+      builder.setStartKey(UnsafeByteOperations.unsafeWrap(info.getStartKey()));
+    }
+    if (info.getEndKey() != null) {
+      builder.setEndKey(UnsafeByteOperations.unsafeWrap(info.getEndKey()));
+    }
+    builder.setOffline(info.isOffline());
+    builder.setSplit(info.isSplit());
+    builder.setReplicaId(info.getReplicaId());
+    return builder.build();
+  }
+
+  /**
+   * Convert HBaseProto.RegionInfo to a RegionInfo
+   *
+   * @param proto the RegionInfo to convert
+   * @return the converted RegionInfo
+   */
+  public static org.apache.hadoop.hbase.client.RegionInfo toRegionInfo(final HBaseProtos.RegionInfo proto) {
+    if (proto == null) return null;
+    TableName tableName = ProtobufUtil.toTableName(proto.getTableName());
+    long regionId = proto.getRegionId();
+    int defaultReplicaId = org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
+    int replicaId = proto.hasReplicaId()? proto.getReplicaId(): defaultReplicaId;
+    if (tableName.equals(TableName.META_TABLE_NAME) && replicaId == defaultReplicaId) {
+      return RegionInfoBuilder.FIRST_META_REGIONINFO;
+    }
+    byte[] startKey = null;
+    byte[] endKey = null;
+    if (proto.hasStartKey()) {
+      startKey = proto.getStartKey().toByteArray();
+    }
+    if (proto.hasEndKey()) {
+      endKey = proto.getEndKey().toByteArray();
+    }
+    boolean split = false;
+    if (proto.hasSplit()) {
+      split = proto.getSplit();
+    }
+    RegionInfoBuilder rib = RegionInfoBuilder.newBuilder(tableName)
+    .setStartKey(startKey)
+    .setEndKey(endKey)
+    .setRegionId(regionId)
+    .setReplicaId(replicaId)
+    .setSplit(split);
+    if (proto.hasOffline()) {
+      rib.setOffline(proto.getOffline());
+    }
+    return rib.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index 0787f11..dada632 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -443,7 +443,8 @@ public class MetaTableLocator {
       LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
       return;
     }
-    LOG.info("Setting hbase:meta region location in ZooKeeper as " + serverName);
+    LOG.info("Setting hbase:meta (replicaId=" + replicaId + ") location in ZooKeeper as " +
+      serverName);
     // Make the MetaRegionServer pb and then get its bytes and save this as
     // the znode content.
     MetaRegionServer pbrsr = MetaRegionServer.newBuilder()
@@ -458,7 +459,7 @@ public class MetaTableLocator {
       if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
         LOG.debug("META region location doesn't exist, create it");
       } else {
-        LOG.debug("META region location doesn't exist for replicaId " + replicaId +
+        LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
             ", create it");
       }
       ZKUtil.createAndWatch(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data);

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java
new file mode 100644
index 0000000..4644641
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHRegionInfo.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.Consumer;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import static org.junit.Assert.fail;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test ImmutableHRegionInfo
+ */
+@Category({ClientTests.class, SmallTests.class})
+public class TestImmutableHRegionInfo {
+
+  @Rule
+  public TestName name = new TestName();
+
+  private final List<Consumer<ImmutableHRegionInfo>> TEST_FUNCTIONS = Arrays.asList(
+    hri -> hri.setOffline(true),
+    hri -> hri.setSplit(true)
+  );
+
+  @Test
+  public void testImmutable() {
+    HRegionInfo hri = new HRegionInfo(TableName.valueOf(name.getMethodName()));
+    ImmutableHRegionInfo immutableHri = new ImmutableHRegionInfo(hri);
+
+    TEST_FUNCTIONS.forEach(f -> {
+      try {
+        f.accept(immutableHri);
+        fail("ImmutableHRegionInfo can't be modified !!!");
+      } catch(UnsupportedOperationException e) {
+      }
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java
new file mode 100644
index 0000000..978e8c8
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoDisplay.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionInfoDisplay;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+
+import java.io.IOException;
+
+@Category({MasterTests.class, SmallTests.class})
+public class TestRegionInfoDisplay {
+  @Rule
+  public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+  withLookingForStuckThread(true).build();
+  @Rule public TestName name = new TestName();
+
+  @Test
+  public void testRegionDetailsForDisplay() throws IOException {
+    byte[] startKey = new byte[] {0x01, 0x01, 0x02, 0x03};
+    byte[] endKey = new byte[] {0x01, 0x01, 0x02, 0x04};
+    Configuration conf = new Configuration();
+    conf.setBoolean("hbase.display.keys", false);
+    RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+      .setStartKey(startKey).setEndKey(endKey).build();
+    checkEquality(ri, conf);
+    // check HRIs with non-default replicaId
+    ri = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+    .setStartKey(startKey)
+    .setEndKey(endKey)
+    .setSplit(false)
+    .setRegionId(System.currentTimeMillis())
+    .setReplicaId(1).build();
+    checkEquality(ri, conf);
+    Assert.assertArrayEquals(RegionInfoDisplay.HIDDEN_END_KEY,
+    RegionInfoDisplay.getEndKeyForDisplay(ri, conf));
+    Assert.assertArrayEquals(RegionInfoDisplay.HIDDEN_START_KEY,
+    RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
+
+    RegionState state = new RegionState(convert(ri), RegionState.State.OPEN);
+    String descriptiveNameForDisplay =
+    RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf);
+    checkDescriptiveNameEquality(descriptiveNameForDisplay,state.toDescriptiveString(), startKey);
+
+    conf.setBoolean("hbase.display.keys", true);
+    Assert.assertArrayEquals(endKey, RegionInfoDisplay.getEndKeyForDisplay(ri, conf));
+    Assert.assertArrayEquals(startKey, RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
+    Assert.assertEquals(state.toDescriptiveString(),
+    RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf));
+  }
+
+  private void checkDescriptiveNameEquality(String descriptiveNameForDisplay, String origDesc,
+                                            byte[] startKey) {
+    // except for the "hidden-start-key" substring everything else should exactly match
+    String firstPart = descriptiveNameForDisplay.substring(0,
+    descriptiveNameForDisplay.indexOf(new String(RegionInfoDisplay.HIDDEN_START_KEY)));
+    String secondPart = descriptiveNameForDisplay.substring(
+    descriptiveNameForDisplay.indexOf(new String(RegionInfoDisplay.HIDDEN_START_KEY)) +
+    RegionInfoDisplay.HIDDEN_START_KEY.length);
+    String firstPartOrig = origDesc.substring(0,
+    origDesc.indexOf(Bytes.toStringBinary(startKey)));
+    String secondPartOrig = origDesc.substring(
+    origDesc.indexOf(Bytes.toStringBinary(startKey)) +
+    Bytes.toStringBinary(startKey).length());
+    assert(firstPart.equals(firstPartOrig));
+    assert(secondPart.equals(secondPartOrig));
+  }
+
+  private void checkEquality(RegionInfo ri, Configuration conf) throws IOException {
+    byte[] modifiedRegionName = RegionInfoDisplay.getRegionNameForDisplay(ri, conf);
+    System.out.println(Bytes.toString(modifiedRegionName) + " " + ri.toString());
+    byte[][] modifiedRegionNameParts = RegionInfo.parseRegionName(modifiedRegionName);
+    byte[][] regionNameParts = RegionInfo.parseRegionName(ri.getRegionName());
+
+    //same number of parts
+    assert(modifiedRegionNameParts.length == regionNameParts.length);
+    for (int i = 0; i < regionNameParts.length; i++) {
+      // all parts should match except for [1] where in the modified one,
+      // we should have "hidden_start_key"
+      if (i != 1) {
+        System.out.println("" + i + " " + Bytes.toString(regionNameParts[i]) + " " +
+          Bytes.toString(modifiedRegionNameParts[i]));
+        Assert.assertArrayEquals(regionNameParts[i], modifiedRegionNameParts[i]);
+      } else {
+        System.out.println("" + i + " " + Bytes.toString(regionNameParts[i]) + " " +
+          Bytes.toString(modifiedRegionNameParts[i]));
+        Assert.assertNotEquals(regionNameParts[i], modifiedRegionNameParts[i]);
+        Assert.assertArrayEquals(modifiedRegionNameParts[1],
+          RegionInfoDisplay.getStartKeyForDisplay(ri, conf));
+      }
+    }
+  }
+
+  private HRegionInfo convert(RegionInfo ri) {
+    HRegionInfo hri = new HRegionInfo(
+    ri.getTable(), ri.getStartKey(), ri.getEndKey(), ri.isSplit(), ri.getRegionId());
+    hri.setOffline(ri.isOffline());
+    return hri;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
index b5e6dd0..bb3686b 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
@@ -17,7 +17,6 @@ See the License for the specific language governing permissions and
 limitations under the License.
 </%doc>
 <%import>
-org.apache.hadoop.hbase.HRegionInfo;
 org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 org.apache.hadoop.hbase.master.assignment.AssignmentManager.RegionInTransitionStat;
 org.apache.hadoop.hbase.master.assignment.RegionStates.RegionFailedOpen;
@@ -25,6 +24,7 @@ org.apache.hadoop.hbase.master.RegionState;
 org.apache.hadoop.conf.Configuration;
 org.apache.hadoop.hbase.HBaseConfiguration;
 org.apache.hadoop.hbase.HConstants;
+org.apache.hadoop.hbase.client.RegionInfoDisplay;
 java.util.HashSet;
 java.util.SortedSet;
 java.util.Map;
@@ -94,7 +94,7 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
                           }
                         </%java>
                         <td><% rs.getRegion().getEncodedName() %></td><td>
-                        <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(rs,
+                        <% RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(rs,
                             assignmentManager.getConfiguration()) %></td>
                         <td><% (currentTime - rs.getStamp()) %> </td>
                         <td> <% retryStatus %> </td>

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index f58cbaf..3a9c34a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -19,37 +19,36 @@
 
 package org.apache.hadoop.hbase.master.assignment;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 
 /**
  * RegionStates contains a set of Maps that describes the in-memory state of the AM, with

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java
index 9a1515b..a89237e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MetaMockingUtil.java
@@ -105,17 +105,4 @@ public class MetaMockingUtil {
     return Result.create(kvs);
   }
 
-  /**
-   * @param sn  ServerName to use making startcode and server in meta
-   * @param hri Region to serialize into HRegionInfo
-   * @return A mocked up Result that fakes a Get on a row in the <code>hbase:meta</code> table.
-   * @throws IOException
-   */
-  public static Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri,
-      final ServerName sn) throws IOException {
-    hri.setOffline(true);
-    hri.setSplit(true);
-    return getMetaTableRowResult(hri, sn);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
index 003dfdd..0087ecd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
@@ -18,16 +18,6 @@
 
 package org.apache.hadoop.hbase.master.assignment;
 
-import static org.junit.Assert.assertEquals;
-
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -45,6 +35,16 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+
 @Category({MasterTests.class, MediumTests.class})
 public class TestRegionStates {
   private static final Log LOG = LogFactory.getLog(TestRegionStates.class);
@@ -217,8 +217,4 @@ public class TestRegionStates {
         StringUtils.humanTimeDiff(et - st),
       StringUtils.humanSize(NRUNS / ((et - st) / 1000.0f))));
   }
-
-  // ==========================================================================
-  //  Server related
-  // ==========================================================================
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
new file mode 100644
index 0000000..497ecc4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
@@ -0,0 +1,323 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.MD5Hash;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+@Category({RegionServerTests.class, SmallTests.class})
+public class TestRegionInfoBuilder {
+  @Rule
+  public TestName name = new TestName();
+
+  @Test
+  public void testBuilder() {
+    TableName tn = TableName.valueOf("test");
+    RegionInfoBuilder builder = RegionInfoBuilder.newBuilder(tn);
+    byte[] startKey = Bytes.toBytes("a");
+    builder.setStartKey(startKey);
+    byte[] endKey = Bytes.toBytes("z");
+    builder.setEndKey(endKey);
+    int regionId = 1;
+    builder.setRegionId(1);
+    int replicaId = 2;
+    builder.setReplicaId(replicaId);
+    boolean offline = true;
+    builder.setOffline(offline);
+    boolean isSplit = true;
+    builder.setSplit(isSplit);
+    RegionInfo ri = builder.build();
+
+    assertEquals(tn, ri.getTable());
+    assertArrayEquals(startKey, ri.getStartKey());
+    assertArrayEquals(endKey, ri.getEndKey());
+    assertEquals(regionId, ri.getRegionId());
+    assertEquals(replicaId, ri.getReplicaId());
+    assertEquals(offline, ri.isOffline());
+    assertEquals(isSplit, ri.isSplit());
+  }
+
+  @Test
+  public void testPb() throws DeserializationException {
+    RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+    byte [] bytes = RegionInfo.toByteArray(ri);
+    RegionInfo pbri = RegionInfo.parseFrom(bytes);
+    assertTrue(ri.equals(pbri));
+  }
+
+  @Test
+  public void testReadAndWriteRegionInfoFile() throws IOException, InterruptedException {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+    Path basedir = htu.getDataTestDir();
+    // Create a region.  That'll write the .regioninfo file.
+    FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration());
+    HRegion r = HBaseTestingUtility.createRegionAndWAL(convert(ri), basedir, htu.getConfiguration(),
+        fsTableDescriptors.get(TableName.META_TABLE_NAME));
+    // Get modtime on the file.
+    long modtime = getModTime(r);
+    HBaseTestingUtility.closeRegionAndWAL(r);
+    Thread.sleep(1001);
+    r = HRegion.openHRegion(basedir, convert(ri), fsTableDescriptors.get(TableName.META_TABLE_NAME),
+        null, htu.getConfiguration());
+    // Ensure the file is not written for a second time.
+    long modtime2 = getModTime(r);
+    assertEquals(modtime, modtime2);
+    // Now load the file.
+    RegionInfo deserializedRi = HRegionFileSystem.loadRegionInfoFileContent(
+        r.getRegionFileSystem().getFileSystem(), r.getRegionFileSystem().getRegionDir());
+    HBaseTestingUtility.closeRegionAndWAL(r);
+  }
+
+  long getModTime(final HRegion r) throws IOException {
+    FileStatus[] statuses = r.getRegionFileSystem().getFileSystem().listStatus(
+      new Path(r.getRegionFileSystem().getRegionDir(), HRegionFileSystem.REGION_INFO_FILE));
+    assertTrue(statuses != null && statuses.length == 1);
+    return statuses[0].getModificationTime();
+  }
+
+  @Test
+  public void testCreateRegionInfoName() throws Exception {
+    final String tableName = name.getMethodName();
+    final TableName tn = TableName.valueOf(tableName);
+    String startKey = "startkey";
+    final byte[] sk = Bytes.toBytes(startKey);
+    String id = "id";
+
+    // old format region name
+    byte [] name = RegionInfo.createRegionName(tn, sk, id, false);
+    String nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + "," + id, nameStr);
+
+
+    // new format region name.
+    String md5HashInHex = MD5Hash.getMD5AsHex(name);
+    assertEquals(RegionInfo.MD5_HEX_LENGTH, md5HashInHex.length());
+    name = RegionInfo.createRegionName(tn, sk, id, true);
+    nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + ","
+                 + id + "." + md5HashInHex + ".",
+                 nameStr);
+  }
+
+  @Test
+  public void testContainsRange() {
+    TableDescriptor tableDesc = TableDescriptorBuilder.newBuilder(
+            TableName.valueOf(name.getMethodName())).build();
+    RegionInfo ri = RegionInfoBuilder.newBuilder(tableDesc.getTableName())
+            .setStartKey(Bytes.toBytes("a"))
+            .setEndKey(Bytes.toBytes("g")).build();
+    // Single row range at start of region
+    assertTrue(ri.containsRange(Bytes.toBytes("a"), Bytes.toBytes("a")));
+    // Fully contained range
+    assertTrue(ri.containsRange(Bytes.toBytes("b"), Bytes.toBytes("c")));
+    // Range overlapping start of region
+    assertTrue(ri.containsRange(Bytes.toBytes("a"), Bytes.toBytes("c")));
+    // Fully contained single-row range
+    assertTrue(ri.containsRange(Bytes.toBytes("c"), Bytes.toBytes("c")));
+    // Range that overlaps end key and hence doesn't fit
+    assertFalse(ri.containsRange(Bytes.toBytes("a"), Bytes.toBytes("g")));
+    // Single row range on end key
+    assertFalse(ri.containsRange(Bytes.toBytes("g"), Bytes.toBytes("g")));
+    // Single row range entirely outside
+    assertFalse(ri.containsRange(Bytes.toBytes("z"), Bytes.toBytes("z")));
+
+    // Degenerate range
+    try {
+      ri.containsRange(Bytes.toBytes("z"), Bytes.toBytes("a"));
+      fail("Invalid range did not throw IAE");
+    } catch (IllegalArgumentException iae) {
+    }
+  }
+
+  @Test
+  public void testLastRegionCompare() {
+    TableDescriptor tableDesc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    RegionInfo rip = RegionInfoBuilder.newBuilder(tableDesc.getTableName())
+            .setStartKey(Bytes.toBytes("a"))
+            .setEndKey(new byte[0]).build();
+    RegionInfo ric = RegionInfoBuilder.newBuilder(tableDesc.getTableName())
+            .setStartKey(Bytes.toBytes("a"))
+            .setEndKey(Bytes.toBytes("b")).build();
+    assertTrue(RegionInfo.COMPARATOR.compare(rip, ric) > 0);
+  }
+
+  @Test
+  public void testMetaTables() {
+    assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isMetaTable());
+  }
+
+  @Test
+  public void testComparator() {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    byte[] empty = new byte[0];
+    RegionInfo older = RegionInfoBuilder.newBuilder(tableName)
+            .setStartKey(empty)
+            .setEndKey(empty)
+            .setSplit(false)
+            .setRegionId(0L).build();
+    RegionInfo newer = RegionInfoBuilder.newBuilder(tableName)
+            .setStartKey(empty)
+            .setEndKey(empty)
+            .setSplit(false)
+            .setRegionId(1L).build();
+    assertTrue(RegionInfo.COMPARATOR.compare(older, newer) < 0);
+    assertTrue(RegionInfo.COMPARATOR.compare(newer, older) > 0);
+    assertTrue(RegionInfo.COMPARATOR.compare(older, older) == 0);
+    assertTrue(RegionInfo.COMPARATOR.compare(newer, newer) == 0);
+  }
+
+  @Test
+  public void testRegionNameForRegionReplicas() throws Exception {
+    String tableName = name.getMethodName();
+    final TableName tn = TableName.valueOf(tableName);
+    String startKey = "startkey";
+    final byte[] sk = Bytes.toBytes(startKey);
+    String id = "id";
+
+    // assert with only the region name without encoding
+
+    // primary, replicaId = 0
+    byte [] name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0, false);
+    String nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + "," + id, nameStr);
+
+    // replicaId = 1
+    name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 1, false);
+    nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + "," + id + "_" +
+      String.format(RegionInfo.REPLICA_ID_FORMAT, 1), nameStr);
+
+    // replicaId = max
+    name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0xFFFF, false);
+    nameStr = Bytes.toString(name);
+    assertEquals(tableName + "," + startKey + "," + id + "_" +
+        String.format(RegionInfo.REPLICA_ID_FORMAT, 0xFFFF), nameStr);
+  }
+
+  @Test
+  public void testParseName() throws IOException {
+    final TableName tableName = TableName.valueOf(name.getMethodName());
+    byte[] startKey = Bytes.toBytes("startKey");
+    long regionId = System.currentTimeMillis();
+    int replicaId = 42;
+
+    // test without replicaId
+    byte[] regionName = RegionInfo.createRegionName(tableName, startKey, regionId, false);
+
+    byte[][] fields = RegionInfo.parseRegionName(regionName);
+    assertArrayEquals(Bytes.toString(fields[0]),tableName.getName(), fields[0]);
+    assertArrayEquals(Bytes.toString(fields[1]),startKey, fields[1]);
+    assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)),fields[2]);
+    assertEquals(3, fields.length);
+
+    // test with replicaId
+    regionName = RegionInfo.createRegionName(tableName, startKey, regionId,
+      replicaId, false);
+
+    fields = RegionInfo.parseRegionName(regionName);
+    assertArrayEquals(Bytes.toString(fields[0]),tableName.getName(), fields[0]);
+    assertArrayEquals(Bytes.toString(fields[1]),startKey, fields[1]);
+    assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)),fields[2]);
+    assertArrayEquals(Bytes.toString(fields[3]), Bytes.toBytes(
+      String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)), fields[3]);
+  }
+
+  @Test
+  public void testConvert() {
+    final TableName tableName = TableName.valueOf("ns1:" + name.getMethodName());
+    byte[] startKey = Bytes.toBytes("startKey");
+    byte[] endKey = Bytes.toBytes("endKey");
+    boolean split = false;
+    long regionId = System.currentTimeMillis();
+    int replicaId = 42;
+
+
+    RegionInfo ri = RegionInfoBuilder.newBuilder(tableName)
+            .setStartKey(startKey)
+            .setEndKey(endKey)
+            .setSplit(split)
+            .setRegionId(regionId)
+            .setReplicaId(replicaId).build();
+
+    // convert two times, compare
+    RegionInfo convertedRi = ProtobufUtil.toRegionInfo(ProtobufUtil.toProtoRegionInfo(ri));
+
+    assertEquals(ri, convertedRi);
+
+    // test convert RegionInfo without replicaId
+    HBaseProtos.RegionInfo info = HBaseProtos.RegionInfo.newBuilder()
+      .setTableName(HBaseProtos.TableName.newBuilder()
+        .setQualifier(UnsafeByteOperations.unsafeWrap(tableName.getQualifier()))
+        .setNamespace(UnsafeByteOperations.unsafeWrap(tableName.getNamespace()))
+        .build())
+      .setStartKey(UnsafeByteOperations.unsafeWrap(startKey))
+      .setEndKey(UnsafeByteOperations.unsafeWrap(endKey))
+      .setSplit(split)
+      .setRegionId(regionId)
+      .build();
+
+    convertedRi = ProtobufUtil.toRegionInfo(info);
+    RegionInfo expectedRi = RegionInfoBuilder.newBuilder(tableName)
+            .setStartKey(startKey)
+            .setEndKey(endKey)
+            .setSplit(split)
+            .setRegionId(regionId)
+            .setReplicaId(0).build();
+
+    assertEquals(expectedRi, convertedRi);
+  }
+
+  // Duplicated method in TestRegionInfoDisplay too.
+  private HRegionInfo convert(RegionInfo ri) {
+    HRegionInfo hri = new HRegionInfo(
+    ri.getTable(), ri.getStartKey(), ri.getEndKey(), ri.isSplit(), ri.getRegionId());
+    hri.setOffline(ri.isOffline());
+    return hri;
+  }
+}
\ No newline at end of file


[2/2] hbase git commit: HBASE-17980 Any HRegionInfo we give out should be immutable

Posted by st...@apache.org.
HBASE-17980 Any HRegionInfo we give out should be immutable

Signed-off-by: Michael Stack <st...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/58988cb5
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/58988cb5
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/58988cb5

Branch: refs/heads/master
Commit: 58988cb53c6287dc82a7a6242b1fe3f50dbf9dfb
Parents: a4277f3
Author: brandboat <br...@gmail.com>
Authored: Wed Sep 13 21:00:18 2017 +0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Sep 14 14:25:07 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HRegionInfo.java    | 579 +++++---------
 .../org/apache/hadoop/hbase/client/Admin.java   |  25 +
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  37 +-
 .../hbase/client/ImmutableHRegionInfo.java      |  49 ++
 .../apache/hadoop/hbase/client/RegionInfo.java  | 751 +++++++++++++++++++
 .../hadoop/hbase/client/RegionInfoBuilder.java  | 624 +++++++++++++++
 .../hadoop/hbase/client/RegionInfoDisplay.java  | 135 ++++
 .../hadoop/hbase/client/RegionReplicaUtil.java  |  32 +
 .../hbase/client/UnmodifyableHRegionInfo.java   |   2 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  64 ++
 .../hbase/zookeeper/MetaTableLocator.java       |   5 +-
 .../hbase/client/TestImmutableHRegionInfo.java  |  61 ++
 .../hbase/client/TestRegionInfoDisplay.java     | 129 ++++
 .../master/AssignmentManagerStatusTmpl.jamon    |   4 +-
 .../hbase/master/assignment/RegionStates.java   |  35 +-
 .../apache/hadoop/hbase/MetaMockingUtil.java    |  13 -
 .../master/assignment/TestRegionStates.java     |  24 +-
 .../regionserver/TestRegionInfoBuilder.java     | 323 ++++++++
 18 files changed, 2445 insertions(+), 447 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index 2e735a0..cc88733 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -28,22 +28,17 @@ import java.util.stream.Collectors;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
-import org.apache.hadoop.hbase.util.ByteArrayHashKey;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.HashKey;
-import org.apache.hadoop.hbase.util.JenkinsHash;
-import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.hadoop.hbase.client.RegionInfoDisplay;
 import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * Information about a region. A region is a range of keys in the whole keyspace of a table, an
@@ -74,10 +69,12 @@ import org.apache.hadoop.util.StringUtils;
  * correspond to multiple HRegionInfo's. These HRI's share the same fields however except the
  * replicaId field. If the replicaId is not set, it defaults to 0, which is compatible with the
  * previous behavior of a range corresponding to 1 region.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+ *             use {@link RegionInfoBuilder} to build {@link RegionInfo}.
  */
+@Deprecated
 @InterfaceAudience.Public
-public class HRegionInfo implements Comparable<HRegionInfo> {
-
+public class HRegionInfo implements RegionInfo, Comparable<HRegionInfo> {
   private static final Log LOG = LogFactory.getLog(HRegionInfo.class);
 
   /**
@@ -103,62 +100,20 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * old region name format.
    */
 
-  /** Separator used to demarcate the encodedName in a region name
-   * in the new format. See description on new format above.
-   */
-  private static final int ENC_SEPARATOR = '.';
-  public  static final int MD5_HEX_LENGTH   = 32;
-
   /** A non-capture group so that this can be embedded. */
-  public static final String ENCODED_REGION_NAME_REGEX = "(?:[a-f0-9]+)";
-
-  // to keep appended int's sorted in string format. Only allows 2 bytes to be
-  // sorted for replicaId
-  public static final String REPLICA_ID_FORMAT = "%04X";
-
-  public static final byte REPLICA_ID_DELIMITER = (byte)'_';
+  public static final String ENCODED_REGION_NAME_REGEX = RegionInfoBuilder.ENCODED_REGION_NAME_REGEX;
 
   private static final int MAX_REPLICA_ID = 0xFFFF;
-  public static final int DEFAULT_REPLICA_ID = 0;
-
-  public static final String INVALID_REGION_NAME_FORMAT_MESSAGE = "Invalid regionName format";
-
-  /**
-   * Does region name contain its encoded name?
-   * @param regionName region name
-   * @return boolean indicating if this a new format region
-   *         name which contains its encoded name.
-   */
-  private static boolean hasEncodedName(final byte[] regionName) {
-    // check if region name ends in ENC_SEPARATOR
-    if ((regionName.length >= 1)
-        && (regionName[regionName.length - 1] == ENC_SEPARATOR)) {
-      // region name is new format. it contains the encoded name.
-      return true;
-    }
-    return false;
-  }
 
   /**
    * @param regionName
    * @return the encodedName
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#encodeRegionName(byte[])}.
    */
+  @Deprecated
   public static String encodeRegionName(final byte [] regionName) {
-    String encodedName;
-    if (hasEncodedName(regionName)) {
-      // region is in new format:
-      // <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
-      encodedName = Bytes.toString(regionName,
-          regionName.length - MD5_HEX_LENGTH - 1,
-          MD5_HEX_LENGTH);
-    } else {
-      // old format region name. First hbase:meta region also
-      // use this format.EncodedName is the JenkinsHash value.
-      HashKey<byte[]> key = new ByteArrayHashKey(regionName, 0, regionName.length);
-      int hashVal = Math.abs(JenkinsHash.getInstance().hash(key, 0));
-      encodedName = String.valueOf(hashVal);
-    }
-    return encodedName;
+    return RegionInfo.encodeRegionName(regionName);
   }
 
   /**
@@ -168,17 +123,24 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
     return prettyPrint(this.getEncodedName());
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(RegionInfo...)}.
+   */
+  @Deprecated
   public static String getShortNameToLog(HRegionInfo...hris) {
-    return getShortNameToLog(Arrays.asList(hris));
+    return RegionInfo.getShortNameToLog(Arrays.asList(hris));
   }
 
   /**
    * @return Return a String of short, printable names for <code>hris</code>
    * (usually encoded name) for us logging.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(List)})}.
    */
+  @Deprecated
   public static String getShortNameToLog(final List<HRegionInfo> hris) {
-    return hris.stream().map(hri -> hri.getShortNameToLog()).
-        collect(Collectors.toList()).toString();
+    return RegionInfo.getShortNameToLog(hris.stream().collect(Collectors.toList()));
   }
 
   /**
@@ -186,12 +148,13 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param encodedRegionName The encoded regionname.
    * @return <code>hbase:meta</code> if passed <code>1028785192</code> else returns
    * <code>encodedRegionName</code>
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#prettyPrint(String)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static String prettyPrint(final String encodedRegionName) {
-    if (encodedRegionName.equals("1028785192")) {
-      return encodedRegionName + "/hbase:meta";
-    }
-    return encodedRegionName;
+    return RegionInfo.prettyPrint(encodedRegionName);
   }
 
   private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
@@ -212,9 +175,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
 
   // Current TableName
   private TableName tableName = null;
-  final static String DISPLAY_KEYS_KEY = "hbase.display.keys";
-  public final static byte[] HIDDEN_END_KEY = Bytes.toBytes("hidden-end-key");
-  public final static byte[] HIDDEN_START_KEY = Bytes.toBytes("hidden-start-key");
+
+  // Duplicated over in RegionInfoDisplay
+  final static String DISPLAY_KEYS_KEY = RegionInfoDisplay.DISPLAY_KEYS_KEY;
+  public final static byte[] HIDDEN_END_KEY = RegionInfoDisplay.HIDDEN_END_KEY;
+  public final static byte[] HIDDEN_START_KEY = RegionInfoDisplay.HIDDEN_START_KEY;
 
   /** HRegionInfo for first meta region */
   // TODO: How come Meta regions still do not have encoded region names? Fix.
@@ -232,7 +197,6 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
     this.hashCode = result;
   }
 
-
   /**
    * Private constructor used constructing HRegionInfo for the
    * first meta regions
@@ -354,8 +318,8 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
     this.startKey = other.getStartKey();
     this.hashCode = other.hashCode();
     this.encodedName = other.getEncodedName();
-    this.tableName = other.tableName;
-    this.replicaId = other.replicaId;
+    this.tableName = other.getTable();
+    this.replicaId = other.getReplicaId();
   }
 
   public HRegionInfo(HRegionInfo other, int replicaId) {
@@ -372,10 +336,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey and id
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], long, boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final long regionid, boolean newFormat) {
-    return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
+    return RegionInfo.createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
   }
 
   /**
@@ -386,10 +354,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey and id
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], String, boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final String id, boolean newFormat) {
-    return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
+    return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
   }
 
   /**
@@ -401,10 +373,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey, id and replicaId
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], long, int, boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final long regionid, int replicaId, boolean newFormat) {
-    return createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
+    return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
         replicaId, newFormat);
   }
 
@@ -416,10 +392,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param newFormat should we create the region name in the new format
    *                  (such that it contains its encoded name?).
    * @return Region name made of passed tableName, startKey and id
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], byte[], boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final byte [] id, boolean newFormat) {
-    return createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
+    return RegionInfo.createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
   }
   /**
    * Make a region name of passed parameters.
@@ -429,94 +409,38 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param replicaId
    * @param newFormat should we create the region name in the new format
    * @return Region name made of passed tableName, startKey, id and replicaId
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#createRegionName(TableName, byte[], byte[], int, boolean)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [] createRegionName(final TableName tableName,
       final byte [] startKey, final byte [] id, final int replicaId, boolean newFormat) {
-    int len = tableName.getName().length + 2 + id.length +
-        (startKey == null? 0: startKey.length);
-    if (newFormat) {
-      len += MD5_HEX_LENGTH + 2;
-    }
-    byte[] replicaIdBytes = null;
-    // Special casing: replicaId is only appended if replicaId is greater than
-    // 0. This is because all regions in meta would have to be migrated to the new
-    // name otherwise
-    if (replicaId > 0) {
-      // use string representation for replica id
-      replicaIdBytes = Bytes.toBytes(String.format(REPLICA_ID_FORMAT, replicaId));
-      len += 1 + replicaIdBytes.length;
-    }
-
-    byte [] b = new byte [len];
-
-    int offset = tableName.getName().length;
-    System.arraycopy(tableName.getName(), 0, b, 0, offset);
-    b[offset++] = HConstants.DELIMITER;
-    if (startKey != null && startKey.length > 0) {
-      System.arraycopy(startKey, 0, b, offset, startKey.length);
-      offset += startKey.length;
-    }
-    b[offset++] = HConstants.DELIMITER;
-    System.arraycopy(id, 0, b, offset, id.length);
-    offset += id.length;
-
-    if (replicaIdBytes != null) {
-      b[offset++] = REPLICA_ID_DELIMITER;
-      System.arraycopy(replicaIdBytes, 0, b, offset, replicaIdBytes.length);
-      offset += replicaIdBytes.length;
-    }
-
-    if (newFormat) {
-      //
-      // Encoded name should be built into the region name.
-      //
-      // Use the region name thus far (namely, <tablename>,<startKey>,<id>_<replicaId>)
-      // to compute a MD5 hash to be used as the encoded name, and append
-      // it to the byte buffer.
-      //
-      String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
-      byte [] md5HashBytes = Bytes.toBytes(md5Hash);
-
-      if (md5HashBytes.length != MD5_HEX_LENGTH) {
-        LOG.error("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
-                  "; Got=" + md5HashBytes.length);
-      }
-
-      // now append the bytes '.<encodedName>.' to the end
-      b[offset++] = ENC_SEPARATOR;
-      System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
-      offset += MD5_HEX_LENGTH;
-      b[offset++] = ENC_SEPARATOR;
-    }
-
-    return b;
+    return RegionInfo.createRegionName(tableName, startKey, id, replicaId, newFormat);
   }
 
   /**
    * Gets the table name from the specified region name.
    * @param regionName to extract the table name from
    * @return Table name
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getTable(byte[])}.
    */
+  @Deprecated
   public static TableName getTable(final byte [] regionName) {
-    int offset = -1;
-    for (int i = 0; i < regionName.length; i++) {
-      if (regionName[i] == HConstants.DELIMITER) {
-        offset = i;
-        break;
-      }
-    }
-    byte[] buff  = new byte[offset];
-    System.arraycopy(regionName, 0, buff, 0, offset);
-    return TableName.valueOf(buff);
+    return RegionInfo.getTable(regionName);
   }
 
   /**
    * Gets the start key from the specified region name.
    * @param regionName
    * @return Start key.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#getStartKey(byte[])}.
    */
+  @Deprecated
   public static byte[] getStartKey(final byte[] regionName) throws IOException {
-    return parseRegionName(regionName)[1];
+    return RegionInfo.getStartKey(regionName);
   }
 
   /**
@@ -524,88 +448,27 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param regionName
    * @return Array of byte[] containing tableName, startKey and id
    * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#parseRegionName(byte[])}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte [][] parseRegionName(final byte [] regionName)
   throws IOException {
-    // Region name is of the format:
-    // tablename,startkey,regionIdTimestamp[_replicaId][.encodedName.]
-    // startkey can contain the delimiter (',') so we parse from the start and end
-
-    // parse from start
-    int offset = -1;
-    for (int i = 0; i < regionName.length; i++) {
-      if (regionName[i] == HConstants.DELIMITER) {
-        offset = i;
-        break;
-      }
-    }
-    if (offset == -1) {
-      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE
-        + ": " + Bytes.toStringBinary(regionName));
-    }
-    byte[] tableName = new byte[offset];
-    System.arraycopy(regionName, 0, tableName, 0, offset);
-    offset = -1;
-
-    int endOffset = regionName.length;
-    // check whether regionName contains encodedName
-    if (regionName.length > MD5_HEX_LENGTH + 2
-        && regionName[regionName.length-1] == ENC_SEPARATOR
-        && regionName[regionName.length-MD5_HEX_LENGTH-2] == ENC_SEPARATOR) {
-      endOffset = endOffset - MD5_HEX_LENGTH - 2;
-    }
-
-    // parse from end
-    byte[] replicaId = null;
-    int idEndOffset = endOffset;
-    for (int i = endOffset - 1; i > 0; i--) {
-      if (regionName[i] == REPLICA_ID_DELIMITER) { //replicaId may or may not be present
-        replicaId = new byte[endOffset - i - 1];
-        System.arraycopy(regionName, i + 1, replicaId, 0,
-          endOffset - i - 1);
-        idEndOffset = i;
-        // do not break, continue to search for id
-      }
-      if (regionName[i] == HConstants.DELIMITER) {
-        offset = i;
-        break;
-      }
-    }
-    if (offset == -1) {
-      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE
-        + ": " + Bytes.toStringBinary(regionName));
-    }
-    byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
-    if(offset != tableName.length + 1) {
-      startKey = new byte[offset - tableName.length - 1];
-      System.arraycopy(regionName, tableName.length + 1, startKey, 0,
-          offset - tableName.length - 1);
-    }
-    byte [] id = new byte[idEndOffset - offset - 1];
-    System.arraycopy(regionName, offset + 1, id, 0,
-      idEndOffset - offset - 1);
-    byte [][] elements = new byte[replicaId == null ? 3 : 4][];
-    elements[0] = tableName;
-    elements[1] = startKey;
-    elements[2] = id;
-    if (replicaId != null) {
-      elements[3] = replicaId;
-    }
-
-    return elements;
+    return RegionInfo.parseRegionName(regionName);
   }
 
+  /**
+   *
+   * @param regionName
+   * @return if region name is encoded.
+   * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#isEncodedRegionName(byte[])}.
+   */
+  @Deprecated
   public static boolean isEncodedRegionName(byte[] regionName) throws IOException {
-    try {
-      HRegionInfo.parseRegionName(regionName);
-      return false;
-    } catch (IOException e) {
-      if (StringUtils.stringifyException(e)
-          .contains(HRegionInfo.INVALID_REGION_NAME_FORMAT_MESSAGE)) {
-        return true;
-      }
-      throw e;
-    }
+    return RegionInfo.isEncodedRegionName(regionName);
   }
 
   /** @return the regionId */
@@ -625,7 +488,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return Region name as a String for use in logging, etc.
    */
   public String getRegionNameAsString() {
-    if (hasEncodedName(this.regionName)) {
+    if (RegionInfo.hasEncodedName(this.regionName)) {
       // new format region names already have their encoded name.
       return Bytes.toStringBinary(this.regionName);
     }
@@ -639,7 +502,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
   /** @return the encoded region name */
   public synchronized String getEncodedName() {
     if (this.encodedName == null) {
-      this.encodedName = encodeRegionName(this.regionName);
+      this.encodedName = RegionInfo.encodeRegionName(this.regionName);
     }
     return this.encodedName;
   }
@@ -819,53 +682,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
 
   @Override
   public int compareTo(HRegionInfo o) {
-    if (o == null) {
-      return 1;
-    }
-
-    // Are regions of same table?
-    int result = this.tableName.compareTo(o.tableName);
-    if (result != 0) {
-      return result;
-    }
-
-    // Compare start keys.
-    result = Bytes.compareTo(this.startKey, o.startKey);
-    if (result != 0) {
-      return result;
-    }
-
-    // Compare end keys.
-    result = Bytes.compareTo(this.endKey, o.endKey);
-
-    if (result != 0) {
-      if (this.getStartKey().length != 0
-              && this.getEndKey().length == 0) {
-          return 1; // this is last region
-      }
-      if (o.getStartKey().length != 0
-              && o.getEndKey().length == 0) {
-          return -1; // o is the last region
-      }
-      return result;
-    }
-
-    // regionId is usually milli timestamp -- this defines older stamps
-    // to be "smaller" than newer stamps in sort order.
-    if (this.regionId > o.regionId) {
-      return 1;
-    } else if (this.regionId < o.regionId) {
-      return -1;
-    }
-
-    int replicaDiff = this.getReplicaId() - o.getReplicaId();
-    if (replicaDiff != 0) return replicaDiff;
-
-    if (this.offLine == o.offLine)
-      return 0;
-    if (this.offLine == true) return -1;
-
-    return 1;
+    return RegionInfo.COMPARATOR.compare(this, o);
   }
 
   /**
@@ -883,7 +700,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    *
    * @return the converted RegionInfo
    */
-  RegionInfo convert() {
+  HBaseProtos.RegionInfo convert() {
     return convert(this);
   }
 
@@ -892,58 +709,47 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    *
    * @param info the HRegionInfo to convert
    * @return the converted RegionInfo
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use toProtoRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
+   *             in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
    */
-  public static RegionInfo convert(final HRegionInfo info) {
-    if (info == null) return null;
-    RegionInfo.Builder builder = RegionInfo.newBuilder();
-    builder.setTableName(ProtobufUtil.toProtoTableName(info.getTable()));
-    builder.setRegionId(info.getRegionId());
-    if (info.getStartKey() != null) {
-      builder.setStartKey(UnsafeByteOperations.unsafeWrap(info.getStartKey()));
-    }
-    if (info.getEndKey() != null) {
-      builder.setEndKey(UnsafeByteOperations.unsafeWrap(info.getEndKey()));
-    }
-    builder.setOffline(info.isOffline());
-    builder.setSplit(info.isSplit());
-    builder.setReplicaId(info.getReplicaId());
-    return builder.build();
+  @Deprecated
+  @InterfaceAudience.Private
+  public static HBaseProtos.RegionInfo convert(final HRegionInfo info) {
+    return ProtobufUtil.toProtoRegionInfo(info);
   }
 
   /**
    * Convert a RegionInfo to a HRegionInfo
    *
    * @param proto the RegionInfo to convert
-   * @return the converted HRegionInfho
+   * @return the converted HRegionInfo
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use toRegionInfo(HBaseProtos.RegionInfo)
+   *             in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
    */
-  public static HRegionInfo convert(final RegionInfo proto) {
-    if (proto == null) return null;
-    TableName tableName =
-        ProtobufUtil.toTableName(proto.getTableName());
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
-      return RegionReplicaUtil.getRegionInfoForReplica(FIRST_META_REGIONINFO,
-          proto.getReplicaId());
-    }
-    long regionId = proto.getRegionId();
-    int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : DEFAULT_REPLICA_ID;
-    byte[] startKey = null;
-    byte[] endKey = null;
-    if (proto.hasStartKey()) {
-      startKey = proto.getStartKey().toByteArray();
-    }
-    if (proto.hasEndKey()) {
-      endKey = proto.getEndKey().toByteArray();
-    }
-    boolean split = false;
-    if (proto.hasSplit()) {
-      split = proto.getSplit();
-    }
-    HRegionInfo hri = new HRegionInfo(
-        tableName,
-        startKey,
-        endKey, split, regionId, replicaId);
-    if (proto.hasOffline()) {
-      hri.setOffline(proto.getOffline());
+  @Deprecated
+  @InterfaceAudience.Private
+  public static HRegionInfo convert(final HBaseProtos.RegionInfo proto) {
+    RegionInfo ri = ProtobufUtil.toRegionInfo(proto);
+    // This is hack of what is in RegionReplicaUtil but it is doing translation of
+    // RegionInfo into HRegionInfo which is what is wanted here.
+    HRegionInfo hri;
+    if (ri.isMetaRegion()) {
+      hri = ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID ?
+      HRegionInfo.FIRST_META_REGIONINFO :
+      new HRegionInfo(ri.getRegionId(), ri.getTable(), ri.getReplicaId());
+    } else {
+      hri = new HRegionInfo(
+        ri.getTable(),
+        ri.getStartKey(),
+        ri.getEndKey(),
+        ri.isSplit(),
+        ri.getRegionId(),
+        ri.getReplicaId());
+      if (proto.hasOffline()) {
+        hri.setOffline(proto.getOffline());
+      }
     }
     return hri;
   }
@@ -951,17 +757,22 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
   /**
    * @return This instance serialized as protobuf w/ a magic pb prefix.
    * @see #parseFrom(byte[])
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#toByteArray(RegionInfo)}.
    */
+  @Deprecated
   public byte [] toByteArray() {
-    byte [] bytes = convert().toByteArray();
-    return ProtobufUtil.prependPBMagic(bytes);
+    return RegionInfo.toByteArray(this);
   }
 
   /**
    * @return A deserialized {@link HRegionInfo}
    * or null if we failed deserialize or passed bytes null
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[])}.
    */
+  @Deprecated
   public static HRegionInfo parseFromOrNull(final byte [] bytes) {
     if (bytes == null) return null;
     return parseFromOrNull(bytes, 0, bytes.length);
@@ -971,7 +782,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return A deserialized {@link HRegionInfo} or null
    *  if we failed deserialize or passed bytes null
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[], int, int)}.
    */
+  @Deprecated
   public static HRegionInfo parseFromOrNull(final byte [] bytes, int offset, int len) {
     if (bytes == null || len <= 0) return null;
     try {
@@ -986,6 +800,8 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return A deserialized {@link HRegionInfo}
    * @throws DeserializationException
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[])}.
    */
   public static HRegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
     if (bytes == null) return null;
@@ -999,7 +815,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return A deserialized {@link HRegionInfo}
    * @throws DeserializationException
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[], int, int)}.
    */
+  @Deprecated
   public static HRegionInfo parseFrom(final byte [] bytes, int offset, int len)
       throws DeserializationException {
     if (ProtobufUtil.isPBMagicPrefix(bytes, offset, len)) {
@@ -1023,9 +842,12 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
    * @throws IOException
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#toDelimitedByteArray(RegionInfo)}.
    */
+  @Deprecated
   public byte [] toDelimitedByteArray() throws IOException {
-    return ProtobufUtil.toDelimitedByteArray(convert());
+    return RegionInfo.toDelimitedByteArray(this);
   }
 
   /**
@@ -1034,14 +856,15 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param state
    * @param conf
    * @return descriptive string
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getDescriptiveNameFromRegionStateForDisplay(RegionState, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static String getDescriptiveNameFromRegionStateForDisplay(RegionState state,
       Configuration conf) {
-    if (conf.getBoolean(DISPLAY_KEYS_KEY, true)) return state.toDescriptiveString();
-    String descriptiveStringFromState = state.toDescriptiveString();
-    int idx = descriptiveStringFromState.lastIndexOf(" state=");
-    String regionName = getRegionNameAsStringForDisplay(state.getRegion(), conf);
-    return regionName + descriptiveStringFromState.substring(idx);
+    return RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf);
   }
 
   /**
@@ -1049,11 +872,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param hri
    * @param conf
    * @return the endkey
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getEndKeyForDisplay(RegionInfo, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte[] getEndKeyForDisplay(HRegionInfo hri, Configuration conf) {
-    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
-    if (displayKey) return hri.getEndKey();
-    return HIDDEN_END_KEY;
+    return RegionInfoDisplay.getEndKeyForDisplay(hri, conf);
   }
 
   /**
@@ -1061,11 +887,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param hri
    * @param conf
    * @return the startkey
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getStartKeyForDisplay(RegionInfo, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte[] getStartKeyForDisplay(HRegionInfo hri, Configuration conf) {
-    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
-    if (displayKey) return hri.getStartKey();
-    return HIDDEN_START_KEY;
+    return RegionInfoDisplay.getStartKeyForDisplay(hri, conf);
   }
 
   /**
@@ -1073,9 +902,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param hri
    * @param conf
    * @return region name as String
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getRegionNameAsStringForDisplay(RegionInfo, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static String getRegionNameAsStringForDisplay(HRegionInfo hri, Configuration conf) {
-    return Bytes.toStringBinary(getRegionNameForDisplay(hri, conf));
+    return RegionInfoDisplay.getRegionNameAsStringForDisplay(hri, conf);
   }
 
   /**
@@ -1083,47 +917,14 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param hri
    * @param conf
    * @return region name bytes
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use RegionInfoDisplay#getRegionNameForDisplay(RegionInfo, Configuration)
+   *             over in hbase-server module.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte[] getRegionNameForDisplay(HRegionInfo hri, Configuration conf) {
-    boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true);
-    if (displayKey || hri.getTable().equals(TableName.META_TABLE_NAME)) {
-      return hri.getRegionName();
-    } else {
-      // create a modified regionname with the startkey replaced but preserving
-      // the other parts including the encodedname.
-      try {
-        byte[][]regionNameParts = parseRegionName(hri.getRegionName());
-        regionNameParts[1] = HIDDEN_START_KEY; //replace the real startkey
-        int len = 0;
-        // get the total length
-        for (byte[] b : regionNameParts) {
-          len += b.length;
-        }
-        byte[] encodedRegionName =
-            Bytes.toBytes(encodeRegionName(hri.getRegionName()));
-        len += encodedRegionName.length;
-        //allocate some extra bytes for the delimiters and the last '.'
-        byte[] modifiedName = new byte[len + regionNameParts.length + 1];
-        int lengthSoFar = 0;
-        int loopCount = 0;
-        for (byte[] b : regionNameParts) {
-          System.arraycopy(b, 0, modifiedName, lengthSoFar, b.length);
-          lengthSoFar += b.length;
-          if (loopCount++ == 2) modifiedName[lengthSoFar++] = REPLICA_ID_DELIMITER;
-          else  modifiedName[lengthSoFar++] = HConstants.DELIMITER;
-        }
-        // replace the last comma with '.'
-        modifiedName[lengthSoFar - 1] = ENC_SEPARATOR;
-        System.arraycopy(encodedRegionName, 0, modifiedName, lengthSoFar,
-            encodedRegionName.length);
-        lengthSoFar += encodedRegionName.length;
-        modifiedName[lengthSoFar] = ENC_SEPARATOR;
-        return modifiedName;
-      } catch (IOException e) {
-        //LOG.warn("Encountered exception " + e);
-        throw new RuntimeException(e);
-      }
-    }
+    return RegionInfoDisplay.getRegionNameForDisplay(hri, conf);
   }
 
   /**
@@ -1132,7 +933,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param in
    * @return An instance of HRegionInfo.
    * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#parseFrom(DataInputStream)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static HRegionInfo parseFrom(final DataInputStream in) throws IOException {
     // I need to be able to move back in the stream if this is not a pb serialization so I can
     // do the Writable decoding instead.
@@ -1161,22 +966,13 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
    * @throws IOException
    * @see #toByteArray()
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#toDelimitedByteArray(RegionInfo...)}.
    */
+  @Deprecated
+  @InterfaceAudience.Private
   public static byte[] toDelimitedByteArray(HRegionInfo... infos) throws IOException {
-    byte[][] bytes = new byte[infos.length][];
-    int size = 0;
-    for (int i = 0; i < infos.length; i++) {
-      bytes[i] = infos[i].toDelimitedByteArray();
-      size += bytes[i].length;
-    }
-
-    byte[] result = new byte[size];
-    int offset = 0;
-    for (byte[] b : bytes) {
-      System.arraycopy(b, 0, result, offset, b.length);
-      offset += b.length;
-    }
-    return result;
+    return RegionInfo.toDelimitedByteArray(infos);
   }
 
   /**
@@ -1186,7 +982,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param offset the start offset into the byte[] buffer
    * @param length how far we should read into the byte[] buffer
    * @return All the hregioninfos that are in the byte array. Keeps reading till we hit the end.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link RegionInfo#parseDelimitedFrom(byte[], int, int)}.
    */
+  @Deprecated
   public static List<HRegionInfo> parseDelimitedFrom(final byte[] bytes, final int offset,
       final int length) throws IOException {
     if (bytes == null) {
@@ -1211,21 +1010,11 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
    * @param regionA
    * @param regionB
    * @return true if two regions are adjacent
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link org.apache.hadoop.hbase.client.RegionInfo#areAdjacent(RegionInfo, RegionInfo)}.
    */
+  @Deprecated
   public static boolean areAdjacent(HRegionInfo regionA, HRegionInfo regionB) {
-    if (regionA == null || regionB == null) {
-      throw new IllegalArgumentException(
-          "Can't check whether adjacent for null region");
-    }
-    HRegionInfo a = regionA;
-    HRegionInfo b = regionB;
-    if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) {
-      a = regionB;
-      b = regionA;
-    }
-    if (Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0) {
-      return true;
-    }
-    return false;
+    return RegionInfo.areAdjacent(regionA, regionB);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 0a82a6b..2a92409 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -811,10 +811,22 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Get all the online regions on a region server.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>).
+   *             Use {@link #getRegions(ServerName sn)}.
    */
+  @Deprecated
   List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException;
 
   /**
+   * Get all the online regions on a region server.
+   *
+   * @return List of {@link RegionInfo}
+   * @throws java.io.IOException
+   */
+  List<RegionInfo> getRegions(ServerName serverName) throws IOException;
+
+  /**
    * Flush a table. Synchronous operation.
    *
    * @param tableName table to flush
@@ -1510,10 +1522,23 @@ public interface Admin extends Abortable, Closeable {
    * @param tableName the name of the table
    * @return List of {@link HRegionInfo}.
    * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>).
+   *             Use {@link #getRegions(TableName)}.
    */
+  @Deprecated
   List<HRegionInfo> getTableRegions(TableName tableName)
     throws IOException;
 
+  /**
+   * Get the regions of a given table.
+   *
+   * @param tableName the name of the table
+   * @return List of {@link RegionInfo}.
+   * @throws IOException
+   */
+  List<RegionInfo> getRegions(TableName tableName) throws IOException;
+
   @Override
   void close() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 4e17335..fac3ef1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -397,6 +397,16 @@ public class HBaseAdmin implements Admin {
     });
   }
 
+  @Override
+  public List<RegionInfo> getRegions(final ServerName sn) throws IOException {
+    return getOnlineRegions(sn).stream().collect(Collectors.toList());
+  }
+
+  @Override
+  public List<RegionInfo> getRegions(final TableName tableName) throws IOException {
+    return getTableRegions(tableName).stream().collect(Collectors.toList());
+  }
+
   private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
     private boolean isAbortInProgress;
 
@@ -1143,12 +1153,24 @@ public class HBaseAdmin implements Admin {
     unassign(hri.getRegionName(), true);
   }
 
+  /**
+   *
+   * @param sn
+   * @return List of {@link HRegionInfo}.
+   * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegions(ServerName)}.
+   */
+  @Deprecated
   @Override
   public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
     // TODO: There is no timeout on this controller. Set one!
     HBaseRpcController controller = rpcControllerFactory.newController();
-    return ProtobufUtil.getOnlineRegions(controller, admin);
+    List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(controller, admin);
+    return onlineRegions == null ? null : onlineRegions.stream()
+            .map(hri -> new ImmutableHRegionInfo(hri))
+            .collect(Collectors.toList());
   }
 
   @Override
@@ -2340,6 +2362,15 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  /**
+   *
+   * @param tableName
+   * @return List of {@link HRegionInfo}.
+   * @throws IOException
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegions(TableName)}.
+   */
+  @Deprecated
   @Override
   public List<HRegionInfo> getTableRegions(final TableName tableName)
   throws IOException {
@@ -2356,7 +2387,9 @@ public class HBaseAdmin implements Admin {
     } finally {
       zookeeper.close();
     }
-    return regions;
+    return regions == null ? null : regions.stream()
+            .map(hri -> new ImmutableHRegionInfo(hri))
+            .collect(Collectors.toList());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
new file mode 100644
index 0000000..16329c8
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHRegionInfo.java
@@ -0,0 +1,49 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Read-only Region info.
+ */
+@Deprecated // deprecated for hbase 2.0, remove for hbase 3.0. see HRegionInfo.
+@InterfaceAudience.Private
+public class ImmutableHRegionInfo extends HRegionInfo {
+
+  /*
+   * Creates an immutable copy of an HRegionInfo.
+   *
+   * @param other
+   */
+  public ImmutableHRegionInfo(HRegionInfo other) {
+    super(other);
+  }
+
+  @Override
+  public void setSplit(boolean split) {
+    throw new UnsupportedOperationException("HRegionInfo is read-only");
+  }
+
+  @Override
+  public void setOffline(boolean offline) {
+    throw new UnsupportedOperationException("HRegionInfo is read-only");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/58988cb5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
new file mode 100644
index 0000000..3646722
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -0,0 +1,751 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.util.ByteArrayHashKey;
+import org.apache.hadoop.hbase.util.HashKey;
+import org.apache.hadoop.hbase.util.JenkinsHash;
+
+/**
+ * Information about a region. A region is a range of keys in the whole keyspace
+ * of a table, an identifier (a timestamp) for differentiating between subset
+ * ranges (after region split) and a replicaId for differentiating the instance
+ * for the same range and some status information about the region.
+ *
+ * The region has a unique name which consists of the following fields:
+ * <ul>
+ * <li> tableName   : The name of the table </li>
+ * <li> startKey    : The startKey for the region. </li>
+ * <li> regionId    : A timestamp when the region is created. </li>
+ * <li> replicaId   : An id starting from 0 to differentiate replicas of the
+ * same region range but hosted in separated servers. The same region range can
+ * be hosted in multiple locations.</li>
+ * <li> encodedName : An MD5 encoded string for the region name.</li>
+ * </ul>
+ *
+ * <br> Other than the fields in the region name, region info contains:
+ * <ul>
+ * <li> endKey      : the endKey for the region (exclusive) </li>
+ * <li> split       : Whether the region is split </li>
+ * <li> offline     : Whether the region is offline </li>
+ * </ul>
+ *
+ */
+@InterfaceAudience.Public
+public interface RegionInfo {
+  /**
+   * Separator used to demarcate the encodedName in a region name
+   * in the new format. See description on new format above.
+   */
+  @InterfaceAudience.Private
+  int ENC_SEPARATOR = '.';
+
+  @InterfaceAudience.Private
+  int MD5_HEX_LENGTH = 32;
+
+  @InterfaceAudience.Private
+  int DEFAULT_REPLICA_ID = 0;
+
+  /**
+   * to keep appended int's sorted in string format. Only allows 2 bytes
+   * to be sorted for replicaId.
+   */
+  @InterfaceAudience.Private
+  String REPLICA_ID_FORMAT = "%04X";
+
+  @InterfaceAudience.Private
+  byte REPLICA_ID_DELIMITER = (byte)'_';
+
+  @InterfaceAudience.Private
+  String INVALID_REGION_NAME_FORMAT_MESSAGE = "Invalid regionName format";
+
+  @InterfaceAudience.Private
+  Comparator<RegionInfo> COMPARATOR
+    = (RegionInfo lhs, RegionInfo rhs) -> {
+      if (rhs == null) {
+        return 1;
+      }
+
+      // Are regions of same table?
+      int result = lhs.getTable().compareTo(rhs.getTable());
+      if (result != 0) {
+        return result;
+      }
+
+      // Compare start keys.
+      result = Bytes.compareTo(lhs.getStartKey(), rhs.getStartKey());
+      if (result != 0) {
+        return result;
+      }
+
+      // Compare end keys.
+      result = Bytes.compareTo(lhs.getEndKey(), rhs.getEndKey());
+
+      if (result != 0) {
+        if (lhs.getStartKey().length != 0
+                && lhs.getEndKey().length == 0) {
+            return 1; // this is last region
+        }
+        if (rhs.getStartKey().length != 0
+                && rhs.getEndKey().length == 0) {
+            return -1; // o is the last region
+        }
+        return result;
+      }
+
+      // regionId is usually milli timestamp -- this defines older stamps
+      // to be "smaller" than newer stamps in sort order.
+      if (lhs.getRegionId() > rhs.getRegionId()) {
+        return 1;
+      } else if (lhs.getRegionId() < rhs.getRegionId()) {
+        return -1;
+      }
+
+      int replicaDiff = lhs.getReplicaId() - rhs.getReplicaId();
+      if (replicaDiff != 0) return replicaDiff;
+
+      if (lhs.isOffline() == rhs.isOffline())
+        return 0;
+      if (lhs.isOffline() == true) return -1;
+
+      return 1;
+  };
+
+
+  /**
+   * @return Return a short, printable name for this region
+   * (usually encoded name) for us logging.
+   */
+  String getShortNameToLog();
+
+  /**
+   * @return the regionId.
+   */
+  long getRegionId();
+
+  /**
+   * @return the regionName as an array of bytes.
+   * @see #getRegionNameAsString()
+   */
+  byte [] getRegionName();
+
+  /**
+   * @return Region name as a String for use in logging, etc.
+   */
+  String getRegionNameAsString();
+
+  /**
+   * @return the encoded region name.
+   */
+  String getEncodedName();
+
+  /**
+   * @return the encoded region name as an array of bytes.
+   */
+  byte [] getEncodedNameAsBytes();
+
+  /**
+   * @return the startKey.
+   */
+  byte [] getStartKey();
+
+  /**
+   * @return the endKey.
+   */
+  byte [] getEndKey();
+
+  /**
+   * @return current table name of the region
+   */
+  TableName getTable();
+
+  /**
+   * @return returns region replica id
+   */
+  int getReplicaId();
+
+  /**
+   * @return True if has been split and has daughters.
+   */
+  boolean isSplit();
+
+  /**
+   * @return True if this region is offline.
+   */
+  boolean isOffline();
+
+  /**
+   * @return True if this is a split parent region.
+   */
+  boolean isSplitParent();
+
+  /**
+   * @return true if this region is from hbase:meta.
+   */
+  boolean isMetaTable();
+
+  /**
+   * @return true if this region is from a system table.
+   */
+  boolean isSystemTable();
+
+  /**
+   * @return true if this region is a meta region.
+   */
+  boolean isMetaRegion();
+
+  /**
+   * @param rangeStartKey
+   * @param rangeEndKey
+   * @return true if the given inclusive range of rows is fully contained
+   * by this region. For example, if the region is foo,a,g and this is
+   * passed ["b","c"] or ["a","c"] it will return true, but if this is passed
+   * ["b","z"] it will return false.
+   * @throws IllegalArgumentException if the range passed is invalid (ie. end &lt; start)
+   */
+  boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey);
+
+  /**
+   * @param row
+   * @return true if the given row falls in this region.
+   */
+  boolean containsRow(byte[] row);
+
+  /**
+   * Does region name contain its encoded name?
+   * @param regionName region name
+   * @return boolean indicating if this a new format region
+   *         name which contains its encoded name.
+   */
+  @InterfaceAudience.Private
+  static boolean hasEncodedName(final byte[] regionName) {
+    // check if region name ends in ENC_SEPARATOR
+    return (regionName.length >= 1) &&
+      (regionName[regionName.length - 1] == RegionInfo.ENC_SEPARATOR);
+  }
+
+  /**
+   * @return the encodedName
+   */
+  @InterfaceAudience.Private
+  static String encodeRegionName(final byte [] regionName) {
+    String encodedName;
+    if (hasEncodedName(regionName)) {
+      // region is in new format:
+      // <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
+      encodedName = Bytes.toString(regionName,
+      regionName.length - MD5_HEX_LENGTH - 1,
+      MD5_HEX_LENGTH);
+    } else {
+      // old format region name. First hbase:meta region also
+      // use this format.EncodedName is the JenkinsHash value.
+      HashKey<byte[]> key = new ByteArrayHashKey(regionName, 0, regionName.length);
+      int hashVal = Math.abs(JenkinsHash.getInstance().hash(key, 0));
+      encodedName = String.valueOf(hashVal);
+    }
+    return encodedName;
+  }
+
+  /**
+   * @return Return a String of short, printable names for <code>hris</code>
+   * (usually encoded name) for us logging.
+   */
+  static String getShortNameToLog(RegionInfo...hris) {
+    return getShortNameToLog(Arrays.asList(hris));
+  }
+
+  /**
+   * @return Return a String of short, printable names for <code>hris</code>
+   * (usually encoded name) for us logging.
+   */
+  static String getShortNameToLog(final List<RegionInfo> ris) {
+    return ris.stream().map(ri -> ri.getShortNameToLog()).
+    collect(Collectors.toList()).toString();
+  }
+
+  /**
+   * Gets the table name from the specified region name.
+   * @param regionName to extract the table name from
+   * @return Table name
+   */
+  @InterfaceAudience.Private
+  // This method should never be used. Its awful doing parse from bytes.
+  // It is fallback in case we can't get the tablename any other way. Could try removing it.
+  // Keeping it Audience Private so can remove at later date.
+  static TableName getTable(final byte [] regionName) {
+    int offset = -1;
+    for (int i = 0; i < regionName.length; i++) {
+      if (regionName[i] == HConstants.DELIMITER) {
+        offset = i;
+        break;
+      }
+    }
+    byte[] buff  = new byte[offset];
+    System.arraycopy(regionName, 0, buff, 0, offset);
+    return TableName.valueOf(buff);
+  }
+
+  /**
+   * Gets the start key from the specified region name.
+   * @param regionName
+   * @return Start key.
+   * @throws java.io.IOException
+   */
+  static byte[] getStartKey(final byte[] regionName) throws IOException {
+    return parseRegionName(regionName)[1];
+  }
+
+  @InterfaceAudience.Private
+  static boolean isEncodedRegionName(byte[] regionName) throws IOException {
+    try {
+      parseRegionName(regionName);
+      return false;
+    } catch (IOException e) {
+      if (StringUtils.stringifyException(e)
+      .contains(INVALID_REGION_NAME_FORMAT_MESSAGE)) {
+        return true;
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * @param bytes
+   * @return A deserialized {@link RegionInfo}
+   * or null if we failed deserialize or passed bytes null
+   */
+  @InterfaceAudience.Private
+  static RegionInfo parseFromOrNull(final byte [] bytes) {
+    if (bytes == null) return null;
+    return parseFromOrNull(bytes, 0, bytes.length);
+  }
+
+  /**
+   * @param bytes
+   * @param offset
+   * @param len
+   * @return A deserialized {@link RegionInfo} or null
+   *  if we failed deserialize or passed bytes null
+   */
+  @InterfaceAudience.Private
+  static RegionInfo parseFromOrNull(final byte [] bytes, int offset, int len) {
+    if (bytes == null || len <= 0) return null;
+    try {
+      return parseFrom(bytes, offset, len);
+    } catch (DeserializationException e) {
+      return null;
+    }
+  }
+
+  /**
+   * @param bytes A pb RegionInfo serialized with a pb magic prefix.
+   * @return A deserialized {@link RegionInfo}
+   * @throws DeserializationException
+   */
+  @InterfaceAudience.Private
+  static RegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
+    if (bytes == null) return null;
+    return parseFrom(bytes, 0, bytes.length);
+  }
+
+  /**
+   * @param bytes A pb RegionInfo serialized with a pb magic prefix.
+   * @param offset starting point in the byte array
+   * @param len length to read on the byte array
+   * @return A deserialized {@link RegionInfo}
+   * @throws DeserializationException
+   */
+  @InterfaceAudience.Private
+  static RegionInfo parseFrom(final byte [] bytes, int offset, int len)
+  throws DeserializationException {
+    if (ProtobufUtil.isPBMagicPrefix(bytes, offset, len)) {
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      try {
+        HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
+        ProtobufUtil.mergeFrom(builder, bytes, pblen + offset, len - pblen);
+        HBaseProtos.RegionInfo ri = builder.build();
+        return ProtobufUtil.toRegionInfo(ri);
+      } catch (IOException e) {
+        throw new DeserializationException(e);
+      }
+    } else {
+      throw new DeserializationException("PB encoded RegionInfo expected");
+    }
+  }
+
+  /**
+   * Check whether two regions are adjacent
+   * @param regionA
+   * @param regionB
+   * @return true if two regions are adjacent
+   */
+  static boolean areAdjacent(RegionInfo regionA, RegionInfo regionB) {
+    if (regionA == null || regionB == null) {
+      throw new IllegalArgumentException(
+      "Can't check whether adjacent for null region");
+    }
+    RegionInfo a = regionA;
+    RegionInfo b = regionB;
+    if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) {
+      a = regionB;
+      b = regionA;
+    }
+    if (Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * @param ri
+   * @return This instance serialized as protobuf w/ a magic pb prefix.
+   * @see #parseFrom(byte[])
+   */
+  static byte [] toByteArray(RegionInfo ri) {
+    byte [] bytes = ProtobufUtil.toProtoRegionInfo(ri).toByteArray();
+    return ProtobufUtil.prependPBMagic(bytes);
+  }
+
+  /**
+   * Use logging.
+   * @param encodedRegionName The encoded regionname.
+   * @return <code>hbase:meta</code> if passed <code>1028785192</code> else returns
+   * <code>encodedRegionName</code>
+   */
+  static String prettyPrint(final String encodedRegionName) {
+    if (encodedRegionName.equals("1028785192")) {
+      return encodedRegionName + "/hbase:meta";
+    }
+    return encodedRegionName;
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param regionid Region id (Usually timestamp from when region was created).
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey and id
+   */
+  static byte [] createRegionName(final TableName tableName, final byte[] startKey,
+                                  final long regionid, boolean newFormat) {
+    return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param id Region id (Usually timestamp from when region was created).
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey and id
+   */
+  static byte [] createRegionName(final TableName tableName,
+                                  final byte[] startKey, final String id, boolean newFormat) {
+    return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param regionid Region id (Usually timestamp from when region was created).
+   * @param replicaId
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey, id and replicaId
+   */
+  static byte [] createRegionName(final TableName tableName,
+      final byte[] startKey, final long regionid, int replicaId, boolean newFormat) {
+    return createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
+      replicaId, newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param id Region id (Usually timestamp from when region was created).
+   * @param newFormat should we create the region name in the new format
+   *                  (such that it contains its encoded name?).
+   * @return Region name made of passed tableName, startKey and id
+   */
+  static byte [] createRegionName(final TableName tableName,
+      final byte[] startKey, final byte[] id, boolean newFormat) {
+    return createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
+  }
+
+  /**
+   * Make a region name of passed parameters.
+   * @param tableName
+   * @param startKey Can be null
+   * @param id Region id (Usually timestamp from when region was created).
+   * @param replicaId
+   * @param newFormat should we create the region name in the new format
+   * @return Region name made of passed tableName, startKey, id and replicaId
+   */
+  static byte [] createRegionName(final TableName tableName,
+      final byte[] startKey, final byte[] id, final int replicaId, boolean newFormat) {
+    int len = tableName.getName().length + 2 + id.length + (startKey == null? 0: startKey.length);
+    if (newFormat) {
+      len += MD5_HEX_LENGTH + 2;
+    }
+    byte[] replicaIdBytes = null;
+    // Special casing: replicaId is only appended if replicaId is greater than
+    // 0. This is because all regions in meta would have to be migrated to the new
+    // name otherwise
+    if (replicaId > 0) {
+      // use string representation for replica id
+      replicaIdBytes = Bytes.toBytes(String.format(REPLICA_ID_FORMAT, replicaId));
+      len += 1 + replicaIdBytes.length;
+    }
+
+    byte [] b = new byte [len];
+
+    int offset = tableName.getName().length;
+    System.arraycopy(tableName.getName(), 0, b, 0, offset);
+    b[offset++] = HConstants.DELIMITER;
+    if (startKey != null && startKey.length > 0) {
+      System.arraycopy(startKey, 0, b, offset, startKey.length);
+      offset += startKey.length;
+    }
+    b[offset++] = HConstants.DELIMITER;
+    System.arraycopy(id, 0, b, offset, id.length);
+    offset += id.length;
+
+    if (replicaIdBytes != null) {
+      b[offset++] = REPLICA_ID_DELIMITER;
+      System.arraycopy(replicaIdBytes, 0, b, offset, replicaIdBytes.length);
+      offset += replicaIdBytes.length;
+    }
+
+    if (newFormat) {
+      //
+      // Encoded name should be built into the region name.
+      //
+      // Use the region name thus far (namely, <tablename>,<startKey>,<id>_<replicaId>)
+      // to compute a MD5 hash to be used as the encoded name, and append
+      // it to the byte buffer.
+      //
+      String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
+      byte [] md5HashBytes = Bytes.toBytes(md5Hash);
+
+      if (md5HashBytes.length != MD5_HEX_LENGTH) {
+        System.out.println("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
+        "; Got=" + md5HashBytes.length);
+      }
+
+      // now append the bytes '.<encodedName>.' to the end
+      b[offset++] = ENC_SEPARATOR;
+      System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
+      offset += MD5_HEX_LENGTH;
+      b[offset++] = ENC_SEPARATOR;
+    }
+
+    return b;
+  }
+
+  /**
+   * Separate elements of a regionName.
+   * @param regionName
+   * @return Array of byte[] containing tableName, startKey and id
+   * @throws IOException
+   */
+  static byte [][] parseRegionName(final byte[] regionName)
+  throws IOException {
+    // Region name is of the format:
+    // tablename,startkey,regionIdTimestamp[_replicaId][.encodedName.]
+    // startkey can contain the delimiter (',') so we parse from the start and end
+
+    // parse from start
+    int offset = -1;
+    for (int i = 0; i < regionName.length; i++) {
+      if (regionName[i] == HConstants.DELIMITER) {
+        offset = i;
+        break;
+      }
+    }
+    if (offset == -1) {
+      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE
+      + ": " + Bytes.toStringBinary(regionName));
+    }
+    byte[] tableName = new byte[offset];
+    System.arraycopy(regionName, 0, tableName, 0, offset);
+    offset = -1;
+
+    int endOffset = regionName.length;
+    // check whether regionName contains encodedName
+    if (regionName.length > MD5_HEX_LENGTH + 2
+    && regionName[regionName.length-1] == ENC_SEPARATOR
+    && regionName[regionName.length-MD5_HEX_LENGTH-2] == ENC_SEPARATOR) {
+      endOffset = endOffset - MD5_HEX_LENGTH - 2;
+    }
+
+    // parse from end
+    byte[] replicaId = null;
+    int idEndOffset = endOffset;
+    for (int i = endOffset - 1; i > 0; i--) {
+      if (regionName[i] == REPLICA_ID_DELIMITER) { //replicaId may or may not be present
+        replicaId = new byte[endOffset - i - 1];
+        System.arraycopy(regionName, i + 1, replicaId, 0,
+        endOffset - i - 1);
+        idEndOffset = i;
+        // do not break, continue to search for id
+      }
+      if (regionName[i] == HConstants.DELIMITER) {
+        offset = i;
+        break;
+      }
+    }
+    if (offset == -1) {
+      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE
+      + ": " + Bytes.toStringBinary(regionName));
+    }
+    byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
+    if(offset != tableName.length + 1) {
+      startKey = new byte[offset - tableName.length - 1];
+      System.arraycopy(regionName, tableName.length + 1, startKey, 0,
+      offset - tableName.length - 1);
+    }
+    byte [] id = new byte[idEndOffset - offset - 1];
+    System.arraycopy(regionName, offset + 1, id, 0,
+    idEndOffset - offset - 1);
+    byte [][] elements = new byte[replicaId == null ? 3 : 4][];
+    elements[0] = tableName;
+    elements[1] = startKey;
+    elements[2] = id;
+    if (replicaId != null) {
+      elements[3] = replicaId;
+    }
+    return elements;
+  }
+
+  /**
+   * Serializes given RegionInfo's as a byte array. Use this instead of
+   * {@link RegionInfo#toByteArray(RegionInfo)} when
+   * writing to a stream and you want to use the pb mergeDelimitedFrom (w/o the delimiter, pb reads
+   * to EOF which may not be what you want). {@link #parseDelimitedFrom(byte[], int, int)} can
+   * be used to read back the instances.
+   * @param infos RegionInfo objects to serialize
+   * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
+   * @throws IOException
+   */
+  static byte[] toDelimitedByteArray(RegionInfo... infos) throws IOException {
+    byte[][] bytes = new byte[infos.length][];
+    int size = 0;
+    for (int i = 0; i < infos.length; i++) {
+      bytes[i] = toDelimitedByteArray(infos[i]);
+      size += bytes[i].length;
+    }
+
+    byte[] result = new byte[size];
+    int offset = 0;
+    for (byte[] b : bytes) {
+      System.arraycopy(b, 0, result, offset, b.length);
+      offset += b.length;
+    }
+    return result;
+  }
+
+  /**
+   * Use this instead of {@link RegionInfo#toByteArray(RegionInfo)} when writing to a stream and you want to use
+   * the pb mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want).
+   * @param ri
+   * @return This instance serialized as a delimied protobuf w/ a magic pb prefix.
+   * @throws IOException
+   */
+  static byte [] toDelimitedByteArray(RegionInfo ri) throws IOException {
+    return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toProtoRegionInfo(ri));
+  }
+
+  /**
+   * Parses an RegionInfo instance from the passed in stream.
+   * Presumes the RegionInfo was serialized to the stream with
+   * {@link #toDelimitedByteArray(RegionInfo)}.
+   * @param in
+   * @return An instance of RegionInfo.
+   * @throws IOException
+   */
+  static RegionInfo parseFrom(final DataInputStream in) throws IOException {
+    // I need to be able to move back in the stream if this is not a pb
+    // serialization so I can do the Writable decoding instead.
+    int pblen = ProtobufUtil.lengthOfPBMagic();
+    byte [] pbuf = new byte[pblen];
+    if (in.markSupported()) { //read it with mark()
+      in.mark(pblen);
+    }
+
+    //assumption: if Writable serialization, it should be longer than pblen.
+    int read = in.read(pbuf);
+    if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
+    if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
+      return ProtobufUtil.toRegionInfo(HBaseProtos.RegionInfo.parseDelimitedFrom(in));
+    } else {
+      throw new IOException("PB encoded RegionInfo expected");
+    }
+  }
+
+  /**
+   * Parses all the RegionInfo instances from the passed in stream until EOF. Presumes the
+   * RegionInfo's were serialized to the stream with oDelimitedByteArray()
+   * @param bytes serialized bytes
+   * @param offset the start offset into the byte[] buffer
+   * @param length how far we should read into the byte[] buffer
+   * @return All the RegionInfos that are in the byte array. Keeps reading till we hit the end.
+   * @throws IOException
+   */
+  static List<RegionInfo> parseDelimitedFrom(final byte[] bytes, final int offset,
+                                             final int length) throws IOException {
+    if (bytes == null) {
+      throw new IllegalArgumentException("Can't build an object with empty bytes array");
+    }
+    DataInputBuffer in = new DataInputBuffer();
+    List<RegionInfo> ris = new ArrayList<>();
+    try {
+      in.reset(bytes, offset, length);
+      while (in.available() > 0) {
+        RegionInfo ri = parseFrom(in);
+        ris.add(ri);
+      }
+    } finally {
+      in.close();
+    }
+    return ris;
+  }
+}
\ No newline at end of file