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 > 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, 0));
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 = infoMap.tailMap(serverColumn, false);
744     if (serverMap.isEmpty()) return new RegionLocations(locations);
745 
746     for (Map.Entry<byte[], byte[]> entry : serverMap.entrySet()) {
747       replicaId = parseReplicaIdFromServerColumn(entry.getKey());
748       if (replicaId < 0) {
749         break;
750       }
751       HRegionLocation location = getRegionLocation(r, regionInfo, replicaId);
752       // In case the region replica is newly created, it's location might be null. We usually do not
753       // have HRL's in RegionLocations object with null ServerName. They are handled as null HRLs.
754       if (location == null || location.getServerName() == null) {
755         locations.add(null);
756       } else {
757         locations.add(location);
758       }
759     }
760 
761     return new RegionLocations(locations);
762   }
763 
764   /**
765    * Returns the HRegionLocation parsed from the given meta row Result
766    * for the given regionInfo and replicaId. The regionInfo can be the default region info
767    * for the replica.
768    * @param r the meta row result
769    * @param regionInfo RegionInfo for default replica
770    * @param replicaId the replicaId for the HRegionLocation
771    * @return HRegionLocation parsed from the given meta row Result for the given replicaId
772    */
773   private static HRegionLocation getRegionLocation(final Result r, final HRegionInfo regionInfo,
774                                                    final int replicaId) {
775     ServerName serverName = getServerName(r, replicaId);
776     long seqNum = getSeqNumDuringOpen(r, replicaId);
777     HRegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
778     return new HRegionLocation(replicaInfo, serverName, seqNum);
779   }
780 
781   /**
782    * Returns HRegionInfo object from the column
783    * HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
784    * table Result.
785    * @param data a Result object from the catalog table scan
786    * @return HRegionInfo or null
787    */
788   public static HRegionInfo getHRegionInfo(Result data) {
789     return getHRegionInfo(data, HConstants.REGIONINFO_QUALIFIER);
790   }
791 
792   /**
793    * Returns the HRegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
794    * <code>qualifier</code> of the catalog table result.
795    * @param r a Result object from the catalog table scan
796    * @param qualifier Column family qualifier
797    * @return An HRegionInfo instance or null.
798    */
799   private static HRegionInfo getHRegionInfo(final Result r, byte [] qualifier) {
800     Cell cell = r.getColumnLatestCell(getFamily(), qualifier);
801     if (cell == null) return null;
802     return HRegionInfo.parseFromOrNull(cell.getValueArray(),
803       cell.getValueOffset(), cell.getValueLength());
804   }
805 
806   /**
807    * Returns the daughter regions by reading the corresponding columns of the catalog table
808    * Result.
809    * @param data a Result object from the catalog table scan
810    * @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
811    * parent
812    */
813   public static PairOfSameType<HRegionInfo> getDaughterRegions(Result data) {
814     HRegionInfo splitA = getHRegionInfo(data, HConstants.SPLITA_QUALIFIER);
815     HRegionInfo splitB = getHRegionInfo(data, HConstants.SPLITB_QUALIFIER);
816 
817     return new PairOfSameType<HRegionInfo>(splitA, splitB);
818   }
819 
820   /**
821    * Returns the merge regions by reading the corresponding columns of the catalog table
822    * Result.
823    * @param data a Result object from the catalog table scan
824    * @return a pair of HRegionInfo or PairOfSameType(null, null) if the region is not a split
825    * parent
826    */
827   public static PairOfSameType<HRegionInfo> getMergeRegions(Result data) {
828     HRegionInfo mergeA = getHRegionInfo(data, HConstants.MERGEA_QUALIFIER);
829     HRegionInfo mergeB = getHRegionInfo(data, HConstants.MERGEB_QUALIFIER);
830 
831     return new PairOfSameType<HRegionInfo>(mergeA, mergeB);
832   }
833 
834   /**
835    * Implementations 'visit' a catalog table row.
836    */
837   public interface Visitor {
838     /**
839      * Visit the catalog table row.
840      * @param r A row from catalog table
841      * @return True if we are to proceed scanning the table, else false if
842      * we are to stop now.
843      */
844     boolean visit(final Result r) throws IOException;
845   }
846 
847   /**
848    * A {@link Visitor} that collects content out of passed {@link Result}.
849    */
850   static abstract class CollectingVisitor<T> implements Visitor {
851     final List<T> results = new ArrayList<T>();
852     @Override
853     public boolean visit(Result r) throws IOException {
854       if (r ==  null || r.isEmpty()) return true;
855       add(r);
856       return true;
857     }
858 
859     abstract void add(Result r);
860 
861     /**
862      * @return Collected results; wait till visits complete to collect all
863      * possible results
864      */
865     List<T> getResults() {
866       return this.results;
867     }
868   }
869 
870   /**
871    * Collects all returned.
872    */
873   static class CollectAllVisitor extends CollectingVisitor<Result> {
874     @Override
875     void add(Result r) {
876       this.results.add(r);
877     }
878   }
879 
880   /**
881    * Count regions in <code>hbase:meta</code> for passed table.
882    * @param c Configuration object
883    * @param tableName table name to count regions for
884    * @return Count or regions in table <code>tableName</code>
885    * @throws IOException
886    */
887   @Deprecated
888   public static int getRegionCount(final Configuration c, final String tableName)
889       throws IOException {
890     return getRegionCount(c, TableName.valueOf(tableName));
891   }
892 
893   /**
894    * Count regions in <code>hbase:meta</code> for passed table.
895    * @param c Configuration object
896    * @param tableName table name to count regions for
897    * @return Count or regions in table <code>tableName</code>
898    * @throws IOException
899    */
900   public static int getRegionCount(final Configuration c, final TableName tableName)
901   throws IOException {
902     try (Connection connection = ConnectionFactory.createConnection(c)) {
903       return getRegionCount(connection, tableName);
904     }
905   }
906 
907   /**
908    * Count regions in <code>hbase:meta</code> for passed table.
909    * @param connection Connection object
910    * @param tableName table name to count regions for
911    * @return Count or regions in table <code>tableName</code>
912    * @throws IOException
913    */
914   public static int getRegionCount(final Connection connection, final TableName tableName)
915   throws IOException {
916     try (RegionLocator locator = connection.getRegionLocator(tableName)) {
917       List<HRegionLocation> locations = locator.getAllRegionLocations();
918       return locations == null? 0: locations.size();
919     }
920   }
921 
922   ////////////////////////
923   // Editing operations //
924   ////////////////////////
925 
926   /**
927    * Generates and returns a Put containing the region into for the catalog table
928    */
929   public static Put makePutFromRegionInfo(HRegionInfo regionInfo)
930     throws IOException {
931     return makePutFromRegionInfo(regionInfo, HConstants.LATEST_TIMESTAMP);
932   }
933   /**
934    * Generates and returns a Put containing the region into for the catalog table
935    */
936   public static Put makePutFromRegionInfo(HRegionInfo regionInfo, long ts)
937     throws IOException {
938     Put put = new Put(regionInfo.getRegionName(), ts);
939     addRegionInfo(put, regionInfo);
940     return put;
941   }
942 
943   /**
944    * Generates and returns a Delete containing the region info for the catalog
945    * table
946    */
947   public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo) {
948     return makeDeleteFromRegionInfo(regionInfo, HConstants.LATEST_TIMESTAMP);
949   }
950 
951   /**
952    * Generates and returns a Delete containing the region info for the catalog
953    * table
954    */
955   public static Delete makeDeleteFromRegionInfo(HRegionInfo regionInfo, long ts) {
956     if (regionInfo == null) {
957       throw new IllegalArgumentException("Can't make a delete for null region");
958     }
959     Delete delete = new Delete(regionInfo.getRegionName(), ts);
960     return delete;
961   }
962 
963   /**
964    * Adds split daughters to the Put
965    */
966   public static Put addDaughtersToPut(Put put, HRegionInfo splitA, HRegionInfo splitB) {
967     if (splitA != null) {
968       put.addImmutable(
969         HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER, splitA.toByteArray());
970     }
971     if (splitB != null) {
972       put.addImmutable(
973         HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER, splitB.toByteArray());
974     }
975     return put;
976   }
977 
978   /**
979    * Put the passed <code>p</code> to the <code>hbase:meta</code> table.
980    * @param connection connection we're using
981    * @param p Put to add to hbase:meta
982    * @throws IOException
983    */
984   static void putToMetaTable(final Connection connection, final Put p)
985     throws IOException {
986     put(getMetaHTable(connection), p);
987   }
988 
989   /**
990    * @param t Table to use (will be closed when done).
991    * @param p put to make
992    * @throws IOException
993    */
994   private static void put(final Table t, final Put p) throws IOException {
995     try {
996       t.put(p);
997     } finally {
998       t.close();
999     }
1000   }
1001 
1002   /**
1003    * Put the passed <code>ps</code> to the <code>hbase:meta</code> table.
1004    * @param connection connection we're using
1005    * @param ps Put to add to hbase:meta
1006    * @throws IOException
1007    */
1008   public static void putsToMetaTable(final Connection connection, final List<Put> ps)
1009     throws IOException {
1010     Table t = getMetaHTable(connection);
1011     try {
1012       t.put(ps);
1013     } finally {
1014       t.close();
1015     }
1016   }
1017 
1018   /**
1019    * Delete the passed <code>d</code> from the <code>hbase:meta</code> table.
1020    * @param connection connection we're using
1021    * @param d Delete to add to hbase:meta
1022    * @throws IOException
1023    */
1024   static void deleteFromMetaTable(final Connection connection, final Delete d)
1025     throws IOException {
1026     List<Delete> dels = new ArrayList<Delete>(1);
1027     dels.add(d);
1028     deleteFromMetaTable(connection, dels);
1029   }
1030 
1031   /**
1032    * Delete the passed <code>deletes</code> from the <code>hbase:meta</code> table.
1033    * @param connection connection we're using
1034    * @param deletes Deletes to add to hbase:meta  This list should support #remove.
1035    * @throws IOException
1036    */
1037   public static void deleteFromMetaTable(final Connection connection, final List<Delete> deletes)
1038     throws IOException {
1039     Table t = getMetaHTable(connection);
1040     try {
1041       t.delete(deletes);
1042     } finally {
1043       t.close();
1044     }
1045   }
1046 
1047   /**
1048    * Deletes some replica columns corresponding to replicas for the passed rows
1049    * @param metaRows rows in hbase:meta
1050    * @param replicaIndexToDeleteFrom the replica ID we would start deleting from
1051    * @param numReplicasToRemove how many replicas to remove
1052    * @param connection connection we're using to access meta table
1053    * @throws IOException
1054    */
1055   public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows,
1056     int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection)
1057       throws IOException {
1058     int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
1059     for (byte[] row : metaRows) {
1060       Delete deleteReplicaLocations = new Delete(row);
1061       for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
1062         deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
1063           getServerColumn(i));
1064         deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
1065           getSeqNumColumn(i));
1066         deleteReplicaLocations.deleteColumns(HConstants.CATALOG_FAMILY,
1067           getStartCodeColumn(i));
1068       }
1069       deleteFromMetaTable(connection, deleteReplicaLocations);
1070     }
1071   }
1072 
1073   /**
1074    * Execute the passed <code>mutations</code> against <code>hbase:meta</code> table.
1075    * @param connection connection we're using
1076    * @param mutations Puts and Deletes to execute on hbase:meta
1077    * @throws IOException
1078    */
1079   public static void mutateMetaTable(final Connection connection,
1080                                      final List<Mutation> mutations)
1081     throws IOException {
1082     Table t = getMetaHTable(connection);
1083     try {
1084       t.batch(mutations);
1085     } catch (InterruptedException e) {
1086       InterruptedIOException ie = new InterruptedIOException(e.getMessage());
1087       ie.initCause(e);
1088       throw ie;
1089     } finally {
1090       t.close();
1091     }
1092   }
1093 
1094   /**
1095    * Adds a hbase:meta row for the specified new region.
1096    * @param connection connection we're using
1097    * @param regionInfo region information
1098    * @throws IOException if problem connecting or updating meta
1099    */
1100   public static void addRegionToMeta(Connection connection,
1101                                      HRegionInfo regionInfo)
1102     throws IOException {
1103     putToMetaTable(connection, makePutFromRegionInfo(regionInfo));
1104     LOG.info("Added " + regionInfo.getRegionNameAsString());
1105   }
1106 
1107   /**
1108    * Adds a hbase:meta row for the specified new region to the given catalog table. The
1109    * Table is not flushed or closed.
1110    * @param meta the Table for META
1111    * @param regionInfo region information
1112    * @throws IOException if problem connecting or updating meta
1113    */
1114   public static void addRegionToMeta(Table meta, HRegionInfo regionInfo) throws IOException {
1115     addRegionToMeta(meta, regionInfo, null, null);
1116   }
1117 
1118   /**
1119    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
1120    * does not add its daughter's as different rows, but adds information about the daughters
1121    * in the same row as the parent. Use
1122    * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)}
1123    * if you want to do that.
1124    * @param meta the Table for META
1125    * @param regionInfo region information
1126    * @param splitA first split daughter of the parent regionInfo
1127    * @param splitB second split daughter of the parent regionInfo
1128    * @throws IOException if problem connecting or updating meta
1129    */
1130   public static void addRegionToMeta(Table meta, HRegionInfo regionInfo,
1131                                      HRegionInfo splitA, HRegionInfo splitB) throws IOException {
1132     Put put = makePutFromRegionInfo(regionInfo);
1133     addDaughtersToPut(put, splitA, splitB);
1134     meta.put(put);
1135     if (LOG.isDebugEnabled()) {
1136       LOG.debug("Added " + regionInfo.getRegionNameAsString());
1137     }
1138   }
1139 
1140   /**
1141    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
1142    * does not add its daughter's as different rows, but adds information about the daughters
1143    * in the same row as the parent. Use
1144    * {@link #splitRegion(Connection, HRegionInfo, HRegionInfo, HRegionInfo, ServerName, int)}
1145    * if you want to do that.
1146    * @param connection connection we're using
1147    * @param regionInfo region information
1148    * @param splitA first split daughter of the parent regionInfo
1149    * @param splitB second split daughter of the parent regionInfo
1150    * @throws IOException if problem connecting or updating meta
1151    */
1152   public static void addRegionToMeta(Connection connection, HRegionInfo regionInfo,
1153                                      HRegionInfo splitA, HRegionInfo splitB) throws IOException {
1154     Table meta = getMetaHTable(connection);
1155     try {
1156       addRegionToMeta(meta, regionInfo, splitA, splitB);
1157     } finally {
1158       meta.close();
1159     }
1160   }
1161 
1162   /**
1163    * Adds a hbase:meta row for each of the specified new regions.
1164    * @param connection connection we're using
1165    * @param regionInfos region information list
1166    * @throws IOException if problem connecting or updating meta
1167    */
1168   public static void addRegionsToMeta(Connection connection,
1169                                       List<HRegionInfo> regionInfos, int regionReplication)
1170     throws IOException {
1171     addRegionsToMeta(connection, regionInfos, regionReplication, HConstants.LATEST_TIMESTAMP);
1172   }
1173   /**
1174    * Adds a hbase:meta row for each of the specified new regions.
1175    * @param connection connection we're using
1176    * @param regionInfos region information list
1177    * @param regionReplication
1178    * @param ts desired timestamp
1179    * @throws IOException if problem connecting or updating meta
1180    */
1181   public static void addRegionsToMeta(Connection connection,
1182       List<HRegionInfo> regionInfos, int regionReplication, long ts)
1183           throws IOException {
1184     List<Put> puts = new ArrayList<Put>();
1185     for (HRegionInfo regionInfo : regionInfos) {
1186       if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
1187         Put put = makePutFromRegionInfo(regionInfo, ts);
1188         // Add empty locations for region replicas so that number of replicas can be cached
1189         // whenever the primary region is looked up from meta
1190         for (int i = 1; i < regionReplication; i++) {
1191           addEmptyLocation(put, i);
1192         }
1193         puts.add(put);
1194       }
1195     }
1196     putsToMetaTable(connection, puts);
1197     LOG.info("Added " + puts.size());
1198   }
1199 
1200   /**
1201    * Adds a daughter region entry to meta.
1202    * @param regionInfo the region to put
1203    * @param sn the location of the region
1204    * @param openSeqNum the latest sequence number obtained when the region was open
1205    */
1206   public static void addDaughter(final Connection connection,
1207       final HRegionInfo regionInfo, final ServerName sn, final long openSeqNum)
1208       throws NotAllMetaRegionsOnlineException, IOException {
1209     Put put = new Put(regionInfo.getRegionName());
1210     addRegionInfo(put, regionInfo);
1211     if (sn != null) {
1212       addLocation(put, sn, openSeqNum, -1, regionInfo.getReplicaId());
1213     }
1214     putToMetaTable(connection, put);
1215     LOG.info("Added daughter " + regionInfo.getEncodedName() +
1216       (sn == null? ", serverName=null": ", serverName=" + sn.toString()));
1217   }
1218 
1219   /**
1220    * Merge the two regions into one in an atomic operation. Deletes the two
1221    * merging regions in hbase:meta and adds the merged region with the information of
1222    * two merging regions.
1223    * @param connection connection we're using
1224    * @param mergedRegion the merged region
1225    * @param regionA
1226    * @param regionB
1227    * @param sn the location of the region
1228    * @param masterSystemTime
1229    * @throws IOException
1230    */
1231   public static void mergeRegions(final Connection connection, HRegionInfo mergedRegion,
1232       HRegionInfo regionA, HRegionInfo regionB, ServerName sn, int regionReplication,
1233       long masterSystemTime)
1234           throws IOException {
1235     Table meta = getMetaHTable(connection);
1236     try {
1237       HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
1238 
1239       // use the maximum of what master passed us vs local time.
1240       long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
1241 
1242       // Put for parent
1243       Put putOfMerged = makePutFromRegionInfo(copyOfMerged, time);
1244       putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER,
1245         regionA.toByteArray());
1246       putOfMerged.addImmutable(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER,
1247         regionB.toByteArray());
1248 
1249       // Deletes for merging regions
1250       Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
1251       Delete deleteB = makeDeleteFromRegionInfo(regionB, time);
1252 
1253       // The merged is a new region, openSeqNum = 1 is fine.
1254       addLocation(putOfMerged, sn, 1, -1, mergedRegion.getReplicaId());
1255 
1256       // Add empty locations for region replicas of the merged region so that number of replicas can
1257       // be cached whenever the primary region is looked up from meta
1258       for (int i = 1; i < regionReplication; i++) {
1259         addEmptyLocation(putOfMerged, i);
1260       }
1261 
1262       byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString()
1263         + HConstants.DELIMITER);
1264       multiMutate(meta, tableRow, putOfMerged, deleteA, deleteB);
1265     } finally {
1266       meta.close();
1267     }
1268   }
1269 
1270   /**
1271    * Splits the region into two in an atomic operation. Offlines the parent
1272    * region with the information that it is split into two, and also adds
1273    * the daughter regions. Does not add the location information to the daughter
1274    * regions since they are not open yet.
1275    * @param connection connection we're using
1276    * @param parent the parent region which is split
1277    * @param splitA Split daughter region A
1278    * @param splitB Split daughter region A
1279    * @param sn the location of the region
1280    */
1281   public static void splitRegion(final Connection connection,
1282                                  HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
1283                                  ServerName sn, int regionReplication) throws IOException {
1284     Table meta = getMetaHTable(connection);
1285     try {
1286       HRegionInfo copyOfParent = new HRegionInfo(parent);
1287       copyOfParent.setOffline(true);
1288       copyOfParent.setSplit(true);
1289 
1290       //Put for parent
1291       Put putParent = makePutFromRegionInfo(copyOfParent);
1292       addDaughtersToPut(putParent, splitA, splitB);
1293 
1294       //Puts for daughters
1295       Put putA = makePutFromRegionInfo(splitA);
1296       Put putB = makePutFromRegionInfo(splitB);
1297 
1298       addLocation(putA, sn, 1, -1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
1299       addLocation(putB, sn, 1, -1, splitB.getReplicaId());
1300 
1301       // Add empty locations for region replicas of daughters so that number of replicas can be
1302       // cached whenever the primary region is looked up from meta
1303       for (int i = 1; i < regionReplication; i++) {
1304         addEmptyLocation(putA, i);
1305         addEmptyLocation(putB, i);
1306       }
1307 
1308       byte[] tableRow = Bytes.toBytes(parent.getRegionNameAsString() + HConstants.DELIMITER);
1309       multiMutate(meta, tableRow, putParent, putA, putB);
1310     } finally {
1311       meta.close();
1312     }
1313   }
1314 
1315   /**
1316    * Performs an atomic multi-Mutate operation against the given table.
1317    */
1318   private static void multiMutate(Table table, byte[] row, Mutation... mutations)
1319       throws IOException {
1320     CoprocessorRpcChannel channel = table.coprocessorService(row);
1321     MultiRowMutationProtos.MutateRowsRequest.Builder mmrBuilder
1322       = MultiRowMutationProtos.MutateRowsRequest.newBuilder();
1323     for (Mutation mutation : mutations) {
1324       if (mutation instanceof Put) {
1325         mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(
1326           ClientProtos.MutationProto.MutationType.PUT, mutation));
1327       } else if (mutation instanceof Delete) {
1328         mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(
1329           ClientProtos.MutationProto.MutationType.DELETE, mutation));
1330       } else {
1331         throw new DoNotRetryIOException("multi in MetaEditor doesn't support "
1332           + mutation.getClass().getName());
1333       }
1334     }
1335 
1336     MultiRowMutationProtos.MultiRowMutationService.BlockingInterface service =
1337       MultiRowMutationProtos.MultiRowMutationService.newBlockingStub(channel);
1338     try {
1339       service.mutateRows(null, mmrBuilder.build());
1340     } catch (ServiceException ex) {
1341       ProtobufUtil.toIOException(ex);
1342     }
1343   }
1344 
1345   /**
1346    * Updates the location of the specified region in hbase:meta to be the specified
1347    * server hostname and startcode.
1348    * <p>
1349    * Uses passed catalog tracker to get a connection to the server hosting
1350    * hbase:meta and makes edits to that region.
1351    *
1352    * @param connection connection we're using
1353    * @param regionInfo region to update location of
1354    * @param openSeqNum the latest sequence number obtained when the region was open
1355    * @param sn Server name
1356    * @param masterSystemTime wall clock time from master if passed in the open region RPC or -1
1357    * @throws IOException
1358    */
1359   public static void updateRegionLocation(Connection connection,
1360                                           HRegionInfo regionInfo, ServerName sn, long openSeqNum,
1361                                           long masterSystemTime)
1362     throws IOException {
1363     updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
1364   }
1365 
1366   /**
1367    * Updates the location of the specified region to be the specified server.
1368    * <p>
1369    * Connects to the specified server which should be hosting the specified
1370    * catalog region name to perform the edit.
1371    *
1372    * @param connection connection we're using
1373    * @param regionInfo region to update location of
1374    * @param sn Server name
1375    * @param openSeqNum the latest sequence number obtained when the region was open
1376    * @param masterSystemTime wall clock time from master if passed in the open region RPC or -1
1377    * @throws IOException In particular could throw {@link java.net.ConnectException}
1378    * if the server is down on other end.
1379    */
1380   private static void updateLocation(final Connection connection,
1381                                      HRegionInfo regionInfo, ServerName sn, long openSeqNum,
1382                                      long masterSystemTime)
1383     throws IOException {
1384 
1385     // use the maximum of what master passed us vs local time.
1386     long time = Math.max(EnvironmentEdgeManager.currentTime(), masterSystemTime);
1387 
1388     // region replicas are kept in the primary region's row
1389     Put put = new Put(getMetaKeyForRegion(regionInfo), time);
1390     addLocation(put, sn, openSeqNum, time, regionInfo.getReplicaId());
1391     putToMetaTable(connection, put);
1392     LOG.info("Updated row " + regionInfo.getRegionNameAsString() +
1393       " with server=" + sn);
1394   }
1395 
1396   /**
1397    * Deletes the specified region from META.
1398    * @param connection connection we're using
1399    * @param regionInfo region to be deleted from META
1400    * @throws IOException
1401    */
1402   public static void deleteRegion(Connection connection,
1403                                   HRegionInfo regionInfo)
1404     throws IOException {
1405     Delete delete = new Delete(regionInfo.getRegionName());
1406     deleteFromMetaTable(connection, delete);
1407     LOG.info("Deleted " + regionInfo.getRegionNameAsString());
1408   }
1409 
1410   /**
1411    * Deletes the specified regions from META.
1412    * @param connection connection we're using
1413    * @param regionsInfo list of regions to be deleted from META
1414    * @param ts desired timestamp
1415    * @throws IOException
1416    */
1417   public static void deleteRegions(Connection connection,
1418                                    List<HRegionInfo> regionsInfo, long ts) throws IOException {
1419     List<Delete> deletes = new ArrayList<Delete>(regionsInfo.size());
1420     for (HRegionInfo hri: regionsInfo) {
1421       deletes.add(new Delete(hri.getRegionName(), ts));
1422     }
1423     deleteFromMetaTable(connection, deletes);
1424     LOG.info("Deleted " + regionsInfo);
1425   }
1426   /**
1427    * Deletes the specified regions from META.
1428    * @param connection connection we're using
1429    * @param regionsInfo list of regions to be deleted from META
1430    * @throws IOException
1431    */
1432   public static void deleteRegions(Connection connection,
1433                                    List<HRegionInfo> regionsInfo) throws IOException {
1434     deleteRegions(connection, regionsInfo, HConstants.LATEST_TIMESTAMP);
1435   }
1436 
1437   /**
1438    * Adds and Removes the specified regions from hbase:meta
1439    * @param connection connection we're using
1440    * @param regionsToRemove list of regions to be deleted from META
1441    * @param regionsToAdd list of regions to be added to META
1442    * @throws IOException
1443    */
1444   public static void mutateRegions(Connection connection,
1445                                    final List<HRegionInfo> regionsToRemove,
1446                                    final List<HRegionInfo> regionsToAdd)
1447     throws IOException {
1448     List<Mutation> mutation = new ArrayList<Mutation>();
1449     if (regionsToRemove != null) {
1450       for (HRegionInfo hri: regionsToRemove) {
1451         mutation.add(new Delete(hri.getRegionName()));
1452       }
1453     }
1454     if (regionsToAdd != null) {
1455       for (HRegionInfo hri: regionsToAdd) {
1456         mutation.add(makePutFromRegionInfo(hri));
1457       }
1458     }
1459     mutateMetaTable(connection, mutation);
1460     if (regionsToRemove != null && regionsToRemove.size() > 0) {
1461       LOG.debug("Deleted " + regionsToRemove);
1462     }
1463     if (regionsToAdd != null && regionsToAdd.size() > 0) {
1464       LOG.debug("Added " + regionsToAdd);
1465     }
1466   }
1467 
1468   /**
1469    * Overwrites the specified regions from hbase:meta
1470    * @param connection connection we're using
1471    * @param regionInfos list of regions to be added to META
1472    * @throws IOException
1473    */
1474   public static void overwriteRegions(Connection connection,
1475       List<HRegionInfo> regionInfos, int regionReplication) throws IOException {
1476     // use master time for delete marker and the Put
1477     long now = EnvironmentEdgeManager.currentTime();
1478     deleteRegions(connection, regionInfos, now);
1479     // Why sleep? This is the easiest way to ensure that the previous deletes does not
1480     // eclipse the following puts, that might happen in the same ts from the server.
1481     // See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed,
1482     // or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep.
1483     //
1484     // HBASE-13875 uses master timestamp for the mutations. The 20ms sleep is not needed
1485     addRegionsToMeta(connection, regionInfos, regionReplication, now+1);
1486     LOG.info("Overwritten " + regionInfos);
1487   }
1488 
1489   /**
1490    * Deletes merge qualifiers for the specified merged region.
1491    * @param connection connection we're using
1492    * @param mergedRegion
1493    * @throws IOException
1494    */
1495   public static void deleteMergeQualifiers(Connection connection,
1496                                            final HRegionInfo mergedRegion) throws IOException {
1497     Delete delete = new Delete(mergedRegion.getRegionName());
1498     delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER);
1499     delete.deleteColumns(HConstants.CATALOG_FAMILY, HConstants.MERGEB_QUALIFIER);
1500     deleteFromMetaTable(connection, delete);
1501     LOG.info("Deleted references in merged region "
1502       + mergedRegion.getRegionNameAsString() + ", qualifier="
1503       + Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier="
1504       + Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER));
1505   }
1506 
1507   private static Put addRegionInfo(final Put p, final HRegionInfo hri)
1508     throws IOException {
1509     p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
1510       hri.toByteArray());
1511     return p;
1512   }
1513 
1514   public static Put addLocation(final Put p, final ServerName sn, long openSeqNum,
1515       long time, int replicaId){
1516     if (time <= 0) {
1517       time = EnvironmentEdgeManager.currentTime();
1518     }
1519     p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), time,
1520       Bytes.toBytes(sn.getHostAndPort()));
1521     p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), time,
1522       Bytes.toBytes(sn.getStartcode()));
1523     p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), time,
1524       Bytes.toBytes(openSeqNum));
1525     return p;
1526   }
1527 
1528   public static Put addEmptyLocation(final Put p, int replicaId) {
1529     long now = EnvironmentEdgeManager.currentTime();
1530     p.addImmutable(HConstants.CATALOG_FAMILY, getServerColumn(replicaId), now, null);
1531     p.addImmutable(HConstants.CATALOG_FAMILY, getStartCodeColumn(replicaId), now, null);
1532     p.addImmutable(HConstants.CATALOG_FAMILY, getSeqNumColumn(replicaId), now, null);
1533     return p;
1534   }
1535 }