001/*
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.client;
020
021import edu.umd.cs.findbugs.annotations.CheckForNull;
022
023import java.io.DataInputStream;
024import java.io.IOException;
025import java.util.ArrayList;
026import java.util.Arrays;
027import java.util.Comparator;
028import java.util.List;
029import java.util.stream.Collectors;
030
031import org.apache.hadoop.hbase.HConstants;
032import org.apache.hadoop.hbase.TableName;
033import org.apache.hadoop.hbase.exceptions.DeserializationException;
034import org.apache.hadoop.hbase.util.ByteArrayHashKey;
035import org.apache.hadoop.hbase.util.Bytes;
036import org.apache.hadoop.hbase.util.HashKey;
037import org.apache.hadoop.hbase.util.JenkinsHash;
038import org.apache.hadoop.hbase.util.MD5Hash;
039import org.apache.hadoop.io.DataInputBuffer;
040import org.apache.yetus.audience.InterfaceAudience;
041
042import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
043import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
044
045/**
046 * Information about a region. A region is a range of keys in the whole keyspace
047 * of a table, an identifier (a timestamp) for differentiating between subset
048 * ranges (after region split) and a replicaId for differentiating the instance
049 * for the same range and some status information about the region.
050 *
051 * The region has a unique name which consists of the following fields:
052 * <ul>
053 * <li> tableName   : The name of the table </li>
054 * <li> startKey    : The startKey for the region. </li>
055 * <li> regionId    : A timestamp when the region is created. </li>
056 * <li> replicaId   : An id starting from 0 to differentiate replicas of the
057 * same region range but hosted in separated servers. The same region range can
058 * be hosted in multiple locations.</li>
059 * <li> encodedName : An MD5 encoded string for the region name.</li>
060 * </ul>
061 *
062 * <br> Other than the fields in the region name, region info contains:
063 * <ul>
064 * <li> endKey      : the endKey for the region (exclusive) </li>
065 * <li> split       : Whether the region is split </li>
066 * <li> offline     : Whether the region is offline </li>
067 * </ul>
068 *
069 */
070@InterfaceAudience.Public
071public interface RegionInfo {
072  RegionInfo UNDEFINED = RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
073  /**
074   * Separator used to demarcate the encodedName in a region name
075   * in the new format. See description on new format above.
076   */
077  @InterfaceAudience.Private
078  int ENC_SEPARATOR = '.';
079
080  @InterfaceAudience.Private
081  int MD5_HEX_LENGTH = 32;
082
083  @InterfaceAudience.Private
084  int DEFAULT_REPLICA_ID = 0;
085
086  /**
087   * to keep appended int's sorted in string format. Only allows 2 bytes
088   * to be sorted for replicaId.
089   */
090  @InterfaceAudience.Private
091  String REPLICA_ID_FORMAT = "%04X";
092
093  @InterfaceAudience.Private
094  byte REPLICA_ID_DELIMITER = (byte)'_';
095
096  @InterfaceAudience.Private
097  String INVALID_REGION_NAME_FORMAT_MESSAGE = "Invalid regionName format";
098
099  @InterfaceAudience.Private
100  Comparator<RegionInfo> COMPARATOR
101    = (RegionInfo lhs, RegionInfo rhs) -> {
102      if (rhs == null) {
103        return 1;
104      }
105
106      // Are regions of same table?
107      int result = lhs.getTable().compareTo(rhs.getTable());
108      if (result != 0) {
109        return result;
110      }
111
112      // Compare start keys.
113      result = Bytes.compareTo(lhs.getStartKey(), rhs.getStartKey());
114      if (result != 0) {
115        return result;
116      }
117
118      // Compare end keys.
119      result = Bytes.compareTo(lhs.getEndKey(), rhs.getEndKey());
120
121      if (result != 0) {
122        if (lhs.getStartKey().length != 0
123                && lhs.getEndKey().length == 0) {
124            return 1; // this is last region
125        }
126        if (rhs.getStartKey().length != 0
127                && rhs.getEndKey().length == 0) {
128            return -1; // o is the last region
129        }
130        return result;
131      }
132
133      // regionId is usually milli timestamp -- this defines older stamps
134      // to be "smaller" than newer stamps in sort order.
135      if (lhs.getRegionId() > rhs.getRegionId()) {
136        return 1;
137      } else if (lhs.getRegionId() < rhs.getRegionId()) {
138        return -1;
139      }
140
141      int replicaDiff = lhs.getReplicaId() - rhs.getReplicaId();
142      if (replicaDiff != 0) {
143        return replicaDiff;
144      }
145
146      if (lhs.isOffline() == rhs.isOffline()) {
147        return 0;
148      }
149      if (lhs.isOffline()) {
150        return -1;
151      }
152
153      return 1;
154  };
155
156
157  /**
158   * @return Return a short, printable name for this region
159   * (usually encoded name) for us logging.
160   */
161  String getShortNameToLog();
162
163  /**
164   * @return the regionId.
165   */
166  long getRegionId();
167
168  /**
169   * @return the regionName as an array of bytes.
170   * @see #getRegionNameAsString()
171   */
172  byte [] getRegionName();
173
174  /**
175   * @return Region name as a String for use in logging, etc.
176   */
177  String getRegionNameAsString();
178
179  /**
180   * @return the encoded region name.
181   */
182  String getEncodedName();
183
184  /**
185   * @return the encoded region name as an array of bytes.
186   */
187  byte [] getEncodedNameAsBytes();
188
189  /**
190   * @return the startKey.
191   */
192  byte [] getStartKey();
193
194  /**
195   * @return the endKey.
196   */
197  byte [] getEndKey();
198
199  /**
200   * @return current table name of the region
201   */
202  TableName getTable();
203
204  /**
205   * @return returns region replica id
206   */
207  int getReplicaId();
208
209  /**
210   * @return True if has been split and has daughters.
211   */
212  boolean isSplit();
213
214  /**
215   * @return True if this region is offline.
216   */
217  boolean isOffline();
218
219  /**
220   * @return True if this is a split parent region.
221   */
222  boolean isSplitParent();
223
224  /**
225   * @return true if this region is a meta region.
226   */
227  boolean isMetaRegion();
228
229  /**
230   * @return true if the given inclusive range of rows is fully contained
231   * by this region. For example, if the region is foo,a,g and this is
232   * passed ["b","c"] or ["a","c"] it will return true, but if this is passed
233   * ["b","z"] it will return false.
234   * @throws IllegalArgumentException if the range passed is invalid (ie. end &lt; start)
235   */
236  boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey);
237
238  /**
239   * @return true if the given row falls in this region.
240   */
241  boolean containsRow(byte[] row);
242
243  /**
244   * Does region name contain its encoded name?
245   * @param regionName region name
246   * @return boolean indicating if this a new format region
247   *         name which contains its encoded name.
248   */
249  @InterfaceAudience.Private
250  static boolean hasEncodedName(final byte[] regionName) {
251    // check if region name ends in ENC_SEPARATOR
252    return (regionName.length >= 1) &&
253      (regionName[regionName.length - 1] == RegionInfo.ENC_SEPARATOR);
254  }
255
256  /**
257   * @return the encodedName
258   */
259  @InterfaceAudience.Private
260  static String encodeRegionName(final byte [] regionName) {
261    String encodedName;
262    if (hasEncodedName(regionName)) {
263      // region is in new format:
264      // <tableName>,<startKey>,<regionIdTimeStamp>/encodedName/
265      encodedName = Bytes.toString(regionName,
266      regionName.length - MD5_HEX_LENGTH - 1,
267      MD5_HEX_LENGTH);
268    } else {
269      // old format region name. First hbase:meta region also
270      // use this format.EncodedName is the JenkinsHash value.
271      HashKey<byte[]> key = new ByteArrayHashKey(regionName, 0, regionName.length);
272      int hashVal = Math.abs(JenkinsHash.getInstance().hash(key, 0));
273      encodedName = String.valueOf(hashVal);
274    }
275    return encodedName;
276  }
277
278  @InterfaceAudience.Private
279  static String getRegionNameAsString(byte[] regionName) {
280    return getRegionNameAsString(null, regionName);
281  }
282
283  @InterfaceAudience.Private
284  static String getRegionNameAsString(@CheckForNull RegionInfo ri, byte[] regionName) {
285    if (RegionInfo.hasEncodedName(regionName)) {
286      // new format region names already have their encoded name.
287      return Bytes.toStringBinary(regionName);
288    }
289
290    // old format. regionNameStr doesn't have the region name.
291    if (ri == null) {
292      return Bytes.toStringBinary(regionName) + "." + RegionInfo.encodeRegionName(regionName);
293    } else {
294      return Bytes.toStringBinary(regionName) + "." + ri.getEncodedName();
295    }
296  }
297
298  /**
299   * @return Return a String of short, printable names for <code>hris</code>
300   * (usually encoded name) for us logging.
301   */
302  static String getShortNameToLog(RegionInfo...hris) {
303    return getShortNameToLog(Arrays.asList(hris));
304  }
305
306  /**
307   * @return Return a String of short, printable names for <code>hris</code>
308   * (usually encoded name) for us logging.
309   */
310  static String getShortNameToLog(final List<RegionInfo> ris) {
311    return ris.stream().map(ri -> ri.getShortNameToLog()).
312    collect(Collectors.toList()).toString();
313  }
314
315  /**
316   * Gets the table name from the specified region name.
317   * @param regionName to extract the table name from
318   * @return Table name
319   */
320  @InterfaceAudience.Private
321  // This method should never be used. Its awful doing parse from bytes.
322  // It is fallback in case we can't get the tablename any other way. Could try removing it.
323  // Keeping it Audience Private so can remove at later date.
324  static TableName getTable(final byte [] regionName) {
325    int offset = -1;
326    for (int i = 0; i < regionName.length; i++) {
327      if (regionName[i] == HConstants.DELIMITER) {
328        offset = i;
329        break;
330      }
331    }
332    if (offset <= 0) {
333      throw new IllegalArgumentException("offset=" + offset);
334    }
335    byte[] buff  = new byte[offset];
336    System.arraycopy(regionName, 0, buff, 0, offset);
337    return TableName.valueOf(buff);
338  }
339
340  /**
341   * Gets the start key from the specified region name.
342   * @return Start key.
343   */
344  static byte[] getStartKey(final byte[] regionName) throws IOException {
345    return parseRegionName(regionName)[1];
346  }
347
348  /**
349   * Figure if the passed bytes represent an encoded region name or not.
350   * @param regionName A Region name either encoded or not.
351   * @return True if <code>regionName</code> represents an encoded name.
352   */
353  @InterfaceAudience.Private // For use by internals only.
354  public static boolean isEncodedRegionName(byte[] regionName) throws IOException {
355    // If not parseable as region name, presume encoded. TODO: add stringency; e.g. if hex.
356    return parseRegionNameOrReturnNull(regionName) == null && regionName.length <= MD5_HEX_LENGTH;
357  }
358
359  /**
360   * @return A deserialized {@link RegionInfo}
361   * or null if we failed deserialize or passed bytes null
362   */
363  @InterfaceAudience.Private
364  static RegionInfo parseFromOrNull(final byte [] bytes) {
365    if (bytes == null) return null;
366    return parseFromOrNull(bytes, 0, bytes.length);
367  }
368
369  /**
370   * @return A deserialized {@link RegionInfo} or null
371   *  if we failed deserialize or passed bytes null
372   */
373  @InterfaceAudience.Private
374  static RegionInfo parseFromOrNull(final byte [] bytes, int offset, int len) {
375    if (bytes == null || len <= 0) return null;
376    try {
377      return parseFrom(bytes, offset, len);
378    } catch (DeserializationException e) {
379      return null;
380    }
381  }
382
383  /**
384   * @param bytes A pb RegionInfo serialized with a pb magic prefix.
385   * @return A deserialized {@link RegionInfo}
386   */
387  @InterfaceAudience.Private
388  static RegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
389    if (bytes == null) return null;
390    return parseFrom(bytes, 0, bytes.length);
391  }
392
393  /**
394   * @param bytes A pb RegionInfo serialized with a pb magic prefix.
395   * @param offset starting point in the byte array
396   * @param len length to read on the byte array
397   * @return A deserialized {@link RegionInfo}
398   */
399  @InterfaceAudience.Private
400  static RegionInfo parseFrom(final byte [] bytes, int offset, int len)
401  throws DeserializationException {
402    if (ProtobufUtil.isPBMagicPrefix(bytes, offset, len)) {
403      int pblen = ProtobufUtil.lengthOfPBMagic();
404      try {
405        HBaseProtos.RegionInfo.Builder builder = HBaseProtos.RegionInfo.newBuilder();
406        ProtobufUtil.mergeFrom(builder, bytes, pblen + offset, len - pblen);
407        HBaseProtos.RegionInfo ri = builder.build();
408        return ProtobufUtil.toRegionInfo(ri);
409      } catch (IOException e) {
410        throw new DeserializationException(e);
411      }
412    } else {
413      throw new DeserializationException("PB encoded RegionInfo expected");
414    }
415  }
416
417  /**
418   * Check whether two regions are adjacent; i.e. lies just before or just
419   * after in a table.
420   * @return true if two regions are adjacent
421   */
422  static boolean areAdjacent(RegionInfo regionA, RegionInfo regionB) {
423    if (regionA == null || regionB == null) {
424      throw new IllegalArgumentException(
425      "Can't check whether adjacent for null region");
426    }
427    if (!regionA.getTable().equals(regionB.getTable())) {
428      return false;
429    }
430    RegionInfo a = regionA;
431    RegionInfo b = regionB;
432    if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) {
433      a = regionB;
434      b = regionA;
435    }
436    return Bytes.equals(a.getEndKey(), b.getStartKey());
437  }
438
439  /**
440   * @return This instance serialized as protobuf w/ a magic pb prefix.
441   * @see #parseFrom(byte[])
442   */
443  static byte [] toByteArray(RegionInfo ri) {
444    byte [] bytes = ProtobufUtil.toRegionInfo(ri).toByteArray();
445    return ProtobufUtil.prependPBMagic(bytes);
446  }
447
448  /**
449   * Use logging.
450   * @param encodedRegionName The encoded regionname.
451   * @return <code>hbase:meta</code> if passed <code>1028785192</code> else returns
452   * <code>encodedRegionName</code>
453   */
454  static String prettyPrint(final String encodedRegionName) {
455    if (encodedRegionName.equals("1028785192")) {
456      return encodedRegionName + "/hbase:meta";
457    }
458    return encodedRegionName;
459  }
460
461  /**
462   * Make a region name of passed parameters.
463   * @param startKey Can be null
464   * @param regionid Region id (Usually timestamp from when region was created).
465   * @param newFormat should we create the region name in the new format
466   *                  (such that it contains its encoded name?).
467   * @return Region name made of passed tableName, startKey and id
468   */
469  static byte [] createRegionName(final TableName tableName, final byte[] startKey,
470                                  final long regionid, boolean newFormat) {
471    return createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
472  }
473
474  /**
475   * Make a region name of passed parameters.
476   * @param startKey Can be null
477   * @param id Region id (Usually timestamp from when region was created).
478   * @param newFormat should we create the region name in the new format
479   *                  (such that it contains its encoded name?).
480   * @return Region name made of passed tableName, startKey and id
481   */
482  static byte [] createRegionName(final TableName tableName,
483                                  final byte[] startKey, final String id, boolean newFormat) {
484    return createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
485  }
486
487  /**
488   * Make a region name of passed parameters.
489   * @param startKey Can be null
490   * @param regionid Region id (Usually timestamp from when region was created).
491   * @param newFormat should we create the region name in the new format
492   *                  (such that it contains its encoded name?).
493   * @return Region name made of passed tableName, startKey, id and replicaId
494   */
495  static byte [] createRegionName(final TableName tableName,
496      final byte[] startKey, final long regionid, int replicaId, boolean newFormat) {
497    return createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
498      replicaId, newFormat);
499  }
500
501  /**
502   * Make a region name of passed parameters.
503   * @param startKey Can be null
504   * @param id Region id (Usually timestamp from when region was created).
505   * @param newFormat should we create the region name in the new format
506   *                  (such that it contains its encoded name?).
507   * @return Region name made of passed tableName, startKey and id
508   */
509  static byte [] createRegionName(final TableName tableName,
510      final byte[] startKey, final byte[] id, boolean newFormat) {
511    return createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
512  }
513
514  /**
515   * Make a region name of passed parameters.
516   * @param startKey Can be null
517   * @param id Region id (Usually timestamp from when region was created).
518   * @param newFormat should we create the region name in the new format
519   * @return Region name made of passed tableName, startKey, id and replicaId
520   */
521  static byte [] createRegionName(final TableName tableName,
522      final byte[] startKey, final byte[] id, final int replicaId, boolean newFormat) {
523    int len = tableName.getName().length + 2 + id.length + (startKey == null? 0: startKey.length);
524    if (newFormat) {
525      len += MD5_HEX_LENGTH + 2;
526    }
527    byte[] replicaIdBytes = null;
528    // Special casing: replicaId is only appended if replicaId is greater than
529    // 0. This is because all regions in meta would have to be migrated to the new
530    // name otherwise
531    if (replicaId > 0) {
532      // use string representation for replica id
533      replicaIdBytes = Bytes.toBytes(String.format(REPLICA_ID_FORMAT, replicaId));
534      len += 1 + replicaIdBytes.length;
535    }
536
537    byte [] b = new byte [len];
538
539    int offset = tableName.getName().length;
540    System.arraycopy(tableName.getName(), 0, b, 0, offset);
541    b[offset++] = HConstants.DELIMITER;
542    if (startKey != null && startKey.length > 0) {
543      System.arraycopy(startKey, 0, b, offset, startKey.length);
544      offset += startKey.length;
545    }
546    b[offset++] = HConstants.DELIMITER;
547    System.arraycopy(id, 0, b, offset, id.length);
548    offset += id.length;
549
550    if (replicaIdBytes != null) {
551      b[offset++] = REPLICA_ID_DELIMITER;
552      System.arraycopy(replicaIdBytes, 0, b, offset, replicaIdBytes.length);
553      offset += replicaIdBytes.length;
554    }
555
556    if (newFormat) {
557      //
558      // Encoded name should be built into the region name.
559      //
560      // Use the region name thus far (namely, <tablename>,<startKey>,<id>_<replicaId>)
561      // to compute a MD5 hash to be used as the encoded name, and append
562      // it to the byte buffer.
563      //
564      String md5Hash = MD5Hash.getMD5AsHex(b, 0, offset);
565      byte [] md5HashBytes = Bytes.toBytes(md5Hash);
566
567      if (md5HashBytes.length != MD5_HEX_LENGTH) {
568        System.out.println("MD5-hash length mismatch: Expected=" + MD5_HEX_LENGTH +
569        "; Got=" + md5HashBytes.length);
570      }
571
572      // now append the bytes '.<encodedName>.' to the end
573      b[offset++] = ENC_SEPARATOR;
574      System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH);
575      offset += MD5_HEX_LENGTH;
576      b[offset] = ENC_SEPARATOR;
577    }
578
579    return b;
580  }
581
582  /**
583   * Creates a RegionInfo object for MOB data.
584   *
585   * @param tableName the name of the table
586   * @return the MOB {@link RegionInfo}.
587   */
588  static RegionInfo createMobRegionInfo(TableName tableName) {
589    return RegionInfoBuilder.newBuilder(tableName)
590        .setStartKey(Bytes.toBytes(".mob")).setRegionId(0).build();
591  }
592
593  /**
594   * Separate elements of a regionName.
595   * @return Array of byte[] containing tableName, startKey and id OR null if
596   *   not parseable as a region name.
597   * @throws IOException if not parseable as regionName.
598   */
599  static byte [][] parseRegionName(final byte[] regionName) throws IOException {
600    byte [][] result = parseRegionNameOrReturnNull(regionName);
601    if (result == null) {
602      throw new IOException(INVALID_REGION_NAME_FORMAT_MESSAGE + ": " + Bytes.toStringBinary(regionName));
603    }
604    return result;
605  }
606
607  /**
608   * Separate elements of a regionName.
609   * Region name is of the format:
610   * <code>tablename,startkey,regionIdTimestamp[_replicaId][.encodedName.]</code>.
611   * Startkey can contain the delimiter (',') so we parse from the start and then parse from
612   * the end.
613   * @return Array of byte[] containing tableName, startKey and id OR null if not parseable
614   * as a region name.
615   */
616  static byte [][] parseRegionNameOrReturnNull(final byte[] regionName) {
617    int offset = -1;
618    for (int i = 0; i < regionName.length; i++) {
619      if (regionName[i] == HConstants.DELIMITER) {
620        offset = i;
621        break;
622      }
623    }
624    if (offset == -1) {
625      return null;
626    }
627    byte[] tableName = new byte[offset];
628    System.arraycopy(regionName, 0, tableName, 0, offset);
629    offset = -1;
630
631    int endOffset = regionName.length;
632    // check whether regionName contains encodedName
633    if (regionName.length > MD5_HEX_LENGTH + 2 &&
634        regionName[regionName.length-1] == ENC_SEPARATOR &&
635        regionName[regionName.length-MD5_HEX_LENGTH-2] == ENC_SEPARATOR) {
636      endOffset = endOffset - MD5_HEX_LENGTH - 2;
637    }
638
639    // parse from end
640    byte[] replicaId = null;
641    int idEndOffset = endOffset;
642    for (int i = endOffset - 1; i > 0; i--) {
643      if (regionName[i] == REPLICA_ID_DELIMITER) { //replicaId may or may not be present
644        replicaId = new byte[endOffset - i - 1];
645        System.arraycopy(regionName, i + 1, replicaId, 0,
646        endOffset - i - 1);
647        idEndOffset = i;
648        // do not break, continue to search for id
649      }
650      if (regionName[i] == HConstants.DELIMITER) {
651        offset = i;
652        break;
653      }
654    }
655    if (offset == -1) {
656      return null;
657    }
658    byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
659    if(offset != tableName.length + 1) {
660      startKey = new byte[offset - tableName.length - 1];
661      System.arraycopy(regionName, tableName.length + 1, startKey, 0,
662      offset - tableName.length - 1);
663    }
664    byte [] id = new byte[idEndOffset - offset - 1];
665    System.arraycopy(regionName, offset + 1, id, 0,
666    idEndOffset - offset - 1);
667    byte [][] elements = new byte[replicaId == null ? 3 : 4][];
668    elements[0] = tableName;
669    elements[1] = startKey;
670    elements[2] = id;
671    if (replicaId != null) {
672      elements[3] = replicaId;
673    }
674    return elements;
675  }
676
677  /**
678   * Serializes given RegionInfo's as a byte array. Use this instead of
679   * {@link RegionInfo#toByteArray(RegionInfo)} when
680   * writing to a stream and you want to use the pb mergeDelimitedFrom (w/o the delimiter, pb reads
681   * to EOF which may not be what you want). {@link #parseDelimitedFrom(byte[], int, int)} can
682   * be used to read back the instances.
683   * @param infos RegionInfo objects to serialize
684   * @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
685   */
686  static byte[] toDelimitedByteArray(RegionInfo... infos) throws IOException {
687    byte[][] bytes = new byte[infos.length][];
688    int size = 0;
689    for (int i = 0; i < infos.length; i++) {
690      bytes[i] = toDelimitedByteArray(infos[i]);
691      size += bytes[i].length;
692    }
693
694    byte[] result = new byte[size];
695    int offset = 0;
696    for (byte[] b : bytes) {
697      System.arraycopy(b, 0, result, offset, b.length);
698      offset += b.length;
699    }
700    return result;
701  }
702
703  /**
704   * Use this instead of {@link RegionInfo#toByteArray(RegionInfo)} when writing to a stream and you want to use
705   * the pb mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want).
706   * @return This instance serialized as a delimied protobuf w/ a magic pb prefix.
707   */
708  static byte [] toDelimitedByteArray(RegionInfo ri) throws IOException {
709    return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toRegionInfo(ri));
710  }
711
712  /**
713   * Parses an RegionInfo instance from the passed in stream.
714   * Presumes the RegionInfo was serialized to the stream with
715   * {@link #toDelimitedByteArray(RegionInfo)}.
716   * @return An instance of RegionInfo.
717   */
718  static RegionInfo parseFrom(final DataInputStream in) throws IOException {
719    // I need to be able to move back in the stream if this is not a pb
720    // serialization so I can do the Writable decoding instead.
721    int pblen = ProtobufUtil.lengthOfPBMagic();
722    byte [] pbuf = new byte[pblen];
723    if (in.markSupported()) { //read it with mark()
724      in.mark(pblen);
725    }
726
727    //assumption: if Writable serialization, it should be longer than pblen.
728    int read = in.read(pbuf);
729    if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
730    if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
731      return ProtobufUtil.toRegionInfo(HBaseProtos.RegionInfo.parseDelimitedFrom(in));
732    } else {
733      throw new IOException("PB encoded RegionInfo expected");
734    }
735  }
736
737  /**
738   * Parses all the RegionInfo instances from the passed in stream until EOF. Presumes the
739   * RegionInfo's were serialized to the stream with oDelimitedByteArray()
740   * @param bytes serialized bytes
741   * @param offset the start offset into the byte[] buffer
742   * @param length how far we should read into the byte[] buffer
743   * @return All the RegionInfos that are in the byte array. Keeps reading till we hit the end.
744   */
745  static List<RegionInfo> parseDelimitedFrom(final byte[] bytes, final int offset,
746                                             final int length) throws IOException {
747    if (bytes == null) {
748      throw new IllegalArgumentException("Can't build an object with empty bytes array");
749    }
750    List<RegionInfo> ris = new ArrayList<>();
751    try (DataInputBuffer in = new DataInputBuffer()) {
752      in.reset(bytes, offset, length);
753      while (in.available() > 0) {
754        RegionInfo ri = parseFrom(in);
755        ris.add(ri);
756      }
757    }
758    return ris;
759  }
760
761  /**
762   * @return True if this is first Region in Table
763   */
764  default boolean isFirst() {
765    return Bytes.equals(getStartKey(), HConstants.EMPTY_START_ROW);
766  }
767
768  /**
769   * @return True if this is last Region in Table
770   */
771  default boolean isLast() {
772    return Bytes.equals(getEndKey(), HConstants.EMPTY_START_ROW);
773  }
774
775  /**
776   * @return True if region is next, adjacent but 'after' this one.
777   * @see #isAdjacent(RegionInfo)
778   * @see #areAdjacent(RegionInfo, RegionInfo)
779   */
780  default boolean isNext(RegionInfo after) {
781    return getTable().equals(after.getTable()) && Bytes.equals(getEndKey(), after.getStartKey());
782  }
783
784  /**
785   * @return True if region is adjacent, either just before or just after this one.
786   * @see #isNext(RegionInfo)
787   */
788  default boolean isAdjacent(RegionInfo other) {
789    return getTable().equals(other.getTable()) && areAdjacent(this, other);
790  }
791
792  /**
793   * @return True if RegionInfo is degenerate... if startKey > endKey.
794   */
795  default boolean isDegenerate() {
796    return !isLast() && Bytes.compareTo(getStartKey(), getEndKey()) > 0;
797  }
798
799  /**
800   * @return True if an overlap in region range.
801   * @see #isDegenerate()
802   */
803  default boolean isOverlap(RegionInfo other) {
804    if (other == null) {
805      return false;
806    }
807    if (!getTable().equals(other.getTable())) {
808      return false;
809    }
810    int startKeyCompare = Bytes.compareTo(getStartKey(), other.getStartKey());
811    if (startKeyCompare == 0) {
812      return true;
813    }
814    if (startKeyCompare < 0) {
815      if (isLast()) {
816        return true;
817      }
818      return Bytes.compareTo(getEndKey(), other.getStartKey()) > 0;
819    }
820    if (other.isLast()) {
821      return true;
822    }
823    return Bytes.compareTo(getStartKey(), other.getEndKey()) < 0;
824  }
825}