Revert "HBASE-19598 Fix TestAssignmentManagerMetrics flaky test"

Pushed prematurely. Revert.

This reverts commit 016cf0c64b.
This commit is contained in:
Michael Stack 2018-02-28 23:39:54 -08:00
parent 5b64de45ed
commit 0732ef5ebf
12 changed files with 94 additions and 91 deletions

View File

@ -327,7 +327,7 @@ public abstract class RpcExecutor {
int failedCount = failedHandlerCount.incrementAndGet();
if (this.handlerFailureThreshhold >= 0
&& failedCount > handlerCount * this.handlerFailureThreshhold) {
String message = "Number of failed RpcServer handler runs exceeded threshold "
String message = "Number of failed RpcServer handler runs exceeded threshhold "
+ this.handlerFailureThreshhold + "; reason: " + StringUtils.stringifyException(e);
if (abortable != null) {
abortable.abort(message, e);

View File

@ -653,12 +653,14 @@ public class HMaster extends HRegionServer implements MasterServices {
}
/**
* Wait here if backup Master. This avoids showing backup masters as regionservers in master
* web UI, or assigning any region to them.
* If configured to put regions on active master,
* wait till a backup master becomes active.
* Otherwise, loop till the server is stopped or aborted.
*/
@Override
protected void waitForMasterActive() {
while (!this.activeMaster && keepLooping()) {
protected void waitForMasterActive(){
boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(conf);
while (!(tablesOnMaster && activeMaster) && !isStopped() && !isAborted()) {
sleeper.sleep();
}
}

View File

@ -884,7 +884,7 @@ public class ServerManager {
if (isClusterShutdown()) {
this.master.stop("Cluster shutdown");
}
LOG.info("RegionServer count=" + count + "; waited=" + slept + "ms," +
LOG.info("Finished waiting on RegionServer count=" + count + "; waited=" + slept + "ms," +
" expected min=" + minToStart + " server(s), max=" + getStrForMax(maxToStart) + " server(s),"+
" master is "+ (this.master.isStopped() ? "stopped.": "running"));
}

View File

@ -260,7 +260,9 @@ public class SplitLogManager {
if (batch.done != batch.installed) {
batch.isDead = true;
SplitLogCounters.tot_mgr_log_split_batch_err.increment();
String msg = "Error or interrupted while splitting WALs in " + logDirs + "; task=" + batch;
LOG.warn("error while splitting logs in " + logDirs + " installed = " + batch.installed
+ " but only " + batch.done + " done");
String msg = "error or interrupted while splitting logs in " + logDirs + " Task = " + batch;
status.abort(msg);
throw new IOException(msg);
}
@ -474,7 +476,7 @@ public class SplitLogManager {
@Override
public String toString() {
return ("installed=" + installed + ", done=" + done + ", error=" + error);
return ("installed = " + installed + " done = " + done + " error = " + error);
}
}

View File

@ -1727,15 +1727,15 @@ public class AssignmentManager implements ServerListener {
}
// TODO: Optimize balancer. pass a RegionPlan?
final HashMap<RegionInfo, ServerName> retainPlan = new HashMap<>();
final List<RegionInfo> userRegionInfos = new ArrayList<>();
final HashMap<RegionInfo, ServerName> retainMap = new HashMap<>();
final List<RegionInfo> userHRIs = new ArrayList<>(regions.size());
// Regions for system tables requiring reassignment
final List<RegionInfo> systemRegionInfos = new ArrayList<>();
final List<RegionInfo> systemHRIs = new ArrayList<>();
for (RegionStateNode regionStateNode: regions.values()) {
boolean sysTable = regionStateNode.isSystemTable();
final List<RegionInfo> hris = sysTable? systemRegionInfos: userRegionInfos;
final List<RegionInfo> hris = sysTable? systemHRIs: userHRIs;
if (regionStateNode.getRegionLocation() != null) {
retainPlan.put(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation());
retainMap.put(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation());
} else {
hris.add(regionStateNode.getRegionInfo());
}
@ -1748,42 +1748,38 @@ public class AssignmentManager implements ServerListener {
for (int i = 0; servers.size() < 1; ++i) {
// Report every fourth time around this loop; try not to flood log.
if (i % 4 == 0) {
// Log every 4th time; we wait 250ms below so means every second.
LOG.warn("No server available; unable to find a location for " + regions.size() +
" regions. waiting...");
LOG.warn("No servers available; cannot place " + regions.size() + " unassigned regions.");
}
if (!isRunning()) {
LOG.debug("Aborting assignment-queue with " + regions.size() + " unassigned");
LOG.debug("Stopped! Dropping assign of " + regions.size() + " queued regions.");
return;
}
Threads.sleep(250);
// Refresh server list.
servers = master.getServerManager().createDestinationServersList();
}
if (!systemRegionInfos.isEmpty()) {
if (!systemHRIs.isEmpty()) {
// System table regions requiring reassignment are present, get region servers
// not available for system table regions. Here we are filtering out any regionservers
// that might be running older versions of the RegionServer; we want system tables on any
// newer servers that may be present. Newer servers means we are probably doing a rolling
// upgrade.
// not available for system table regions
final List<ServerName> excludeServers = getExcludedServersForSystemTable();
List<ServerName> serversForSysTables = servers.stream()
.filter(s -> !excludeServers.contains(s)).collect(Collectors.toList());
if (serversForSysTables.isEmpty()) {
LOG.warn("All servers excluded! Considering all servers!");
LOG.warn("Filtering old server versions and the excluded produced an empty set; " +
"instead considering all candidate servers!");
}
LOG.debug("Candidate servers to host system regions=" + serversForSysTables.size() +
"; totalServersCount=" + servers.size());
processAssignmentPlans(regions, null, systemRegionInfos,
LOG.debug("Processing assignQueue; systemServersCount=" + serversForSysTables.size() +
", allServersCount=" + servers.size());
processAssignmentPlans(regions, null, systemHRIs,
serversForSysTables.isEmpty()? servers: serversForSysTables);
}
processAssignmentPlans(regions, retainPlan, userRegionInfos, servers);
processAssignmentPlans(regions, retainMap, userHRIs, servers);
}
private void processAssignmentPlans(final HashMap<RegionInfo, RegionStateNode> regions,
final HashMap<RegionInfo, ServerName> retain, final List<RegionInfo> hris,
final HashMap<RegionInfo, ServerName> retainMap, final List<RegionInfo> hris,
final List<ServerName> servers) {
boolean isTraceEnabled = LOG.isTraceEnabled();
if (isTraceEnabled) {
@ -1792,15 +1788,15 @@ public class AssignmentManager implements ServerListener {
final LoadBalancer balancer = getBalancer();
// ask the balancer where to place regions
if (retain != null && !retain.isEmpty()) {
if (retainMap != null && !retainMap.isEmpty()) {
if (isTraceEnabled) {
LOG.trace("Retain assign regions=" + retain);
LOG.trace("retain assign regions=" + retainMap);
}
try {
acceptPlan(regions, balancer.retainAssignment(retain, servers));
acceptPlan(regions, balancer.retainAssignment(retainMap, servers));
} catch (HBaseIOException e) {
LOG.warn("unable to retain assignment", e);
addToPendingAssignment(regions, retain.keySet());
addToPendingAssignment(regions, retainMap.keySet());
}
}
@ -1809,12 +1805,12 @@ public class AssignmentManager implements ServerListener {
if (!hris.isEmpty()) {
Collections.sort(hris, RegionInfo.COMPARATOR);
if (isTraceEnabled) {
LOG.trace("Round-robin regions=" + hris);
LOG.trace("round robin regions=" + hris);
}
try {
acceptPlan(regions, balancer.roundRobinAssignment(hris, servers));
} catch (HBaseIOException e) {
LOG.warn("Unable to round-robin assignment", e);
LOG.warn("unable to round-robin assignment", e);
addToPendingAssignment(regions, hris);
}
}
@ -1826,7 +1822,7 @@ public class AssignmentManager implements ServerListener {
final long st = System.currentTimeMillis();
if (plan == null) {
throw new HBaseIOException("Unable to compute plans for " + regions.size() + " regions");
throw new HBaseIOException("unable to compute plans for regions=" + regions.size());
}
if (plan.isEmpty()) return;
@ -1862,9 +1858,8 @@ public class AssignmentManager implements ServerListener {
}
/**
* Get a list of servers that this region can NOT be assigned to.
* For system tables, we must assign them to a server with highest version (rolling upgrade
* scenario).
* Get a list of servers that this region cannot be assigned to.
* For system tables, we must assign them to a server with highest version.
*/
public List<ServerName> getExcludedServersForSystemTable() {
// TODO: This should be a cached list kept by the ServerManager rather than calculated on each

View File

@ -1229,16 +1229,10 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
if (regions == null || regions.isEmpty()) {
return assignments;
}
if (!this.tablesOnMaster) {
// Make sure Master is not in set of possible servers.
if (servers != null && !servers.isEmpty()) {
servers.remove(this.masterServerName);
}
}
int numServers = servers == null? 0: servers.size();
int numServers = servers == null ? 0 : servers.size();
if (numServers == 0) {
LOG.warn("Wanted to round-robin assignment but no server(s) to assign to.");
LOG.warn("Wanted to do round robin assignment but no servers to assign to");
return null;
}

View File

@ -874,15 +874,22 @@ public class HRegionServer extends HasThread implements
}
}
// Watch for snapshots and other procedures. Check we have not been stopped before proceeding.
if (keepLooping()) {
try {
rspmHost = new RegionServerProcedureManagerHost();
rspmHost.loadProcedures(conf);
rspmHost.initialize(this);
} catch (KeeperException e) {
this.abort("Failed setup of RegionServerProcedureManager.", e);
}
// In case colocated master, wait here till it's active.
// So backup masters won't start as regionservers.
// This is to avoid showing backup masters as regionservers
// in master web UI, or assigning any region to them.
waitForMasterActive();
if (isStopped() || isAborted()) {
return; // No need for further initialization
}
// watch for snapshots and other procedures
try {
rspmHost = new RegionServerProcedureManagerHost();
rspmHost.loadProcedures(conf);
rspmHost.initialize(this);
} catch (KeeperException e) {
this.abort("Failed to reach coordination cluster when creating procedure handler.", e);
}
}
@ -923,10 +930,7 @@ public class HRegionServer extends HasThread implements
}
try {
// If we are backup server instance, wait till we become active master before proceeding.
waitForMasterActive();
if (keepLooping()) {
if (!isStopped() && !isAborted()) {
ShutdownHook.install(conf, fs, this, Thread.currentThread());
// Initialize the RegionServerCoprocessorHost now that our ephemeral
// node was created, in case any coprocessors want to use ZooKeeper
@ -2135,7 +2139,7 @@ public class HRegionServer extends HasThread implements
*/
public void stop(final String msg, final boolean force, final User user) {
if (!this.stopped) {
LOG.info("STOPPING server '" + this);
LOG.info("***** STOPPING region server '" + this + "' *****");
if (this.rsHost != null) {
// when forced via abort don't allow CPs to override
try {
@ -2567,7 +2571,7 @@ public class HRegionServer extends HasThread implements
* @return True if we should break loop because cluster is going down or
* this server has been stopped or hdfs has gone bad.
*/
protected boolean keepLooping() {
private boolean keepLooping() {
return !this.stopped && isClusterUp();
}

View File

@ -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,7 +20,6 @@ package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
@ -150,7 +149,10 @@ public class TestClientClusterMetrics {
Assert.assertNotNull(metrics);
// exclude a dead region server
Assert.assertEquals(SLAVES -1, numRs);
Assert.assertEquals(numRs + 1 /*Master*/, metrics.getLiveServerMetrics().size());
// live servers = nums of regionservers
// By default, HMaster don't carry any regions so it won't report its load.
// Hence, it won't be in the server list.
Assert.assertEquals(numRs, metrics.getLiveServerMetrics().size());
Assert.assertTrue(metrics.getRegionCount() > 0);
Assert.assertNotNull(metrics.getDeadServerNames());
Assert.assertEquals(1, metrics.getDeadServerNames().size());

View File

@ -144,7 +144,7 @@ public class TestClientClusterStatus {
// live servers = nums of regionservers
// By default, HMaster don't carry any regions so it won't report its load.
// Hence, it won't be in the server list.
Assert.assertEquals(status.getServers().size(), numRs + 1/*Master*/);
Assert.assertEquals(status.getServers().size(), numRs);
Assert.assertTrue(status.getRegionsCount() > 0);
Assert.assertNotNull(status.getDeadServerNames());
Assert.assertEquals(1, status.getDeadServersSize());

View File

@ -24,21 +24,20 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompatibilityFactory;
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.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -48,6 +47,7 @@ import org.slf4j.LoggerFactory;
import static org.junit.Assert.fail;
@Ignore // Disabled temporarily; reenable
@Category(MediumTests.class)
public class TestAssignmentManagerMetrics {
@ -83,7 +83,10 @@ public class TestAssignmentManagerMetrics {
// set msgInterval to 1 second
conf.setInt("hbase.regionserver.msginterval", msgInterval);
// Set client sync wait timeout to 5sec
// set tablesOnMaster to none
conf.set("hbase.balancer.tablesOnMaster", "none");
// set client sync wait timeout to 5sec
conf.setInt("hbase.client.sync.wait.timeout.msec", 2500);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 2500);
@ -129,25 +132,27 @@ public class TestAssignmentManagerMetrics {
amSource);
// alter table with a non-existing coprocessor
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build();
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TABLENAME).addColumnFamily(hcd).
addCoprocessorWithSpec("hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2").
build();
HTableDescriptor htd = new HTableDescriptor(TABLENAME);
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
htd.addFamily(hcd);
String spec = "hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2";
htd.addCoprocessorWithSpec(spec);
try {
TEST_UTIL.getAdmin().modifyTable(htd);
TEST_UTIL.getAdmin().modifyTable(TABLENAME, htd);
fail("Expected region failed to open");
} catch (IOException e) {
// Expected, the RS will crash and the assignment will spin forever waiting for a RS
// to assign the region.
LOG.info("Expected exception", e);
// expected, the RS will crash and the assignment will spin forever waiting for a RS
// to assign the region. the region will not go to FAILED_OPEN because in this case
// we have just one RS and it will do one retry.
}
// Sleep 3 seconds, wait for doMetrics chore catching up
Thread.sleep(msgInterval * 3);
// Two regions in RIT -- meta and the testRITAssignementManagerMetrics table region.
metricsHelper.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_NAME, 2, amSource);
// Both are over the threshold because no RegionServer to assign to.
metricsHelper.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_OVER_THRESHOLD_NAME, 2,
metricsHelper.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_NAME, 1, amSource);
metricsHelper.assertGauge(MetricsAssignmentManagerSource.RIT_COUNT_OVER_THRESHOLD_NAME, 1,
amSource);
} finally {

View File

@ -51,7 +51,6 @@ import org.slf4j.LoggerFactory;
* It then does kill combinations to make sure the distribution is more than just for startup.
* NOTE: Regions on Master does not work well. See HBASE-19828. Until addressed, disabling this
* test.
* NOTE: System-tables only on Master doesn't work. TODO.
*/
@Ignore
@Category({MediumTests.class})
@ -112,8 +111,8 @@ public class TestRegionsOnMasterOptions {
checkBalance(0, rsCount);
}
@Ignore // Needs a bunch of work. We need to assign meta first and do it ahead of all others.
// This special handling messes up being able to host system tables only on Master w/o hacks.
@Ignore // Fix this. The Master startup doesn't allow Master reporting as a RegionServer, not
// until way late after the Master startup finishes. Needs more work.
@Test
public void testSystemTablesOnMaster() throws Exception {
c.setBoolean(LoadBalancer.TABLES_ON_MASTER, true);

View File

@ -611,13 +611,13 @@ public final class ZKUtil {
logRetrievedMsg(zkw, znode, data, false);
return data;
} catch (KeeperException.NoNodeException e) {
LOG.debug(zkw.prefix("failed to get data of " + znode + " " +
"; does not exist (not an error)"));
LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
"because node does not exist (not an error)"));
return null;
} catch (KeeperException e) {
LOG.debug(zkw.prefix("failed to get data of " + znode + "; " + e.getMessage()));
// Rethrow
throw e;
LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
zkw.keeperException(e);
return null;
}
}