Revert "HBASE-19598 Fix TestAssignmentManagerMetrics flaky test"
Pushed prematurely. Revert.
This reverts commit 016cf0c64b
.
This commit is contained in:
parent
5b64de45ed
commit
0732ef5ebf
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"));
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue