View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase;
19  
20  import com.google.common.annotations.VisibleForTesting;
21  import com.google.protobuf.ServiceException;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.client.ClusterConnection;
28  import org.apache.hadoop.hbase.client.Connection;
29  import org.apache.hadoop.hbase.client.ConnectionFactory;
30  import org.apache.hadoop.hbase.client.Delete;
31  import org.apache.hadoop.hbase.client.Get;
32  import org.apache.hadoop.hbase.client.HTable;
33  import org.apache.hadoop.hbase.client.Mutation;
34  import org.apache.hadoop.hbase.client.Put;
35  import org.apache.hadoop.hbase.client.RegionLocator;
36  import org.apache.hadoop.hbase.client.RegionReplicaUtil;
37  import org.apache.hadoop.hbase.client.Result;
38  import org.apache.hadoop.hbase.client.ResultScanner;
39  import org.apache.hadoop.hbase.client.Scan;
40  import org.apache.hadoop.hbase.client.Table;
41  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
42  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
43  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
44  import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
47  import org.apache.hadoop.hbase.util.Pair;
48  import org.apache.hadoop.hbase.util.PairOfSameType;
49  import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
50  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
51  
52  import java.io.IOException;
53  import java.io.InterruptedIOException;
54  import java.util.ArrayList;
55  import java.util.List;
56  import java.util.Map;
57  import java.util.NavigableMap;
58  import java.util.Set;
59  import java.util.SortedMap;
60  import java.util.TreeMap;
61  import java.util.regex.Matcher;
62  import java.util.regex.Pattern;
63  
64  /**
65   * Read/write operations on region and assignment information store in
66   * <code>hbase:meta</code>.
67   *
68   * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason
69   * for this is because when used on client-side (like from HBaseAdmin), we want to use
70   * short-living connection (opened before each operation, closed right after), while
71   * when used on HM or HRS (like in AssignmentManager) we want permanent connection.
72   */
73  @InterfaceAudience.Private
74  public class MetaTableAccessor {
75  
76    /*
77     * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the
78     * same table range (table, startKey, endKey). For every range, there will be at least one
79     * HRI defined which is called default replica.
80     *
81     * Meta layout (as of 0.98 + HBASE-10070) is like:
82     * For each table range, there is a single row, formatted like:
83     * <tableName>,<startKey>,<regionId>,<encodedRegionName>. This row corresponds to the regionName
84     * of the default region replica.
85     * Columns are:
86     * info:regioninfo         => contains serialized HRI for the default region replica
87     * info:server             => contains hostname:port (in string form) for the server hosting
88     *                            the default regionInfo replica
89     * info:server_<replicaId> => contains hostname:port (in string form) for the server hosting the
90     *                            regionInfo replica with replicaId
91     * info:serverstartcode    => contains server start code (in binary long form) for the server
92     *                            hosting the default regionInfo replica
93     * info:serverstartcode_<replicaId> => contains server start code (in binary long form) for the
94     *                                     server hosting the regionInfo replica with replicaId
95     * info:seqnumDuringOpen    => contains seqNum (in binary long form) for the region at the time
96     *                             the server opened the region with default replicaId
97     * info:seqnumDuringOpen_<replicaId> => contains seqNum (in binary long form) for the region at
98     *                             the time the server opened the region with replicaId
99     * info:splitA              => contains a serialized HRI for the first daughter region if the
100    *                             region is split
101    * info:splitB              => contains a serialized HRI for the second daughter region if the
102    *                             region is split
103    * info:mergeA              => contains a serialized HRI for the first parent region if the
104    *                             region is the result of a merge
105    * info:mergeB              => contains a serialized HRI for the second parent region if the
106    *                             region is the result of a merge
107    *
108    * The actual layout of meta should be encapsulated inside MetaTableAccessor methods,
109    * and should not leak out of it (through Result objects, etc)
110    */
111 
112   private static final Log LOG = LogFactory.getLog(MetaTableAccessor.class);
113 
114   static final byte [] META_REGION_PREFIX;
115   static {
116     // Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
117     // FIRST_META_REGIONINFO == 'hbase:meta,,1'.  META_REGION_PREFIX == 'hbase:meta,'
118     int len = HRegionInfo.FIRST_META_REGIONINFO.getRegionName().length - 2;
119     META_REGION_PREFIX = new byte [len];
120     System.arraycopy(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), 0,
121       META_REGION_PREFIX, 0, len);
122   }
123 
124   /** The delimiter for meta columns for replicaIds &gt; 0 */
125   protected static final char META_REPLICA_ID_DELIMITER = '_';
126 
127   /** A regex for parsing server columns from meta. See above javadoc for meta layout */
128   private static final Pattern SERVER_COLUMN_PATTERN
129     = Pattern.compile("^server(_[0-9a-fA-F]{4})?$");
130 
131   ////////////////////////
132   // Reading operations //
133   ////////////////////////
134 
135  /**
136    * Performs a full scan of a <code>hbase:meta</code> table.
137    * @return List of {@link org.apache.hadoop.hbase.client.Result}
138    * @throws IOException
139    */
140   public static List<Result> fullScanOfMeta(Connection connection)
141   throws IOException {
142     CollectAllVisitor v = new CollectAllVisitor();
143     fullScan(connection, v, null);
144     return v.getResults();
145   }
146 
147   /**
148    * Performs a full scan of <code>hbase:meta</code>.
149    * @param connection connection we're using
150    * @param visitor Visitor invoked against each row.
151    * @throws IOException
152    */
153   public static void fullScan(Connection connection,
154       final Visitor visitor)
155   throws IOException {
156     fullScan(connection, visitor, null);
157   }
158 
159   /**
160    * Performs a full scan of <code>hbase:meta</code>.
161    * @param connection connection we're using
162    * @return List of {@link Result}
163    * @throws IOException
164    */
165   public static List<Result> fullScan(Connection connection)
166     throws IOException {
167     CollectAllVisitor v = new CollectAllVisitor();
168     fullScan(connection, v, null);
169     return v.getResults();
170   }
171 
172   /**
173    * Callers should call close on the returned {@link Table} instance.
174    * @param connection connection we're using to access Meta
175    * @return An {@link Table} for <code>hbase:meta</code>
176    * @throws IOException
177    */
178   static Table getMetaHTable(final Connection connection)
179   throws IOException {
180     // We used to pass whole CatalogTracker in here, now we just pass in Connection
181     if (connection == null) {
182       throw new NullPointerException("No connection");
183     } else if (connection.isClosed()) {
184       throw new IOException("connection is closed");
185     }
186     // If the passed in 'connection' is 'managed' -- i.e. every second test uses
187     // a Table or an HBaseAdmin with managed connections -- then doing
188     // connection.getTable will throw an exception saying you are NOT to use
189     // managed connections getting tables.  Leaving this as it is for now. Will
190     // revisit when inclined to change all tests.  User code probaby makes use of
191     // managed connections too so don't change it till post hbase 1.0.
192     //
193     // There should still be a way to use this method with an unmanaged connection.
194     if (connection instanceof ClusterConnection) {
195       if (((ClusterConnection) connection).isManaged()) {
196         return new HTable(TableName.META_TABLE_NAME, (ClusterConnection) connection);
197       }
198     }
199     return connection.getTable(TableName.META_TABLE_NAME);
200   }
201 
202   /**
203    * @param t Table to use (will be closed when done).
204    * @param g Get to run
205    * @throws IOException
206    */
207   private static Result get(final Table t, final Get g) throws IOException {
208     try {
209       return t.get(g);
210     } finally {
211       t.close();
212     }
213   }
214 
215   /**
216    * Gets the region info and assignment for the specified region.
217    * @param connection connection we're using
218    * @param regionName Region to lookup.
219    * @return Location and HRegionInfo for <code>regionName</code>
220    * @throws IOException
221    * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
222    */
223   @Deprecated
224   public static Pair<HRegionInfo, ServerName> getRegion(Connection connection, byte [] regionName)
225     throws IOException {
226     HRegionLocation location = getRegionLocation(connection, regionName);
227     return location == null
228       ? null
229       : new Pair<HRegionInfo, ServerName>(location.getRegionInfo(), location.getServerName());
230   }
231 
232   /**
233    * Returns the HRegionLocation from meta for the given region
234    * @param connection connection we're using
235    * @param regionName region we're looking for
236    * @return HRegionLocation for the given region
237    * @throws IOException
238    */
239   public static HRegionLocation getRegionLocation(Connection connection,
240                                                   byte[] regionName) throws IOException {
241     byte[] row = regionName;
242     HRegionInfo parsedInfo = null;
243     try {
244       parsedInfo = parseRegionInfoFromRegionName(regionName);
245       row = getMetaKeyForRegion(parsedInfo);
246     } catch (Exception parseEx) {
247       // Ignore. This is used with tableName passed as regionName.
248     }
249     Get get = new Get(row);
250     get.addFamily(HConstants.CATALOG_FAMILY);
251     Result r = get(getMetaHTable(connection), get);
252     RegionLocations locations = getRegionLocations(r);
253     return locations == null
254       ? null
255       : locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
256   }
257 
258   /**
259    * Returns the HRegionLocation from meta for the given region
260    * @param connection connection we're using
261    * @param regionInfo region information
262    * @return HRegionLocation for the given region
263    * @throws IOException
264    */
265   public static HRegionLocation getRegionLocation(Connection connection,
266                                                   HRegionInfo regionInfo) throws IOException {
267     byte[] row = getMetaKeyForRegion(regionInfo);
268     Get get = new Get(row);
269     get.addFamily(HConstants.CATALOG_FAMILY);
270     Result r = get(getMetaHTable(connection), get);
271     return getRegionLocation(r, regionInfo, regionInfo.getReplicaId());
272   }
273 
274   /** Returns the row key to use for this regionInfo */
275   public static byte[] getMetaKeyForRegion(HRegionInfo regionInfo) {
276     return RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo).getRegionName();
277   }
278 
279   /** Returns an HRI parsed from this regionName. Not all the fields of the HRI
280    * is stored in the name, so the returned object should only be used for the fields
281    * in the regionName.
282    */
283   protected static HRegionInfo parseRegionInfoFromRegionName(byte[] regionName)
284     throws IOException {
285     byte[][] fields = HRegionInfo.parseRegionName(regionName);
286     long regionId =  Long.parseLong(Bytes.toString(fields[2]));
287     int replicaId = fields.length > 3 ? Integer.parseInt(Bytes.toString(fields[3]), 16) : 0;
288     return new HRegionInfo(
289       TableName.valueOf(fields[0]), fields[1], fields[1], false, regionId, replicaId);
290   }
291 
292   /**
293    * Gets the result in hbase:meta for the specified region.
294    * @param connection connection we're using
295    * @param regionName region we're looking for
296    * @return result of the specified region
297    * @throws IOException
298    */
299   public static Result getRegionResult(Connection connection,
300       byte[] regionName) throws IOException {
301     Get get = new Get(regionName);
302     get.addFamily(HConstants.CATALOG_FAMILY);
303     return get(getMetaHTable(connection), get);
304   }
305 
306   /**
307    * Get regions from the merge qualifier of the specified merged region
308    * @return null if it doesn't contain merge qualifier, else two merge regions
309    * @throws IOException
310    */
311   public static Pair<HRegionInfo, HRegionInfo> getRegionsFromMergeQualifier(
312       Connection connection, byte[] regionName) throws IOException {
313     Result result = getRegionResult(connection, regionName);
314     HRegionInfo mergeA = getHRegionInfo(result, HConstants.MERGEA_QUALIFIER);
315     HRegionInfo mergeB = getHRegionInfo(result, HConstants.MERGEB_QUALIFIER);
316     if (mergeA == null && mergeB == null) {
317       return null;
318     }
319     return new Pair<HRegionInfo, HRegionInfo>(mergeA, mergeB);
320  }
321 
322   /**
323    * Checks if the specified table exists.  Looks at the hbase:meta table hosted on
324    * the specified server.
325    * @param connection connection we're using
326    * @param tableName table to check
327    * @return true if the table exists in meta, false if not
328    * @throws IOException
329    */
330   public static boolean tableExists(Connection connection,
331       final TableName tableName)
332   throws IOException {
333     if (tableName.equals(TableName.META_TABLE_NAME)) {
334       // Catalog tables always exist.
335       return true;
336     }
337     // Make a version of ResultCollectingVisitor that only collects the first
338     CollectingVisitor<HRegionInfo> visitor = new CollectingVisitor<HRegionInfo>() {
339       private HRegionInfo current = null;
340 
341       @Override
342       public boolean visit(Result r) throws IOException {
343         RegionLocations locations = getRegionLocations(r);
344         if (locations == null || locations.getRegionLocation().getRegionInfo() == null) {
345           LOG.warn("No serialized HRegionInfo in " + r);
346           return true;
347         }
348         this.current = locations.getRegionLocation().getRegionInfo();
349         if (this.current == null) {
350           LOG.warn("No serialized HRegionInfo in " + r);
351           return true;
352         }
353         if (!isInsideTable(this.current, tableName)) return false;
354         // Else call super and add this Result to the collection.
355         super.visit(r);
356         // Stop collecting regions from table after we get one.
357         return false;
358       }
359 
360       @Override
361       void add(Result r) {
362         // Add the current HRI.
363         this.results.add(this.current);
364       }
365     };
366     fullScan(connection, visitor, getTableStartRowForMeta(tableName));
367     // If visitor has results >= 1 then table exists.
368     return visitor.getResults().size() >= 1;
369   }
370 
371   /**
372    * Gets all of the regions of the specified table.
373    * @param zkw zookeeper connection to access meta table
374    * @param connection connection we're using
375    * @param tableName table we're looking for
376    * @return Ordered list of {@link HRegionInfo}.
377    * @throws IOException
378    */
379   public static List<HRegionInfo> getTableRegions(ZooKeeperWatcher zkw,
380       Connection connection, TableName tableName)
381   throws IOException {
382     return getTableRegions(zkw, connection, tableName, false);
383   }
384 
385   /**
386    * Gets all of the regions of the specified table.
387    * @param zkw zookeeper connection to access meta table
388    * @param connection connection we're using
389    * @param tableName table we're looking for
390    * @param excludeOfflinedSplitParents If true, do not include offlined split
391    * parents in the return.
392    * @return Ordered list of {@link HRegionInfo}.
393    * @throws IOException
394    */
395   public static List<HRegionInfo> getTableRegions(ZooKeeperWatcher zkw,
396       Connection connection, TableName tableName, final boolean excludeOfflinedSplitParents)
397         throws IOException {
398     List<Pair<HRegionInfo, ServerName>> result = null;
399       result = getTableRegionsAndLocations(zkw, connection, tableName,
400         excludeOfflinedSplitParents);
401     return getListOfHRegionInfos(result);
402   }
403 
404   static List<HRegionInfo> getListOfHRegionInfos(final List<Pair<HRegionInfo, ServerName>> pairs) {
405     if (pairs == null || pairs.isEmpty()) return null;
406     List<HRegionInfo> result = new ArrayList<HRegionInfo>(pairs.size());
407     for (Pair<HRegionInfo, ServerName> pair: pairs) {
408       result.add(pair.getFirst());
409     }
410     return result;
411   }
412 
413   /**
414    * @param current region of current table we're working with
415    * @param tableName table we're checking against
416    * @return True if <code>current</code> tablename is equal to
417    * <code>tableName</code>
418    */
419   static boolean isInsideTable(final HRegionInfo current, final TableName tableName) {
420     return tableName.equals(current.getTable());
421   }
422 
423   /**
424    * @param tableName table we're working with
425    * @return Place to start Scan in <code>hbase:meta</code> when passed a
426    * <code>tableName</code>; returns &lt;tableName&rt; &lt;,&rt; &lt;,&rt;
427    */
428   static byte [] getTableStartRowForMeta(TableName tableName) {
429     byte [] startRow = new byte[tableName.getName().length + 2];
430     System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length);
431     startRow[startRow.length - 2] = HConstants.DELIMITER;
432     startRow[startRow.length - 1] = HConstants.DELIMITER;
433     return startRow;
434   }
435 
436   /**
437    * This method creates a Scan object that will only scan catalog rows that
438    * belong to the specified table. It doesn't specify any columns.
439    * This is a better alternative to just using a start row and scan until
440    * it hits a new table since that requires parsing the HRI to get the table
441    * name.
442    * @param tableName bytes of table's name
443    * @return configured Scan object
444    */
445   public static Scan getScanForTableName(TableName tableName) {
446     String strName = tableName.getNameAsString();
447     // Start key is just the table name with delimiters
448     byte[] startKey = Bytes.toBytes(strName + ",,");
449     // Stop key appends the smallest possible char to the table name
450     byte[] stopKey = Bytes.toBytes(strName + " ,,");
451 
452     Scan scan = new Scan(startKey);
453     scan.setStopRow(stopKey);
454     return scan;
455   }
456 
457   /**
458    * @param zkw zookeeper connection to access meta table
459    * @param connection connection we're using
460    * @param tableName table we're looking for
461    * @return Return list of regioninfos and server.
462    * @throws IOException
463    */
464   public static List<Pair<HRegionInfo, ServerName>>
465   getTableRegionsAndLocations(ZooKeeperWatcher zkw,
466                               Connection connection, TableName tableName)
467   throws IOException {
468     return getTableRegionsAndLocations(zkw, connection, tableName, true);
469   }
470 
471   /**
472    * @param zkw ZooKeeperWatcher instance we're using to get hbase:meta location
473    * @param connection connection we're using
474    * @param tableName table to work with
475    * @return Return list of regioninfos and server addresses.
476    * @throws IOException
477    */
478   public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
479       ZooKeeperWatcher zkw, Connection connection, final TableName tableName,
480       final boolean excludeOfflinedSplitParents) throws IOException {
481 
482     if (tableName.equals(TableName.META_TABLE_NAME)) {
483       // If meta, do a bit of special handling.
484       ServerName serverName = new MetaTableLocator().getMetaRegionLocation(zkw);
485       List<Pair<HRegionInfo, ServerName>> list =
486         new ArrayList<Pair<HRegionInfo, ServerName>>();
487       list.add(new Pair<HRegionInfo, ServerName>(HRegionInfo.FIRST_META_REGIONINFO,
488         serverName));
489       return list;
490     }
491     // Make a version of CollectingVisitor that collects HRegionInfo and ServerAddress
492     CollectingVisitor<Pair<HRegionInfo, ServerName>> visitor =
493       new CollectingVisitor<Pair<HRegionInfo, ServerName>>() {
494         private RegionLocations current = null;
495 
496         @Override
497         public boolean visit(Result r) throws IOException {
498           current = getRegionLocations(r);
499           if (current == null || current.getRegionLocation().getRegionInfo() == null) {
500             LOG.warn("No serialized HRegionInfo in " + r);
501             return true;
502           }
503           HRegionInfo hri = current.getRegionLocation().getRegionInfo();
504           if (!isInsideTable(hri, tableName)) return false;
505           if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
506           // Else call super and add this Result to the collection.
507           return super.visit(r);
508         }
509 
510         @Override
511         void add(Result r) {
512           if (current == null) {
513             return;
514           }
515           for (HRegionLocation loc : current.getRegionLocations()) {
516             if (loc != null) {
517               this.results.add(new Pair<HRegionInfo, ServerName>(
518                 loc.getRegionInfo(), loc.getServerName()));
519             }
520           }
521         }
522       };
523     fullScan(connection, visitor, getTableStartRowForMeta(tableName));
524     return visitor.getResults();
525   }
526 
527   /**
528    * @param connection connection we're using
529    * @param serverName server whose regions we're interested in
530    * @return List of user regions installed on this server (does not include
531    * catalog regions).
532    * @throws IOException
533    */
534   public static NavigableMap<HRegionInfo, Result>
535   getServerUserRegions(Connection connection, final ServerName serverName)
536     throws IOException {
537     final NavigableMap<HRegionInfo, Result> hris = new TreeMap<HRegionInfo, Result>();
538     // Fill the above hris map with entries from hbase:meta that have the passed
539     // servername.
540     CollectingVisitor<Result> v = new CollectingVisitor<Result>() {
541       @Override
542       void add(Result r) {
543         if (r == null || r.isEmpty()) return;
544         RegionLocations locations = getRegionLocations(r);
545         if (locations == null) return;
546         for (HRegionLocation loc : locations.getRegionLocations()) {
547           if (loc != null) {
548             if (loc.getServerName() != null && loc.getServerName().equals(serverName)) {
549               hris.put(loc.getRegionInfo(), r);
550             }
551           }
552         }
553       }
554     };
555     fullScan(connection, v);
556     return hris;
557   }
558 
559   public static void fullScanMetaAndPrint(Connection connection)
560     throws IOException {
561     Visitor v = new Visitor() {
562       @Override
563       public boolean visit(Result r) throws IOException {
564         if (r ==  null || r.isEmpty()) return true;
565         LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
566         RegionLocations locations = getRegionLocations(r);
567         if (locations == null) return true;
568         for (HRegionLocation loc : locations.getRegionLocations()) {
569           if (loc != null) {
570             LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegionInfo());
571           }
572         }
573         return true;
574       }
575     };
576     fullScan(connection, v);
577   }
578 
579   /**
580    * Performs a full scan of a catalog table.
581    * @param connection connection we're using
582    * @param visitor Visitor invoked against each row.
583    * @param startrow Where to start the scan. Pass null if want to begin scan
584    * at first row.
585    * <code>hbase:meta</code>, the default (pass false to scan hbase:meta)
586    * @throws IOException
587    */
588   public static void fullScan(Connection connection,
589     final Visitor visitor, final byte [] startrow)
590   throws IOException {
591     Scan scan = new Scan();
592     if (startrow != null) scan.setStartRow(startrow);
593     if (startrow == null) {
594       int caching = connection.getConfiguration()
595           .getInt(HConstants.HBASE_META_SCANNER_CACHING, 100);
596       scan.setCaching(caching);
597     }
598     scan.addFamily(HConstants.CATALOG_FAMILY);
599     Table metaTable = getMetaHTable(connection);
600     ResultScanner scanner = null;
601     try {
602       scanner = metaTable.getScanner(scan);
603       Result data;
604       while((data = scanner.next()) != null) {
605         if (data.isEmpty()) continue;
606         // Break if visit returns false.
607         if (!visitor.visit(data)) break;
608       }
609     } finally {
610       if (scanner != null) scanner.close();
611       metaTable.close();
612     }
613   }
614 
615   /**
616    * Returns the column family used for meta columns.
617    * @return HConstants.CATALOG_FAMILY.
618    */
619   protected static byte[] getFamily() {
620     return HConstants.CATALOG_FAMILY;
621   }
622 
623   /**
624    * Returns the column qualifier for serialized region info
625    * @return HConstants.REGIONINFO_QUALIFIER
626    */
627   protected static byte[] getRegionInfoColumn() {
628     return HConstants.REGIONINFO_QUALIFIER;
629   }
630 
631   /**
632    * Returns the column qualifier for server column for replicaId
633    * @param replicaId the replicaId of the region
634    * @return a byte[] for server column qualifier
635    */
636   @VisibleForTesting
637   public static byte[] getServerColumn(int replicaId) {
638     return replicaId == 0
639       ? HConstants.SERVER_QUALIFIER
640       : Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
641       + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
642   }
643 
644   /**
645    * Returns the column qualifier for server start code column for replicaId
646    * @param replicaId the replicaId of the region
647    * @return a byte[] for server start code column qualifier
648    */
649   @VisibleForTesting
650   public static byte[] getStartCodeColumn(int replicaId) {
651     return replicaId == 0
652       ? HConstants.STARTCODE_QUALIFIER
653       : Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
654       + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
655   }
656 
657   /**
658    * Returns the column qualifier for seqNum column for replicaId
659    * @param replicaId the replicaId of the region
660    * @return a byte[] for seqNum column qualifier
661    */
662   @VisibleForTesting
663   public static byte[] getSeqNumColumn(int replicaId) {
664     return replicaId == 0
665       ? HConstants.SEQNUM_QUALIFIER
666       : Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
667       + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
668   }
669 
670   /**
671    * Parses the replicaId from the server column qualifier. See top of the class javadoc
672    * for the actual meta layout
673    * @param serverColumn the column qualifier
674    * @return an int for the replicaId
675    */
676   @VisibleForTesting
677   static int parseReplicaIdFromServerColumn(byte[] serverColumn) {
678     String serverStr = Bytes.toString(serverColumn);
679 
680     Matcher matcher = SERVER_COLUMN_PATTERN.matcher(serverStr);
681     if (matcher.matches() && matcher.groupCount() > 0) {
682       String group = matcher.group(1);
683       if (group != null && group.length() > 0) {
684         return Integer.parseInt(group.substring(1), 16);
685       } else {
686         return 0;
687       }
688     }
689     return -1;
690   }
691 
692   /**
693    * Returns a {@link ServerName} from catalog table {@link Result}.
694    * @param r Result to pull from
695    * @return A ServerName instance or null if necessary fields not found or empty.
696    */
697   private static ServerName getServerName(final Result r, final int replicaId) {
698     byte[] serverColumn = getServerColumn(replicaId);
699     Cell cell = r.getColumnLatestCell(getFamily(), serverColumn);
700     if (cell == null || cell.getValueLength() == 0) return null;
701     String hostAndPort = Bytes.toString(
702       cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
703     byte[] startcodeColumn = getStartCodeColumn(replicaId);
704     cell = r.getColumnLatestCell(getFamily(), startcodeColumn);
705     if (cell == null || cell.getValueLength() == 0) return null;
706     return ServerName.valueOf(hostAndPort,
707       Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
708   }
709 
710   /**
711    * The latest seqnum that the server writing to meta observed when opening the region.
712    * E.g. the seqNum when the result of {@link #getServerName(Result, int)} was written.
713    * @param r Result to pull the seqNum from
714    * @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
715    */
716   private static long getSeqNumDuringOpen(final Result r, final int replicaId) {
717     Cell cell = r.getColumnLatestCell(getFamily(), getSeqNumColumn(replicaId));
718     if (cell == null || cell.getValueLength() == 0) return HConstants.NO_SEQNUM;
719     return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
720   }
721 
722   /**
723    * Returns an HRegionLocationList extracted from the result.
724    * @return an HRegionLocationList containing all locations for the region range or null if
725    *  we can't deserialize the result.
726    */
727   public static RegionLocations getRegionLocations(final Result r) {
728     if (r == null) return null;
729     HRegionInfo regionInfo = getHRegionInfo(r, getRegionInfoColumn());
730     if (regionInfo == null) return null;
731 
732     List<HRegionLocation> locations = new ArrayList<HRegionLocation>(1);
733     NavigableMap<byte[],NavigableMap<byte[],byte[]>> familyMap = r.getNoVersionMap();
734 
735     locations.add(getRegionLocation(r, regionInfo, regionInfo.getReplicaId()));
736 
737     NavigableMap<byte[], byte[]> infoMap = familyMap.get(getFamily());
738     if (infoMap == null) return new RegionLocations(locations);
739 
740     // iterate until all serverName columns are seen
741     int replicaId = 0;
742     byte[] serverColumn = getServerColumn(replicaId);
743     SortedMap<byte[], byte[]> serverMap = null;
744     serverMap = infoMap.tailMap(serverColumn, false);
745 
746     if (serverMap.isEmpty()) return new RegionLocations(locations);
747 
748     for (Map.Entry<byte[], byte[]> entry : serverMap.entrySet()) {
749       replicaId = parseReplicaIdFromServerColumn(entry.getKey());
750       if (replicaId < 0) {
751         break;
752       }
753       HRegionLocation location = getRegionLocation(r, regionInfo, replicaId);
754       // In case the region replica is newly created, it's location might be null. We usually do not
755       // have HRL's in RegionLocations object with null ServerName. They are handled as null HRLs.
756       if (location == null || location.getServerName() == null) {
757         locations.add(null);
758       } else {
759         locations.add(location);
760       }
761     }
762 
763     return new RegionLocations(locations);
764   }
765 
766   /**
767    * Returns the HRegionLocation parsed from the given meta row Result
768    * for the given regionInfo and replicaId. The regionInfo can be the default region info
769    * for the replica.
770    * @param r the meta row result
771    * @param regionInfo RegionInfo for default replica
772    * @param replicaId the replicaId for the HRegionLocation
773    * @return HRegionLocation parsed from the given meta row Result for the given replicaId
774    */
775   private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
776                                                    final int replicaId) {
777     ServerName serverName = getServerName(r, replicaId);
778     long seqNum = getSeqNumDuringOpen(r, replicaId);
779     HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
780     return new HRegionLocation(replicaInfo, serverName, seqNum);
781   }
782 
783   /**
784    * Returns HRegionInfo object from the column
785    * HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
786    * table Result.
787    * @param data a Result object from the catalog table scan
788    * @return HRegionInfo or null
789    */
790   public static HRegionInfo getHRegionInfo(Result data) {
791     return getHRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
792   }
793 
794   /**
795    * Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
796    * <code>qualifier</code> of the catalog table result.
797    * @param r a Result object from the catalog table scan
798    * @param qualifier Column family qualifier
799    * @return An HRegionInfo instance or null.
800    */
801   private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
802     Cell cell = r.getColumnLatestCell(getFamily(), qualifier);
803     if (cell == null) return null;
804     return HRegionInfo.parseFromOrNull(cell.getValueArray(),
805       cell.getValueOffset(), cell.getValueLength());
806   }
807 
808   /**
809    * Returns the daughter regions by reading the corresponding columns of the catalog table
810    * Result.
811    * @param data a Result object from the catalog table scan
812    * @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
813    * parent
814    */
815   public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) {
816     HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
817     HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
818 
819     return new PairOfSameType<HRegionInfo>(splitA, splitB);
820   }
821 
822   /**
823    * Returns the merge regions by reading the corresponding columns of the catalog table
824    * Result.
825    * @param data a Result object from the catalog table scan
826    * @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
827    * parent
828    */
829   public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) {
830     HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
831     HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
832 
833     return new PairOfSameType<HRegionInfo>(mergeA, mergeB);
834   }
835 
836   /**
837    * Implementations 'visit' a catalog table row.
838    */
839   public interface Visitor {
840     /**
841      * Visit the catalog table row.
842      * @param r A row from catalog table
843      * @return True if we are to proceed scanning the table, else false if
844      * we are to stop now.
845      */
846     boolean visit(final Result r) throws IOException;
847   }
848 
849   /**
850    * A {@link Visitor} that collects content out of passed {@link Result}.
851    */
852   static abstract class CollectingVisitor<T> implements Visitor {
853     final List<T> results = new ArrayList<T>();
854     @Override
855     public boolean visit(Result r) throws IOException {
856       if (r ==  null || r.isEmpty()) return true;
857       add(r);
858       return true;
859     }
860 
861     abstract void add(Result r);
862 
863     /**
864      * @return Collected results; wait till visits complete to collect all
865      * possible results
866      */
867     List<T> getResults() {
868       return this.results;
869     }
870   }
871 
872   /**
873    * Collects all returned.
874    */
875   static class CollectAllVisitor extends CollectingVisitor<Result> {
876     @Override
877     void add(Result r) {
878       this.results.add(r);
879     }
880   }
881 
882   /**
883    * Count regions in <code>hbase:meta</code> for passed table.
884    * @param c Configuration object
885    * @param tableName table name to count regions for
886    * @return Count or regions in table <code>tableName</code>
887    * @throws IOException
888    */
889   @Deprecated
890   public static int getRegionCount(final Configuration c, final String tableName)
891       throws IOException {
892     return getRegionCount(c, TableName.valueOf(tableName));
893   }
894 
895   /**
896    * Count regions in <code>hbase:meta</code> for passed table.
897    * @param c Configuration object
898    * @param tableName table name to count regions for
899    * @return Count or regions in table <code>tableName</code>
900    * @throws IOException
901    */
902   public static int getRegionCount(final Configuration c, final TableName tableName)
903   throws IOException {
904     try (Connection connection = ConnectionFactory.createConnection(c)) {
905       return getRegionCount(connection, tableName);
906     }
907   }
908 
909   /**
910    * Count regions in <code>hbase:meta</code> for passed table.
911    * @param connection Connection object
912    * @param tableName table name to count regions for
913    * @return Count or regions in table <code>tableName</code>
914    * @throws IOException
915    */
916   public static int getRegionCount(final Connection connection, final TableName tableName)
917   throws IOException {
918     try (RegionLocator locator = connection.getRegionLocator(tableName)) {
919       List<HRegionLocation> locations = locator.getAllRegionLocations();
920       return locations == null? 0: locations.size();
921     }
922   }
923 
924   ////////////////////////
925   // Editing operations //
926   ////////////////////////
927 
928   /**
929    * Generates and returns a Put containing the region into for the catalog table
930    */
931   public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
932     throws IOException {
933     return makePutFromRegionInfo(regionInfo, HConstants.LATEST_TIMESTAMP);
934   }
935   /**
936    * Generates and returns a Put containing the region into for the catalog table
937    */
938   public static Put makePutFromRegionInfo(HRegionInfo regionInfo, long ts)
939     throws IOException {
940     Put put = new Put(regionInfo.getRegionName(), ts);
941     addRegionInfo(put, regionInfo);
942     return put;
943   }
944 
945   /**
946    * Generates and returns a Delete containing the region info for the catalog
947    * table
948    */
949   public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo) {
950     return makeDeleteFromRegionInfo(regionInfo, HConstants.LATEST_TIMESTAMP);
951   }
952 
953   /**
954    * Generates and returns a Delete containing the region info for the catalog
955    * table
956    */
957   public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo, long ts) {
958     if (regionInfo == null) {
959       throw new IllegalArgumentException("Can't make a delete for null region");
960     }
961     Delete delete = new Delete(regionInfo.getRegionName(), ts);
962     return delete;
963   }
964 
965   /**
966    * Adds split daughters to the Put
967    */
968   public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB) {
969     if (splitA != null) {
970       put.addImmutable(
971         HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray());
972     }
973     if (splitB != null) {
974       put.addImmutable(
975         HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray());
976     }
977     return put;
978   }
979 
980   /**
981    * Put the passed <code>p</code> to the <code>hbase:meta</code> table.
982    * @param connection connection we're using
983    * @param p Put to add to hbase:meta
984    * @throws IOException
985    */
986   static void putToMetaTable(final Connection connection, final Put p)
987     throws IOException {
988     put(getMetaHTable(connection), p);
989   }
990 
991   /**
992    * @param t Table to use (will be closed when done).
993    * @param p put to make
994    * @throws IOException
995    */
996   private static void put(final Table t, final Put p) throws IOException {
997     try {
998       t.put(p);
999     } finally {
1000       t.close();
1001     }
1002   }
1003 
1004   /**
1005    * Put the passed <code>ps</code> to the <code>hbase:meta</code> table.
1006    * @param connection connection we're using
1007    * @param ps Put to add to hbase:meta
1008    * @throws IOException
1009    */
1010   public static void putsToMetaTable(final Connection connection, final List<Put> ps)
1011     throws IOException {
1012     Table t = getMetaHTable(connection);
1013     try {
1014       t.put(ps);
1015     } finally {
1016       t.close();
1017     }
1018   }
1019 
1020   /**
1021    * Delete the passed <code>d</code> from the <code>hbase:meta</code> table.
1022    * @param connection connection we're using
1023    * @param d Delete to add to hbase:meta
1024    * @throws IOException
1025    */
1026   static void deleteFromMetaTable(final Connection connection, final Delete d)
1027     throws IOException {
1028     List<Delete> dels = new ArrayList<Delete>(1);
1029     dels.add(d);
1030     deleteFromMetaTable(connection, dels);
1031   }
1032 
1033   /**
1034    * Delete the passed <code>deletes</code> from the <code>hbase:meta</code> table.
1035    * @param connection connection we're using
1036    * @param deletes Deletes to add to hbase:meta  This list should support #remove.
1037    * @throws IOException
1038    */
1039   public static void deleteFromMetaTable(final Connection connection, final List<Delete> deletes)
1040     throws IOException {
1041     Table t = getMetaHTable(connection);
1042     try {
1043       t.delete(deletes);
1044     } finally {
1045       t.close();
1046     }
1047   }
1048 
1049   /**
1050    * Deletes some replica columns corresponding to replicas for the passed rows
1051    * @param metaRows rows in hbase:meta
1052    * @param replicaIndexToDeleteFrom the replica ID we would start deleting from
1053    * @param numReplicasToRemove how many replicas to remove
1054    * @param connection connection we're using to access meta table
1055    * @throws IOException
1056    */
1057   public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows,
1058     int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection)
1059       throws IOException {
1060     int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
1061     for (byte[] row : metaRows) {
1062       Delete deleteReplicaLocations = new Delete(row);
1063       for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
1064         deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
1065           getServerColumn(i));
1066         deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
1067           getSeqNumColumn(i));
1068         deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
1069           getStartCodeColumn(i));
1070       }
1071       deleteFromMetaTable(connection, deleteReplicaLocations);
1072     }
1073   }
1074 
1075   /**
1076    * Execute the passed <code>mutations</code> against <code>hbase:meta</code> table.
1077    * @param connection connection we're using
1078    * @param mutations Puts and Deletes to execute on hbase:meta
1079    * @throws IOException
1080    */
1081   public static void mutateMetaTable(final Connection connection,
1082                                      final List<Mutation> mutations)
1083     throws IOException {
1084     Table t = getMetaHTable(connection);
1085     try {
1086       t.batch(mutations);
1087     } catch (InterruptedException e) {
1088       InterruptedIOException ie = new InterruptedIOException(e.getMessage());
1089       ie.initCause(e);
1090       throw ie;
1091     } finally {
1092       t.close();
1093     }
1094   }
1095 
1096   /**
1097    * Adds a hbase:meta row for the specified new region.
1098    * @param connection connection we're using
1099    * @param regionInfo region information
1100    * @throws IOException if problem connecting or updating meta
1101    */
1102   public static void addRegionToMeta(Connection connection,
1103                                      HRegionInfo regionInfo)
1104     throws IOException {
1105     putToMetaTable(connection, makePutFromRegionInfo(regionInfo));
1106     LOG.info("Added " + regionInfo.getRegionNameAsString());
1107   }
1108 
1109   /**
1110    * Adds a hbase:meta row for the specified new region to the given catalog table. The
1111    * Table is not flushed or closed.
1112    * @param meta the Table for META
1113    * @param regionInfo region information
1114    * @throws IOException if problem connecting or updating meta
1115    */
1116   public static void addRegionToMeta(Table meta, HRegionInfo regionInfo) throws IOException {
1117     addRegionToMeta(meta, regionInfo, null, null);
1118   }
1119 
1120   /**
1121    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
1122    * does not add its daughter's as different rows, but adds information about the daughters
1123    * in the same row as the parent. Use
1124    * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)}
1125    * if you want to do that.
1126    * @param meta the Table for META
1127    * @param regionInfo region information
1128    * @param splitA first split daughter of the parent regionInfo
1129    * @param splitB second split daughter of the parent regionInfo
1130    * @throws IOException if problem connecting or updating meta
1131    */
1132   public static void addRegionToMeta(Table meta, HRegionInfo regionInfo,
1133                                      HRegionInfo splitA, HRegionInfo splitB) throws IOException {
1134     Put put = makePutFromRegionInfo(regionInfo);
1135     addDaughtersToPut(put, splitA, splitB);
1136     meta.put(put);
1137     if (LOG.isDebugEnabled()) {
1138       LOG.debug("Added " + regionInfo.getRegionNameAsString());
1139     }
1140   }
1141 
1142   /**
1143    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
1144    * does not add its daughter's as different rows, but adds information about the daughters
1145    * in the same row as the parent. Use
1146    * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)}
1147    * if you want to do that.
1148    * @param connection connection we're using
1149    * @param regionInfo region information
1150    * @param splitA first split daughter of the parent regionInfo
1151    * @param splitB second split daughter of the parent regionInfo
1152    * @throws IOException if problem connecting or updating meta
1153    */
1154   public static void addRegionToMeta(Connection connection, HRegionInfo regionInfo,
1155                                      HRegionInfo splitA, HRegionInfo splitB) throws IOException {
1156     Table meta = getMetaHTable(connection);
1157     try {
1158       addRegionToMeta(meta, regionInfo, splitA, splitB);
1159     } finally {
1160       meta.close();
1161     }
1162   }
1163 
1164   /**
1165    * Adds a hbase:meta row for each of the specified new regions.
1166    * @param connection connection we're using
1167    * @param regionInfos region information list
1168    * @throws IOException if problem connecting or updating meta
1169    */
1170   public static void addRegionsToMeta(Connection connection,
1171                                       List<HRegionInfo> regionInfos, int regionReplication)
1172     throws IOException {
1173     addRegionsToMeta(connection, regionInfos, regionReplication, HConstants.LATEST_TIMESTAMP);
1174   }
1175   /**
1176    * Adds a hbase:meta row for each of the specified new regions.
1177    * @param connection connection we're using
1178    * @param regionInfos region information list
1179    * @param regionReplication
1180    * @param ts desired timestamp
1181    * @throws IOException if problem connecting or updating meta
1182    */
1183   public static void addRegionsToMeta(Connection connection,
1184       List<HRegionInfo> regionInfos, int regionReplication, long ts)
1185           throws IOException {
1186     List<Put> puts = new ArrayList<Put>();
1187     for (HRegionInfo regionInfo : regionInfos) {
1188       if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
1189         Put put = makePutFromRegionInfo(regionInfo, ts);
1190         // Add empty locations for region replicas so that number of replicas can be cached
1191         // whenever the primary region is looked up from meta
1192         for (int i = 1; i < regionReplication; i++) {
1193           addEmptyLocation(put, i);
1194         }
1195         puts.add(put);
1196       }
1197     }
1198     putsToMetaTable(connection, puts);
1199     LOG.info("Added " + puts.size());
1200   }
1201 
1202   /**
1203    * Adds a daughter region entry to meta.
1204    * @param regionInfo the region to put
1205    * @param sn the location of the region
1206    * @param openSeqNum the latest sequence number obtained when the region was open
1207    */
1208   public static void addDaughter(final Connection connection,
1209       final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
1210       throws NotAllMetaRegionsOnlineException, IOException {
1211     Put put = new Put(regionInfo.getRegionName());
1212     addRegionInfo(put, regionInfo);
1213     if (sn != null) {
1214       addLocation(put, sn, openSeqNum, -1, regionInfo.getReplicaId());
1215     }
1216     putToMetaTable(connection, put);
1217     LOG.info("Added daughter " + regionInfo.getEncodedName() +
1218       (sn == null? ", serverName=null": ", serverName=" + sn.toString()));
1219   }
1220 
1221   /**
1222    * Merge the two regions into one in an atomic operation. Deletes the two
1223    * merging regions in hbase:meta and adds the merged region with the information of
1224    * two merging regions.
1225    * @param connection connection we're using
1226    * @param mergedRegion the merged region
1227    * @param regionA
1228    * @param regionB
1229    * @param sn the location of the region
1230    * @param masterSystemTime
1231    * @throws IOException
1232    */
1233   public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion,
1234       HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication,
1235       long masterSystemTime)
1236           throws IOException {
1237     Table meta = getMetaHTable(connection);
1238     try {
1239       HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
1240 
1241       // use the maximum of what master passed us vs local time.
1242       long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
1243 
1244       // Put for parent
1245       Put putOfMerged = makePutFromRegionInfo(copyOfMerged, time);
1246       putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
1247         regionA.toByteArray());
1248       putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
1249         regionB.toByteArray());
1250 
1251       // Deletes for merging regions
1252       Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
1253       Delete deleteB = makeDeleteFromRegionInfo(regionB, time);
1254 
1255       // The merged is a new region, openSeqNum = 1 is fine.
1256       addLocation(putOfMerged, sn, 1, -1, mergedRegion.getReplicaId());
1257 
1258       // Add empty locations for region replicas of the merged region so that number of replicas can
1259       // be cached whenever the primary region is looked up from meta
1260       for (int i = 1; i < regionReplication; i++) {
1261         addEmptyLocation(putOfMerged, i);
1262       }
1263 
1264       byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString()
1265         + HConstants.DELIMITER);
1266       multiMutate(meta, tableRow, putOfMerged, deleteA, deleteB);
1267     } finally {
1268       meta.close();
1269     }
1270   }
1271 
1272   /**
1273    * Splits the region into two in an atomic operation. Offlines the parent
1274    * region with the information that it is split into two, and also adds
1275    * the daughter regions. Does not add the location information to the daughter
1276    * regions since they are not open yet.
1277    * @param connection connection we're using
1278    * @param parent the parent region which is split
1279    * @param splitA Split daughter region A
1280    * @param splitB Split daughter region A
1281    * @param sn the location of the region
1282    */
1283   public static void splitRegion(final Connection connection,
1284                                  HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
1285                                  ServerName sn, int regionReplication) throws IOException {
1286     Table meta = getMetaHTable(connection);
1287     try {
1288       HRegionInfo copyOfParent = new HRegionInfo(parent);
1289       copyOfParent.setOffline(true);
1290       copyOfParent.setSplit(true);
1291 
1292       //Put for parent
1293       Put putParent = makePutFromRegionInfo(copyOfParent);
1294       addDaughtersToPut(putParent, splitA, splitB);
1295 
1296       //Puts for daughters
1297       Put putA = makePutFromRegionInfo(splitA);
1298       Put putB = makePutFromRegionInfo(splitB);
1299 
1300       addLocation(putA, sn, 1, -1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
1301       addLocation(putB, sn, 1, -1, splitB.getReplicaId());
1302 
1303       // Add empty locations for region replicas of daughters so that number of replicas can be
1304       // cached whenever the primary region is looked up from meta
1305       for (int i = 1; i < regionReplication; i++) {
1306         addEmptyLocation(putA, i);
1307         addEmptyLocation(putB, i);
1308       }
1309 
1310       byte[] tableRow = Bytes.toBytes(parent.getRegionNameAsString() + HConstants.DELIMITER);
1311       multiMutate(meta, tableRow, putParent, putA, putB);
1312     } finally {
1313       meta.close();
1314     }
1315   }
1316 
1317   /**
1318    * Performs an atomic multi-Mutate operation against the given table.
1319    */
1320   private static void multiMutate(Table table, byte[] row, Mutation... mutations)
1321       throws IOException {
1322     CoprocessorRpcChannel channel = table.coprocessorService(row);
1323     MultiRowMutationProtos.MutateRowsRequest.Builder mmrBuilder
1324       = MultiRowMutationProtos.MutateRowsRequest.newBuilder();
1325     for (Mutation mutation : mutations) {
1326       if (mutation instanceof Put) {
1327         mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(
1328           ClientProtos.MutationProto.MutationType.PUT, mutation));
1329       } else if (mutation instanceof Delete) {
1330         mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(
1331           ClientProtos.MutationProto.MutationType.DELETE, mutation));
1332       } else {
1333         throw new DoNotRetryIOException("multi in MetaEditor doesn't support "
1334           + mutation.getClass().getName());
1335       }
1336     }
1337 
1338     MultiRowMutationProtos.MultiRowMutationService.BlockingInterface service =
1339       MultiRowMutationProtos.MultiRowMutationService.newBlockingStub(channel);
1340     try {
1341       service.mutateRows(null, mmrBuilder.build());
1342     } catch (ServiceException ex) {
1343       ProtobufUtil.toIOException(ex);
1344     }
1345   }
1346 
1347   /**
1348    * Updates the location of the specified region in hbase:meta to be the specified
1349    * server hostname and startcode.
1350    * <p>
1351    * Uses passed catalog tracker to get a connection to the server hosting
1352    * hbase:meta and makes edits to that region.
1353    *
1354    * @param connection connection we're using
1355    * @param regionInfo region to update location of
1356    * @param openSeqNum the latest sequence number obtained when the region was open
1357    * @param sn Server name
1358    * @param masterSystemTime wall clock time from master if passed in the open region RPC or -1
1359    * @throws IOException
1360    */
1361   public static void updateRegionLocation(Connection connection,
1362                                           HRegionInfo regionInfo, ServerName sn, long openSeqNum,
1363                                           long masterSystemTime)
1364     throws IOException {
1365     updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
1366   }
1367 
1368   /**
1369    * Updates the location of the specified region to be the specified server.
1370    * <p>
1371    * Connects to the specified server which should be hosting the specified
1372    * catalog region name to perform the edit.
1373    *
1374    * @param connection connection we're using
1375    * @param regionInfo region to update location of
1376    * @param sn Server name
1377    * @param openSeqNum the latest sequence number obtained when the region was open
1378    * @param masterSystemTime wall clock time from master if passed in the open region RPC or -1
1379    * @throws IOException In particular could throw {@link java.net.ConnectException}
1380    * if the server is down on other end.
1381    */
1382   private static void updateLocation(final Connection connection,
1383                                      HRegionInfo regionInfo, ServerName sn, long openSeqNum,
1384                                      long masterSystemTime)
1385     throws IOException {
1386 
1387     // use the maximum of what master passed us vs local time.
1388     long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
1389 
1390     // region replicas are kept in the primary region's row
1391     Put put = new Put(getMetaKeyForRegion(regionInfo), time);
1392     HRegionInfo defaultRegionInfo = regionInfo;
1393     if (regionInfo.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
1394       defaultRegionInfo = new HRegionInfo(regionInfo, HRegionInfo.DEFAULT_REPLICA_ID);
1395     }
1396     addRegionInfo(put, defaultRegionInfo);
1397     addLocation(put, sn, openSeqNum, time, regionInfo.getReplicaId());
1398     putToMetaTable(connection, put);
1399     LOG.info("Updated row " + defaultRegionInfo.getRegionNameAsString() +
1400       " with server=" + sn);
1401   }
1402 
1403   /**
1404    * Deletes the specified region from META.
1405    * @param connection connection we're using
1406    * @param regionInfo region to be deleted from META
1407    * @throws IOException
1408    */
1409   public static void deleteRegion(Connection connection,
1410                                   HRegionInfo regionInfo)
1411     throws IOException {
1412     Delete delete = new Delete(regionInfo.getRegionName());
1413     deleteFromMetaTable(connection, delete);
1414     LOG.info("Deleted " + regionInfo.getRegionNameAsString());
1415   }
1416 
1417   /**
1418    * Deletes the specified regions from META.
1419    * @param connection connection we're using
1420    * @param regionsInfo list of regions to be deleted from META
1421    * @param ts desired timestamp
1422    * @throws IOException
1423    */
1424   public static void deleteRegions(Connection connection,
1425                                    List<HRegionInfo> regionsInfo, long ts) throws IOException {
1426     List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size());
1427     for (HRegionInfo hri: regionsInfo) {
1428       deletes.add(new Delete(hri.getRegionName(), ts));
1429     }
1430     deleteFromMetaTable(connection, deletes);
1431     LOG.info("Deleted " + regionsInfo);
1432   }
1433   /**
1434    * Deletes the specified regions from META.
1435    * @param connection connection we're using
1436    * @param regionsInfo list of regions to be deleted from META
1437    * @throws IOException
1438    */
1439   public static void deleteRegions(Connection connection,
1440                                    List<HRegionInfo> regionsInfo) throws IOException {
1441     deleteRegions(connection, regionsInfo, HConstants.LATEST_TIMESTAMP);
1442   }
1443 
1444   /**
1445    * Adds and Removes the specified regions from hbase:meta
1446    * @param connection connection we're using
1447    * @param regionsToRemove list of regions to be deleted from META
1448    * @param regionsToAdd list of regions to be added to META
1449    * @throws IOException
1450    */
1451   public static void mutateRegions(Connection connection,
1452                                    final List<HRegionInfo> regionsToRemove,
1453                                    final List<HRegionInfo> regionsToAdd)
1454     throws IOException {
1455     List<Mutation> mutation = new ArrayList<Mutation>();
1456     if (regionsToRemove != null) {
1457       for (HRegionInfo hri: regionsToRemove) {
1458         mutation.add(new Delete(hri.getRegionName()));
1459       }
1460     }
1461     if (regionsToAdd != null) {
1462       for (HRegionInfo hri: regionsToAdd) {
1463         mutation.add(makePutFromRegionInfo(hri));
1464       }
1465     }
1466     mutateMetaTable(connection, mutation);
1467     if (regionsToRemove != null && regionsToRemove.size() > 0) {
1468       LOG.debug("Deleted " + regionsToRemove);
1469     }
1470     if (regionsToAdd != null && regionsToAdd.size() > 0) {
1471       LOG.debug("Added " + regionsToAdd);
1472     }
1473   }
1474 
1475   /**
1476    * Overwrites the specified regions from hbase:meta
1477    * @param connection connection we're using
1478    * @param regionInfos list of regions to be added to META
1479    * @throws IOException
1480    */
1481   public static void overwriteRegions(Connection connection,
1482       List<HRegionInfo> regionInfos, int regionReplication) throws IOException {
1483     // use master time for delete marker and the Put
1484     long now = EnvironmentEdgeManager.currentTime();
1485     deleteRegions(connection, regionInfos, now);
1486     // Why sleep? This is the easiest way to ensure that the previous deletes does not
1487     // eclipse the following puts, that might happen in the same ts from the server.
1488     // See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed,
1489     // or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep.
1490     //
1491     // HBASE-13875 uses master timestamp for the mutations. The 20ms sleep is not needed
1492     addRegionsToMeta(connection, regionInfos, regionReplication, now+1);
1493     LOG.info("Overwritten " + regionInfos);
1494   }
1495 
1496   /**
1497    * Deletes merge qualifiers for the specified merged region.
1498    * @param connection connection we're using
1499    * @param mergedRegion
1500    * @throws IOException
1501    */
1502   public static void deleteMergeQualifiers(Connection connection,
1503                                            final HRegionInfo mergedRegion) throws IOException {
1504     Delete delete = new Delete(mergedRegion.getRegionName());
1505     delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER);
1506     delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER);
1507     deleteFromMetaTable(connection, delete);
1508     LOG.info("Deleted references in merged region "
1509       + mergedRegion.getRegionNameAsString() + ", qualifier="
1510       + Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier="
1511       + Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER));
1512   }
1513 
1514   private static Put addRegionInfo(final Put p, final HRegionInfo hri)
1515     throws IOException {
1516     p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
1517       hri.toByteArray());
1518     return p;
1519   }
1520 
1521   public static Put addLocation(final Put p, final ServerName sn, long openSeqNum,
1522       long time, int replicaId){
1523     if (time <= 0) {
1524       time = EnvironmentEdgeManager.currentTime();
1525     }
1526     p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), time,
1527       Bytes.toBytes(sn.getHostAndPort()));
1528     p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), time,
1529       Bytes.toBytes(sn.getStartcode()));
1530     p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), time,
1531       Bytes.toBytes(openSeqNum));
1532     return p;
1533   }
1534 
1535   public static Put addEmptyLocation(final Put p, int replicaId) {
1536     long now = EnvironmentEdgeManager.currentTime();
1537     p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), now, null);
1538     p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now, null);
1539     p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now, null);
1540     return p;
1541   }
1542 }