From commits-return-91211-archive-asf-public=cust-asf.ponee.io@hbase.apache.org Tue Jan 14 17:00:48 2020 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 09C2118061A for ; Tue, 14 Jan 2020 18:00:47 +0100 (CET) Received: (qmail 6988 invoked by uid 500); 14 Jan 2020 17:00:47 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 6979 invoked by uid 99); 14 Jan 2020 17:00:47 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 14 Jan 2020 17:00:47 +0000 Received: by gitbox.apache.org (ASF Mail Server at gitbox.apache.org, from userid 33) id 4145081F11; Tue, 14 Jan 2020 17:00:47 +0000 (UTC) Date: Tue, 14 Jan 2020 17:00:47 +0000 To: "commits@hbase.apache.org" Subject: [hbase] branch branch-2.2 updated: HBASE-23383 [hbck2] `fixHoles` should queue assignment procedures for any regions its fixing (#917) (#1038) MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit Message-ID: <157902124693.4745.16276814211208048558@gitbox.apache.org> From: ndimiduk@apache.org X-Git-Host: gitbox.apache.org X-Git-Repo: hbase X-Git-Refname: refs/heads/branch-2.2 X-Git-Reftype: branch X-Git-Oldrev: 3d63bff56353cb49cb4ee6ee671d2ada6ebc29cc X-Git-Newrev: 244461aefbd8d4193c1f9117f558b0c6c687e1c0 X-Git-Rev: 244461aefbd8d4193c1f9117f558b0c6c687e1c0 X-Git-NotificationType: ref_changed_plus_diff X-Git-Multimail-Version: 1.5.dev Auto-Submitted: auto-generated This is an automated email from the ASF dual-hosted git repository. ndimiduk pushed a commit to branch branch-2.2 in repository https://gitbox.apache.org/repos/asf/hbase.git The following commit(s) were added to refs/heads/branch-2.2 by this push: new 244461a HBASE-23383 [hbck2] `fixHoles` should queue assignment procedures for any regions its fixing (#917) (#1038) 244461a is described below commit 244461aefbd8d4193c1f9117f558b0c6c687e1c0 Author: Nick Dimiduk AuthorDate: Tue Jan 14 09:00:37 2020 -0800 HBASE-23383 [hbck2] `fixHoles` should queue assignment procedures for any regions its fixing (#917) (#1038) The current process for an operator, after fixing holes in meta, is to manually disable and enable the whole table. Let's try to avoid bringing the whole table offline if we can. Have the master attempt to queue up assignment procedures for any new regions it creates. Signed-off-by: stack --- .../org/apache/hadoop/hbase/master/MetaFixer.java | 214 +++++++++++++++------ .../apache/hadoop/hbase/master/TestMetaFixer.java | 49 +++-- 2 files changed, 191 insertions(+), 72 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java index 15b4e88..8f37441 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaFixer.java @@ -21,26 +21,23 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; - -import org.apache.hadoop.conf.Configuration; +import java.util.stream.Collectors; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.exceptions.MergeRegionException; -import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - - import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -56,6 +53,7 @@ class MetaFixer { private static final Logger LOG = LoggerFactory.getLogger(MetaFixer.class); private static final String MAX_MERGE_COUNT_KEY = "hbase.master.metafixer.max.merge.count"; private static final int MAX_MERGE_COUNT_DEFAULT = 10; + private final MasterServices masterServices; /** * Maximum for many regions to merge at a time. @@ -86,75 +84,134 @@ class MetaFixer { * If hole, it papers it over by adding a region in the filesystem and to hbase:meta. * Does not assign. */ - void fixHoles(CatalogJanitor.Report report) throws IOException { - List> holes = report.getHoles(); + void fixHoles(CatalogJanitor.Report report) { + final List> holes = report.getHoles(); if (holes.isEmpty()) { - LOG.debug("No holes."); + LOG.info("CatalogJanitor Report contains no holes to fix. Skipping."); return; } - for (Pair p: holes) { - RegionInfo ri = getHoleCover(p); - if (ri == null) { - continue; - } - Configuration configuration = this.masterServices.getConfiguration(); - HRegion.createRegionDir(configuration, ri, FSUtils.getRootDir(configuration)); - // If an error here, then we'll have a region in the filesystem but not - // in hbase:meta (if the below fails). Should be able to rerun the fix. - // Add to hbase:meta and then update in-memory state so it knows of new - // Region; addRegionToMeta adds region and adds a state column set to CLOSED. - MetaTableAccessor.addRegionToMeta(this.masterServices.getConnection(), ri); - this.masterServices.getAssignmentManager().getRegionStates(). - updateRegionState(ri, RegionState.State.CLOSED); - LOG.info("Fixed hole by adding {} in CLOSED state; region NOT assigned (assign to ONLINE).", - ri); - } + + LOG.info("Identified {} region holes to fix. Detailed fixup progress logged at DEBUG.", + holes.size()); + + final List newRegionInfos = createRegionInfosForHoles(holes); + final List newMetaEntries = createMetaEntries(masterServices, newRegionInfos); + final TransitRegionStateProcedure[] assignProcedures = masterServices + .getAssignmentManager() + .createRoundRobinAssignProcedures(newMetaEntries); + + masterServices.getMasterProcedureExecutor().submitProcedures(assignProcedures); + LOG.info( + "Scheduled {}/{} new regions for assignment.", assignProcedures.length, holes.size()); + } + + /** + * Create a new {@link RegionInfo} corresponding to each provided "hole" pair. + */ + private static List createRegionInfosForHoles( + final List> holes) { + final List newRegionInfos = holes.stream() + .map(MetaFixer::getHoleCover) + .filter(Optional::isPresent) + .map(Optional::get) + .collect(Collectors.toList()); + LOG.debug("Constructed {}/{} RegionInfo descriptors corresponding to identified holes.", + newRegionInfos.size(), holes.size()); + return newRegionInfos; } /** - * @return Calculated RegionInfo that covers the hole hole + * @return Attempts to calculate a new {@link RegionInfo} that covers the region range described + * in {@code hole}. */ - private RegionInfo getHoleCover(Pair hole) { - RegionInfo holeCover = null; - RegionInfo left = hole.getFirst(); - RegionInfo right = hole.getSecond(); + private static Optional getHoleCover(Pair hole) { + final RegionInfo left = hole.getFirst(); + final RegionInfo right = hole.getSecond(); + if (left.getTable().equals(right.getTable())) { // Simple case. if (Bytes.compareTo(left.getEndKey(), right.getStartKey()) >= 0) { - LOG.warn("Skipping hole fix; left-side endKey is not less than right-side startKey; " + - "left=<{}>, right=<{}>", left, right); - return holeCover; - } - holeCover = buildRegionInfo(left.getTable(), left.getEndKey(), right.getStartKey()); - } else { - boolean leftUndefined = left.equals(RegionInfo.UNDEFINED); - boolean rightUnefined = right.equals(RegionInfo.UNDEFINED); - boolean last = left.isLast(); - boolean first = right.isFirst(); - if (leftUndefined && rightUnefined) { - LOG.warn("Skipping hole fix; both the hole left-side and right-side RegionInfos are " + - "UNDEFINED; left=<{}>, right=<{}>", left, right); - return holeCover; - } - if (leftUndefined || last) { - holeCover = buildRegionInfo(right.getTable(), HConstants.EMPTY_START_ROW, - right.getStartKey()); - } else if (rightUnefined || first) { - holeCover = buildRegionInfo(left.getTable(), left.getEndKey(), HConstants.EMPTY_END_ROW); - } else { - LOG.warn("Skipping hole fix; don't know what to do with left=<{}>, right=<{}>", - left, right); - return holeCover; + LOG.warn("Skipping hole fix; left-side endKey is not less than right-side startKey;" + + " left=<{}>, right=<{}>", left, right); + return Optional.empty(); } + return Optional.of(buildRegionInfo(left.getTable(), left.getEndKey(), right.getStartKey())); + } + + final boolean leftUndefined = left.equals(RegionInfo.UNDEFINED); + final boolean rightUndefined = right.equals(RegionInfo.UNDEFINED); + final boolean last = left.isLast(); + final boolean first = right.isFirst(); + if (leftUndefined && rightUndefined) { + LOG.warn("Skipping hole fix; both the hole left-side and right-side RegionInfos are " + + "UNDEFINED; left=<{}>, right=<{}>", left, right); + return Optional.empty(); + } + if (leftUndefined || last) { + return Optional.of( + buildRegionInfo(right.getTable(), HConstants.EMPTY_START_ROW, right.getStartKey())); } - return holeCover; + if (rightUndefined || first) { + return Optional.of( + buildRegionInfo(left.getTable(), left.getEndKey(), HConstants.EMPTY_END_ROW)); + } + LOG.warn("Skipping hole fix; don't know what to do with left=<{}>, right=<{}>", left, right); + return Optional.empty(); } - private RegionInfo buildRegionInfo(TableName tn, byte [] start, byte [] end) { + private static RegionInfo buildRegionInfo(TableName tn, byte [] start, byte [] end) { return RegionInfoBuilder.newBuilder(tn).setStartKey(start).setEndKey(end).build(); } /** + * Create entries in the {@code hbase:meta} for each provided {@link RegionInfo}. Best effort. + * @param masterServices used to connect to {@code hbase:meta} + * @param newRegionInfos the new {@link RegionInfo} entries to add to the filesystem + * @return a list of {@link RegionInfo} entries for which {@code hbase:meta} entries were + * successfully created + */ + private static List createMetaEntries(final MasterServices masterServices, + final List newRegionInfos) { + + final List> addMetaEntriesResults = newRegionInfos.stream() + .map(regionInfo -> { + try { + MetaTableAccessor.addRegionToMeta(masterServices.getConnection(), regionInfo); + masterServices.getAssignmentManager() + .getRegionStates() + .updateRegionState(regionInfo, RegionState.State.CLOSED); + return Either.ofLeft(regionInfo); + } catch (IOException e) { + return Either.ofRight(e); + } + }) + .collect(Collectors.toList()); + final List createMetaEntriesSuccesses = addMetaEntriesResults.stream() + .filter(Either::hasLeft) + .map(Either::getLeft) + .collect(Collectors.toList()); + final List createMetaEntriesFailures = addMetaEntriesResults.stream() + .filter(Either::hasRight) + .map(Either::getRight) + .collect(Collectors.toList()); + LOG.debug("Added {}/{} entries to hbase:meta", + createMetaEntriesSuccesses.size(), newRegionInfos.size()); + + if (!createMetaEntriesFailures.isEmpty()) { + LOG.warn("Failed to create entries in hbase:meta for {}/{} RegionInfo descriptors. First" + + " failure message included; full list of failures with accompanying stack traces is" + + " available at log level DEBUG. message={}", createMetaEntriesFailures.size(), + addMetaEntriesResults.size(), createMetaEntriesFailures.get(0).getMessage()); + if (LOG.isDebugEnabled()) { + createMetaEntriesFailures.forEach( + ioe -> LOG.debug("Attempt to fix region hole in hbase:meta failed.", ioe)); + } + } + + return createMetaEntriesSuccesses; + } + + /** * Fix overlaps noted in CJ consistency report. */ void fixOverlaps(CatalogJanitor.Report report) throws IOException { @@ -244,4 +301,47 @@ class MetaFixer { } return ri.isOverlap(pair.getFirst()) || ri.isOverlap(pair.getSecond()); } + + /** + * A union over {@link L} and {@link R}. + */ + private static class Either { + private final L left; + private final R right; + + public static Either ofLeft(L left) { + return new Either<>(left, null); + } + + public static Either ofRight(R right) { + return new Either<>(null, right); + } + + Either(L left, R right) { + this.left = left; + this.right = right; + } + + public boolean hasLeft() { + return left != null; + } + + public L getLeft() { + if (!hasLeft()) { + throw new IllegalStateException("Either contains no left."); + } + return left; + } + + public boolean hasRight() { + return right != null; + } + + public R getRight() { + if (!hasRight()) { + throw new IllegalStateException("Either contains no right."); + } + return right; + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java index 16768a2..8697143 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixer.java @@ -17,13 +17,13 @@ */ package org.apache.hadoop.hbase.master; +import static org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils.isNotEmpty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; - import java.io.IOException; import java.util.Collections; import java.util.List; - +import java.util.function.BooleanSupplier; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.util.Threads; - import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -70,7 +69,7 @@ public class TestMetaFixer { } @Test - public void testPlugsHoles() throws IOException { + public void testPlugsHoles() throws Exception { TableName tn = TableName.valueOf(this.name.getMethodName()); TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY); List ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn); @@ -96,9 +95,11 @@ public class TestMetaFixer { assertTrue(report.toString(), report.isEmpty()); assertEquals(initialSize, services.getAssignmentManager().getRegionStates().getRegionStates().size()); - // Disable and reenable so the added regions get reassigned. - TEST_UTIL.getAdmin().disableTable(tn); - TEST_UTIL.getAdmin().enableTable(tn); + + // wait for RITs to settle -- those are the fixed regions being assigned -- or until the + // watchdog TestRule terminates the test. + await(50, () -> isNotEmpty(services.getAssignmentManager().getRegionsInTransition())); + ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn); assertEquals(originalCount, ris.size()); } @@ -143,7 +144,7 @@ public class TestMetaFixer { } @Test - public void testOverlap() throws IOException { + public void testOverlap() throws Exception { TableName tn = TableName.valueOf(this.name.getMethodName()); TEST_UTIL.createMultiRegionTable(tn, HConstants.CATALOG_FAMILY); List ris = MetaTableAccessor.getTableRegions(TEST_UTIL.getConnection(), tn); @@ -163,14 +164,32 @@ public class TestMetaFixer { assertEquals(1, MetaFixer.calculateMerges(10, report.getOverlaps()).size()); MetaFixer fixer = new MetaFixer(services); fixer.fixOverlaps(report); - while (true) { - services.getCatalogJanitor().scan(); - report = services.getCatalogJanitor().getLastReport(); - if (report.isEmpty()) { - break; + await(10, () -> { + try { + services.getCatalogJanitor().scan(); + final CatalogJanitor.Report postReport = services.getCatalogJanitor().getLastReport(); + return postReport.isEmpty(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + /** + * Await the successful return of {@code condition}, sleeping {@code sleepMillis} between + * invocations. + */ + private static void await(final long sleepMillis, final BooleanSupplier condition) + throws InterruptedException { + try { + while (!condition.getAsBoolean()) { + Thread.sleep(sleepMillis); + } + } catch (RuntimeException e) { + if (e.getCause() instanceof AssertionError) { + throw (AssertionError) e.getCause(); } - Threads.sleep(10); + throw e; } - assertTrue(report.toString(), report.isEmpty()); } }