diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index ad95370725a..573a08f5dfc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -22,9 +22,9 @@ import edu.umd.cs.findbugs.annotations.Nullable; import java.io.ByteArrayOutputStream; import java.io.Closeable; import java.io.IOException; -import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.LinkedHashMap; @@ -59,6 +59,10 @@ import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; +import org.apache.hadoop.hbase.filter.QualifierFilter; +import org.apache.hadoop.hbase.filter.RegexStringComparator; +import org.apache.hadoop.hbase.filter.RowFilter; +import org.apache.hadoop.hbase.filter.SubstringComparator; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState.State; @@ -82,11 +86,12 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti /** *

- * Read/write operations on region and assignment information store in hbase:meta. + * Read/write operations on hbase:meta region as well as assignment information stored + * to hbase:meta. *

*

* Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is - * because when used on client-side (like from HBaseAdmin), we want to use short-living connection + * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection * (opened before each operation, closed right after), while when used on HM or HRS (like in * AssignmentManager) we want permanent connection. *

@@ -96,16 +101,17 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti * called default replica. *

*

- * Meta layout (as of 0.98 + HBASE-10070) is like: + *

Meta layout

* *
- * For each table there is single row in column family 'table' formatted:
- * <tableName> including namespace and columns are:
- * table: state             => contains table state
+ * For each table there is single row named for the table with a 'table' column family.
+ * The column family currently has one column in it, the 'state' column:
  *
- * For each table range, there is a single row, formatted like:
+ * table:state             => contains table state
+ *
+ * Then for each table range ('Region'), there is a single row, formatted as:
  * <tableName>,<startKey>,<regionId>,<encodedRegionName>.
- * This row corresponds to the regionName of the default region replica.
+ * This row is the serialized regionName of the default region replica.
  * Columns are:
  * info:regioninfo         => contains serialized HRI for the default region replica
  * info:server             => contains hostname:port (in string form) for the server hosting
@@ -126,10 +132,14 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  *                             region is split
  * info:splitB             => contains a serialized HRI for the second daughter region if the
  *                             region is split
- * info:mergeA             => contains a serialized HRI for the first parent region if the
- *                             region is the result of a merge
- * info:mergeB             => contains a serialized HRI for the second parent region if the
- *                             region is the result of a merge
+ * info:merge*             => contains a serialized HRI for a merge parent region. There will be two
+ *                             or more of these columns in a row. A row that has these columns is
+ *                             undergoing a merge and is the result of the merge. Columns listed
+ *                             in marge* columns are the parents of this merged region. Example
+ *                             columns: info:merge0001, info:merge0002. You make also see 'mergeA',
+ *                             and 'mergeB'. This is old form replaced by the new format that allows
+ *                             for more than two parents to be merged at a time.
+ * TODO: Add rep_barrier for serial replication explaination.
  * 
*

*

@@ -150,36 +160,6 @@ public class MetaTableAccessor { private static final byte SEPARATED_BYTE = 0x00; - /** - * Lists all of the table regions currently in META. - * Deprecated, keep there until some test use this. - * @param connection what we will use - * @param tableName table to list - * @return Map of all user-space regions to servers - * @deprecated use {@link #getTableRegionsAndLocations}, region can have multiple locations - */ - @Deprecated - public static NavigableMap allTableRegions( - Connection connection, final TableName tableName) throws IOException { - final NavigableMap regions = new TreeMap<>(); - Visitor visitor = new TableVisitorBase(tableName) { - @Override - public boolean visitInternal(Result result) throws IOException { - RegionLocations locations = getRegionLocations(result); - if (locations == null) return true; - for (HRegionLocation loc : locations.getRegionLocations()) { - if (loc != null) { - RegionInfo regionInfo = loc.getRegionInfo(); - regions.put(regionInfo, loc.getServerName()); - } - } - return true; - } - }; - scanMetaForTableRegions(connection, visitor, tableName); - return regions; - } - @InterfaceAudience.Private public enum QueryType { ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY), @@ -199,7 +179,7 @@ public class MetaTableAccessor { } /** The delimiter for meta columns for replicaIds > 0 */ - protected static final char META_REPLICA_ID_DELIMITER = '_'; + static final char META_REPLICA_ID_DELIMITER = '_'; /** A regex for parsing server columns from meta. See above javadoc for meta layout */ private static final Pattern SERVER_COLUMN_PATTERN @@ -214,8 +194,7 @@ public class MetaTableAccessor { * @param connection connection we're using * @param visitor Visitor invoked against each row in regions family. */ - public static void fullScanRegions(Connection connection, - final Visitor visitor) + public static void fullScanRegions(Connection connection, final Visitor visitor) throws IOException { scanMeta(connection, null, null, QueryType.REGION, visitor); } @@ -224,8 +203,7 @@ public class MetaTableAccessor { * Performs a full scan of hbase:meta for regions. * @param connection connection we're using */ - public static List fullScanRegions(Connection connection) - throws IOException { + public static List fullScanRegions(Connection connection) throws IOException { return fullScan(connection, QueryType.REGION); } @@ -234,8 +212,7 @@ public class MetaTableAccessor { * @param connection connection we're using * @param visitor Visitor invoked against each row in tables family. */ - public static void fullScanTables(Connection connection, - final Visitor visitor) + public static void fullScanTables(Connection connection, final Visitor visitor) throws IOException { scanMeta(connection, null, null, QueryType.TABLE, visitor); } @@ -246,8 +223,7 @@ public class MetaTableAccessor { * @param type scanned part of meta * @return List of {@link Result} */ - public static List fullScan(Connection connection, QueryType type) - throws IOException { + private static List fullScan(Connection connection, QueryType type) throws IOException { CollectAllVisitor v = new CollectAllVisitor(); scanMeta(connection, null, null, type, v); return v.getResults(); @@ -373,20 +349,108 @@ public class MetaTableAccessor { } /** - * Get regions from the merge qualifier of the specified merged region - * @return null if it doesn't contain merge qualifier, else two merge regions + * Scans META table for a row whose key contains the specified regionEncodedName, + * returning a single related Result instance if any row is found, null otherwise. + * + * @param connection the connection to query META table. + * @param regionEncodedName the region encoded name to look for at META. + * @return Result instance with the row related info in META, null otherwise. + * @throws IOException if any errors occur while querying META. + */ + public static Result scanByRegionEncodedName(Connection connection, + String regionEncodedName) throws IOException { + RowFilter rowFilter = new RowFilter(CompareOperator.EQUAL, + new SubstringComparator(regionEncodedName)); + Scan scan = getMetaScan(connection, 1); + scan.setFilter(rowFilter); + ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan); + return resultScanner.next(); + } + + /** + * @return Return all regioninfos listed in the 'info:merge*' columns of + * the regionName row. */ @Nullable - public static Pair getRegionsFromMergeQualifier( - Connection connection, byte[] regionName) throws IOException { - Result result = getRegionResult(connection, regionName); - RegionInfo mergeA = getRegionInfo(result, HConstants.MERGEA_QUALIFIER); - RegionInfo mergeB = getRegionInfo(result, HConstants.MERGEB_QUALIFIER); - if (mergeA == null && mergeB == null) { + public static List getMergeRegions(Connection connection, byte[] regionName) + throws IOException { + return getMergeRegions(getMergeRegionsRaw(connection, regionName)); + } + + /** + * @return Deserialized regioninfo values taken from column values that match + * the regex 'info:merge.*' in array of cells. + */ + @Nullable + public static List getMergeRegions(Cell [] cells) { + if (cells == null) { return null; } - return new Pair<>(mergeA, mergeB); - } + List regionsToMerge = null; + for (Cell cell: cells) { + if (!isMergeQualifierPrefix(cell)) { + continue; + } + // Ok. This cell is that of a info:merge* column. + RegionInfo ri = RegionInfo.parseFromOrNull(cell.getValueArray(), cell.getValueOffset(), + cell.getValueLength()); + if (ri != null) { + if (regionsToMerge == null) { + regionsToMerge = new ArrayList<>(); + } + regionsToMerge.add(ri); + } + } + return regionsToMerge; + } + + /** + * @return True if any merge regions present in cells; i.e. + * the column in cell matches the regex 'info:merge.*'. + */ + public static boolean hasMergeRegions(Cell [] cells) { + for (Cell cell: cells) { + if (!isMergeQualifierPrefix(cell)) { + continue; + } + return true; + } + return false; + } + + /** + * @return True if the column in cell matches the regex 'info:merge.*'. + */ + private static boolean isMergeQualifierPrefix(Cell cell) { + // Check to see if has family and that qualifier starts with the merge qualifier 'merge' + return CellUtil.matchingFamily(cell, HConstants.CATALOG_FAMILY) && + PrivateCellUtil.qualifierStartsWith(cell, HConstants.MERGE_QUALIFIER_PREFIX); + } + + /** + * @return Array of Cells made from all columns on the regionName row + * that match the regex 'info:merge.*'. + */ + @Nullable + private static Cell [] getMergeRegionsRaw(Connection connection, byte [] regionName) + throws IOException { + Scan scan = new Scan().withStartRow(regionName). + setOneRowLimit(). + readVersions(1). + addFamily(HConstants.CATALOG_FAMILY). + setFilter(new QualifierFilter(CompareOperator.EQUAL, + new RegexStringComparator(HConstants.MERGE_QUALIFIER_PREFIX_STR+ ".*"))); + try (Table m = getMetaHTable(connection); ResultScanner scanner = m.getScanner(scan)) { + // Should be only one result in this scanner if any. + Result result = scanner.next(); + if (result == null) { + return null; + } + // Should be safe to just return all Cells found since we had filter in place. + // All values should be RegionInfos or something wrong. + return result.rawCells(); + } + } /** * Checks if the specified table exists. Looks at the hbase:meta table hosted on @@ -399,8 +463,8 @@ public class MetaTableAccessor { final TableName tableName) throws IOException { // Catalog tables always exist. - return tableName.equals(TableName.META_TABLE_NAME) - || getTableState(connection, tableName) != null; + return tableName.equals(TableName.META_TABLE_NAME) || + getTableState(connection, tableName) != null; } /** @@ -557,7 +621,6 @@ public class MetaTableAccessor { * @param connection connection we're using * @param tableName table we're looking for * @return Return list of regioninfos and server. - * @throws IOException */ public static List> getTableRegionsAndLocations(Connection connection, TableName tableName) @@ -571,7 +634,6 @@ public class MetaTableAccessor { * @param tableName table to work with, can be null for getting all regions * @param excludeOfflinedSplitParents don't return split parents * @return Return list of regioninfos and server addresses. - * @throws IOException */ public static List> getTableRegionsAndLocations( Connection connection, @Nullable final TableName tableName, @@ -651,25 +713,26 @@ public class MetaTableAccessor { public static void fullScanMetaAndPrint(Connection connection) throws IOException { - Visitor v = new Visitor() { - @Override - public boolean visit(Result r) throws IOException { - if (r == null || r.isEmpty()) return true; - LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r); - TableState state = getTableState(r); - if (state != null) { - LOG.info("Table State: " + state); - } else { - RegionLocations locations = getRegionLocations(r); - if (locations == null) return true; - for (HRegionLocation loc : locations.getRegionLocations()) { - if (loc != null) { - LOG.info("fullScanMetaAndPrint.HRI Print= " + loc.getRegion()); - } - } - } + Visitor v = r -> { + if (r == null || r.isEmpty()) { return true; } + LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r); + TableState state = getTableState(r); + if (state != null) { + LOG.info("fullScanMetaAndPrint.Table State={}" + state); + } else { + RegionLocations locations = getRegionLocations(r); + if (locations == null) { + return true; + } + for (HRegionLocation loc : locations.getRegionLocations()) { + if (loc != null) { + LOG.info("fullScanMetaAndPrint.HRI Print={}", loc.getRegion()); + } + } + } + return true; }; scanMeta(connection, null, null, QueryType.ALL, v); } @@ -679,21 +742,19 @@ public class MetaTableAccessor { scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor); } - public static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows, + private static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows, final Visitor visitor) throws IOException { scanMeta(connection, getTableStartRowForMeta(table, type), getTableStopRowForMeta(table, type), type, maxRows, visitor); } - public static void scanMeta(Connection connection, @Nullable final byte[] startRow, + private static void scanMeta(Connection connection, @Nullable final byte[] startRow, @Nullable final byte[] stopRow, QueryType type, final Visitor visitor) throws IOException { scanMeta(connection, startRow, stopRow, type, Integer.MAX_VALUE, visitor); } /** - * Performs a scan of META table for given table starting from - * given row. - * + * Performs a scan of META table for given table starting from given row. * @param connection connection we're using * @param visitor visitor to call * @param tableName table withing we scan @@ -716,7 +777,6 @@ public class MetaTableAccessor { scanMeta(connection, startRow, stopRow, QueryType.REGION, rowLimit, visitor); } - /** * Performs a scan of META table. * @param connection connection we're using @@ -728,8 +788,8 @@ public class MetaTableAccessor { * @param maxRows maximum rows to return * @param visitor Visitor invoked against each row. */ - public static void scanMeta(Connection connection, @Nullable final byte[] startRow, - @Nullable final byte[] stopRow, QueryType type, int maxRows, final Visitor visitor) + static void scanMeta(Connection connection, @Nullable final byte[] startRow, + @Nullable final byte[] stopRow, QueryType type, int maxRows, final Visitor visitor) throws IOException { scanMeta(connection, startRow, stopRow, type, null, maxRows, visitor); } @@ -771,7 +831,7 @@ public class MetaTableAccessor { } } } - if (visitor != null && visitor instanceof Closeable) { + if (visitor instanceof Closeable) { try { ((Closeable) visitor).close(); } catch (Throwable t) { @@ -846,6 +906,30 @@ public class MetaTableAccessor { return HConstants.STATE_QUALIFIER; } + /** + * Returns the column qualifier for serialized region state + * @param replicaId the replicaId of the region + * @return a byte[] for state qualifier + */ + @VisibleForTesting + static byte[] getRegionStateColumn(int replicaId) { + return replicaId == 0 ? HConstants.STATE_QUALIFIER + : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER + + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); + } + + /** + * Returns the column qualifier for serialized region state + * @param replicaId the replicaId of the region + * @return a byte[] for sn column qualifier + */ + @VisibleForTesting + static byte[] getServerNameColumn(int replicaId) { + return replicaId == 0 ? HConstants.SERVERNAME_QUALIFIER + : Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER + + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); + } + /** * Returns the column qualifier for server column for replicaId * @param replicaId the replicaId of the region @@ -944,6 +1028,18 @@ public class MetaTableAccessor { return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); } + /** + * Returns the daughter regions by reading the corresponding columns of the catalog table + * Result. + * @param data a Result object from the catalog table scan + * @return pair of RegionInfo or PairOfSameType(null, null) if region is not a split parent + */ + public static PairOfSameType getDaughterRegions(Result data) { + RegionInfo splitA = getRegionInfo(data, HConstants.SPLITA_QUALIFIER); + RegionInfo splitB = getRegionInfo(data, HConstants.SPLITB_QUALIFIER); + return new PairOfSameType<>(splitA, splitB); + } + /** * Returns an HRegionLocationList extracted from the result. * @return an HRegionLocationList containing all locations for the region range or null if @@ -966,7 +1062,7 @@ public class MetaTableAccessor { // iterate until all serverName columns are seen int replicaId = 0; byte[] serverColumn = getServerColumn(replicaId); - SortedMap serverMap = null; + SortedMap serverMap; serverMap = infoMap.tailMap(serverColumn, false); if (serverMap.isEmpty()) return new RegionLocations(locations); @@ -979,7 +1075,7 @@ public class MetaTableAccessor { HRegionLocation location = getRegionLocation(r, regionInfo, replicaId); // In case the region replica is newly created, it's location might be null. We usually do not // have HRL's in RegionLocations object with null ServerName. They are handled as null HRLs. - if (location == null || location.getServerName() == null) { + if (location.getServerName() == null) { locations.add(null); } else { locations.add(location); @@ -1032,40 +1128,10 @@ public class MetaTableAccessor { cell.getValueOffset(), cell.getValueLength()); } - /** - * Returns the daughter regions by reading the corresponding columns of the catalog table - * Result. - * @param data a Result object from the catalog table scan - * @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split - * parent - */ - public static PairOfSameType getDaughterRegions(Result data) { - RegionInfo splitA = getRegionInfo(data, HConstants.SPLITA_QUALIFIER); - RegionInfo splitB = getRegionInfo(data, HConstants.SPLITB_QUALIFIER); - - return new PairOfSameType<>(splitA, splitB); - } - - /** - * Returns the merge regions by reading the corresponding columns of the catalog table - * Result. - * @param data a Result object from the catalog table scan - * @return a pair of RegionInfo or PairOfSameType(null, null) if the region is not a split - * parent - */ - public static PairOfSameType getMergeRegions(Result data) { - RegionInfo mergeA = getRegionInfo(data, HConstants.MERGEA_QUALIFIER); - RegionInfo mergeB = getRegionInfo(data, HConstants.MERGEB_QUALIFIER); - - return new PairOfSameType<>(mergeA, mergeB); - } - /** * Fetch table state for given table from META table * @param conn connection to use * @param tableName table to fetch state for - * @return state - * @throws IOException */ @Nullable public static TableState getTableState(Connection conn, TableName tableName) @@ -1083,19 +1149,16 @@ public class MetaTableAccessor { * Fetch table states from META table * @param conn connection to use * @return map {tableName -> state} - * @throws IOException */ public static Map getTableStates(Connection conn) throws IOException { final Map states = new LinkedHashMap<>(); - Visitor collector = new Visitor() { - @Override - public boolean visit(Result r) throws IOException { - TableState state = getTableState(r); - if (state != null) - states.put(state.getTableName(), state); - return true; + Visitor collector = r -> { + TableState state = getTableState(r); + if (state != null) { + states.put(state.getTableName(), state); } + return true; }; fullScanTables(conn, collector); return states; @@ -1105,7 +1168,6 @@ public class MetaTableAccessor { * Updates state in META * @param conn connection to use * @param tableName table to look for - * @throws IOException */ public static void updateTableState(Connection conn, TableName tableName, TableState.State actual) throws IOException { @@ -1115,7 +1177,6 @@ public class MetaTableAccessor { /** * Decode table state from META Result. * Should contain cell from HConstants.TABLE_FAMILY - * @param r result * @return null if not found */ @Nullable @@ -1279,8 +1340,7 @@ public class MetaTableAccessor { } /** - * Generates and returns a Delete containing the region info for the catalog - * table + * Generates and returns a Delete containing the region info for the catalog table */ private static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) { if (regionInfo == null) { @@ -1294,7 +1354,7 @@ public class MetaTableAccessor { /** * Adds split daughters to the Put */ - public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB) + private static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB) throws IOException { if (splitA != null) { put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) @@ -1411,28 +1471,6 @@ public class MetaTableAccessor { } } - /** - * Execute the passed mutations against hbase:meta table. - * @param connection connection we're using - * @param mutations Puts and Deletes to execute on hbase:meta - * @throws IOException - */ - public static void mutateMetaTable(final Connection connection, - final List mutations) - throws IOException { - Table t = getMetaHTable(connection); - try { - debugLogMutations(mutations); - t.batch(mutations, null); - } catch (InterruptedException e) { - InterruptedIOException ie = new InterruptedIOException(e.getMessage()); - ie.initCause(e); - throw ie; - } finally { - t.close(); - } - } - private static void addRegionStateToPut(Put put, RegionState.State state) throws IOException { put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) .setRow(put.getRow()) @@ -1458,15 +1496,12 @@ public class MetaTableAccessor { */ public static void addSplitsToParent(Connection connection, RegionInfo regionInfo, RegionInfo splitA, RegionInfo splitB) throws IOException { - Table meta = getMetaHTable(connection); - try { + try (Table meta = getMetaHTable(connection)) { Put put = makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime()); addDaughtersToPut(put, splitA, splitB); meta.put(put); debugLogMutation(put); LOG.debug("Added region {}", regionInfo.getRegionNameAsString()); - } finally { - meta.close(); } } @@ -1504,7 +1539,6 @@ public class MetaTableAccessor { * is CLOSED. * @param connection connection we're using * @param regionInfos region information list - * @param regionReplication * @param ts desired timestamp * @throws IOException if problem connecting or updating meta */ @@ -1528,58 +1562,59 @@ public class MetaTableAccessor { LOG.info("Added {} regions to meta.", puts.size()); } + static Put addMergeRegions(Put put, Collection mergeRegions) throws IOException { + int limit = 10000; // Arbitrary limit. No room in our formatted 'task0000' below for more. + int max = mergeRegions.size(); + if (max > limit) { + // Should never happen!!!!! But just in case. + throw new RuntimeException("Can't merge " + max + " regions in one go; " + limit + + " is upper-limit."); + } + int counter = 0; + for (RegionInfo ri: mergeRegions) { + String qualifier = String.format(HConstants.MERGE_QUALIFIER_PREFIX_STR + "%04d", counter++); + put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY). + setRow(put.getRow()). + setFamily(HConstants.CATALOG_FAMILY). + setQualifier(Bytes.toBytes(qualifier)). + setTimestamp(put.getTimestamp()). + setType(Type.Put). + setValue(RegionInfo.toByteArray(ri)). + build()); + } + return put; + } + /** - * Merge the two regions into one in an atomic operation. Deletes the two merging regions in - * hbase:meta and adds the merged region with the information of two merging regions. + * Merge regions into one in an atomic operation. Deletes the merging regions in + * hbase:meta and adds the merged region. * @param connection connection we're using * @param mergedRegion the merged region - * @param regionA merge parent region A - * @param regionAOpenSeqNum the next open sequence id for region A, used by serial replication. -1 - * if not necessary. - * @param regionB merge parent region B - * @param regionBOpenSeqNum the next open sequence id for region B, used by serial replication. -1 - * if not necessary. + * @param parentSeqNum Parent regions to merge and their next open sequence id used + * by serial replication. Set to -1 if not needed by this table. * @param sn the location of the region */ public static void mergeRegions(Connection connection, RegionInfo mergedRegion, - RegionInfo regionA, long regionAOpenSeqNum, RegionInfo regionB, long regionBOpenSeqNum, - ServerName sn, int regionReplication) throws IOException { + Map parentSeqNum, ServerName sn, int regionReplication) + throws IOException { try (Table meta = getMetaHTable(connection)) { - long time = EnvironmentEdgeManager.currentTime(); + long time = HConstants.LATEST_TIMESTAMP; List mutations = new ArrayList<>(); - - List replicationParents = new ArrayList<>(2); - // Deletes for merging regions - mutations.add(makeDeleteFromRegionInfo(regionA, time)); - if (regionAOpenSeqNum > 0) { - mutations.add(makePutForReplicationBarrier(regionA, regionAOpenSeqNum, time)); - replicationParents.add(regionA); + List replicationParents = new ArrayList<>(); + for (Map.Entry e: parentSeqNum.entrySet()) { + RegionInfo ri = e.getKey(); + long seqNum = e.getValue(); + // Deletes for merging regions + mutations.add(makeDeleteFromRegionInfo(ri, time)); + if (seqNum > 0) { + mutations.add(makePutForReplicationBarrier(ri, seqNum, time)); + replicationParents.add(ri); + } } - mutations.add(makeDeleteFromRegionInfo(regionB, time)); - if (regionBOpenSeqNum > 0) { - mutations.add(makePutForReplicationBarrier(regionB, regionBOpenSeqNum, time)); - replicationParents.add(regionB); - } - // Put for parent Put putOfMerged = makePutFromRegionInfo(mergedRegion, time); - putOfMerged.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) - .setRow(putOfMerged.getRow()) - .setFamily(HConstants.CATALOG_FAMILY) - .setQualifier(HConstants.MERGEA_QUALIFIER) - .setTimestamp(putOfMerged.getTimestamp()) - .setType(Type.Put) - .setValue(RegionInfo.toByteArray(regionA)) - .build()) - .add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) - .setRow(putOfMerged.getRow()) - .setFamily(HConstants.CATALOG_FAMILY) - .setQualifier(HConstants.MERGEB_QUALIFIER) - .setTimestamp(putOfMerged.getTimestamp()) - .setType(Type.Put) - .setValue(RegionInfo.toByteArray(regionB)) - .build()); - // Set initial state to CLOSED + putOfMerged = addMergeRegions(putOfMerged, parentSeqNum.keySet()); + // Set initial state to CLOSED. // NOTE: If initial state is not set to CLOSED then merged region gets added with the // default OFFLINE state. If Master gets restarted after this step, start up sequence of // master tries to assign this offline region. This is followed by re-assignments of the @@ -1594,8 +1629,8 @@ public class MetaTableAccessor { addLocation(putOfMerged, sn, 1, mergedRegion.getReplicaId()); } - // Add empty locations for region replicas of the merged region so that number of replicas can - // be cached whenever the primary region is looked up from meta + // Add empty locations for region replicas of the merged region so that number of replicas + // can be cached whenever the primary region is looked up from meta for (int i = 1; i < regionReplication; i++) { addEmptyLocation(putOfMerged, i); } @@ -1677,7 +1712,7 @@ public class MetaTableAccessor { * Construct PUT for given state * @param state new state */ - public static Put makePutFromTableState(TableState state, long ts) { + private static Put makePutFromTableState(TableState state, long ts) { Put put = new Put(state.getTableName().getName(), ts); put.addColumn(getTableFamily(), getTableStateColumn(), state.convert().toByteArray()); return put; @@ -1706,7 +1741,8 @@ public class MetaTableAccessor { * Performs an atomic multi-mutate operation against the given table. */ private static void multiMutate(Connection connection, final Table table, byte[] row, - final List mutations) throws IOException { + final List mutations) + throws IOException { debugLogMutations(mutations); // TODO: Need rollback!!!! // TODO: Need Retry!!! @@ -1805,7 +1841,6 @@ public class MetaTableAccessor { * Deletes the specified region from META. * @param connection connection we're using * @param regionInfo region to be deleted from META - * @throws IOException */ public static void deleteRegionInfo(Connection connection, RegionInfo regionInfo) throws IOException { @@ -1830,7 +1865,8 @@ public class MetaTableAccessor { * @param connection connection we're using * @param regionsInfo list of regions to be deleted from META */ - public static void deleteRegionInfos(Connection connection, List regionsInfo, long ts) + private static void deleteRegionInfos(Connection connection, List regionsInfo, + long ts) throws IOException { List deletes = new ArrayList<>(regionsInfo.size()); for (RegionInfo hri : regionsInfo) { @@ -1866,21 +1902,29 @@ public class MetaTableAccessor { } /** - * Deletes merge qualifiers for the specified merged region. + * Deletes merge qualifiers for the specified merge region. * @param connection connection we're using - * @param mergedRegion the merged region + * @param mergeRegion the merged region */ - public static void deleteMergeQualifiers(Connection connection, final RegionInfo mergedRegion) + public static void deleteMergeQualifiers(Connection connection, final RegionInfo mergeRegion) throws IOException { - long time = EnvironmentEdgeManager.currentTime(); - Delete delete = new Delete(mergedRegion.getRegionName()); - delete.addColumns(getCatalogFamily(), HConstants.MERGEA_QUALIFIER, time); - delete.addColumns(getCatalogFamily(), HConstants.MERGEB_QUALIFIER, time); + Delete delete = new Delete(mergeRegion.getRegionName()); + // NOTE: We are doing a new hbase:meta read here. + Cell [] cells = getMergeRegionsRaw(connection, mergeRegion.getRegionName()); + if (cells == null || cells.length == 0) { + return; + } + List qualifiers = new ArrayList<>(cells.length); + for (Cell cell : cells) { + byte[] qualifier = CellUtil.cloneQualifier(cell); + qualifiers.add(qualifier); + delete.addColumns(getCatalogFamily(), qualifier, HConstants.LATEST_TIMESTAMP); + } deleteFromMetaTable(connection, delete); - LOG.info("Deleted references in merged region " - + mergedRegion.getRegionNameAsString() + ", qualifier=" - + Bytes.toStringBinary(HConstants.MERGEA_QUALIFIER) + " and qualifier=" - + Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER)); + LOG.info("Deleted merge references in " + mergeRegion.getRegionNameAsString() + + ", deleted qualifiers " + + qualifiers.stream().map(Bytes::toStringBinary). + collect(Collectors.joining(", "))); } public static Put addRegionInfo(final Put p, final RegionInfo hri) @@ -2023,7 +2067,7 @@ public class MetaTableAccessor { private final RegionState.State state; private final List parentRegionNames; - public ReplicationBarrierResult(long[] barriers, State state, List parentRegionNames) { + ReplicationBarrierResult(long[] barriers, State state, List parentRegionNames) { this.barriers = barriers; this.state = state; this.parentRegionNames = parentRegionNames; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java index e8afeada690..f2829e8ec82 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -70,8 +70,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; */ @InterfaceAudience.Public public interface RegionInfo { - public static final RegionInfo UNDEFINED = - RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build(); + RegionInfo UNDEFINED = RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build(); /** * Separator used to demarcate the encodedName in a region name * in the new format. See description on new format above. @@ -141,11 +140,16 @@ public interface RegionInfo { } int replicaDiff = lhs.getReplicaId() - rhs.getReplicaId(); - if (replicaDiff != 0) return replicaDiff; + if (replicaDiff != 0) { + return replicaDiff; + } - if (lhs.isOffline() == rhs.isOffline()) + if (lhs.isOffline() == rhs.isOffline()) { return 0; - if (lhs.isOffline() == true) return -1; + } + if (lhs.isOffline()) { + return -1; + } return 1; }; @@ -224,8 +228,6 @@ public interface RegionInfo { boolean isMetaRegion(); /** - * @param rangeStartKey - * @param rangeEndKey * @return true if the given inclusive range of rows is fully contained * by this region. For example, if the region is foo,a,g and this is * passed ["b","c"] or ["a","c"] it will return true, but if this is passed @@ -235,7 +237,6 @@ public interface RegionInfo { boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey); /** - * @param row * @return true if the given row falls in this region. */ boolean containsRow(byte[] row); @@ -339,9 +340,7 @@ public interface RegionInfo { /** * Gets the start key from the specified region name. - * @param regionName * @return Start key. - * @throws java.io.IOException */ static byte[] getStartKey(final byte[] regionName) throws IOException { return parseRegionName(regionName)[1]; @@ -362,7 +361,6 @@ public interface RegionInfo { } /** - * @param bytes * @return A deserialized {@link RegionInfo} * or null if we failed deserialize or passed bytes null */ @@ -373,9 +371,6 @@ public interface RegionInfo { } /** - * @param bytes - * @param offset - * @param len * @return A deserialized {@link RegionInfo} or null * if we failed deserialize or passed bytes null */ @@ -392,7 +387,6 @@ public interface RegionInfo { /** * @param bytes A pb RegionInfo serialized with a pb magic prefix. * @return A deserialized {@link RegionInfo} - * @throws DeserializationException */ @InterfaceAudience.Private static RegionInfo parseFrom(final byte [] bytes) throws DeserializationException { @@ -405,7 +399,6 @@ public interface RegionInfo { * @param offset starting point in the byte array * @param len length to read on the byte array * @return A deserialized {@link RegionInfo} - * @throws DeserializationException */ @InterfaceAudience.Private static RegionInfo parseFrom(final byte [] bytes, int offset, int len) @@ -426,9 +419,8 @@ public interface RegionInfo { } /** - * Check whether two regions are adjacent - * @param regionA - * @param regionB + * Check whether two regions are adjacent; i.e. lies just before or just + * after in a table. * @return true if two regions are adjacent */ static boolean areAdjacent(RegionInfo regionA, RegionInfo regionB) { @@ -436,20 +428,19 @@ public interface RegionInfo { throw new IllegalArgumentException( "Can't check whether adjacent for null region"); } + if (!regionA.getTable().equals(regionB.getTable())) { + return false; + } RegionInfo a = regionA; RegionInfo b = regionB; if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) { a = regionB; b = regionA; } - if (Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0) { - return true; - } - return false; + return Bytes.equals(a.getEndKey(), b.getStartKey()); } /** - * @param ri * @return This instance serialized as protobuf w/ a magic pb prefix. * @see #parseFrom(byte[]) */ @@ -473,7 +464,6 @@ public interface RegionInfo { /** * Make a region name of passed parameters. - * @param tableName * @param startKey Can be null * @param regionid Region id (Usually timestamp from when region was created). * @param newFormat should we create the region name in the new format @@ -487,7 +477,6 @@ public interface RegionInfo { /** * Make a region name of passed parameters. - * @param tableName * @param startKey Can be null * @param id Region id (Usually timestamp from when region was created). * @param newFormat should we create the region name in the new format @@ -501,10 +490,8 @@ public interface RegionInfo { /** * Make a region name of passed parameters. - * @param tableName * @param startKey Can be null * @param regionid Region id (Usually timestamp from when region was created). - * @param replicaId * @param newFormat should we create the region name in the new format * (such that it contains its encoded name?). * @return Region name made of passed tableName, startKey, id and replicaId @@ -517,7 +504,6 @@ public interface RegionInfo { /** * Make a region name of passed parameters. - * @param tableName * @param startKey Can be null * @param id Region id (Usually timestamp from when region was created). * @param newFormat should we create the region name in the new format @@ -531,10 +517,8 @@ public interface RegionInfo { /** * Make a region name of passed parameters. - * @param tableName * @param startKey Can be null * @param id Region id (Usually timestamp from when region was created). - * @param replicaId * @param newFormat should we create the region name in the new format * @return Region name made of passed tableName, startKey, id and replicaId */ @@ -593,7 +577,7 @@ public interface RegionInfo { b[offset++] = ENC_SEPARATOR; System.arraycopy(md5HashBytes, 0, b, offset, MD5_HEX_LENGTH); offset += MD5_HEX_LENGTH; - b[offset++] = ENC_SEPARATOR; + b[offset] = ENC_SEPARATOR; } return b; @@ -612,9 +596,7 @@ public interface RegionInfo { /** * Separate elements of a regionName. - * @param regionName * @return Array of byte[] containing tableName, startKey and id - * @throws IOException */ static byte [][] parseRegionName(final byte[] regionName) throws IOException { @@ -693,7 +675,6 @@ public interface RegionInfo { * be used to read back the instances. * @param infos RegionInfo objects to serialize * @return This instance serialized as a delimited protobuf w/ a magic pb prefix. - * @throws IOException */ static byte[] toDelimitedByteArray(RegionInfo... infos) throws IOException { byte[][] bytes = new byte[infos.length][]; @@ -715,9 +696,7 @@ public interface RegionInfo { /** * Use this instead of {@link RegionInfo#toByteArray(RegionInfo)} when writing to a stream and you want to use * the pb mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want). - * @param ri * @return This instance serialized as a delimied protobuf w/ a magic pb prefix. - * @throws IOException */ static byte [] toDelimitedByteArray(RegionInfo ri) throws IOException { return ProtobufUtil.toDelimitedByteArray(ProtobufUtil.toRegionInfo(ri)); @@ -727,9 +706,7 @@ public interface RegionInfo { * Parses an RegionInfo instance from the passed in stream. * Presumes the RegionInfo was serialized to the stream with * {@link #toDelimitedByteArray(RegionInfo)}. - * @param in * @return An instance of RegionInfo. - * @throws IOException */ static RegionInfo parseFrom(final DataInputStream in) throws IOException { // I need to be able to move back in the stream if this is not a pb @@ -757,28 +734,23 @@ public interface RegionInfo { * @param offset the start offset into the byte[] buffer * @param length how far we should read into the byte[] buffer * @return All the RegionInfos that are in the byte array. Keeps reading till we hit the end. - * @throws IOException */ static List parseDelimitedFrom(final byte[] bytes, final int offset, final int length) throws IOException { if (bytes == null) { throw new IllegalArgumentException("Can't build an object with empty bytes array"); } - DataInputBuffer in = new DataInputBuffer(); List ris = new ArrayList<>(); - try { + try (DataInputBuffer in = new DataInputBuffer()) { in.reset(bytes, offset, length); while (in.available() > 0) { RegionInfo ri = parseFrom(in); ris.add(ri); } - } finally { - in.close(); } return ris; } - /** * @return True if this is first Region in Table */ @@ -794,10 +766,20 @@ public interface RegionInfo { } /** - * @return True if regions are adjacent, if 'after' next. Does not do tablename compare. + * @return True if region is next, adjacent but 'after' this one. + * @see #isAdjacent(RegionInfo) + * @see #areAdjacent(RegionInfo, RegionInfo) */ default boolean isNext(RegionInfo after) { - return Bytes.equals(getEndKey(), after.getStartKey()); + return getTable().equals(after.getTable()) && Bytes.equals(getEndKey(), after.getStartKey()); + } + + /** + * @return True if region is adjacent, either just before or just after this one. + * @see #isNext(RegionInfo) + */ + default boolean isAdjacent(RegionInfo other) { + return getTable().equals(other.getTable()) && areAdjacent(this, other); } /** @@ -808,11 +790,13 @@ public interface RegionInfo { } /** - * @return True if an overlap in region range. Does not do tablename compare. - * Does not check if other has degenerate range. + * @return True if an overlap in region range. * @see #isDegenerate() */ default boolean isOverlap(RegionInfo other) { + if (!getTable().equals(other.getTable())) { + return false; + } int startKeyCompare = Bytes.compareTo(getStartKey(), other.getStartKey()); if (startKeyCompare == 0) { return true; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index d497c157db1..097e411d6e9 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -527,11 +527,31 @@ public final class HConstants { /** The upper-half split region column qualifier */ public static final byte [] SPLITB_QUALIFIER = Bytes.toBytes("splitB"); - /** The lower-half merge region column qualifier */ - public static final byte[] MERGEA_QUALIFIER = Bytes.toBytes("mergeA"); + /** + * Merge qualifier prefix. + * We used to only allow two regions merge; mergeA and mergeB. + * Now we allow many to merge. Each region to merge will be referenced + * in a column whose qualifier starts with this define. + */ + public static final String MERGE_QUALIFIER_PREFIX_STR = "merge"; + public static final byte [] MERGE_QUALIFIER_PREFIX = + Bytes.toBytes(MERGE_QUALIFIER_PREFIX_STR); - /** The upper-half merge region column qualifier */ - public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes("mergeB"); + /** + * The lower-half merge region column qualifier + * @deprecated Since 2.3.0 and 2.2.1. Not used anymore. Instead we look for + * the {@link #MERGE_QUALIFIER_PREFIX_STR} prefix. + */ + @Deprecated + public static final byte[] MERGEA_QUALIFIER = Bytes.toBytes(MERGE_QUALIFIER_PREFIX_STR + "A"); + + /** + * The upper-half merge region column qualifier + * @deprecated Since 2.3.0 and 2.2.1. Not used anymore. Instead we look for + * the {@link #MERGE_QUALIFIER_PREFIX_STR} prefix. + */ + @Deprecated + public static final byte[] MERGEB_QUALIFIER = Bytes.toBytes(MERGE_QUALIFIER_PREFIX_STR + "B"); /** The catalog family as a string*/ public static final String TABLE_FAMILY_STR = "table"; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java index 6cf6edb733d..e47d9172662 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java @@ -758,6 +758,25 @@ public final class PrivateCellUtil { left.getQualifierLength(), buf, offset, length); } + /** + * Finds if the start of the qualifier part of the Cell matches buf + * @param left the cell with which we need to match the qualifier + * @param startsWith the serialized keyvalue format byte[] + * @return true if the qualifier have same staring characters, false otherwise + */ + public static boolean qualifierStartsWith(final Cell left, final byte[] startsWith) { + if (startsWith == null || startsWith.length == 0) { + throw new IllegalArgumentException("Cannot pass an empty startsWith"); + } + if (left instanceof ByteBufferExtendedCell) { + return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(), + ((ByteBufferExtendedCell) left).getQualifierPosition(), startsWith.length, + startsWith, 0, startsWith.length); + } + return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), + startsWith.length, startsWith, 0, startsWith.length); + } + public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset, final int flength, final byte[] qual, final int qoffset, final int qlength) { if (!matchingFamily(left, fam, foffset, flength)) { diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index d5525f67112..9aa1f0c46a3 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -372,6 +372,8 @@ message GCRegionStateData { required RegionInfo region_info = 1; } +// NOTE: This message is used by GCMergedRegionStateProcedure +// AND GCMultipleMergedRegionStateProcedure. enum GCMergedRegionsState { GC_MERGED_REGIONS_PREPARE = 1; GC_MERGED_REGIONS_PURGE = 2; @@ -379,11 +381,18 @@ enum GCMergedRegionsState { } message GCMergedRegionsStateData { + // Use GCMultipleMergedRegionsStateData instead. + option deprecated = true; required RegionInfo parent_a = 1; required RegionInfo parent_b = 2; required RegionInfo merged_child = 3; } +message GCMultipleMergedRegionsStateData { + repeated RegionInfo parents = 1; + required RegionInfo merged_child = 2; +} + enum PeerModificationState { PRE_PEER_MODIFICATION = 1; UPDATE_PEER_STORAGE = 2; diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java index 1e0f7beb9a0..8e93acf76cb 100644 --- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java +++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RegionsResource.java @@ -27,7 +27,7 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.ResponseBuilder; import javax.ws.rs.core.UriInfo; import java.io.IOException; -import java.util.Map; +import java.util.List; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.rest.model.TableInfoModel; import org.apache.hadoop.hbase.rest.model.TableRegionModel; +import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,13 +79,12 @@ public class RegionsResource extends ResourceBase { TableInfoModel model = new TableInfoModel(tableName.getNameAsString()); Connection connection = ConnectionFactory.createConnection(servlet.getConfiguration()); - @SuppressWarnings("deprecation") - Map regions = MetaTableAccessor - .allTableRegions(connection, tableName); + List> regions = MetaTableAccessor + .getTableRegionsAndLocations(connection, tableName); connection.close(); - for (Map.Entry e: regions.entrySet()) { - RegionInfo hri = e.getKey(); - ServerName addr = e.getValue(); + for (Pair e: regions) { + RegionInfo hri = e.getFirst(); + ServerName addr = e.getSecond(); model.add( new TableRegionModel(tableName.getNameAsString(), hri.getRegionId(), hri.getStartKey(), hri.getEndKey(), addr.getHostAndPort())); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java index 686939016d7..b8d4b09c0f5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeLoadBalancer.java @@ -428,10 +428,10 @@ public class FavoredNodeLoadBalancer extends BaseLoadBalancer implements Favored * keep it simple. */ @Override - public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo regionA, - RegionInfo regionB) throws IOException { + public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo [] mergeParents) + throws IOException { Map> regionFNMap = Maps.newHashMap(); - regionFNMap.put(merged, getFavoredNodes(regionA)); + regionFNMap.put(merged, getFavoredNodes(mergeParents[0])); fnm.updateFavoredNodes(regionFNMap); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java index 95160a6db63..322eb1df0d2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodesPromoter.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -33,6 +33,6 @@ public interface FavoredNodesPromoter { void generateFavoredNodesForDaughter(List servers, RegionInfo parent, RegionInfo hriA, RegionInfo hriB) throws IOException; - void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo hriA, - RegionInfo hriB) throws IOException; + void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo [] mergeParents) + throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java index 1980c8ab348..bfea5a71f6c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Properties; import java.util.TreeMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -50,7 +51,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; -import org.apache.hadoop.hbase.master.assignment.GCMergedRegionsProcedure; +import org.apache.hadoop.hbase.master.assignment.GCMultipleMergedRegionsProcedure; import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; @@ -184,26 +185,15 @@ public class CatalogJanitor extends ScheduledChore { break; } - PairOfSameType p = MetaTableAccessor.getMergeRegions(e.getValue()); - RegionInfo regionA = p.getFirst(); - RegionInfo regionB = p.getSecond(); - if (regionA == null || regionB == null) { - LOG.warn("Unexpected references regionA=" - + (regionA == null ? "null" : regionA.getShortNameToLog()) - + ",regionB=" - + (regionB == null ? "null" : regionB.getShortNameToLog()) - + " in merged region " + e.getKey().getShortNameToLog()); - } else { - if (cleanMergeRegion(e.getKey(), regionA, regionB)) { - gcs++; - } + List parents = MetaTableAccessor.getMergeRegions(e.getValue().rawCells()); + if (parents != null && cleanMergeRegion(e.getKey(), parents)) { + gcs++; } } // Clean split parents Map splitParents = report.splitParents; // Now work on our list of found parents. See if any we can clean up. - // regions whose parents are still around HashSet parentNotCleaned = new HashSet<>(); for (Map.Entry e : splitParents.entrySet()) { if (this.services.isInMaintenanceMode()) { @@ -252,10 +242,10 @@ public class CatalogJanitor extends ScheduledChore { * If merged region no longer holds reference to the merge regions, archive * merge region on hdfs and perform deleting references in hbase:meta * @return true if we delete references in merged region on hbase:meta and archive - * the files on the file system + * the files on the file system */ - private boolean cleanMergeRegion(final RegionInfo mergedRegion, - final RegionInfo regionA, final RegionInfo regionB) throws IOException { + private boolean cleanMergeRegion(final RegionInfo mergedRegion, List parents) + throws IOException { FileSystem fs = this.services.getMasterFileSystem().getFileSystem(); Path rootdir = this.services.getMasterFileSystem().getRootDir(); Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable()); @@ -268,17 +258,19 @@ public class CatalogJanitor extends ScheduledChore { LOG.warn("Merged region does not exist: " + mergedRegion.getEncodedName()); } if (regionFs == null || !regionFs.hasReferences(htd)) { - LOG.debug("Deleting region " + regionA.getShortNameToLog() + " and " - + regionB.getShortNameToLog() - + " from fs because merged region no longer holds references"); + LOG.debug("Deleting parents ({}) from fs; merged child {} no longer holds references", + parents.stream().map(r -> RegionInfo.getShortNameToLog(r)). + collect(Collectors.joining(", ")), + mergedRegion); ProcedureExecutor pe = this.services.getMasterProcedureExecutor(); - pe.submitProcedure(new GCMergedRegionsProcedure(pe.getEnvironment(), - mergedRegion, regionA, regionB)); - // Remove from in-memory states - this.services.getAssignmentManager().getRegionStates().deleteRegion(regionA); - this.services.getAssignmentManager().getRegionStates().deleteRegion(regionB); - this.services.getServerManager().removeRegion(regionA); - this.services.getServerManager().removeRegion(regionB); + pe.submitProcedure(new GCMultipleMergedRegionsProcedure(pe.getEnvironment(), + mergedRegion, parents)); + for (RegionInfo ri: parents) { + // The above scheduled GCMultipleMergedRegionsProcedure does the below. + // Do we need this? + this.services.getAssignmentManager().getRegionStates().deleteRegion(ri); + this.services.getServerManager().removeRegion(ri); + } return true; } return false; @@ -326,11 +318,9 @@ public class CatalogJanitor extends ScheduledChore { */ boolean cleanParent(final RegionInfo parent, Result rowContent) throws IOException { - // Check whether it is a merged region and not clean reference - // No necessary to check MERGEB_QUALIFIER because these two qualifiers will - // be inserted/deleted together - if (rowContent.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null) { - // wait cleaning merge region first + // Check whether it is a merged region and if it is clean of references. + if (MetaTableAccessor.hasMergeRegions(rowContent.rawCells())) { + // Wait until clean of merge parent regions first return false; } // Run checks on each daughter split. @@ -423,28 +413,19 @@ public class CatalogJanitor extends ScheduledChore { } /** - * Checks if the specified region has merge qualifiers, if so, try to clean - * them - * @return true if the specified region doesn't have merge qualifier now + * Checks if the specified region has merge qualifiers, if so, try to clean them. + * @return true if no info:merge* columns; i.e. the specified region doesn't have + * any merge qualifiers. */ public boolean cleanMergeQualifier(final RegionInfo region) throws IOException { - // Get merge regions if it is a merged region and already has merge - // qualifier - Pair mergeRegions = MetaTableAccessor - .getRegionsFromMergeQualifier(this.services.getConnection(), - region.getRegionName()); - if (mergeRegions == null - || (mergeRegions.getFirst() == null && mergeRegions.getSecond() == null)) { + // Get merge regions if it is a merged region and already has merge qualifier + List parents = MetaTableAccessor.getMergeRegions(this.services.getConnection(), + region.getRegionName()); + if (parents == null || parents.isEmpty()) { // It doesn't have merge qualifier, no need to clean return true; } - // It shouldn't happen, we must insert/delete these two qualifiers together - if (mergeRegions.getFirst() == null || mergeRegions.getSecond() == null) { - LOG.error("Merged region " + region.getRegionNameAsString() - + " has only one merge qualifier in META."); - return false; - } - return cleanMergeRegion(region, mergeRegions.getFirst(), mergeRegions.getSecond()); + return cleanMergeRegion(region, parents); } /** @@ -579,7 +560,7 @@ public class CatalogJanitor extends ScheduledChore { if (regionInfo.isSplitParent()) { // splitParent means split and offline. this.report.splitParents.put(regionInfo, r); } - if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null) { + if (MetaTableAccessor.hasMergeRegions(r.rawCells())) { this.report.mergedRegions.put(regionInfo, r); } } @@ -719,7 +700,7 @@ public class CatalogJanitor extends ScheduledChore { try (Table t = connection.getTable(TableName.META_TABLE_NAME)) { Result r = t.get(g); byte [] row = g.getRow(); - row[row.length - 2] <<= ((byte)row[row.length - 2]); + row[row.length - 2] <<= row[row.length - 2]; Put p = new Put(g.getRow()); p.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, r.getValue(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER)); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 0cb4f0804f0..5a7eadb5f10 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -90,7 +90,6 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.exceptions.MergeRegionException; import org.apache.hadoop.hbase.executor.ExecutorType; import org.apache.hadoop.hbase.favored.FavoredNodesManager; import org.apache.hadoop.hbase.favored.FavoredNodesPromoter; @@ -128,6 +127,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler; import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable; import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure; import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure; @@ -1812,40 +1812,20 @@ public class HMaster extends HRegionServer implements MasterServices { public long mergeRegions( final RegionInfo[] regionsToMerge, final boolean forcible, - final long nonceGroup, + final long ng, final long nonce) throws IOException { checkInitialized(); - assert(regionsToMerge.length == 2); - - TableName tableName = regionsToMerge[0].getTable(); - if (tableName == null || regionsToMerge[1].getTable() == null) { - throw new UnknownRegionException ("Can't merge regions without table associated"); - } - - if (!tableName.equals(regionsToMerge[1].getTable())) { - throw new IOException ( - "Cannot merge regions from two different tables " + regionsToMerge[0].getTable() - + " and " + regionsToMerge[1].getTable()); - } - - if (RegionInfo.COMPARATOR.compare(regionsToMerge[0], regionsToMerge[1]) == 0) { - throw new MergeRegionException( - "Cannot merge a region to itself " + regionsToMerge[0] + ", " + regionsToMerge[1]); - } - - return MasterProcedureUtil.submitProcedure( - new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) { + final String mergeRegionsStr = Arrays.stream(regionsToMerge). + map(r -> RegionInfo.getShortNameToLog(r)).collect(Collectors.joining(", ")); + return MasterProcedureUtil.submitProcedure(new NonceProcedureRunnable(this, ng, nonce) { @Override protected void run() throws IOException { getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge); - - LOG.info(getClientIdAuditPrefix() + " Merge regions " + - regionsToMerge[0].getEncodedName() + " and " + regionsToMerge[1].getEncodedName()); - + String aid = getClientIdAuditPrefix(); + LOG.info("{} merge regions {}", aid, mergeRegionsStr); submitProcedure(new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(), - regionsToMerge, forcible)); - + regionsToMerge, forcible)); getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index b28a79d285d..99fd04d5d33 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -750,9 +750,8 @@ public class AssignmentManager implements ServerListener { return new SplitTableRegionProcedure(getProcedureEnvironment(), regionToSplit, splitKey); } - public MergeTableRegionsProcedure createMergeProcedure(final RegionInfo regionToMergeA, - final RegionInfo regionToMergeB) throws IOException { - return new MergeTableRegionsProcedure(getProcedureEnvironment(), regionToMergeA,regionToMergeB); + public MergeTableRegionsProcedure createMergeProcedure(RegionInfo ... ris) throws IOException { + return new MergeTableRegionsProcedure(getProcedureEnvironment(), ris, false); } /** @@ -1543,23 +1542,28 @@ public class AssignmentManager implements ServerListener { } /** - * When called here, the merge has happened. The two merged regions have been + * When called here, the merge has happened. The merged regions have been * unassigned and the above markRegionClosed has been called on each so they have been * disassociated from a hosting Server. The merged region will be open after this call. The - * merged regions are removed from hbase:meta below> Later they are deleted from the filesystem + * merged regions are removed from hbase:meta below. Later they are deleted from the filesystem * by the catalog janitor running against hbase:meta. It notices when the merged region no - * longer holds references to the old regions. + * longer holds references to the old regions (References are deleted after a compaction + * rewrites what the Reference points at but not until the archiver chore runs, are the + * References removed). */ public void markRegionAsMerged(final RegionInfo child, final ServerName serverName, - final RegionInfo mother, final RegionInfo father) throws IOException { + RegionInfo [] mergeParents) + throws IOException { final RegionStateNode node = regionStates.getOrCreateRegionStateNode(child); node.setState(State.MERGED); - regionStates.deleteRegion(mother); - regionStates.deleteRegion(father); - regionStateStore.mergeRegions(child, mother, father, serverName); + for (RegionInfo ri: mergeParents) { + regionStates.deleteRegion(ri); + + } + regionStateStore.mergeRegions(child, mergeParents, serverName); if (shouldAssignFavoredNodes(child)) { ((FavoredNodesPromoter)getBalancer()). - generateFavoredNodesForMergedRegion(child, mother, father); + generateFavoredNodesForMergedRegion(child, mergeParents); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java index 610003df911..a8097e0396f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java @@ -40,8 +40,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.G *

This is a Table Procedure. We take a read lock on the Table. * We do NOT keep a lock for the life of this procedure. The subprocedures * take locks on the Regions they are purging. + * @deprecated 2.3.0 Use {@link GCMultipleMergedRegionsProcedure}. */ @InterfaceAudience.Private +@Deprecated public class GCMergedRegionsProcedure extends AbstractStateMachineTableProcedure { private static final Logger LOG = LoggerFactory.getLogger(GCMergedRegionsProcedure.class); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMultipleMergedRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMultipleMergedRegionsProcedure.java new file mode 100644 index 00000000000..285891eb2b2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMultipleMergedRegionsProcedure.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master.assignment; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMultipleMergedRegionsStateData; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * GC regions that have been Merged. Caller determines if it is GC time. This Procedure does not + * check. This is a Table Procedure. We take a read lock on the Table. We do NOT keep a lock for + * the life of this procedure. The sub-procedures take locks on the Regions they are purging. + * Replaces a Procedure that did two regions only at a time instead doing multiple merges in the + * one go; only difference from the old {@link GCMergedRegionsState} is the serialization; this + * class has a different serialization profile writing out more than just two regions. + */ +@org.apache.yetus.audience.InterfaceAudience.Private +public class GCMultipleMergedRegionsProcedure extends + AbstractStateMachineTableProcedure { + private static final Logger LOG = LoggerFactory.getLogger(GCMultipleMergedRegionsProcedure.class); + private List parents; + private RegionInfo mergedChild; + + public GCMultipleMergedRegionsProcedure(final MasterProcedureEnv env, + final RegionInfo mergedChild, final List parents) { + super(env); + this.parents = parents; + this.mergedChild = mergedChild; + } + + public GCMultipleMergedRegionsProcedure() { + // Required by the Procedure framework to create the procedure on replay + super(); + } + + @Override + public TableOperationType getTableOperationType() { + return TableOperationType.MERGED_REGIONS_GC; + } + + @Override + protected Flow executeFromState(MasterProcedureEnv env, GCMergedRegionsState state) + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { + if (LOG.isTraceEnabled()) { + LOG.trace(this + " execute state=" + state); + } + try { + switch (state) { + case GC_MERGED_REGIONS_PREPARE: + // Nothing to do to prepare. + setNextState(GCMergedRegionsState.GC_MERGED_REGIONS_PURGE); + break; + case GC_MERGED_REGIONS_PURGE: + addChildProcedure(createGCRegionProcedures(env)); + setNextState(GCMergedRegionsState.GC_REGION_EDIT_METADATA); + break; + case GC_REGION_EDIT_METADATA: + MetaTableAccessor.deleteMergeQualifiers(env.getMasterServices().getConnection(), + mergedChild); + return Flow.NO_MORE_STATE; + default: + throw new UnsupportedOperationException(this + " unhandled state=" + state); + } + } catch (IOException ioe) { + // TODO: This is going to spew log? + LOG.warn("Error trying to GC merged regions {}; retrying...", + this.parents.stream().map(r -> RegionInfo.getShortNameToLog(r)). + collect(Collectors.joining(", ")), + ioe); + } + return Flow.HAS_MORE_STATE; + } + + private GCRegionProcedure[] createGCRegionProcedures(final MasterProcedureEnv env) { + GCRegionProcedure [] procs = new GCRegionProcedure[this.parents.size()]; + int index = 0; + for (RegionInfo ri: this.parents) { + GCRegionProcedure proc = new GCRegionProcedure(env, ri); + proc.setOwner(env.getRequestUser().getShortName()); + procs[index++] = proc; + } + return procs; + } + + @Override + protected void rollbackState(MasterProcedureEnv env, GCMergedRegionsState state) + throws IOException, InterruptedException { + // no-op + } + + @Override + protected GCMergedRegionsState getState(int stateId) { + return GCMergedRegionsState.forNumber(stateId); + } + + @Override + protected int getStateId(GCMergedRegionsState state) { + return state.getNumber(); + } + + @Override + protected GCMergedRegionsState getInitialState() { + return GCMergedRegionsState.GC_MERGED_REGIONS_PREPARE; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) + throws IOException { + super.serializeStateData(serializer); + final GCMultipleMergedRegionsStateData.Builder msg = + GCMultipleMergedRegionsStateData.newBuilder(). + addAllParents(this.parents.stream().map(ProtobufUtil::toRegionInfo). + collect(Collectors.toList())). + setMergedChild(ProtobufUtil.toRegionInfo(this.mergedChild)); + serializer.serialize(msg.build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) + throws IOException { + super.deserializeStateData(serializer); + final GCMultipleMergedRegionsStateData msg = + serializer.deserialize(GCMultipleMergedRegionsStateData.class); + this.parents = msg.getParentsList().stream().map(ProtobufUtil::toRegionInfo). + collect(Collectors.toList()); + this.mergedChild = ProtobufUtil.toRegionInfo(msg.getMergedChild()); + } + + @Override + public void toStringClassDetails(StringBuilder sb) { + sb.append(getClass().getSimpleName()); + sb.append(" child="); + sb.append(this.mergedChild.getShortNameToLog()); + sb.append(", parents:"); + sb.append(this.parents.stream().map(r -> RegionInfo.getShortNameToLog(r)). + collect(Collectors.joining(", "))); + } + + @Override + public TableName getTableName() { + return this.mergedChild.getTable(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index ec99a73f6b9..81093163f91 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -28,7 +28,6 @@ import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaMutationAnnotation; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -60,7 +59,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.yetus.audience.InterfaceAudience; @@ -75,149 +73,120 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState; /** - * The procedure to Merge a region in a table. - * This procedure takes an exclusive table lock since it is working over multiple regions. - * It holds the lock for the life of the procedure. - *

Throws exception on construction if determines context hostile to merge (cluster going - * down or master is shutting down or table is disabled).

+ * The procedure to Merge regions in a table. This procedure takes an exclusive table + * lock since it is working over multiple regions. It holds the lock for the life of the procedure. + * Throws exception on construction if determines context hostile to merge (cluster going down or + * master is shutting down or table is disabled). */ @InterfaceAudience.Private public class MergeTableRegionsProcedure extends AbstractStateMachineTableProcedure { private static final Logger LOG = LoggerFactory.getLogger(MergeTableRegionsProcedure.class); - private Boolean traceEnabled; private ServerName regionLocation; + + /** + * Two or more regions to merge, the 'merge parents'. + */ private RegionInfo[] regionsToMerge; + + /** + * The resulting merged region. + */ private RegionInfo mergedRegion; - private boolean forcible; + + private boolean force; public MergeTableRegionsProcedure() { // Required by the Procedure framework to create the procedure on replay } public MergeTableRegionsProcedure(final MasterProcedureEnv env, - final RegionInfo regionToMergeA, final RegionInfo regionToMergeB) throws IOException { - this(env, regionToMergeA, regionToMergeB, false); - } - - public MergeTableRegionsProcedure(final MasterProcedureEnv env, - final RegionInfo regionToMergeA, final RegionInfo regionToMergeB, - final boolean forcible) throws IOException { - this(env, new RegionInfo[] {regionToMergeA, regionToMergeB}, forcible); - } - - public MergeTableRegionsProcedure(final MasterProcedureEnv env, - final RegionInfo[] regionsToMerge, final boolean forcible) + final RegionInfo[] regionsToMerge, final boolean force) throws IOException { super(env); - - // Check daughter regions and make sure that we have valid daughter regions - // before doing the real work. This check calls the super method #checkOnline also. - checkRegionsToMerge(env, regionsToMerge, forcible); - - // WARN: make sure there is no parent region of the two merging regions in - // hbase:meta If exists, fixing up daughters would cause daughter regions(we - // have merged one) online again when we restart master, so we should clear - // the parent region to prevent the above case - // Since HBASE-7721, we don't need fix up daughters any more. so here do nothing + // Check parent regions. Make sure valid before starting work. + // This check calls the super method #checkOnline also. + checkRegionsToMerge(env, regionsToMerge, force); + // Sort the regions going into the merge. + Arrays.sort(regionsToMerge); this.regionsToMerge = regionsToMerge; this.mergedRegion = createMergedRegionInfo(regionsToMerge); + // Preflight depends on mergedRegion being set (at least). preflightChecks(env, true); - this.forcible = forcible; - } - - private static void checkRegionsToMerge(MasterProcedureEnv env, final RegionInfo[] regionsToMerge, - final boolean forcible) throws MergeRegionException { - // For now, we only merge 2 regions. - // It could be extended to more than 2 regions in the future. - if (regionsToMerge == null || regionsToMerge.length != 2) { - throw new MergeRegionException("Expected to merge 2 regions, got: " + - Arrays.toString(regionsToMerge)); - } - - checkRegionsToMerge(env, regionsToMerge[0], regionsToMerge[1], forcible); + this.force = force; } /** - * One time checks. + * @throws MergeRegionException If unable to merge regions for whatever reasons. */ - private static void checkRegionsToMerge(MasterProcedureEnv env, final RegionInfo regionToMergeA, - final RegionInfo regionToMergeB, final boolean forcible) throws MergeRegionException { - if (!regionToMergeA.getTable().equals(regionToMergeB.getTable())) { - throw new MergeRegionException("Can't merge regions from two different tables: " + - regionToMergeA + ", " + regionToMergeB); + private static void checkRegionsToMerge(MasterProcedureEnv env, final RegionInfo[] regions, + final boolean force) throws MergeRegionException { + long count = Arrays.stream(regions).distinct().count(); + if (regions.length != count) { + throw new MergeRegionException("Duplicate regions specified; cannot merge a region to " + + "itself. Passed in " + regions.length + " but only " + count + " unique."); } - - if (regionToMergeA.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID || - regionToMergeB.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { - throw new MergeRegionException("Can't merge non-default replicas"); + if (count < 2) { + throw new MergeRegionException("Need two Regions at least to run a Merge"); } - - try { - checkOnline(env, regionToMergeA); - checkOnline(env, regionToMergeB); - } catch (DoNotRetryRegionException dnrre) { - throw new MergeRegionException(dnrre); - } - - if (!RegionInfo.areAdjacent(regionToMergeA, regionToMergeB)) { - String msg = "Unable to merge non-adjacent regions " + regionToMergeA.getShortNameToLog() + - ", " + regionToMergeB.getShortNameToLog() + " where forcible = " + forcible; - LOG.warn(msg); - if (!forcible) { - throw new MergeRegionException(msg); + RegionInfo previous = null; + for (RegionInfo ri: regions) { + if (previous != null) { + if (!previous.getTable().equals(ri.getTable())) { + String msg = "Can't merge regions from different tables: " + previous + ", " + ri; + LOG.warn(msg); + throw new MergeRegionException(msg); + } + if (!force && !ri.isAdjacent(previous) && !ri.isOverlap(previous)) { + String msg = "Unable to merge non-adjacent or non-overlapping regions " + + previous.getShortNameToLog() + ", " + ri.getShortNameToLog() + " when force=false"; + LOG.warn(msg); + throw new MergeRegionException(msg); + } } + + if (ri.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { + throw new MergeRegionException("Can't merge non-default replicas; " + ri); + } + try { + checkOnline(env, ri); + } catch (DoNotRetryRegionException dnrre) { + throw new MergeRegionException(dnrre); + } + + previous = ri; } } - - private static RegionInfo createMergedRegionInfo(final RegionInfo[] regionsToMerge) { - return createMergedRegionInfo(regionsToMerge[0], regionsToMerge[1]); - } - /** - * Create merged region info through the specified two regions + * Create merged region info by looking at passed in regionsToMerge + * to figure what extremes for start and end keys to use; merged region needs + * to have an extent sufficient to cover all regions-to-merge. */ - private static RegionInfo createMergedRegionInfo(final RegionInfo regionToMergeA, - final RegionInfo regionToMergeB) { - // Choose the smaller as start key - final byte[] startKey; - if (RegionInfo.COMPARATOR.compare(regionToMergeA, regionToMergeB) <= 0) { - startKey = regionToMergeA.getStartKey(); - } else { - startKey = regionToMergeB.getStartKey(); + private static RegionInfo createMergedRegionInfo(final RegionInfo[] regionsToMerge) { + byte [] lowestStartKey = null; + byte [] highestEndKey = null; + long highestRegionId = -1; + for (RegionInfo ri: regionsToMerge) { + if (lowestStartKey == null) { + lowestStartKey = ri.getStartKey(); + } else if (Bytes.compareTo(ri.getStartKey(), lowestStartKey) < 0) { + lowestStartKey = ri.getStartKey(); + } + if (highestEndKey == null) { + highestEndKey = ri.getEndKey(); + } else if (ri.isLast() || Bytes.compareTo(ri.getEndKey(), highestEndKey) > 0) { + highestEndKey = ri.getEndKey(); + } + highestRegionId = ri.getRegionId() > highestRegionId? ri.getRegionId(): highestRegionId; } - - // Choose the bigger as end key - final byte[] endKey; - if (Bytes.equals(regionToMergeA.getEndKey(), HConstants.EMPTY_BYTE_ARRAY) - || (!Bytes.equals(regionToMergeB.getEndKey(), HConstants.EMPTY_BYTE_ARRAY) - && Bytes.compareTo(regionToMergeA.getEndKey(), regionToMergeB.getEndKey()) > 0)) { - endKey = regionToMergeA.getEndKey(); - } else { - endKey = regionToMergeB.getEndKey(); - } - // Merged region is sorted between two merging regions in META - return RegionInfoBuilder.newBuilder(regionToMergeA.getTable()) - .setStartKey(startKey) - .setEndKey(endKey) - .setSplit(false) - .setRegionId(getMergedRegionIdTimestamp(regionToMergeA, regionToMergeB)) - .build(); - } - - private static long getMergedRegionIdTimestamp(final RegionInfo regionToMergeA, - final RegionInfo regionToMergeB) { - long rid = EnvironmentEdgeManager.currentTime(); - // Region Id is a timestamp. Merged region's id can't be less than that of - // merging regions else will insert at wrong location in hbase:meta (See HBASE-710). - if (rid < regionToMergeA.getRegionId() || rid < regionToMergeB.getRegionId()) { - LOG.warn("Clock skew; merging regions id are " + regionToMergeA.getRegionId() - + " and " + regionToMergeB.getRegionId() + ", but current time here is " + rid); - rid = Math.max(regionToMergeA.getRegionId(), regionToMergeB.getRegionId()) + 1; - } - return rid; + return RegionInfoBuilder.newBuilder(regionsToMerge[0].getTable()). + setStartKey(lowestStartKey). + setEndKey(highestEndKey). + setSplit(false). + setRegionId(highestRegionId + 1/*Add one so new merged region is highest*/). + build(); } @Override @@ -289,9 +258,8 @@ public class MergeTableRegionsProcedure throw new UnsupportedOperationException(this + " unhandled state=" + state); } } catch (IOException e) { - String msg = "Error trying to merge regions " + - RegionInfo.getShortNameToLog(regionsToMerge) + " in the table " + getTableName() + - " (in state=" + state + ")"; + String msg = "Error trying to merge " + RegionInfo.getShortNameToLog(regionsToMerge) + + " in " + getTableName() + " (in state=" + state + ")"; if (!isRollbackSupported(state)) { // We reach a state that cannot be rolled back. We just need to keep retrying. LOG.warn(msg, e); @@ -312,9 +280,7 @@ public class MergeTableRegionsProcedure @Override protected void rollbackState(final MasterProcedureEnv env, final MergeTableRegionsState state) throws IOException { - if (isTraceEnabled()) { - LOG.trace(this + " rollback state=" + state); - } + LOG.trace("{} rollback state={}", this, state); try { switch (state) { @@ -398,9 +364,9 @@ public class MergeTableRegionsProcedure MasterProcedureProtos.MergeTableRegionsStateData.newBuilder() .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser())) .setMergedRegionInfo(ProtobufUtil.toRegionInfo(mergedRegion)) - .setForcible(forcible); - for (int i = 0; i < regionsToMerge.length; ++i) { - mergeTableRegionsMsg.addRegionInfo(ProtobufUtil.toRegionInfo(regionsToMerge[i])); + .setForcible(force); + for (RegionInfo ri: regionsToMerge) { + mergeTableRegionsMsg.addRegionInfo(ProtobufUtil.toRegionInfo(ri)); } serializer.serialize(mergeTableRegionsMsg.build()); } @@ -430,8 +396,8 @@ public class MergeTableRegionsProcedure sb.append(getTableName()); sb.append(", regions="); sb.append(RegionInfo.getShortNameToLog(regionsToMerge)); - sb.append(", forcibly="); - sb.append(forcible); + sb.append(", force="); + sb.append(force); } @Override @@ -497,43 +463,14 @@ public class MergeTableRegionsProcedure */ private boolean prepareMergeRegion(final MasterProcedureEnv env) throws IOException { // Fail if we are taking snapshot for the given table - if (env.getMasterServices().getSnapshotManager() - .isTakingSnapshot(regionsToMerge[0].getTable())) { - throw new MergeRegionException( - "Skip merging regions " + RegionInfo.getShortNameToLog(regionsToMerge) + - ", because we are taking snapshot for the table " + regionsToMerge[0].getTable()); + TableName tn = regionsToMerge[0].getTable(); + if (env.getMasterServices().getSnapshotManager().isTakingSnapshot(tn)) { + throw new MergeRegionException("Skip merging regions " + + RegionInfo.getShortNameToLog(regionsToMerge) + ", because we are snapshotting " + tn); } - // Note: the following logic assumes that we only have 2 regions to merge. In the future, - // if we want to extend to more than 2 regions, the code needs to be modified a little bit. - CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor(); - boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]); - if (regionAHasMergeQualifier - || !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) { - String msg = "Skip merging regions " + RegionInfo.getShortNameToLog(regionsToMerge) + - ", because region " - + (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1] - .getEncodedName()) + " has merge qualifier"; - LOG.warn(msg); - throw new MergeRegionException(msg); - } - - RegionStates regionStates = env.getAssignmentManager().getRegionStates(); - RegionState regionStateA = regionStates.getRegionState(regionsToMerge[0].getEncodedName()); - RegionState regionStateB = regionStates.getRegionState(regionsToMerge[1].getEncodedName()); - if (regionStateA == null || regionStateB == null) { - throw new UnknownRegionException( - regionStateA == null ? - regionsToMerge[0].getEncodedName() : regionsToMerge[1].getEncodedName()); - } - - if (!regionStateA.isOpened() || !regionStateB.isOpened()) { - throw new MergeRegionException( - "Unable to merge regions that are not online " + regionStateA + ", " + regionStateB); - } - if (!env.getMasterServices().isSplitOrMergeEnabled(MasterSwitchType.MERGE)) { - String regionsStr = Arrays.deepToString(regionsToMerge); - LOG.warn("merge switch is off! skip merge of " + regionsStr); + String regionsStr = Arrays.deepToString(this.regionsToMerge); + LOG.warn("Merge switch is off! skip merge of " + regionsStr); super.setFailure(getClass().getSimpleName(), new IOException("Merge of " + regionsStr + " failed because merge switch is off")); return false; @@ -557,29 +494,33 @@ public class MergeTableRegionsProcedure .toString()); } - // Ask the remote regionserver if regions are mergeable. If we get an IOE, report it - // along with the failure, so we can see why regions are not mergeable at this time. - IOException mergeableCheckIOE = null; - boolean mergeable = false; - RegionState current = regionStateA; - try { - mergeable = isMergeable(env, current); - } catch (IOException e) { - mergeableCheckIOE = e; - } - if (mergeable && mergeableCheckIOE == null) { - current = regionStateB; - try { - mergeable = isMergeable(env, current); - } catch (IOException e) { - mergeableCheckIOE = e; + CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor(); + RegionStates regionStates = env.getAssignmentManager().getRegionStates(); + for (RegionInfo ri: this.regionsToMerge) { + if (!catalogJanitor.cleanMergeQualifier(ri)) { + String msg = "Skip merging " + RegionInfo.getShortNameToLog(regionsToMerge) + + ", because parent " + RegionInfo.getShortNameToLog(ri) + " has a merge qualifier"; + LOG.warn(msg); + throw new MergeRegionException(msg); + } + RegionState state = regionStates.getRegionState(ri.getEncodedName()); + if (state == null) { + throw new UnknownRegionException("No state for " + RegionInfo.getShortNameToLog(ri)); + } + if (!state.isOpened()) { + throw new MergeRegionException("Unable to merge regions that are not online: " + ri); + } + // Ask the remote regionserver if regions are mergeable. If we get an IOE, report it + // along with the failure, so we can see why regions are not mergeable at this time. + try { + if (!isMergeable(env, state)) { + return false; + } + } catch (IOException e) { + IOException ioe = new IOException(RegionInfo.getShortNameToLog(ri) + " NOT mergeable", e); + super.setFailure(getClass().getSimpleName(), ioe); + return false; } - } - if (!mergeable) { - IOException e = new IOException(current.getRegion().getShortNameToLog() + " NOT mergeable"); - if (mergeableCheckIOE != null) e.initCause(mergeableCheckIOE); - super.setFailure(getClass().getSimpleName(), e); - return false; } // Update region states to Merging @@ -615,8 +556,6 @@ public class MergeTableRegionsProcedure /** * Action after rollback a merge table regions action. - * @param env MasterProcedureEnv - * @throws IOException */ private void postRollBackMergeRegions(final MasterProcedureEnv env) throws IOException { final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); @@ -632,70 +571,88 @@ public class MergeTableRegionsProcedure public void setRegionStateToMerging(final MasterProcedureEnv env) { // Set State.MERGING to regions to be merged RegionStates regionStates = env.getAssignmentManager().getRegionStates(); - regionStates.getRegionStateNode(regionsToMerge[0]).setState(State.MERGING); - regionStates.getRegionStateNode(regionsToMerge[1]).setState(State.MERGING); + for (RegionInfo ri: this.regionsToMerge) { + regionStates.getRegionStateNode(ri).setState(State.MERGING); + } } /** - * Create a merged region - * @param env MasterProcedureEnv + * Create merged region. + * The way the merge works is that we make a 'merges' temporary + * directory in the FIRST parent region to merge (Do not change this without + * also changing the rollback where we look in this FIRST region for the + * merge dir). We then collect here references to all the store files in all + * the parent regions including those of the FIRST parent region into a + * subdirectory, named for the resultant merged region. We then call + * commitMergeRegion. It finds this subdirectory of storefile references + * and moves them under the new merge region (creating the region layout + * as side effect). After assign of the new merge region, we will run a + * compaction. This will undo the references but the reference files remain + * in place until the archiver runs (which it does on a period as a chore + * in the RegionServer that hosts the merge region -- see + * CompactedHFilesDischarger). Once the archiver has moved aside the + * no-longer used references, the merge region no longer has references. + * The catalog janitor will notice when it runs next and it will remove + * the old parent regions. */ private void createMergedRegion(final MasterProcedureEnv env) throws IOException { final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable()); final FileSystem fs = mfs.getFileSystem(); - HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( - env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false); - regionFs.createMergesDir(); + HRegionFileSystem mergeRegionFs = null; + for (RegionInfo ri: this.regionsToMerge) { + HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( + env.getMasterConfiguration(), fs, tabledir, ri, false); + if (mergeRegionFs == null) { + mergeRegionFs = regionFs; + mergeRegionFs.createMergesDir(); + } + mergeStoreFiles(env, regionFs, mergeRegionFs.getMergesDir()); + } + assert mergeRegionFs != null; + mergeRegionFs.commitMergedRegion(mergedRegion); - mergeStoreFiles(env, regionFs, regionFs.getMergesDir()); - HRegionFileSystem regionFs2 = HRegionFileSystem.openRegionFromFileSystem( - env.getMasterConfiguration(), fs, tabledir, regionsToMerge[1], false); - mergeStoreFiles(env, regionFs2, regionFs.getMergesDir()); - - regionFs.commitMergedRegion(mergedRegion); - - //Prepare to create merged regions + // Prepare to create merged regions env.getAssignmentManager().getRegionStates(). getOrCreateRegionStateNode(mergedRegion).setState(State.MERGING_NEW); } /** - * Create reference file(s) of merging regions under the merged directory - * @param env MasterProcedureEnv - * @param regionFs region file system - * @param mergedDir the temp directory of merged region + * Create reference file(s) to parent region hfiles in the mergeDir + * @param regionFs merge parent region file system + * @param mergeDir the temp directory in which we are accumulating references. */ private void mergeStoreFiles(final MasterProcedureEnv env, final HRegionFileSystem regionFs, - final Path mergedDir) throws IOException { + final Path mergeDir) throws IOException { final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); final Configuration conf = env.getMasterConfiguration(); final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName()); - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { String family = hcd.getNameAsString(); final Collection storeFiles = regionFs.getStoreFiles(family); - if (storeFiles != null && storeFiles.size() > 0) { - final CacheConfig cacheConf = new CacheConfig(conf, hcd); - for (StoreFileInfo storeFileInfo: storeFiles) { - // Create reference file(s) of the region in mergedDir + for (StoreFileInfo storeFileInfo : storeFiles) { + // Create reference file(s) to parent region file here in mergedDir. + // As this procedure is running on master, use CacheConfig.DISABLED means + // don't cache any block. regionFs.mergeStoreFile(mergedRegion, family, new HStoreFile(mfs.getFileSystem(), - storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true), - mergedDir); + storeFileInfo, conf, CacheConfig.DISABLED, hcd.getBloomFilterType(), true), + mergeDir); } } } } /** - * Clean up a merged region - * @param env MasterProcedureEnv + * Clean up a merged region on rollback after failure. */ private void cleanupMergedRegion(final MasterProcedureEnv env) throws IOException { final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); - final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable()); + TableName tn = this.regionsToMerge[0].getTable(); + final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), tn); final FileSystem fs = mfs.getFileSystem(); + // See createMergedRegion above where we specify the merge dir as being in the + // FIRST merge parent region. HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false); regionFs.cleanupMergedRegion(mergedRegion); @@ -767,8 +724,8 @@ public class MergeTableRegionsProcedure } private int getRegionReplication(final MasterProcedureEnv env) throws IOException { - final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName()); - return htd.getRegionReplication(); + return env.getMasterServices().getTableDescriptors().get(getTableName()). + getRegionReplication(); } /** @@ -797,9 +754,8 @@ public class MergeTableRegionsProcedure * Add merged region to META and delete original regions. */ private void updateMetaForMergedRegions(final MasterProcedureEnv env) throws IOException { - final ServerName serverName = getServerName(env); - env.getAssignmentManager().markRegionAsMerged(mergedRegion, serverName, - regionsToMerge[0], regionsToMerge[1]); + env.getAssignmentManager().markRegionAsMerged(mergedRegion, getServerName(env), + this.regionsToMerge); } /** @@ -856,23 +812,11 @@ public class MergeTableRegionsProcedure } } - /** - * The procedure could be restarted from a different machine. If the variable is null, we need to - * retrieve it. - * @return traceEnabled - */ - private Boolean isTraceEnabled() { - if (traceEnabled == null) { - traceEnabled = LOG.isTraceEnabled(); - } - return traceEnabled; - } - /** * @return The merged region. Maybe be null if called to early or we failed. */ @VisibleForTesting - public RegionInfo getMergedRegion() { + RegionInfo getMergedRegion() { return this.mergedRegion; } @@ -881,6 +825,6 @@ public class MergeTableRegionsProcedure // Abort means rollback. We can't rollback all steps. HBASE-18018 added abort to all // Procedures. Here is a Procedure that has a PONR and cannot be aborted once it enters this // range of steps; what do we do for these should an operator want to cancel them? HBASE-20022. - return isRollbackSupported(getCurrentState())? super.abort(env): false; + return isRollbackSupported(getCurrentState()) && super.abort(env); } -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index 9023e355f4e..fd484973d99 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java @@ -19,7 +19,10 @@ package org.apache.hadoop.hbase.master.assignment; import java.io.IOException; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; + import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellBuilderFactory; import org.apache.hadoop.hbase.CellBuilderType; @@ -239,17 +242,16 @@ public class RegionStateStore { // ============================================================================================ // Update Region Merging State helpers // ============================================================================================ - public void mergeRegions(RegionInfo child, RegionInfo hriA, RegionInfo hriB, - ServerName serverName) throws IOException { + public void mergeRegions(RegionInfo child, RegionInfo [] parents, ServerName serverName) + throws IOException { TableDescriptor htd = getTableDescriptor(child.getTable()); - long regionAOpenSeqNum = -1L; - long regionBOpenSeqNum = -1L; - if (htd.hasGlobalReplicationScope()) { - regionAOpenSeqNum = getOpenSeqNumForParentRegion(hriA); - regionBOpenSeqNum = getOpenSeqNumForParentRegion(hriB); + boolean globalScope = htd.hasGlobalReplicationScope(); + Map parentSeqNums = new HashMap<>(parents.length); + for (RegionInfo ri: parents) { + parentSeqNums.put(ri, globalScope? getOpenSeqNumForParentRegion(ri): -1); } - MetaTableAccessor.mergeRegions(master.getConnection(), child, hriA, regionAOpenSeqNum, hriB, - regionBOpenSeqNum, serverName, getRegionReplication(htd)); + MetaTableAccessor.mergeRegions(master.getConnection(), child, parentSeqNums, + serverName, getRegionReplication(htd)); } // ============================================================================================ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java index 5a6659e2713..1daa9e71c8b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -543,9 +543,9 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements * keep it simple. */ @Override - public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo regionA, - RegionInfo regionB) throws IOException { - updateFavoredNodesForRegion(merged, fnm.getFavoredNodes(regionA)); + public void generateFavoredNodesForMergedRegion(RegionInfo merged, RegionInfo [] mergeParents) + throws IOException { + updateFavoredNodesForRegion(merged, fnm.getFavoredNodes(mergeParents[0])); } /* diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index b71c250f628..3bbe380ff62 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -326,8 +326,8 @@ public class HRegionFileSystem { if(stat.isDirectory()) { continue; } - if(StoreFileInfo.isReference(stat.getPath())) { - if (LOG.isTraceEnabled()) LOG.trace("Reference " + stat.getPath()); + if (StoreFileInfo.isReference(stat.getPath())) { + LOG.trace("Reference {}", stat.getPath()); return true; } } @@ -758,22 +758,22 @@ public class HRegionFileSystem { } /** - * Create the region merges directory. + * Create the region merges directory, a temporary directory to accumulate + * merges in. * @throws IOException If merges dir already exists or we fail to create it. * @see HRegionFileSystem#cleanupMergesDir() */ public void createMergesDir() throws IOException { Path mergesdir = getMergesDir(); if (fs.exists(mergesdir)) { - LOG.info("The " + mergesdir - + " directory exists. Hence deleting it to recreate it"); + LOG.info("{} directory exists. Deleting it to recreate it anew", mergesdir); if (!fs.delete(mergesdir, true)) { - throw new IOException("Failed deletion of " + mergesdir - + " before creating them again."); + throw new IOException("Failed deletion of " + mergesdir + " before recreate."); } } - if (!mkdirs(fs, conf, mergesdir)) + if (!mkdirs(fs, conf, mergesdir)) { throw new IOException("Failed create of " + mergesdir); + } } /** @@ -813,7 +813,7 @@ public class HRegionFileSystem { public void commitMergedRegion(final RegionInfo mergedRegionInfo) throws IOException { Path regionDir = new Path(this.tableDir, mergedRegionInfo.getEncodedName()); Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo); - // Move the tmp dir in the expected location + // Move the tmp dir to the expected location if (mergedRegionTmpDir != null && fs.exists(mergedRegionTmpDir)) { if (!fs.rename(mergedRegionTmpDir, regionDir)) { throw new IOException("Unable to rename " + mergedRegionTmpDir + " to " diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 932590994e9..73172e9df6b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1912,8 +1912,7 @@ public class HRegionServer extends HasThread implements this.procedureResultReporter = new RemoteProcedureResultReporter(this); // Create the CompactedFileDischarger chore executorService. This chore helps to - // remove the compacted files - // that will no longer be used in reads. + // remove the compacted files that will no longer be used in reads. // Default is 2 mins. The default value for TTLCleaner is 5 mins so we set this to // 2 mins so that compacted files can be archived before the TTLCleaner runs int cleanerInterval = diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index d642f5525bc..050a261170d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1869,21 +1869,12 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat } /** - *

It works by processing a compaction that's been written to disk. - * - *

It is usually invoked at the end of a compaction, but might also be - * invoked at HStore startup, if the prior execution died midway through. - * - *

Moving the compacted TreeMap into place means: - *

-   * 1) Unload all replaced StoreFile, close and collect list to delete.
-   * 2) Compute new store size
-   * 
- * + * Update counts. * @param compactedFiles list of files that were compacted */ @VisibleForTesting protected void completeCompaction(Collection compactedFiles) + // Rename this method! TODO. throws IOException { this.storeSize.set(0L); this.totalUncompressedBytes.set(0L); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java index 779ca4499e0..eb12765c88a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java @@ -389,7 +389,7 @@ public class StoreFileInfo { @Override public String toString() { return this.getPath() + - (isReference() ? "-" + getReferredToFile(this.getPath()) + "-" + reference : ""); + (isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : ""); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index a9be057c971..9be24f98a4b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -2801,13 +2801,14 @@ public class HBaseFsck extends Configured implements Closeable { throw new IOException("Two entries in hbase:meta are same " + previous); } } - PairOfSameType mergeRegions = MetaTableAccessor.getMergeRegions(result); - for (RegionInfo mergeRegion : new RegionInfo[] { - mergeRegions.getFirst(), mergeRegions.getSecond() }) { - if (mergeRegion != null) { - // This region is already been merged - HbckRegionInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName()); - hbInfo.setMerged(true); + List mergeParents = MetaTableAccessor.getMergeRegions(result.rawCells()); + if (mergeParents != null) { + for (RegionInfo mergeRegion : mergeParents) { + if (mergeRegion != null) { + // This region is already being merged + HbckRegionInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName()); + hbInfo.setMerged(true); + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java index 0ccf26e9c24..680a7515337 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -17,12 +17,7 @@ */ package org.apache.hadoop.hbase; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyObject; +import static org.mockito.ArgumentMatchers.anyObject; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; @@ -30,13 +25,17 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Random; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionLocator; @@ -47,6 +46,7 @@ import org.apache.hadoop.hbase.ipc.DelegatingRpcScheduler; import org.apache.hadoop.hbase.ipc.PriorityFunction; import org.apache.hadoop.hbase.ipc.RpcScheduler; import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory; @@ -57,6 +57,9 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -68,7 +71,6 @@ import org.junit.rules.TestName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** * Test {@link org.apache.hadoop.hbase.MetaTableAccessor}. @@ -76,7 +78,6 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; @Category({MiscTests.class, MediumTests.class}) @SuppressWarnings("deprecation") public class TestMetaTableAccessor { - @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestMetaTableAccessor.class); @@ -105,10 +106,35 @@ public class TestMetaTableAccessor { UTIL.shutdownMiniCluster(); } + @Test + public void testGettingMergeRegions() throws IOException { + TableName tn = TableName.valueOf(this.name.getMethodName()); + Put put = new Put(Bytes.toBytes(this.name.getMethodName())); + List ris = new ArrayList<>(); + int limit = 10; + byte [] previous = HConstants.EMPTY_START_ROW; + for (int i = 0; i < limit; i++) { + RegionInfo ri = RegionInfoBuilder.newBuilder(tn). + setStartKey(previous).setEndKey(Bytes.toBytes(i)).build(); + ris.add(ri); + } + put = MetaTableAccessor.addMergeRegions(put, ris); + List cells = put.getFamilyCellMap().get(HConstants.CATALOG_FAMILY); + String previousQualifier = null; + Assert.assertEquals(limit, cells.size()); + for (Cell cell: cells) { + LOG.info(cell.toString()); + String qualifier = Bytes.toString(cell.getQualifierArray()); + Assert.assertTrue(qualifier.startsWith(HConstants.MERGE_QUALIFIER_PREFIX_STR)); + Assert.assertNotEquals(qualifier, previousQualifier); + previousQualifier = qualifier; + } + } + @Test public void testIsMetaWhenAllHealthy() throws InterruptedException { HMaster m = UTIL.getMiniHBaseCluster().getMaster(); - assertTrue(m.waitForMetaOnline()); + Assert.assertTrue(m.waitForMetaOnline()); } @Test @@ -117,7 +143,7 @@ public class TestMetaTableAccessor { int index = UTIL.getMiniHBaseCluster().getServerWithMeta(); HRegionServer rsWithMeta = UTIL.getMiniHBaseCluster().getRegionServer(index); rsWithMeta.abort("TESTING"); - assertTrue(m.waitForMetaOnline()); + Assert.assertTrue(m.waitForMetaOnline()); } /** @@ -162,8 +188,8 @@ public class TestMetaTableAccessor { try { // Make sure reader and writer are working. - assertTrue(reader.isProgressing()); - assertTrue(writer.isProgressing()); + Assert.assertTrue(reader.isProgressing()); + Assert.assertTrue(writer.isProgressing()); // Kill server hosting meta -- twice . See if our reader/writer ride over the // meta moves. They'll need to retry. @@ -182,8 +208,8 @@ public class TestMetaTableAccessor { } } - assertTrue("reader: " + reader.toString(), reader.isProgressing()); - assertTrue("writer: " + writer.toString(), writer.isProgressing()); + Assert.assertTrue("reader: " + reader.toString(), reader.isProgressing()); + Assert.assertTrue("writer: " + writer.toString(), writer.isProgressing()); } catch (IOException e) { throw e; } finally { @@ -194,7 +220,7 @@ public class TestMetaTableAccessor { t.close(); } long exeTime = System.currentTimeMillis() - startTime; - assertTrue("Timeout: test took " + exeTime / 1000 + " sec", exeTime < timeOut); + Assert.assertTrue("Timeout: test took " + exeTime / 1000 + " sec", exeTime < timeOut); } /** @@ -246,31 +272,31 @@ public class TestMetaTableAccessor { abstract void metaTask() throws Throwable; } - @Test public void testGetRegionsFromMetaTable() - throws IOException, InterruptedException { - List regions = - new MetaTableLocator().getMetaRegions(UTIL.getZooKeeperWatcher()); - assertTrue(regions.size() >= 1); - assertTrue(new MetaTableLocator().getMetaRegionsAndLocations( - UTIL.getZooKeeperWatcher()).size() >= 1); + @Test + public void testGetRegionsFromMetaTable() throws IOException, InterruptedException { + MetaTableLocator mtl = new MetaTableLocator(); + List regions = mtl.getMetaRegions(UTIL.getZooKeeperWatcher()); + Assert.assertTrue(regions.size() >= 1); + Assert.assertTrue( + mtl.getMetaRegionsAndLocations(UTIL.getZooKeeperWatcher()).size() >= 1); } @Test public void testTableExists() throws IOException { final TableName tableName = TableName.valueOf(name.getMethodName()); - assertFalse(MetaTableAccessor.tableExists(connection, tableName)); + Assert.assertFalse(MetaTableAccessor.tableExists(connection, tableName)); UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); - assertTrue(MetaTableAccessor.tableExists(connection, tableName)); + Assert.assertTrue(MetaTableAccessor.tableExists(connection, tableName)); Admin admin = UTIL.getAdmin(); admin.disableTable(tableName); admin.deleteTable(tableName); - assertFalse(MetaTableAccessor.tableExists(connection, tableName)); - assertTrue(MetaTableAccessor.tableExists(connection, + Assert.assertFalse(MetaTableAccessor.tableExists(connection, tableName)); + Assert.assertTrue(MetaTableAccessor.tableExists(connection, TableName.META_TABLE_NAME)); UTIL.createTable(tableName, HConstants.CATALOG_FAMILY); - assertTrue(MetaTableAccessor.tableExists(connection, tableName)); + Assert.assertTrue(MetaTableAccessor.tableExists(connection, tableName)); admin.disableTable(tableName); admin.deleteTable(tableName); - assertFalse(MetaTableAccessor.tableExists(connection, tableName)); + Assert.assertFalse(MetaTableAccessor.tableExists(connection, tableName)); } @Test public void testGetRegion() throws IOException, InterruptedException { @@ -279,7 +305,7 @@ public class TestMetaTableAccessor { // Test get on non-existent region. Pair pair = MetaTableAccessor.getRegion(connection, Bytes.toBytes("nonexistent-region")); - assertNull(pair); + Assert.assertNull(pair); LOG.info("Finished " + name); } @@ -302,18 +328,18 @@ public class TestMetaTableAccessor { // Now make sure we only get the regions from 1 of the tables at a time - assertEquals(1, MetaTableAccessor.getTableRegions(connection, tableName).size()); - assertEquals(1, MetaTableAccessor.getTableRegions(connection, greaterName).size()); + Assert.assertEquals(1, MetaTableAccessor.getTableRegions(connection, tableName).size()); + Assert.assertEquals(1, MetaTableAccessor.getTableRegions(connection, greaterName).size()); } private static List testGettingTableRegions(final Connection connection, final TableName name, final int regionCount) throws IOException, InterruptedException { List regions = MetaTableAccessor.getTableRegions(connection, name); - assertEquals(regionCount, regions.size()); + Assert.assertEquals(regionCount, regions.size()); Pair pair = MetaTableAccessor.getRegion(connection, regions.get(0).getRegionName()); - assertEquals(regions.get(0).getEncodedName(), + Assert.assertEquals(regions.get(0).getEncodedName(), pair.getFirst().getEncodedName()); return regions; } @@ -323,24 +349,30 @@ public class TestMetaTableAccessor { throws IOException, InterruptedException { Pair pair = MetaTableAccessor.getRegion(connection, region.getRegionName()); - assertEquals(region.getEncodedName(), + Assert.assertEquals(region.getEncodedName(), pair.getFirst().getEncodedName()); } @Test public void testParseReplicaIdFromServerColumn() { String column1 = HConstants.SERVER_QUALIFIER_STR; - assertEquals(0, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column1))); + Assert.assertEquals(0, + MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column1))); String column2 = column1 + MetaTableAccessor.META_REPLICA_ID_DELIMITER; - assertEquals(-1, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column2))); + Assert.assertEquals(-1, + MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column2))); String column3 = column2 + "00"; - assertEquals(-1, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column3))); + Assert.assertEquals(-1, + MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column3))); String column4 = column3 + "2A"; - assertEquals(42, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column4))); + Assert.assertEquals(42, + MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column4))); String column5 = column4 + "2A"; - assertEquals(-1, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column5))); + Assert.assertEquals(-1, + MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column5))); String column6 = HConstants.STARTCODE_QUALIFIER_STR; - assertEquals(-1, MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column6))); + Assert.assertEquals(-1, + MetaTableAccessor.parseReplicaIdFromServerColumn(Bytes.toBytes(column6))); } @Test @@ -427,14 +459,14 @@ public class TestMetaTableAccessor { long seqNum, int replicaId, boolean checkSeqNum) throws IOException { Get get = new Get(row); Result result = meta.get(get); - assertTrue(Bytes.equals( + Assert.assertTrue(Bytes.equals( result.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(replicaId)), Bytes.toBytes(serverName.getHostAndPort()))); - assertTrue(Bytes.equals( + Assert.assertTrue(Bytes.equals( result.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(replicaId)), Bytes.toBytes(serverName.getStartcode()))); if (checkSeqNum) { - assertTrue(Bytes.equals( + Assert.assertTrue(Bytes.equals( result.getValue(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(replicaId)), Bytes.toBytes(seqNum))); } @@ -448,10 +480,48 @@ public class TestMetaTableAccessor { MetaTableAccessor.getServerColumn(replicaId)); Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(replicaId)); - assertNotNull(serverCell); - assertNotNull(startCodeCell); - assertEquals(0, serverCell.getValueLength()); - assertEquals(0, startCodeCell.getValueLength()); + Assert.assertNotNull(serverCell); + Assert.assertNotNull(startCodeCell); + Assert.assertEquals(0, serverCell.getValueLength()); + Assert.assertEquals(0, startCodeCell.getValueLength()); + } + + @Test + public void testMetaLocationForRegionReplicasIsRemovedAtTableDeletion() throws IOException { + long regionId = System.currentTimeMillis(); + RegionInfo primary = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName())) + .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false) + .setRegionId(regionId).setReplicaId(0).build(); + + Table meta = MetaTableAccessor.getMetaHTable(connection); + try { + List regionInfos = Lists.newArrayList(primary); + MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3); + MetaTableAccessor.removeRegionReplicasFromMeta(Sets.newHashSet(primary.getRegionName()), 1, 2, + connection); + Get get = new Get(primary.getRegionName()); + Result result = meta.get(get); + for (int replicaId = 0; replicaId < 3; replicaId++) { + Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, + MetaTableAccessor.getServerColumn(replicaId)); + Cell startCodeCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, + MetaTableAccessor.getStartCodeColumn(replicaId)); + Cell stateCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, + MetaTableAccessor.getRegionStateColumn(replicaId)); + Cell snCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, + MetaTableAccessor.getServerNameColumn(replicaId)); + if (replicaId == 0) { + Assert.assertNotNull(stateCell); + } else { + Assert.assertNull(serverCell); + Assert.assertNull(startCodeCell); + Assert.assertNull(stateCell); + Assert.assertNull(snCell); + } + } + } finally { + meta.close(); + } } @Test @@ -548,15 +618,21 @@ public class TestMetaTableAccessor { try (Table meta = MetaTableAccessor.getMetaHTable(connection)) { List regionInfos = Lists.newArrayList(parentA, parentB); MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3); - - MetaTableAccessor.mergeRegions(connection, merged, parentA, -1L, parentB, -1L, serverName0, - 3); - + MetaTableAccessor.mergeRegions(connection, merged, getMapOfRegionsToSeqNum(parentA, parentB), + serverName0, 3); assertEmptyMetaLocation(meta, merged.getRegionName(), 1); assertEmptyMetaLocation(meta, merged.getRegionName(), 2); } } + private Map getMapOfRegionsToSeqNum(RegionInfo ... regions) { + Map mids = new HashMap<>(regions.length); + for (RegionInfo region: regions) { + mids.put(region, -1L); + } + return mids; + } + @Test public void testMetaScanner() throws Exception { LOG.info("Starting " + name.getMethodName()); @@ -631,15 +707,15 @@ public class TestMetaTableAccessor { MetaTableAccessor.getStartCodeColumn(0)); Cell seqNumCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(0)); - assertNotNull(serverCell); - assertNotNull(startCodeCell); - assertNotNull(seqNumCell); - assertTrue(serverCell.getValueLength() > 0); - assertTrue(startCodeCell.getValueLength() > 0); - assertTrue(seqNumCell.getValueLength() > 0); - assertEquals(masterSystemTime, serverCell.getTimestamp()); - assertEquals(masterSystemTime, startCodeCell.getTimestamp()); - assertEquals(masterSystemTime, seqNumCell.getTimestamp()); + Assert.assertNotNull(serverCell); + Assert.assertNotNull(startCodeCell); + Assert.assertNotNull(seqNumCell); + Assert.assertTrue(serverCell.getValueLength() > 0); + Assert.assertTrue(startCodeCell.getValueLength() > 0); + Assert.assertTrue(seqNumCell.getValueLength() > 0); + Assert.assertEquals(masterSystemTime, serverCell.getTimestamp()); + Assert.assertEquals(masterSystemTime, startCodeCell.getTimestamp()); + Assert.assertEquals(masterSystemTime, seqNumCell.getTimestamp()); } } @@ -689,16 +765,16 @@ public class TestMetaTableAccessor { Result result = meta.get(get); Cell serverCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(0)); - assertNotNull(serverCell); - assertEquals(serverNameTime, serverCell.getTimestamp()); + Assert.assertNotNull(serverCell); + Assert.assertEquals(serverNameTime, serverCell.getTimestamp()); ManualEnvironmentEdge edge = new ManualEnvironmentEdge(); edge.setValue(masterSystemTime); EnvironmentEdgeManager.injectEdge(edge); try { // now merge the regions, effectively deleting the rows for region a and b. - MetaTableAccessor.mergeRegions(connection, mergedRegionInfo, regionInfoA, -1L, regionInfoB, - -1L, sn, 1); + MetaTableAccessor.mergeRegions(connection, mergedRegionInfo, + getMapOfRegionsToSeqNum(regionInfoA, regionInfoB), sn, 1); } finally { EnvironmentEdgeManager.reset(); } @@ -711,9 +787,9 @@ public class TestMetaTableAccessor { MetaTableAccessor.getStartCodeColumn(0)); Cell seqNumCell = result.getColumnLatestCell(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(0)); - assertNull(serverCell); - assertNull(startCodeCell); - assertNull(seqNumCell); + Assert.assertNull(serverCell); + Assert.assertNull(startCodeCell); + Assert.assertNull(seqNumCell); } } @@ -794,7 +870,7 @@ public class TestMetaTableAccessor { MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, loc.getServerName(), 1); - assertTrue(prevCalls < scheduler.numPriorityCalls); + Assert.assertTrue(prevCalls < scheduler.numPriorityCalls); } } @@ -836,8 +912,8 @@ public class TestMetaTableAccessor { MetaTableAccessor.getServerColumn(splitA.getReplicaId())); Cell startCodeCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(splitA.getReplicaId())); - assertNull(serverCellA); - assertNull(startCodeCellA); + Assert.assertNull(serverCellA); + Assert.assertNull(startCodeCellA); Get get2 = new Get(splitA.getRegionName()); Result resultB = meta.get(get2); @@ -845,13 +921,36 @@ public class TestMetaTableAccessor { MetaTableAccessor.getServerColumn(splitB.getReplicaId())); Cell startCodeCellB = resultB.getColumnLatestCell(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(splitB.getReplicaId())); - assertNull(serverCellB); - assertNull(startCodeCellB); + Assert.assertNull(serverCellB); + Assert.assertNull(startCodeCellB); } finally { if (meta != null) { meta.close(); } } } + + @Test + public void testScanByRegionEncodedNameExistingRegion() throws Exception { + final TableName tableName = TableName.valueOf("testScanByRegionEncodedNameExistingRegion"); + UTIL.createTable(tableName, "cf"); + final List regions = UTIL.getHBaseCluster().getRegions(tableName); + final String encodedName = regions.get(0).getRegionInfo().getEncodedName(); + final Result result = MetaTableAccessor.scanByRegionEncodedName(UTIL.getConnection(), + encodedName); + Assert.assertNotNull(result); + Assert.assertTrue(result.advance()); + final String resultingRowKey = CellUtil.getCellKeyAsString(result.current()); + Assert.assertTrue(resultingRowKey.contains(encodedName)); + UTIL.deleteTable(tableName); + } + + @Test + public void testScanByRegionEncodedNameNonExistingRegion() throws Exception { + final String encodedName = "nonexistingregion"; + final Result result = MetaTableAccessor.scanByRegionEncodedName(UTIL.getConnection(), + encodedName); + Assert.assertNull(result); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java index 9608e5cc7e4..5b851ee1364 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java @@ -101,7 +101,7 @@ public class TestMergeTableRegionsWhileRSCrash { .getMaster().getMasterProcedureExecutor(); List regionInfos = admin.getRegions(TABLE_NAME); MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure( - env, regionInfos.get(0), regionInfos.get(1)); + env, new RegionInfo [] {regionInfos.get(0), regionInfos.get(1)}, false); executor.submitProcedure(mergeTableRegionsProcedure); UTIL.waitFor(30000, () -> executor.getProcedures().stream() .filter(p -> p instanceof UnassignProcedure) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java index 1af9bd0da59..ac1726cff40 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java @@ -89,7 +89,7 @@ public class TestMasterAbortWhileMergingTable { List regionInfos = admin.getRegions(TABLE_NAME); MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure( UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor() - .getEnvironment(), regionInfos.get(0), regionInfos.get(1)); + .getEnvironment(), new RegionInfo [] {regionInfos.get(0), regionInfos.get(1)}, false); long procID = UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor() .submitProcedure(mergeTableRegionsProcedure); mergeCommitArrive.await(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java index e1774cb4ea8..ebded69cfc2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -20,16 +20,22 @@ package org.apache.hadoop.hbase.master.assignment; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.ArrayList; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility; @@ -40,7 +46,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Threads; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -66,10 +72,8 @@ public class TestMergeTableRegionsProcedure { protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); private static final int initialRegionCount = 4; private final static byte[] FAMILY = Bytes.toBytes("FAMILY"); - final static Configuration conf = UTIL.getConfiguration(); private static Admin admin; - private AssignmentManager am; private ProcedureMetrics mergeProcMetrics; private ProcedureMetrics assignProcMetrics; private ProcedureMetrics unassignProcMetrics; @@ -91,7 +95,7 @@ public class TestMergeTableRegionsProcedure { @BeforeClass public static void setupCluster() throws Exception { - setupConf(conf); + setupConf(UTIL.getConfiguration()); UTIL.startMiniCluster(1); admin = UTIL.getHBaseAdmin(); } @@ -115,7 +119,7 @@ public class TestMergeTableRegionsProcedure { // Turn off the meta scanner so it don't remove parent on us. UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false); resetProcExecutorTestingKillFlag(); - am = UTIL.getHBaseCluster().getMaster().getAssignmentManager(); + AssignmentManager am = UTIL.getHBaseCluster().getMaster().getAssignmentManager(); mergeProcMetrics = am.getAssignmentManagerMetrics().getMergeProcMetrics(); assignProcMetrics = am.getAssignmentManagerMetrics().getAssignProcMetrics(); unassignProcMetrics = am.getAssignmentManagerMetrics().getUnassignProcMetrics(); @@ -124,7 +128,7 @@ public class TestMergeTableRegionsProcedure { @After public void tearDown() throws Exception { resetProcExecutorTestingKillFlag(); - for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) { + for (TableDescriptor htd: UTIL.getAdmin().listTables()) { LOG.info("Tear down, remove table=" + htd.getTableName()); UTIL.deleteTable(htd.getTableName()); } @@ -136,41 +140,68 @@ public class TestMergeTableRegionsProcedure { assertTrue("expected executor to be running", procExec.isRunning()); } + private int loadARowPerRegion(final Table t, List ris) + throws IOException { + List puts = new ArrayList<>(); + for (RegionInfo ri: ris) { + Put put = new Put(ri.getStartKey() == null || ri.getStartKey().length == 0? + new byte [] {'a'}: ri.getStartKey()); + put.addColumn(HConstants.CATALOG_FAMILY, HConstants.CATALOG_FAMILY, + HConstants.CATALOG_FAMILY); + puts.add(put); + } + t.put(puts); + return puts.size(); + } + + /** * This tests two region merges */ @Test public void testMergeTwoRegions() throws Exception { final TableName tableName = TableName.valueOf(this.name.getMethodName()); - final ProcedureExecutor procExec = getMasterProcedureExecutor(); + UTIL.createTable(tableName, new byte[][]{HConstants.CATALOG_FAMILY}, + new byte[][]{new byte[]{'b'}, new byte[]{'c'}, new byte[]{'d'}, new byte[]{'e'}}); + testMerge(tableName, 2); + } - List tableRegions = createTable(tableName); - - RegionInfo[] regionsToMerge = new RegionInfo[2]; - regionsToMerge[0] = tableRegions.get(0); - regionsToMerge[1] = tableRegions.get(1); + private void testMerge(TableName tableName, int mergeCount) throws IOException { + List ris = MetaTableAccessor.getTableRegions(UTIL.getConnection(), tableName); + int originalRegionCount = ris.size(); + assertTrue(originalRegionCount > mergeCount); + RegionInfo[] regionsToMerge = ris.subList(0, mergeCount).toArray(new RegionInfo [] {}); + int countOfRowsLoaded = 0; + try (Table table = UTIL.getConnection().getTable(tableName)) { + countOfRowsLoaded = loadARowPerRegion(table, ris); + } + assertEquals(countOfRowsLoaded, UTIL.countRows(tableName)); // collect AM metrics before test collectAssignmentManagerMetrics(); - + final ProcedureExecutor procExec = getMasterProcedureExecutor(); MergeTableRegionsProcedure proc = new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true); long procId = procExec.submitProcedure(proc); ProcedureTestingUtility.waitProcedure(procExec, procId); ProcedureTestingUtility.assertProcNotFailed(procExec, procId); - assertRegionCount(tableName, initialRegionCount - 1); + MetaTableAccessor.fullScanMetaAndPrint(UTIL.getConnection()); + assertEquals(originalRegionCount - mergeCount + 1, + MetaTableAccessor.getTableRegions(UTIL.getConnection(), tableName).size()); assertEquals(mergeSubmittedCount + 1, mergeProcMetrics.getSubmittedCounter().getCount()); assertEquals(mergeFailedCount, mergeProcMetrics.getFailedCounter().getCount()); assertEquals(assignSubmittedCount + 1, assignProcMetrics.getSubmittedCounter().getCount()); assertEquals(assignFailedCount, assignProcMetrics.getFailedCounter().getCount()); - assertEquals(unassignSubmittedCount + 2, unassignProcMetrics.getSubmittedCounter().getCount()); + assertEquals(unassignSubmittedCount + mergeCount, + unassignProcMetrics.getSubmittedCounter().getCount()); assertEquals(unassignFailedCount, unassignProcMetrics.getFailedCounter().getCount()); - Pair pair = - MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(), - proc.getMergedRegion().getRegionName()); - assertTrue(pair.getFirst() != null && pair.getSecond() != null); + // Need to get the references cleaned out. Close of region will move them + // to archive so disable and reopen just to get rid of references to later + // when the catalogjanitor runs, it can do merged region cleanup. + admin.disableTable(tableName); + admin.enableTable(tableName); // Can I purge the merged regions from hbase:meta? Check that all went // well by looking at the merged row up in hbase:meta. It should have no @@ -178,10 +209,24 @@ public class TestMergeTableRegionsProcedure { // the merged regions cleanup. UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true); UTIL.getHBaseCluster().getMaster().getCatalogJanitor().triggerNow(); - while (pair != null && pair.getFirst() != null && pair.getSecond() != null) { - pair = MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(), - proc.getMergedRegion().getRegionName()); + byte [] mergedRegion = proc.getMergedRegion().getRegionName(); + while (ris != null && ris.get(0) != null && ris.get(1) != null) { + ris = MetaTableAccessor.getMergeRegions(UTIL.getConnection(), mergedRegion); + LOG.info("{} {}", Bytes.toStringBinary(mergedRegion), ris); + Threads.sleep(1000); } + assertEquals(countOfRowsLoaded, UTIL.countRows(tableName)); + } + + /** + * This tests ten region merges in one go. + */ + @Test + public void testMergeTenRegions() throws Exception { + final TableName tableName = TableName.valueOf(this.name.getMethodName()); + final ProcedureExecutor procExec = getMasterProcedureExecutor(); + UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY); + testMerge(tableName, 10); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java index 16ad3731594..b1969d1e75f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestModifyTableWhileMerging.java @@ -92,7 +92,7 @@ public class TestModifyTableWhileMerging { List regionInfos = admin.getRegions(TABLE_NAME); MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure( UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor() - .getEnvironment(), regionInfos.get(0), regionInfos.get(1)); + .getEnvironment(), new RegionInfo [] {regionInfos.get(0), regionInfos.get(1)}, false); ModifyTableProcedure modifyTableProcedure = new ModifyTableProcedure(env, tableDescriptor); long procModify = executor.submitProcedure(modifyTableProcedure); UTIL.waitFor(30000, () -> executor.getProcedures().stream() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java index 52ed7ba2f46..2cf9d6457d6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -34,7 +34,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; @@ -62,6 +61,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.PairOfSameType; @@ -213,15 +213,12 @@ public class TestRegionMergeTransactionOnCluster { MASTER.getConnection(), mergedRegionInfo.getRegionName()); // contains merge reference in META - assertTrue(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY, - HConstants.MERGEA_QUALIFIER) != null); - assertTrue(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY, - HConstants.MERGEB_QUALIFIER) != null); + assertTrue(MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells())); // merging regions' directory are in the file system all the same - PairOfSameType p = MetaTableAccessor.getMergeRegions(mergedRegionResult); - RegionInfo regionA = p.getFirst(); - RegionInfo regionB = p.getSecond(); + List p = MetaTableAccessor.getMergeRegions(mergedRegionResult.rawCells()); + RegionInfo regionA = p.get(0); + RegionInfo regionB = p.get(1); FileSystem fs = MASTER.getMasterFileSystem().getFileSystem(); Path rootDir = MASTER.getMasterFileSystem().getRootDir(); @@ -288,11 +285,7 @@ public class TestRegionMergeTransactionOnCluster { mergedRegionResult = MetaTableAccessor.getRegionResult( TEST_UTIL.getConnection(), mergedRegionInfo.getRegionName()); - assertFalse(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY, - HConstants.MERGEA_QUALIFIER) != null); - assertFalse(mergedRegionResult.getValue(HConstants.CATALOG_FAMILY, - HConstants.MERGEB_QUALIFIER) != null); - + assertFalse(MetaTableAccessor.hasMergeRegions(mergedRegionResult.rawCells())); } finally { ADMIN.enableCatalogJanitor(true); } @@ -333,8 +326,9 @@ public class TestRegionMergeTransactionOnCluster { try { // Merge the same region: b and b. - admin.mergeRegionsAsync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true); - fail("A region should not be able to merge with itself, even forcifully"); + FutureUtils + .get(admin.mergeRegionsAsync(b.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), true)); + fail("A region should not be able to merge with itself, even forcfully"); } catch (IOException ie) { assertTrue("Exception should mention regions not online", StringUtils.stringifyException(ie).contains("region to itself")