You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2020/05/04 18:37:11 UTC
[hbase] branch branch-2.3 updated: HBASE-24295 [Chaos Monkey]
abstract logging through the class hierarchy
This is an automated email from the ASF dual-hosted git repository.
ndimiduk pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2.3 by this push:
new f6ee316 HBASE-24295 [Chaos Monkey] abstract logging through the class hierarchy
f6ee316 is described below
commit f6ee3163bddd787a8d0c44ffe28adb32d1fc84db
Author: Nick Dimiduk <nd...@apache.org>
AuthorDate: Thu Apr 30 15:00:57 2020 -0700
HBASE-24295 [Chaos Monkey] abstract logging through the class hierarchy
Adds `protected abstract Logger getLogger()` to `Action` so that
implementation's names are logged when actions are performed.
Signed-off-by: stack <st...@apache.org>
Signed-off-by: Jan Hentschel <ja...@ultratendency.com>
---
.../apache/hadoop/hbase/chaos/actions/Action.java | 98 +++++++++++-----------
.../hbase/chaos/actions/AddCPULoadAction.java | 17 ++--
.../hbase/chaos/actions/AddColumnAction.java | 6 +-
.../hbase/chaos/actions/BatchRestartRsAction.java | 20 +++--
.../chaos/actions/ChangeBloomFilterAction.java | 13 +--
.../chaos/actions/ChangeCompressionAction.java | 11 ++-
.../hbase/chaos/actions/ChangeEncodingAction.java | 11 ++-
.../chaos/actions/ChangeSplitPolicyAction.java | 10 ++-
.../hbase/chaos/actions/ChangeVersionsAction.java | 15 ++--
.../hbase/chaos/actions/CompactMobAction.java | 10 ++-
.../actions/CompactRandomRegionOfTableAction.java | 29 ++++---
.../hbase/chaos/actions/CompactTableAction.java | 13 ++-
.../chaos/actions/CorruptDataFilesAction.java | 14 ++--
.../chaos/actions/CorruptPacketsCommandAction.java | 17 ++--
.../chaos/actions/DecreaseMaxHFileSizeAction.java | 12 ++-
.../chaos/actions/DelayPacketsCommandAction.java | 17 ++--
.../hbase/chaos/actions/DeleteDataFilesAction.java | 16 ++--
.../chaos/actions/DumpClusterStatusAction.java | 14 ++--
.../actions/DuplicatePacketsCommandAction.java | 17 ++--
.../hbase/chaos/actions/FillDiskCommandAction.java | 21 +++--
.../actions/FlushRandomRegionOfTableAction.java | 23 ++---
.../hbase/chaos/actions/FlushTableAction.java | 13 +--
.../hbase/chaos/actions/ForceBalancerAction.java | 11 ++-
.../actions/GracefulRollingRestartRsAction.java | 21 +++--
.../chaos/actions/LosePacketsCommandAction.java | 17 ++--
.../MergeRandomAdjacentRegionsOfTableAction.java | 23 ++---
.../actions/MoveRandomRegionOfTableAction.java | 21 +++--
.../chaos/actions/MoveRegionsOfTableAction.java | 34 ++++----
.../hbase/chaos/actions/RemoveColumnAction.java | 11 ++-
.../chaos/actions/ReorderPacketsCommandAction.java | 19 +++--
.../chaos/actions/RestartActionBaseAction.java | 36 ++++----
.../chaos/actions/RestartActiveMasterAction.java | 9 +-
.../chaos/actions/RestartActiveNameNodeAction.java | 12 ++-
.../chaos/actions/RestartRandomDataNodeAction.java | 10 ++-
.../hbase/chaos/actions/RestartRandomRsAction.java | 8 +-
.../actions/RestartRandomRsExceptMetaAction.java | 10 ++-
.../chaos/actions/RestartRandomZKNodeAction.java | 8 +-
.../chaos/actions/RestartRsHoldingMetaAction.java | 11 ++-
.../chaos/actions/RestartRsHoldingTableAction.java | 13 +--
.../chaos/actions/RollingBatchRestartRsAction.java | 27 +++---
.../actions/RollingBatchSuspendResumeRsAction.java | 21 +++--
.../hbase/chaos/actions/SnapshotTableAction.java | 11 ++-
.../chaos/actions/SplitAllRegionOfTableAction.java | 12 +--
.../actions/SplitRandomRegionOfTableAction.java | 23 ++---
.../hbase/chaos/actions/SudoCommandAction.java | 14 ++--
.../chaos/actions/TestChangeSplitPolicyAction.java | 14 ++--
.../hbase/chaos/actions/TruncateTableAction.java | 14 ++--
.../actions/UnbalanceKillAndRebalanceAction.java | 17 ++--
.../chaos/actions/UnbalanceRegionsAction.java | 16 ++--
.../chaos/policies/CompositeSequentialPolicy.java | 4 +-
50 files changed, 499 insertions(+), 365 deletions(-)
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
index c4f87ac..13b67ae 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.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
@@ -50,12 +50,11 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.util.Bytes;
import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* A (possibly mischievous) action that the ChaosMonkey can perform.
*/
-public class Action {
+public abstract class Action {
public static final String KILL_MASTER_TIMEOUT_KEY =
"hbase.chaosmonkey.action.killmastertimeout";
@@ -76,8 +75,6 @@ public class Action {
public static final String START_NAMENODE_TIMEOUT_KEY =
"hbase.chaosmonkey.action.startnamenodetimeout";
- private static final Logger LOG = LoggerFactory.getLogger(Action.class);
-
protected static final long KILL_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
protected static final long KILL_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
@@ -107,12 +104,17 @@ public class Action {
protected long startNameNodeTimeout;
protected boolean skipMetaRS;
+ /**
+ * Retrieve the instance's {@link Logger}, for use throughout the class hierarchy.
+ */
+ protected abstract Logger getLogger();
+
public void init(ActionContext context) throws IOException {
this.context = context;
cluster = context.getHBaseCluster();
initialStatus = cluster.getInitialClusterMetrics();
Collection<ServerName> regionServers = initialStatus.getLiveServerMetrics().keySet();
- initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
+ initialServers = regionServers.toArray(new ServerName[0]);
monkeyProps = context.getMonkeyProps();
if (monkeyProps == null){
@@ -150,12 +152,12 @@ public class Action {
protected ServerName[] getCurrentServers() throws IOException {
ClusterMetrics clusterStatus = cluster.getClusterMetrics();
Collection<ServerName> regionServers = clusterStatus.getLiveServerMetrics().keySet();
- int count = regionServers == null ? 0 : regionServers.size();
+ int count = regionServers.size();
if (count <= 0) {
return new ServerName [] {};
}
ServerName master = clusterStatus.getMasterName();
- Set<ServerName> masters = new HashSet<ServerName>();
+ Set<ServerName> masters = new HashSet<>();
masters.add(master);
masters.addAll(clusterStatus.getBackupMasterNames());
ArrayList<ServerName> tmp = new ArrayList<>(count);
@@ -167,110 +169,110 @@ public class Action {
tmp.remove(metaServer);
}
- return tmp.toArray(new ServerName[tmp.size()]);
+ return tmp.toArray(new ServerName[0]);
}
protected void killMaster(ServerName server) throws IOException {
- LOG.info("Killing master {}", server);
+ getLogger().info("Killing master {}", server);
cluster.killMaster(server);
cluster.waitForMasterToStop(server, killMasterTimeout);
- LOG.info("Killed master " + server);
+ getLogger().info("Killed master " + server);
}
protected void startMaster(ServerName server) throws IOException {
- LOG.info("Starting master {}", server.getHostname());
+ getLogger().info("Starting master {}", server.getHostname());
cluster.startMaster(server.getHostname(), server.getPort());
cluster.waitForActiveAndReadyMaster(startMasterTimeout);
- LOG.info("Started master " + server.getHostname());
+ getLogger().info("Started master " + server.getHostname());
}
protected void stopRs(ServerName server) throws IOException {
- LOG.info("Stopping regionserver {}", server);
+ getLogger().info("Stopping regionserver {}", server);
cluster.stopRegionServer(server);
cluster.waitForRegionServerToStop(server, killRsTimeout);
- LOG.info("Stoppiong regionserver {}. Reported num of rs:{}", server,
- cluster.getClusterMetrics().getLiveServerMetrics().size());
+ getLogger().info("Stoppiong regionserver {}. Reported num of rs:{}", server,
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void suspendRs(ServerName server) throws IOException {
- LOG.info("Suspending regionserver {}", server);
+ getLogger().info("Suspending regionserver {}", server);
cluster.suspendRegionServer(server);
if(!(cluster instanceof MiniHBaseCluster)){
cluster.waitForRegionServerToStop(server, killRsTimeout);
}
- LOG.info("Suspending regionserver {}. Reported num of rs:{}", server,
- cluster.getClusterMetrics().getLiveServerMetrics().size());
+ getLogger().info("Suspending regionserver {}. Reported num of rs:{}", server,
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void resumeRs(ServerName server) throws IOException {
- LOG.info("Resuming regionserver {}", server);
+ getLogger().info("Resuming regionserver {}", server);
cluster.resumeRegionServer(server);
if(!(cluster instanceof MiniHBaseCluster)){
cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
}
- LOG.info("Resuming regionserver {}. Reported num of rs:{}", server,
- cluster.getClusterMetrics().getLiveServerMetrics().size());
+ getLogger().info("Resuming regionserver {}. Reported num of rs:{}", server,
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void killRs(ServerName server) throws IOException {
- LOG.info("Killing regionserver {}", server);
+ getLogger().info("Killing regionserver {}", server);
cluster.killRegionServer(server);
cluster.waitForRegionServerToStop(server, killRsTimeout);
- LOG.info("Killed regionserver {}. Reported num of rs:{}", server,
- cluster.getClusterMetrics().getLiveServerMetrics().size());
+ getLogger().info("Killed regionserver {}. Reported num of rs:{}", server,
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void startRs(ServerName server) throws IOException {
- LOG.info("Starting regionserver {}", server.getAddress());
+ getLogger().info("Starting regionserver {}", server.getAddress());
cluster.startRegionServer(server.getHostname(), server.getPort());
cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
- LOG.info("Started regionserver {}. Reported num of rs:{}", server.getAddress(),
+ getLogger().info("Started regionserver {}. Reported num of rs:{}", server.getAddress(),
cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void killZKNode(ServerName server) throws IOException {
- LOG.info("Killing zookeeper node {}", server);
+ getLogger().info("Killing zookeeper node {}", server);
cluster.killZkNode(server);
cluster.waitForZkNodeToStop(server, killZkNodeTimeout);
- LOG.info("Killed zookeeper node {}. Reported num of rs:{}", server,
+ getLogger().info("Killed zookeeper node {}. Reported num of rs:{}", server,
cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void startZKNode(ServerName server) throws IOException {
- LOG.info("Starting zookeeper node {}", server.getHostname());
+ getLogger().info("Starting zookeeper node {}", server.getHostname());
cluster.startZkNode(server.getHostname(), server.getPort());
cluster.waitForZkNodeToStart(server, startZkNodeTimeout);
- LOG.info("Started zookeeper node {}", server);
+ getLogger().info("Started zookeeper node {}", server);
}
protected void killDataNode(ServerName server) throws IOException {
- LOG.info("Killing datanode {}", server);
+ getLogger().info("Killing datanode {}", server);
cluster.killDataNode(server);
cluster.waitForDataNodeToStop(server, killDataNodeTimeout);
- LOG.info("Killed datanode {}. Reported num of rs:{}", server,
+ getLogger().info("Killed datanode {}. Reported num of rs:{}", server,
cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void startDataNode(ServerName server) throws IOException {
- LOG.info("Starting datanode {}", server.getHostname());
+ getLogger().info("Starting datanode {}", server.getHostname());
cluster.startDataNode(server);
cluster.waitForDataNodeToStart(server, startDataNodeTimeout);
- LOG.info("Started datanode {}", server);
+ getLogger().info("Started datanode {}", server);
}
protected void killNameNode(ServerName server) throws IOException {
- LOG.info("Killing namenode :-{}", server.getHostname());
+ getLogger().info("Killing namenode :-{}", server.getHostname());
cluster.killNameNode(server);
cluster.waitForNameNodeToStop(server, killNameNodeTimeout);
- LOG.info("Killed namenode:{}. Reported num of rs:{}", server,
+ getLogger().info("Killed namenode:{}. Reported num of rs:{}", server,
cluster.getClusterMetrics().getLiveServerMetrics().size());
}
protected void startNameNode(ServerName server) throws IOException {
- LOG.info("Starting Namenode :-{}", server.getHostname());
+ getLogger().info("Starting Namenode :-{}", server.getHostname());
cluster.startNameNode(server);
cluster.waitForNameNodeToStart(server, startNameNodeTimeout);
- LOG.info("Started namenode:{}", server);
+ getLogger().info("Started namenode:{}", server);
}
protected void unbalanceRegions(ClusterMetrics clusterStatus,
List<ServerName> fromServers, List<ServerName> toServers,
@@ -283,7 +285,7 @@ public class Action {
// Ugh.
List<byte[]> regions = new LinkedList<>(serverLoad.getRegionMetrics().keySet());
int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
- LOG.debug("Removing {} regions from {}", victimRegionCount, sn);
+ getLogger().debug("Removing {} regions from {}", victimRegionCount, sn);
for (int i = 0; i < victimRegionCount; ++i) {
int victimIx = RandomUtils.nextInt(0, regions.size());
String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));
@@ -291,8 +293,8 @@ public class Action {
}
}
- LOG.info("Moving {} regions from {} servers to {} different servers", victimRegions.size(),
- fromServers.size(), toServers.size());
+ getLogger().info("Moving {} regions from {} servers to {} different servers",
+ victimRegions.size(), fromServers.size(), toServers.size());
Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
for (byte[] victimRegion : victimRegions) {
// Don't keep moving regions if we're
@@ -311,10 +313,10 @@ public class Action {
try {
result = admin.balancer();
} catch (Exception e) {
- LOG.warn("Got exception while doing balance ", e);
+ getLogger().warn("Got exception while doing balance ", e);
}
if (!result) {
- LOG.error("Balancer didn't succeed");
+ getLogger().error("Balancer didn't succeed");
}
}
@@ -323,7 +325,7 @@ public class Action {
try {
admin.balancerSwitch(onOrOff, synchronous);
} catch (Exception e) {
- LOG.warn("Got exception while switching balance ", e);
+ getLogger().warn("Got exception while switching balance ", e);
}
}
@@ -338,7 +340,8 @@ public class Action {
* @param transform the modification to perform. Callers will have the
* column name as a string and a column family builder available to them
*/
- protected void modifyAllTableColumns(TableName tableName, BiConsumer<String, ColumnFamilyDescriptorBuilder> transform) throws IOException {
+ protected void modifyAllTableColumns(TableName tableName,
+ BiConsumer<String, ColumnFamilyDescriptorBuilder> transform) throws IOException {
HBaseTestingUtility util = this.context.getHBaseIntegrationTestingUtility();
Admin admin = util.getAdmin();
@@ -369,7 +372,8 @@ public class Action {
* @param tableName the table to modify
* @param transform the modification to perform on each column family descriptor builder
*/
- protected void modifyAllTableColumns(TableName tableName, Consumer<ColumnFamilyDescriptorBuilder> transform) throws IOException {
+ protected void modifyAllTableColumns(TableName tableName,
+ Consumer<ColumnFamilyDescriptorBuilder> transform) throws IOException {
modifyAllTableColumns(tableName, (name, cfd) -> transform.accept(cfd));
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddCPULoadAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddCPULoadAction.java
index 9d6437e..da8963b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddCPULoadAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddCPULoadAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.slf4j.Logger;
@@ -29,13 +28,13 @@ import org.slf4j.LoggerFactory;
* Action that adds high cpu load to a random regionserver for a given duration
*/
public class AddCPULoadAction extends SudoCommandAction {
- protected static final Logger LOG = LoggerFactory.getLogger(AddCPULoadAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(AddCPULoadAction.class);
private static final String CPU_LOAD_COMMAND =
"seq 1 %s | xargs -I{} -n 1 -P %s timeout %s dd if=/dev/urandom of=/dev/null bs=1M " +
"iflag=fullblock";
private final long duration;
- private long processes;
+ private final long processes;
/**
* Add high load to cpu
@@ -49,18 +48,22 @@ public class AddCPULoadAction extends SudoCommandAction {
this.processes = processes;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
protected void localPerform() throws IOException {
- LOG.info("Starting to execute AddCPULoadAction");
+ getLogger().info("Starting to execute AddCPULoadAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
String hostname = server.getHostname();
try {
clusterManager.execSudo(hostname, timeout, getCommand());
} catch (IOException ex){
- //This will always happen. We use timeout to kill a continously running process
+ //This will always happen. We use timeout to kill a continuously running process
//after the duration expires
}
- LOG.info("Finished to execute AddCPULoadAction");
+ getLogger().info("Finished to execute AddCPULoadAction");
}
private String getCommand(){
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddColumnAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddColumnAction.java
index a01222a..ff16562 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddColumnAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/AddColumnAction.java
@@ -42,6 +42,10 @@ public class AddColumnAction extends Action {
this.tableName = tableName;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void init(ActionContext context) throws IOException {
super.init(context);
@@ -63,7 +67,7 @@ public class AddColumnAction extends Action {
return;
}
- LOG.debug("Performing action: Adding " + columnDescriptor + " to " + tableName);
+ getLogger().debug("Performing action: Adding " + columnDescriptor + " to " + tableName);
TableDescriptor modifiedTable = TableDescriptorBuilder.newBuilder(tableDescriptor)
.setColumnFamily(columnDescriptor).build();
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java
index 2026a91..9065a7d 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/BatchRestartRsAction.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
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.slf4j.Logger;
@@ -32,17 +31,20 @@ import org.slf4j.LoggerFactory;
*/
public class BatchRestartRsAction extends RestartActionBaseAction {
float ratio; //ratio of regionservers to restart
- private static final Logger LOG =
- LoggerFactory.getLogger(BatchRestartRsAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(BatchRestartRsAction.class);
public BatchRestartRsAction(long sleepTime, float ratio) {
super(sleepTime);
this.ratio = ratio;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info(String.format("Performing action: Batch restarting %d%% of region servers",
+ getLogger().info(String.format("Performing action: Batch restarting %d%% of region servers",
(int)(ratio * 100)));
List<ServerName> selectedServers = PolicyBasedChaosMonkey.selectRandomItems(getCurrentServers(),
ratio);
@@ -55,7 +57,7 @@ public class BatchRestartRsAction extends RestartActionBaseAction {
if (context.isStopping()) {
break;
}
- LOG.info("Killing region server:" + server);
+ getLogger().info("Killing region server:" + server);
cluster.killRegionServer(server);
killedServers.add(server);
}
@@ -64,13 +66,13 @@ public class BatchRestartRsAction extends RestartActionBaseAction {
cluster.waitForRegionServerToStop(server, PolicyBasedChaosMonkey.TIMEOUT);
}
- LOG.info("Killed " + killedServers.size() + " region servers. Reported num of rs:"
+ getLogger().info("Killed " + killedServers.size() + " region servers. Reported num of rs:"
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
sleep(sleepTime);
for (ServerName server : killedServers) {
- LOG.info("Starting region server:" + server.getHostname());
+ getLogger().info("Starting region server:" + server.getHostname());
cluster.startRegionServer(server.getHostname(), server.getPort());
}
@@ -79,7 +81,7 @@ public class BatchRestartRsAction extends RestartActionBaseAction {
server.getPort(),
PolicyBasedChaosMonkey.TIMEOUT);
}
- LOG.info("Started " + killedServers.size() +" region servers. Reported num of rs:"
+ getLogger().info("Started " + killedServers.size() +" region servers. Reported num of rs:"
+ cluster.getClusterMetrics().getLiveServerMetrics().size());
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java
index e76675a..18ea664 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeBloomFilterAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.Random;
-
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.util.BloomFilterUtil;
@@ -44,17 +43,21 @@ public class ChangeBloomFilterAction extends Action {
this.tableName = tableName;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
final Random random = new Random();
final BloomType[] bloomArray = BloomType.values();
final int bloomArraySize = bloomArray.length;
- LOG.info("Performing action: Change bloom filter on all columns of table " + tableName);
+ getLogger().info("Performing action: Change bloom filter on all columns of table " + tableName);
modifyAllTableColumns(tableName, (columnName, columnBuilder) -> {
BloomType bloomType = bloomArray[random.nextInt(bloomArraySize)];
- LOG.debug("Performing action: About to set bloom filter type to "
+ getLogger().debug("Performing action: About to set bloom filter type to "
+ bloomType + " on column " + columnName + " of table " + tableName);
columnBuilder.setBloomFilterType(bloomType);
if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {
@@ -62,6 +65,6 @@ public class ChangeBloomFilterAction extends Action {
}
});
- LOG.debug("Performing action: Just set bloom filter types on table " + tableName);
+ getLogger().debug("Performing action: Just set bloom filter types on table " + tableName);
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java
index 5330284..8828bc8 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeCompressionAction.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,7 +20,6 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.Random;
-
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.io.compress.Compressor;
@@ -40,6 +39,10 @@ public class ChangeCompressionAction extends Action {
this.random = new Random();
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws IOException {
// Possible compression algorithms. If an algorithm is not supported,
@@ -63,13 +66,13 @@ public class ChangeCompressionAction extends Action {
algo.returnCompressor(c);
break;
} catch (Throwable t) {
- LOG.info("Performing action: Changing compression algorithms to " + algo +
+ getLogger().info("Performing action: Changing compression algorithms to " + algo +
" is not supported, pick another one");
}
} while (true);
final Algorithm chosenAlgo = algo; // for use in lambda
- LOG.debug("Performing action: Changing compression algorithms on "
+ getLogger().debug("Performing action: Changing compression algorithms on "
+ tableName.getNameAsString() + " to " + chosenAlgo);
modifyAllTableColumns(tableName, columnFamilyDescriptorBuilder -> {
if (random.nextBoolean()) {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java
index 55a308c..0ff81d9 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeEncodingAction.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,7 +20,6 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.Random;
-
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.slf4j.Logger;
@@ -39,9 +38,13 @@ public class ChangeEncodingAction extends Action {
this.random = new Random();
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws IOException {
- LOG.debug("Performing action: Changing encodings on " + tableName);
+ getLogger().debug("Performing action: Changing encodings on " + tableName);
// possible DataBlockEncoding id's
final int[] possibleIds = {0, 2, 3, 4, 6};
@@ -49,7 +52,7 @@ public class ChangeEncodingAction extends Action {
short id = (short) possibleIds[random.nextInt(possibleIds.length)];
DataBlockEncoding encoding = DataBlockEncoding.getEncodingById(id);
columnBuilder.setDataBlockEncoding(encoding);
- LOG.debug("Set encoding of column family " + columnName + " to: " + encoding);
+ getLogger().debug("Set encoding of column family " + columnName + " to: " + encoding);
});
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java
index 72ff50f..14a11ee 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeSplitPolicyAction.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
@@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.Random;
-
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
@@ -46,18 +45,21 @@ public class ChangeSplitPolicyAction extends Action {
this.random = new Random();
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
Admin admin = util.getAdmin();
- LOG.info("Performing action: Change split policy of table " + tableName);
+ getLogger().info("Performing action: Change split policy of table " + tableName);
TableDescriptor tableDescriptor = admin.getDescriptor(tableName);
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor);
String chosenPolicy = possiblePolicies[random.nextInt(possiblePolicies.length)];
builder.setRegionSplitPolicyClassName(chosenPolicy);
- LOG.info("Changing " + tableName + " split policy to " + chosenPolicy);
+ getLogger().info("Changing " + tableName + " split policy to " + chosenPolicy);
admin.modifyTable(builder.build());
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java
index 18dba94..1228db1 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ChangeVersionsAction.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,7 +20,6 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.Random;
-
import org.apache.hadoop.hbase.TableName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -31,24 +30,28 @@ import org.slf4j.LoggerFactory;
* Always keeps at least 1 as the number of versions.
*/
public class ChangeVersionsAction extends Action {
- private final TableName tableName;
private static final Logger LOG = LoggerFactory.getLogger(ChangeVersionsAction.class);
+ private final TableName tableName;
- private Random random;
+ private final Random random;
public ChangeVersionsAction(TableName tableName) {
this.tableName = tableName;
this.random = new Random();
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws IOException {
final int versions = random.nextInt(3) + 1;
- LOG.debug("Performing action: Changing versions on " + tableName + " to " + versions);
+ getLogger().debug("Performing action: Changing versions on " + tableName + " to " + versions);
modifyAllTableColumns(tableName, columnBuilder -> {
columnBuilder.setMinVersions(versions).setMaxVersions(versions);
});
- LOG.debug("Performing action: Just changed versions on " + tableName);
+ getLogger().debug("Performing action: Just changed versions on " + tableName);
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactMobAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactMobAction.java
index 6ac71d5..889b3fb 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactMobAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactMobAction.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
@@ -46,6 +46,10 @@ public class CompactMobAction extends Action {
this.sleepTime = sleepTime;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
@@ -57,7 +61,7 @@ public class CompactMobAction extends Action {
return;
}
- LOG.info("Performing action: Compact mob of table " + tableName + ", major=" + major);
+ getLogger().info("Performing action: Compact mob of table " + tableName + ", major=" + major);
try {
if (major) {
admin.majorCompact(tableName, CompactType.MOB);
@@ -65,7 +69,7 @@ public class CompactMobAction extends Action {
admin.compact(tableName, CompactType.MOB);
}
} catch (Exception ex) {
- LOG.warn("Mob Compaction failed, might be caused by other chaos: " + ex.getMessage());
+ getLogger().warn("Mob Compaction failed, might be caused by other chaos: " + ex.getMessage());
}
if (sleepTime > 0) {
Thread.sleep(sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java
index 14d27ed..ffc05b8 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactRandomRegionOfTableAction.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
@@ -19,13 +19,12 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
-
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -33,11 +32,11 @@ import org.slf4j.LoggerFactory;
* Region that queues a compaction of a random region from the table.
*/
public class CompactRandomRegionOfTableAction extends Action {
+ private static final Logger LOG = LoggerFactory.getLogger(CompactRandomRegionOfTableAction.class);
+
private final int majorRatio;
private final long sleepTime;
private final TableName tableName;
- private static final Logger LOG =
- LoggerFactory.getLogger(CompactRandomRegionOfTableAction.class);
public CompactRandomRegionOfTableAction(
TableName tableName, float majorRatio) {
@@ -51,33 +50,37 @@ public class CompactRandomRegionOfTableAction extends Action {
this.tableName = tableName;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
Admin admin = util.getAdmin();
boolean major = RandomUtils.nextInt(0, 100) < majorRatio;
- LOG.info("Performing action: Compact random region of table "
+ getLogger().info("Performing action: Compact random region of table "
+ tableName + ", major=" + major);
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
- LOG.info("Table " + tableName + " doesn't have regions to compact");
+ getLogger().info("Table " + tableName + " doesn't have regions to compact");
return;
}
- HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
- regions.toArray(new HRegionInfo[regions.size()]));
+ RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+ regions.toArray(new RegionInfo[0]));
try {
if (major) {
- LOG.debug("Major compacting region " + region.getRegionNameAsString());
+ getLogger().debug("Major compacting region " + region.getRegionNameAsString());
admin.majorCompactRegion(region.getRegionName());
} else {
- LOG.debug("Compacting region " + region.getRegionNameAsString());
+ getLogger().debug("Compacting region " + region.getRegionNameAsString());
admin.compactRegion(region.getRegionName());
}
} catch (Exception ex) {
- LOG.warn("Compaction failed, might be caused by other chaos: " + ex.getMessage());
+ getLogger().warn("Compaction failed, might be caused by other chaos: " + ex.getMessage());
}
if (sleepTime > 0) {
Thread.sleep(sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java
index c05d43b..7278d2b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CompactTableAction.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
@@ -29,10 +29,11 @@ import org.slf4j.LoggerFactory;
* Action that queues a table compaction.
*/
public class CompactTableAction extends Action {
+ private static final Logger LOG = LoggerFactory.getLogger(CompactTableAction.class);
+
private final TableName tableName;
private final int majorRatio;
private final long sleepTime;
- private static final Logger LOG = LoggerFactory.getLogger(CompactTableAction.class);
public CompactTableAction(TableName tableName, float majorRatio) {
this(-1, tableName, majorRatio);
@@ -45,13 +46,17 @@ public class CompactTableAction extends Action {
this.sleepTime = sleepTime;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
Admin admin = util.getAdmin();
boolean major = RandomUtils.nextInt(0, 100) < majorRatio;
- LOG.info("Performing action: Compact table " + tableName + ", major=" + major);
+ getLogger().info("Performing action: Compact table " + tableName + ", major=" + major);
try {
if (major) {
admin.majorCompact(tableName);
@@ -59,7 +64,7 @@ public class CompactTableAction extends Action {
admin.compact(tableName);
}
} catch (Exception ex) {
- LOG.warn("Compaction failed, might be caused by other chaos: " + ex.getMessage());
+ getLogger().warn("Compaction failed, might be caused by other chaos: " + ex.getMessage());
}
if (sleepTime > 0) {
Thread.sleep(sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptDataFilesAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptDataFilesAction.java
index 83e8fe0..3b81f2e 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptDataFilesAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptDataFilesAction.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,7 @@ import org.slf4j.LoggerFactory;
*/
public class CorruptDataFilesAction extends Action {
private static final Logger LOG = LoggerFactory.getLogger(CorruptDataFilesAction.class);
- private float chance;
+ private final float chance;
/**
* Corrupts HFiles with a certain chance
@@ -44,9 +44,13 @@ public class CorruptDataFilesAction extends Action {
this.chance = chance * 100;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Start corrupting data files");
+ getLogger().info("Start corrupting data files");
FileSystem fs = CommonFSUtils.getRootDirFileSystem(getConf());
Path rootDir = CommonFSUtils.getRootDir(getConf());
@@ -67,9 +71,9 @@ public class CorruptDataFilesAction extends Action {
} finally {
out.close();
}
- LOG.info("Corrupting {}", status.getPath());
+ getLogger().info("Corrupting {}", status.getPath());
}
- LOG.info("Done corrupting data files");
+ getLogger().info("Done corrupting data files");
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptPacketsCommandAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptPacketsCommandAction.java
index 965bcbb..52b7e87 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptPacketsCommandAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/CorruptPacketsCommandAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.slf4j.Logger;
@@ -31,8 +30,8 @@ import org.slf4j.LoggerFactory;
*/
public class CorruptPacketsCommandAction extends TCCommandAction {
private static final Logger LOG = LoggerFactory.getLogger(CorruptPacketsCommandAction.class);
- private float ratio;
- private long duration;
+ private final float ratio;
+ private final long duration;
/**
* Corrupt network packets on a random regionserver.
@@ -48,8 +47,12 @@ public class CorruptPacketsCommandAction extends TCCommandAction {
this.duration = duration;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
protected void localPerform() throws IOException {
- LOG.info("Starting to execute CorruptPacketsCommandAction");
+ getLogger().info("Starting to execute CorruptPacketsCommandAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
String hostname = server.getHostname();
@@ -57,12 +60,12 @@ public class CorruptPacketsCommandAction extends TCCommandAction {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(ADD));
Thread.sleep(duration);
} catch (InterruptedException e) {
- LOG.debug("Failed to run the command for the full duration", e);
+ getLogger().debug("Failed to run the command for the full duration", e);
} finally {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(DELETE));
}
- LOG.info("Finished to execute CorruptPacketsCommandAction");
+ getLogger().info("Finished to execute CorruptPacketsCommandAction");
}
private String getCommand(String operation){
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DecreaseMaxHFileSizeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DecreaseMaxHFileSizeAction.java
index 4610ef0..0764d00 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DecreaseMaxHFileSizeAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DecreaseMaxHFileSizeAction.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,18 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.Random;
-
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class DecreaseMaxHFileSizeAction extends Action {
+ private static final Logger LOG = LoggerFactory.getLogger(DecreaseMaxHFileSizeAction.class);
- private static final long minFileSize = 1 * 1024 * 1024 * 1024L;
+ private static final long minFileSize = 1024 * 1024 * 1024L;
private final long sleepTime;
private final TableName tableName;
@@ -42,6 +44,10 @@ public class DecreaseMaxHFileSizeAction extends Action {
this.random = new Random();
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void init(ActionContext context) throws IOException {
super.init(context);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DelayPacketsCommandAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DelayPacketsCommandAction.java
index 38578d8..3f9cfef 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DelayPacketsCommandAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DelayPacketsCommandAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.slf4j.Logger;
@@ -30,8 +29,8 @@ import org.slf4j.LoggerFactory;
*/
public class DelayPacketsCommandAction extends TCCommandAction {
private static final Logger LOG = LoggerFactory.getLogger(DelayPacketsCommandAction.class);
- private long delay;
- private long duration;
+ private final long delay;
+ private final long duration;
/**
* Adds latency to communication on a random region server
@@ -47,8 +46,12 @@ public class DelayPacketsCommandAction extends TCCommandAction {
this.duration = duration;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
protected void localPerform() throws IOException {
- LOG.info("Starting to execute DelayPacketsCommandAction");
+ getLogger().info("Starting to execute DelayPacketsCommandAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
String hostname = server.getHostname();
@@ -56,12 +59,12 @@ public class DelayPacketsCommandAction extends TCCommandAction {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(ADD));
Thread.sleep(duration);
} catch (InterruptedException e) {
- LOG.debug("Failed to run the command for the full duration", e);
+ getLogger().debug("Failed to run the command for the full duration", e);
} finally {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(DELETE));
}
- LOG.info("Finished to execute DelayPacketsCommandAction");
+ getLogger().info("Finished to execute DelayPacketsCommandAction");
}
private String getCommand(String operation){
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DeleteDataFilesAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DeleteDataFilesAction.java
index 4919adc..2b86970 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DeleteDataFilesAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DeleteDataFilesAction.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,7 +33,7 @@ import org.slf4j.LoggerFactory;
*/
public class DeleteDataFilesAction extends Action {
private static final Logger LOG = LoggerFactory.getLogger(DeleteDataFilesAction.class);
- private float chance;
+ private final float chance;
/**
* Delets HFiles with a certain chance
@@ -43,9 +43,13 @@ public class DeleteDataFilesAction extends Action {
this.chance = chance * 100;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Start deleting data files");
+ getLogger().info("Start deleting data files");
FileSystem fs = CommonFSUtils.getRootDirFileSystem(getConf());
Path rootDir = CommonFSUtils.getRootDir(getConf());
Path defaultDir = rootDir.suffix("/data/default");
@@ -58,9 +62,9 @@ public class DeleteDataFilesAction extends Action {
if(RandomUtils.nextFloat(0, 100) > chance){
continue;
}
- fs.delete(status.getPath());
- LOG.info("Deleting {}", status.getPath());
+ fs.delete(status.getPath(), true);
+ getLogger().info("Deleting {}", status.getPath());
}
- LOG.info("Done deleting data files");
+ getLogger().info("Done deleting data files");
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java
index 3c70af0..dfb506d 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DumpClusterStatusAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -27,8 +26,11 @@ import org.slf4j.LoggerFactory;
* Action to dump the cluster status.
*/
public class DumpClusterStatusAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(DumpClusterStatusAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(DumpClusterStatusAction.class);
+
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
@Override
public void init(ActionContext context) throws IOException {
@@ -37,7 +39,7 @@ public class DumpClusterStatusAction extends Action {
@Override
public void perform() throws Exception {
- LOG.debug("Performing action: Dump cluster status");
- LOG.info("Cluster status\n" + cluster.getClusterMetrics());
+ getLogger().debug("Performing action: Dump cluster status");
+ getLogger().info("Cluster status\n" + cluster.getClusterMetrics());
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DuplicatePacketsCommandAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DuplicatePacketsCommandAction.java
index 7ed74ec..f517e8e 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DuplicatePacketsCommandAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/DuplicatePacketsCommandAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.slf4j.Logger;
@@ -31,8 +30,8 @@ import org.slf4j.LoggerFactory;
*/
public class DuplicatePacketsCommandAction extends TCCommandAction {
private static final Logger LOG = LoggerFactory.getLogger(DuplicatePacketsCommandAction.class);
- private float ratio;
- private long duration;
+ private final float ratio;
+ private final long duration;
/**
* Duplicate network packets on a random regionserver.
@@ -48,8 +47,12 @@ public class DuplicatePacketsCommandAction extends TCCommandAction {
this.duration = duration;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
protected void localPerform() throws IOException {
- LOG.info("Starting to execute DuplicatePacketsCommandAction");
+ getLogger().info("Starting to execute DuplicatePacketsCommandAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
String hostname = server.getHostname();
@@ -57,12 +60,12 @@ public class DuplicatePacketsCommandAction extends TCCommandAction {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(ADD));
Thread.sleep(duration);
} catch (InterruptedException e) {
- LOG.debug("Failed to run the command for the full duration", e);
+ getLogger().debug("Failed to run the command for the full duration", e);
} finally {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(DELETE));
}
- LOG.info("Finished to execute DuplicatePacketsCommandAction");
+ getLogger().info("Finished to execute DuplicatePacketsCommandAction");
}
private String getCommand(String operation){
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FillDiskCommandAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FillDiskCommandAction.java
index f5ca1d7..29d2a09 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FillDiskCommandAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FillDiskCommandAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.slf4j.Logger;
@@ -31,9 +30,9 @@ import org.slf4j.LoggerFactory;
*/
public class FillDiskCommandAction extends SudoCommandAction {
private static final Logger LOG = LoggerFactory.getLogger(FillDiskCommandAction.class);
- private long size;
- private long duration;
- private String path;
+ private final long size;
+ private final long duration;
+ private final String path;
/**
* Fill the disk on a random regionserver.
@@ -52,20 +51,24 @@ public class FillDiskCommandAction extends SudoCommandAction {
this.path = path;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
protected void localPerform() throws IOException {
- LOG.info("Starting to execute FillDiskCommandAction");
+ getLogger().info("Starting to execute FillDiskCommandAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
String hostname = server.getHostname();
try {
clusterManager.execSudo(hostname, duration, getFillCommand());
} catch (IOException ex) {
- LOG.info("Potential timeout. We try to stop the dd process on target machine");
+ getLogger().info("Potential timeout. We try to stop the dd process on target machine");
clusterManager.execSudoWithRetries(hostname, timeout, getStopCommand());
throw ex;
} finally {
clusterManager.execSudoWithRetries(hostname, timeout, getClearCommand());
- LOG.info("Finished to execute FillDiskCommandAction");
+ getLogger().info("Finished to execute FillDiskCommandAction");
}
}
@@ -82,6 +85,6 @@ public class FillDiskCommandAction extends SudoCommandAction {
}
private String getStopCommand() {
- return String.format("killall dd");
+ return "killall dd";
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java
index 4259bf80..ff23c34 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushRandomRegionOfTableAction.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
@@ -19,12 +19,11 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
-
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -46,25 +45,29 @@ public class FlushRandomRegionOfTableAction extends Action {
this.tableName = tableName;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
Admin admin = util.getAdmin();
- LOG.info("Performing action: Flush random region of table " + tableName);
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ getLogger().info("Performing action: Flush random region of table " + tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
- LOG.info("Table " + tableName + " doesn't have regions to flush");
+ getLogger().info("Table " + tableName + " doesn't have regions to flush");
return;
}
- HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
- regions.toArray(new HRegionInfo[regions.size()]));
- LOG.debug("Flushing region " + region.getRegionNameAsString());
+ RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+ regions.toArray(new RegionInfo[0]));
+ getLogger().debug("Flushing region " + region.getRegionNameAsString());
try {
admin.flushRegion(region.getRegionName());
} catch (Exception ex) {
- LOG.warn("Flush failed, might be caused by other chaos: " + ex.getMessage());
+ getLogger().warn("Flush failed, might be caused by other chaos: " + ex.getMessage());
}
if (sleepTime > 0) {
Thread.sleep(sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushTableAction.java
index 93f7347..d73b85a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/FlushTableAction.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
@@ -28,8 +28,7 @@ import org.slf4j.LoggerFactory;
* Action that tries to flush a table.
*/
public class FlushTableAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(FlushTableAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(FlushTableAction.class);
private final long sleepTime;
private final TableName tableName;
@@ -42,6 +41,10 @@ public class FlushTableAction extends Action {
this.tableName = tableName;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
@@ -52,11 +55,11 @@ public class FlushTableAction extends Action {
return;
}
- LOG.info("Performing action: Flush table " + tableName);
+ getLogger().info("Performing action: Flush table " + tableName);
try {
admin.flush(tableName);
} catch (Exception ex) {
- LOG.warn("Flush failed, might be caused by other chaos: " + ex.getMessage());
+ getLogger().warn("Flush failed, might be caused by other chaos: " + ex.getMessage());
}
if (sleepTime > 0) {
Thread.sleep(sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ForceBalancerAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ForceBalancerAction.java
index d754754..00767b0 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ForceBalancerAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ForceBalancerAction.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
@@ -25,8 +25,11 @@ import org.slf4j.LoggerFactory;
* Action that tries to force a balancer run.
*/
public class ForceBalancerAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(ForceBalancerAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(ForceBalancerAction.class);
+
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
@Override
public void perform() throws Exception {
@@ -34,7 +37,7 @@ public class ForceBalancerAction extends Action {
if (context.isStopping()) {
return;
}
- LOG.info("Balancing regions");
+ getLogger().info("Balancing regions");
forceBalancer();
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java
index 82005bb..f4ef797 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.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
@@ -39,35 +39,38 @@ public class GracefulRollingRestartRsAction extends RestartActionBaseAction {
super(sleepTime);
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Performing action: Rolling restarting non-master region servers");
+ getLogger().info("Performing action: Rolling restarting non-master region servers");
List<ServerName> selectedServers = selectServers();
- LOG.info("Disabling balancer to make unloading possible");
+ getLogger().info("Disabling balancer to make unloading possible");
setBalancer(false, true);
for (ServerName server : selectedServers) {
String rsName = server.getAddress().toString();
try (RegionMover rm =
new RegionMover.RegionMoverBuilder(rsName, getConf()).ack(true).build()) {
- LOG.info("Unloading {}", server);
+ getLogger().info("Unloading {}", server);
rm.unload();
- LOG.info("Restarting {}", server);
+ getLogger().info("Restarting {}", server);
gracefulRestartRs(server, sleepTime);
- LOG.info("Loading {}", server);
+ getLogger().info("Loading {}", server);
rm.load();
} catch (Shell.ExitCodeException e) {
- LOG.info("Problem restarting but presume successful; code={}", e.getExitCode(), e);
+ getLogger().info("Problem restarting but presume successful; code={}", e.getExitCode(), e);
}
sleep(RandomUtils.nextInt(0, (int)sleepTime));
}
- LOG.info("Enabling balancer");
+ getLogger().info("Enabling balancer");
setBalancer(true, true);
}
protected List<ServerName> selectServers() throws IOException {
return Arrays.asList(getCurrentServers());
}
-
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/LosePacketsCommandAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/LosePacketsCommandAction.java
index 8b09daf..d306558 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/LosePacketsCommandAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/LosePacketsCommandAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.slf4j.Logger;
@@ -31,8 +30,8 @@ import org.slf4j.LoggerFactory;
*/
public class LosePacketsCommandAction extends TCCommandAction {
private static final Logger LOG = LoggerFactory.getLogger(LosePacketsCommandAction.class);
- private float ratio;
- private long duration;
+ private final float ratio;
+ private final long duration;
/**
* Lose network packets on a random regionserver.
@@ -48,8 +47,12 @@ public class LosePacketsCommandAction extends TCCommandAction {
this.duration = duration;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
protected void localPerform() throws IOException {
- LOG.info("Starting to execute LosePacketsCommandAction");
+ getLogger().info("Starting to execute LosePacketsCommandAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
String hostname = server.getHostname();
@@ -57,12 +60,12 @@ public class LosePacketsCommandAction extends TCCommandAction {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(ADD));
Thread.sleep(duration);
} catch (InterruptedException e) {
- LOG.debug("Failed to run the command for the full duration", e);
+ getLogger().debug("Failed to run the command for the full duration", e);
} finally {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(DELETE));
}
- LOG.info("Finished to execute LosePacketsCommandAction");
+ getLogger().info("Finished to execute LosePacketsCommandAction");
}
private String getCommand(String operation){
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
index da120a9..a7e0723 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MergeRandomAdjacentRegionsOfTableAction.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
@@ -19,12 +19,11 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
-
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -46,22 +45,26 @@ public class MergeRandomAdjacentRegionsOfTableAction extends Action {
this.sleepTime = sleepTime;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
Admin admin = util.getAdmin();
- LOG.info("Performing action: Merge random adjacent regions of table " + tableName);
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ getLogger().info("Performing action: Merge random adjacent regions of table " + tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.size() < 2) {
- LOG.info("Table " + tableName + " doesn't have enough regions to merge");
+ getLogger().info("Table " + tableName + " doesn't have enough regions to merge");
return;
}
int i = RandomUtils.nextInt(0, regions.size() - 1);
- HRegionInfo a = regions.get(i++);
- HRegionInfo b = regions.get(i);
- LOG.debug("Merging " + a.getRegionNameAsString() + " and " + b.getRegionNameAsString());
+ RegionInfo a = regions.get(i++);
+ RegionInfo b = regions.get(i);
+ getLogger().debug("Merging " + a.getRegionNameAsString() + " and " + b.getRegionNameAsString());
// Don't try the merge if we're stopping
if (context.isStopping()) {
@@ -71,7 +74,7 @@ public class MergeRandomAdjacentRegionsOfTableAction extends Action {
try {
admin.mergeRegionsAsync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false);
} catch (Exception ex) {
- LOG.warn("Merge failed, might be caused by other chaos: " + ex.getMessage());
+ getLogger().warn("Merge failed, might be caused by other chaos: " + ex.getMessage());
}
if (sleepTime > 0) {
Thread.sleep(sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRandomRegionOfTableAction.java
index cdd2f20..2e9b3b9 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRandomRegionOfTableAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
-
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
@@ -32,8 +31,7 @@ import org.slf4j.LoggerFactory;
* Action that tries to move a random region of a table.
*/
public class MoveRandomRegionOfTableAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(MoveRandomRegionOfTableAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(MoveRandomRegionOfTableAction.class);
private final long sleepTime;
private final TableName tableName;
@@ -46,6 +44,10 @@ public class MoveRandomRegionOfTableAction extends Action {
this.tableName = tableName;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
if (sleepTime > 0) {
@@ -55,18 +57,19 @@ public class MoveRandomRegionOfTableAction extends Action {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
Admin admin = util.getAdmin();
- LOG.info("Performing action: Move random region of table " + tableName);
+ getLogger().info("Performing action: Move random region of table " + tableName);
List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
- LOG.info("Table " + tableName + " doesn't have regions to move");
+ getLogger().info("Table " + tableName + " doesn't have regions to move");
return;
}
RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
- regions.toArray(new RegionInfo[regions.size()]));
- LOG.debug("Move random region {}", region.getRegionNameAsString());
+ regions.toArray(new RegionInfo[0]));
+ getLogger().debug("Move random region {}", region.getRegionNameAsString());
// Use facility over in MoveRegionsOfTableAction...
- MoveRegionsOfTableAction.moveRegion(admin, MoveRegionsOfTableAction.getServers(admin), region);
+ MoveRegionsOfTableAction.moveRegion(admin, MoveRegionsOfTableAction.getServers(admin),
+ region, getLogger());
if (sleepTime > 0) {
Thread.sleep(sleepTime);
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
index edfc503..df6a2c8 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.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,12 +22,9 @@ import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
-
import org.apache.commons.lang3.RandomUtils;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.slf4j.Logger;
@@ -37,22 +34,21 @@ import org.slf4j.LoggerFactory;
* Action that tries to move every region of a table.
*/
public class MoveRegionsOfTableAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(MoveRegionsOfTableAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(MoveRegionsOfTableAction.class);
private final long sleepTime;
private final TableName tableName;
private final long maxTime;
- public MoveRegionsOfTableAction(TableName tableName) {
- this(-1, MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, tableName);
- }
-
public MoveRegionsOfTableAction(long sleepTime, long maxSleepTime, TableName tableName) {
this.sleepTime = sleepTime;
this.tableName = tableName;
this.maxTime = maxSleepTime;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
if (sleepTime > 0) {
@@ -62,24 +58,24 @@ public class MoveRegionsOfTableAction extends Action {
Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
ServerName[] servers = getServers(admin);
- LOG.info("Performing action: Move regions of table {}", tableName);
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ getLogger().info("Performing action: Move regions of table {}", tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
- LOG.info("Table {} doesn't have regions to move", tableName);
+ getLogger().info("Table {} doesn't have regions to move", tableName);
return;
}
Collections.shuffle(regions);
long start = System.currentTimeMillis();
- for (HRegionInfo regionInfo:regions) {
+ for (RegionInfo regionInfo:regions) {
// Don't try the move if we're stopping
if (context.isStopping()) {
return;
}
- moveRegion(admin, servers, regionInfo);
+ moveRegion(admin, servers, regionInfo, getLogger());
if (sleepTime > 0) {
Thread.sleep(sleepTime);
}
@@ -94,16 +90,16 @@ public class MoveRegionsOfTableAction extends Action {
static ServerName [] getServers(Admin admin) throws IOException {
Collection<ServerName> serversList = admin.getRegionServers();
- return serversList.toArray(new ServerName[serversList.size()]);
+ return serversList.toArray(new ServerName[0]);
}
- static void moveRegion(Admin admin, ServerName [] servers, RegionInfo regionInfo) {
+ static void moveRegion(Admin admin, ServerName [] servers, RegionInfo regionInfo, Logger logger) {
try {
ServerName destServerName = servers[RandomUtils.nextInt(0, servers.length)];
- LOG.debug("Moving {} to {}", regionInfo.getRegionNameAsString(), destServerName);
+ logger.debug("Moving {} to {}", regionInfo.getRegionNameAsString(), destServerName);
admin.move(regionInfo.getEncodedNameAsBytes(), destServerName);
} catch (Exception ex) {
- LOG.warn("Move failed, might be caused by other chaos: {}", ex.getMessage());
+ logger.warn("Move failed, might be caused by other chaos: {}", ex.getMessage());
}
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java
index 37e5401..a4361f7 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RemoveColumnAction.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
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.Random;
import java.util.Set;
-
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -40,7 +39,7 @@ public class RemoveColumnAction extends Action {
private final TableName tableName;
private final Set<String> protectedColumns;
private Admin admin;
- private Random random;
+ private final Random random;
public RemoveColumnAction(TableName tableName, Set<String> protectedColumns) {
this.tableName = tableName;
@@ -48,6 +47,10 @@ public class RemoveColumnAction extends Action {
random = new Random();
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void init(ActionContext context) throws IOException {
super.init(context);
@@ -69,7 +72,7 @@ public class RemoveColumnAction extends Action {
index = random.nextInt(columnDescriptors.length);
}
byte[] colDescName = columnDescriptors[index].getName();
- LOG.debug("Performing action: Removing " + Bytes.toString(colDescName)+ " from "
+ getLogger().debug("Performing action: Removing " + Bytes.toString(colDescName)+ " from "
+ tableName.getNameAsString());
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ReorderPacketsCommandAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ReorderPacketsCommandAction.java
index 1bb7969..c4adcd1 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ReorderPacketsCommandAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/ReorderPacketsCommandAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.slf4j.Logger;
@@ -31,9 +30,9 @@ import org.slf4j.LoggerFactory;
*/
public class ReorderPacketsCommandAction extends TCCommandAction {
private static final Logger LOG = LoggerFactory.getLogger(ReorderPacketsCommandAction.class);
- private float ratio;
- private long duration;
- private long delay;
+ private final float ratio;
+ private final long duration;
+ private final long delay;
/**
* Reorder network packets on a random regionserver.
@@ -52,8 +51,12 @@ public class ReorderPacketsCommandAction extends TCCommandAction {
this.delay = delay;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
protected void localPerform() throws IOException {
- LOG.info("Starting to execute ReorderPacketsCommandAction");
+ getLogger().info("Starting to execute ReorderPacketsCommandAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
String hostname = server.getHostname();
@@ -61,12 +64,12 @@ public class ReorderPacketsCommandAction extends TCCommandAction {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(ADD));
Thread.sleep(duration);
} catch (InterruptedException e) {
- LOG.debug("Failed to run the command for the full duration", e);
+ getLogger().debug("Failed to run the command for the full duration", e);
} finally {
clusterManager.execSudoWithRetries(hostname, timeout, getCommand(DELETE));
}
- LOG.info("Finished to execute ReorderPacketsCommandAction");
+ getLogger().info("Finished to execute ReorderPacketsCommandAction");
}
private String getCommand(String operation){
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java
index c53de90..ce41522 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.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
@@ -19,18 +19,13 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.util.Threads;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Base class for restarting HBaseServer's
*/
-public class RestartActionBaseAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(RestartActionBaseAction.class);
+public abstract class RestartActionBaseAction extends Action {
long sleepTime; // how long should we sleep
public RestartActionBaseAction(long sleepTime) {
@@ -38,7 +33,7 @@ public class RestartActionBaseAction extends Action {
}
void sleep(long sleepTime) {
- LOG.info("Sleeping for:" + sleepTime);
+ getLogger().info("Sleeping for:" + sleepTime);
Threads.sleep(sleepTime);
}
@@ -49,10 +44,10 @@ public class RestartActionBaseAction extends Action {
return;
}
- LOG.info("Killing master: {}", server);
+ getLogger().info("Killing master: {}", server);
killMaster(server);
sleep(sleepTime);
- LOG.info("Starting master: {}", server);
+ getLogger().info("Starting master: {}", server);
startMaster(server);
}
@@ -68,10 +63,10 @@ public class RestartActionBaseAction extends Action {
if (context.isStopping()) {
return;
}
- LOG.info("Stopping region server: {}", server);
+ getLogger().info("Stopping region server: {}", server);
stopRs(server);
sleep(sleepTime);
- LOG.info("Starting region server: {}", server);
+ getLogger().info("Starting region server: {}", server);
startRs(server);
}
@@ -81,10 +76,10 @@ public class RestartActionBaseAction extends Action {
if (context.isStopping()) {
return;
}
- LOG.info("Killing region server: {}", server);
+ getLogger().info("Killing region server: {}", server);
killRs(server);
sleep(sleepTime);
- LOG.info("Starting region server: {}", server);
+ getLogger().info("Starting region server: {}", server);
startRs(server);
}
@@ -94,10 +89,10 @@ public class RestartActionBaseAction extends Action {
if (context.isStopping()) {
return;
}
- LOG.info("Killing zookeeper node: {}", server);
+ getLogger().info("Killing zookeeper node: {}", server);
killZKNode(server);
sleep(sleepTime);
- LOG.info("Starting zookeeper node: {}", server);
+ getLogger().info("Starting zookeeper node: {}", server);
startZKNode(server);
}
@@ -107,10 +102,10 @@ public class RestartActionBaseAction extends Action {
if (context.isStopping()) {
return;
}
- LOG.info("Killing data node: {}", server);
+ getLogger().info("Killing data node: {}", server);
killDataNode(server);
sleep(sleepTime);
- LOG.info("Starting data node: {}", server);
+ getLogger().info("Starting data node: {}", server);
startDataNode(server);
}
@@ -120,11 +115,10 @@ public class RestartActionBaseAction extends Action {
if (context.isStopping()) {
return;
}
- LOG.info("Killing name node: {}", server);
+ getLogger().info("Killing name node: {}", server);
killNameNode(server);
sleep(sleepTime);
- LOG.info("Starting name node: {}", server);
+ getLogger().info("Starting name node: {}", server);
startNameNode(server);
}
-
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java
index ddd719a..9ce9056 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveMasterAction.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
@@ -31,9 +31,14 @@ public class RestartActiveMasterAction extends RestartActionBaseAction {
public RestartActiveMasterAction(long sleepTime) {
super(sleepTime);
}
+
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Performing action: Restart active master");
+ getLogger().info("Performing action: Restart active master");
ServerName master = cluster.getClusterMetrics().getMasterName();
restartMaster(master, sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java
index 672169e..f4e52dc 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActiveNameNodeAction.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
@@ -51,9 +51,13 @@ public class RestartActiveNameNodeAction extends RestartActionBaseAction {
super(sleepTime);
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Performing action: Restart active namenode");
+ getLogger().info("Performing action: Restart active namenode");
Configuration conf = CommonFSUtils.getRootDir(getConf()).getFileSystem(getConf()).getConf();
String nameServiceID = DFSUtil.getNamenodeNameServiceId(conf);
if (!HAUtil.isHAEnabled(conf, nameServiceID)) {
@@ -85,9 +89,9 @@ public class RestartActiveNameNodeAction extends RestartActionBaseAction {
if (activeNamenode == null) {
throw new Exception("No active Name node found in zookeeper under " + hadoopHAZkNode);
}
- LOG.info("Found active namenode host:" + activeNamenode);
+ getLogger().info("Found active namenode host:" + activeNamenode);
ServerName activeNNHost = ServerName.valueOf(activeNamenode, -1, -1);
- LOG.info("Restarting Active NameNode :" + activeNamenode);
+ getLogger().info("Restarting Active NameNode :" + activeNamenode);
restartNameNode(activeNNHost, sleepTime);
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java
index 81920c8..8720812 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomDataNodeAction.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
@@ -41,9 +41,13 @@ public class RestartRandomDataNodeAction extends RestartActionBaseAction {
super(sleepTime);
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Performing action: Restart random data node");
+ getLogger().info("Performing action: Restart random data node");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getDataNodes());
restartDataNode(server, sleepTime);
}
@@ -56,6 +60,6 @@ public class RestartRandomDataNodeAction extends RestartActionBaseAction {
for (DatanodeInfo dataNode: dfsClient.datanodeReport(HdfsConstants.DatanodeReportType.LIVE)) {
hosts.add(ServerName.valueOf(dataNode.getHostName(), -1, -1));
}
- return hosts.toArray(new ServerName[hosts.size()]);
+ return hosts.toArray(new ServerName[0]);
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsAction.java
index 48458b6..607c80f 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsAction.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,9 +33,13 @@ public class RestartRandomRsAction extends RestartActionBaseAction {
super(sleepTime);
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Performing action: Restart random region server");
+ getLogger().info("Performing action: Restart random region server");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
restartRs(server, sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsExceptMetaAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsExceptMetaAction.java
index b78144a..a120026 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsExceptMetaAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomRsExceptMetaAction.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,12 +20,20 @@ package org.apache.hadoop.hbase.chaos.actions;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class RestartRandomRsExceptMetaAction extends RestartRandomRsAction {
+ private static final Logger LOG = LoggerFactory.getLogger(RestartRandomRsExceptMetaAction.class);
+
public RestartRandomRsExceptMetaAction(long sleepTime) {
super(sleepTime);
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
int tries = 10;
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomZKNodeAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomZKNodeAction.java
index 3ed7a0d..9f4c2ef 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomZKNodeAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRandomZKNodeAction.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,9 +34,13 @@ public class RestartRandomZKNodeAction extends RestartActionBaseAction {
super(sleepTime);
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Performing action: Restart random zookeeper node");
+ getLogger().info("Performing action: Restart random zookeeper node");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(
ZKServerTool.readZKNodes(getConf()));
restartZKNode(server, sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java
index aeecf0a..8df41da 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingMetaAction.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,12 +33,17 @@ public class RestartRsHoldingMetaAction extends RestartActionBaseAction {
public RestartRsHoldingMetaAction(long sleepTime) {
super(sleepTime);
}
+
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Performing action: Restart region server holding META");
+ getLogger().info("Performing action: Restart region server holding META");
ServerName server = cluster.getServerHoldingMeta();
if (server == null) {
- LOG.warn("No server is holding hbase:meta right now.");
+ getLogger().warn("No server is holding hbase:meta right now.");
return;
}
ClusterMetrics clusterStatus = cluster.getClusterMetrics();
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java
index a63c672..fcc5333 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartRsHoldingTableAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
-
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.client.RegionLocator;
@@ -30,8 +29,7 @@ import org.slf4j.LoggerFactory;
* Action that restarts an HRegionServer holding one of the regions of the table.
*/
public class RestartRsHoldingTableAction extends RestartActionBaseAction {
- private static final Logger LOG =
- LoggerFactory.getLogger(RestartRsHoldingTableAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(RestartRsHoldingTableAction.class);
private final RegionLocator locator;
@@ -40,9 +38,14 @@ public class RestartRsHoldingTableAction extends RestartActionBaseAction {
this.locator = locator;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Performing action: Restart random RS holding table " + this.locator.getName());
+ getLogger().info(
+ "Performing action: Restart random RS holding table " + this.locator.getName());
List<HRegionLocation> locations = locator.getAllRegionLocations();
restartRs(locations.get(RandomUtils.nextInt(0, locations.size())).getServerName(), sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java
index c76fcb1..bd136bb 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.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
@@ -23,7 +23,6 @@ import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
-
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
@@ -60,10 +59,14 @@ public class RollingBatchRestartRsAction extends BatchRestartRsAction {
START
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info(String.format("Performing action: Rolling batch restarting %d%% of region servers",
- (int)(ratio * 100)));
+ getLogger().info("Performing action: Rolling batch restarting {}% of region servers",
+ (int)(ratio * 100));
List<ServerName> selectedServers = selectServers();
Queue<ServerName> serversToBeKilled = new LinkedList<>(selectedServers);
@@ -71,8 +74,8 @@ public class RollingBatchRestartRsAction extends BatchRestartRsAction {
// loop while there are servers to be killed or dead servers to be restarted
while ((!serversToBeKilled.isEmpty() || !deadServers.isEmpty()) && !context.isStopping()) {
- KillOrStart action = KillOrStart.KILL;
+ final KillOrStart action;
if (serversToBeKilled.isEmpty()) { // no more servers to kill
action = KillOrStart.START;
} else if (deadServers.isEmpty()) {
@@ -95,7 +98,7 @@ public class RollingBatchRestartRsAction extends BatchRestartRsAction {
} catch (org.apache.hadoop.util.Shell.ExitCodeException e) {
// We've seen this in test runs where we timeout but the kill went through. HBASE-9743
// So, add to deadServers even if exception so the start gets called.
- LOG.info("Problem killing but presume successful; code=" + e.getExitCode(), e);
+ getLogger().info("Problem killing but presume successful; code={}", e.getExitCode(), e);
}
deadServers.add(server);
break;
@@ -106,7 +109,7 @@ public class RollingBatchRestartRsAction extends BatchRestartRsAction {
} catch (org.apache.hadoop.util.Shell.ExitCodeException e) {
// The start may fail but better to just keep going though we may lose server.
//
- LOG.info("Problem starting, will retry; code=" + e.getExitCode(), e);
+ getLogger().info("Problem starting, will retry; code={}", e.getExitCode(), e);
}
break;
}
@@ -121,25 +124,23 @@ public class RollingBatchRestartRsAction extends BatchRestartRsAction {
/**
* Small test to ensure the class basically works.
- * @param args
- * @throws Exception
*/
public static void main(final String[] args) throws Exception {
RollingBatchRestartRsAction action = new RollingBatchRestartRsAction(1, 1.0f) {
private int invocations = 0;
@Override
- protected ServerName[] getCurrentServers() throws IOException {
+ protected ServerName[] getCurrentServers() {
final int count = 4;
List<ServerName> serverNames = new ArrayList<>(count);
for (int i = 0; i < 4; i++) {
serverNames.add(ServerName.valueOf(i + ".example.org", i, i));
}
- return serverNames.toArray(new ServerName[serverNames.size()]);
+ return serverNames.toArray(new ServerName[0]);
}
@Override
protected void killRs(ServerName server) throws IOException {
- LOG.info("Killed " + server);
+ LOG.info("Killed {}", server);
if (this.invocations++ % 3 == 0) {
throw new org.apache.hadoop.util.Shell.ExitCodeException(-1, "Failed");
}
@@ -147,7 +148,7 @@ public class RollingBatchRestartRsAction extends BatchRestartRsAction {
@Override
protected void startRs(ServerName server) throws IOException {
- LOG.info("Started " + server);
+ LOG.info("Started {}", server);
if (this.invocations++ % 3 == 0) {
throw new org.apache.hadoop.util.Shell.ExitCodeException(-1, "Failed");
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java
index d4ad3e4..dfae09a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.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,7 +22,6 @@ import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
-
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
@@ -40,9 +39,9 @@ import org.slf4j.LoggerFactory;
public class RollingBatchSuspendResumeRsAction extends Action {
private static final Logger LOG =
LoggerFactory.getLogger(RollingBatchSuspendResumeRsAction.class);
- private float ratio;
- private long sleepTime;
- private int maxSuspendedServers; // number of maximum suspended servers at any given time.
+ private final float ratio;
+ private final long sleepTime;
+ private final int maxSuspendedServers; // number of maximum suspended servers at any given time.
public RollingBatchSuspendResumeRsAction(long sleepTime, float ratio) {
this(sleepTime, ratio, 5);
@@ -58,10 +57,14 @@ public class RollingBatchSuspendResumeRsAction extends Action {
SUSPEND, RESUME
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info(String.format("Performing action: Rolling batch restarting %d%% of region servers",
- (int) (ratio * 100)));
+ getLogger().info("Performing action: Rolling batch restarting {}% of region servers",
+ (int) (ratio * 100));
List<ServerName> selectedServers = selectServers();
Queue<ServerName> serversToBeSuspended = new LinkedList<>(selectedServers);
@@ -70,8 +73,8 @@ public class RollingBatchSuspendResumeRsAction extends Action {
// loop while there are servers to be suspended or suspended servers to be resumed
while ((!serversToBeSuspended.isEmpty() || !suspendedServers.isEmpty()) && !context
.isStopping()) {
- SuspendOrResume action;
+ final SuspendOrResume action;
if (serversToBeSuspended.isEmpty()) { // no more servers to suspend
action = SuspendOrResume.RESUME;
} else if (suspendedServers.isEmpty()) {
@@ -105,7 +108,7 @@ public class RollingBatchSuspendResumeRsAction extends Action {
break;
}
- LOG.info("Sleeping for:{}", sleepTime);
+ getLogger().info("Sleeping for:{}", sleepTime);
Threads.sleep(sleepTime);
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SnapshotTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SnapshotTableAction.java
index 6ee9b57..ea5729e 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SnapshotTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SnapshotTableAction.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
@@ -28,8 +28,7 @@ import org.slf4j.LoggerFactory;
* Action that tries to take a snapshot of a table.
*/
public class SnapshotTableAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(SnapshotTableAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(SnapshotTableAction.class);
private final TableName tableName;
private final long sleepTime;
@@ -42,6 +41,10 @@ public class SnapshotTableAction extends Action {
this.sleepTime = sleepTime;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
@@ -53,7 +56,7 @@ public class SnapshotTableAction extends Action {
return;
}
- LOG.info("Performing action: Snapshot table " + tableName);
+ getLogger().info("Performing action: Snapshot table {}", tableName);
admin.snapshot(snapshotName, tableName);
if (sleepTime > 0) {
Thread.sleep(sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java
index 10d5440..ffd841b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitAllRegionOfTableAction.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
@@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
import java.util.concurrent.ThreadLocalRandom;
-
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
public class SplitAllRegionOfTableAction extends Action {
private static final Logger LOG =
LoggerFactory.getLogger(SplitAllRegionOfTableAction.class);
@@ -47,6 +45,10 @@ public class SplitAllRegionOfTableAction extends Action {
this.maxFullTableSplits = getConf().getInt(MAX_SPLIT_KEY, DEFAULT_MAX_SPLITS);
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
@@ -61,10 +63,10 @@ public class SplitAllRegionOfTableAction extends Action {
if (ThreadLocalRandom.current().nextDouble()
< (((double) splits) / ((double) maxFullTableSplits)) / ((double) 2)) {
splits++;
- LOG.info("Performing action: Split all regions of " + tableName);
+ getLogger().info("Performing action: Split all regions of {}", tableName);
admin.split(tableName);
} else {
- LOG.info("Skipping split of all regions.");
+ getLogger().info("Skipping split of all regions.");
}
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java
index 837c6c5..3a68339 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SplitRandomRegionOfTableAction.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
@@ -19,12 +19,11 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.List;
-
import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -46,15 +45,19 @@ public class SplitRandomRegionOfTableAction extends Action {
this.tableName = tableName;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
Admin admin = util.getAdmin();
- LOG.info("Performing action: Split random region of table " + tableName);
- List<HRegionInfo> regions = admin.getTableRegions(tableName);
+ getLogger().info("Performing action: Split random region of table " + tableName);
+ List<RegionInfo> regions = admin.getRegions(tableName);
if (regions == null || regions.isEmpty()) {
- LOG.info("Table " + tableName + " doesn't have regions to split");
+ getLogger().info("Table " + tableName + " doesn't have regions to split");
return;
}
// Don't try the split if we're stopping
@@ -62,13 +65,13 @@ public class SplitRandomRegionOfTableAction extends Action {
return;
}
- HRegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
- regions.toArray(new HRegionInfo[regions.size()]));
- LOG.debug("Splitting region " + region.getRegionNameAsString());
+ RegionInfo region = PolicyBasedChaosMonkey.selectRandomItem(
+ regions.toArray(new RegionInfo[0]));
+ getLogger().debug("Splitting region " + region.getRegionNameAsString());
try {
admin.splitRegionAsync(region.getRegionName()).get();
} catch (Exception ex) {
- LOG.warn("Split failed, might be caused by other chaos: " + ex.getMessage());
+ getLogger().warn("Split failed, might be caused by other chaos: " + ex.getMessage());
}
if (sleepTime > 0) {
Thread.sleep(sleepTime);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SudoCommandAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SudoCommandAction.java
index 6092a5d..f1712c8 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SudoCommandAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/SudoCommandAction.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
@@ -19,18 +19,14 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.io.IOException;
-
import org.apache.hadoop.hbase.DistributedHBaseCluster;
import org.apache.hadoop.hbase.HBaseCluster;
import org.apache.hadoop.hbase.HBaseClusterManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Base class for performing Actions based on linux commands requiring sudo privileges
*/
abstract public class SudoCommandAction extends Action {
- private static final Logger LOG = LoggerFactory.getLogger(SudoCommandAction.class);
protected long timeout;
protected HBaseClusterManager clusterManager;
@@ -43,9 +39,9 @@ abstract public class SudoCommandAction extends Action {
public void init(ActionContext context) throws IOException {
super.init(context);
HBaseCluster cluster = context.getHBaseCluster();
- if(cluster != null && cluster instanceof DistributedHBaseCluster){
+ if (cluster instanceof DistributedHBaseCluster){
Object manager = ((DistributedHBaseCluster)cluster).getClusterManager();
- if(manager != null && manager instanceof HBaseClusterManager){
+ if (manager instanceof HBaseClusterManager){
clusterManager = (HBaseClusterManager) manager;
}
}
@@ -53,8 +49,8 @@ abstract public class SudoCommandAction extends Action {
@Override
public void perform() throws Exception {
- if(clusterManager == null){
- LOG.info("Couldn't perform command action, it requires a distributed cluster.");
+ if (clusterManager == null){
+ getLogger().info("Couldn't perform command action, it requires a distributed cluster.");
return;
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TestChangeSplitPolicyAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TestChangeSplitPolicyAction.java
index 66cdff4..d19233a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TestChangeSplitPolicyAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TestChangeSplitPolicyAction.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,16 +33,14 @@ import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
@Category({MediumTests.class})
-public class TestChangeSplitPolicyAction extends Action {
+public class TestChangeSplitPolicyAction {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestChangeSplitPolicyAction.class);
private final static IntegrationTestingUtility TEST_UTIL = new IntegrationTestingUtility();
- private static ChangeSplitPolicyAction action;
- private Admin admin;
- private TableName tableName = TableName.valueOf("ChangeSplitPolicyAction");
+ private final TableName tableName = TableName.valueOf("ChangeSplitPolicyAction");
@BeforeClass
public static void setUpBeforeClass() throws Exception {
@@ -54,17 +52,17 @@ public class TestChangeSplitPolicyAction extends Action {
}
@Before
public void setUp() throws Exception {
- this.admin = TEST_UTIL.getAdmin();
+ Admin admin = TEST_UTIL.getAdmin();
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
admin.createTable(builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of("fam")).build());
}
@Test
public void testChangeSplitPolicyAction() throws Exception {
- ActionContext ctx = Mockito.mock(ActionContext.class);
+ Action.ActionContext ctx = Mockito.mock(Action.ActionContext.class);
Mockito.when(ctx.getHBaseIntegrationTestingUtility()).thenReturn(TEST_UTIL);
Mockito.when(ctx.getHBaseCluster()).thenReturn(TEST_UTIL.getHBaseCluster());
- action = new ChangeSplitPolicyAction(tableName);
+ ChangeSplitPolicyAction action = new ChangeSplitPolicyAction(tableName);
action.init(ctx);
action.perform();
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java
index b79dafa..5da10c7 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/TruncateTableAction.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
@@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.chaos.actions;
import java.util.Random;
-
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
@@ -30,8 +29,7 @@ import org.slf4j.LoggerFactory;
* Action that tries to truncate of a table.
*/
public class TruncateTableAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(TruncateTableAction.class);
+ private static final Logger LOG = LoggerFactory.getLogger(TruncateTableAction.class);
private final TableName tableName;
private final Random random;
@@ -40,6 +38,10 @@ public class TruncateTableAction extends Action {
this.random = new Random();
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
@@ -51,8 +53,8 @@ public class TruncateTableAction extends Action {
}
boolean preserveSplits = random.nextBoolean();
- LOG.info("Performing action: Truncate table " + tableName.getNameAsString() +
- "preserve splits " + preserveSplits);
+ getLogger().info("Performing action: Truncate table {} preserve splits {}",
+ tableName.getNameAsString(), preserveSplits);
admin.truncateTable(tableName, preserveSplits);
}
}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
index 5e5504a..623d41d 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.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
@@ -23,7 +23,6 @@ import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
-
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ServerName;
@@ -42,10 +41,10 @@ public class UnbalanceKillAndRebalanceAction extends Action {
private static final double HOARD_FRC_OF_REGIONS = 0.8;
/** Waits between calling unbalance and killing servers, kills and rebalance, and rebalance
* and restarting the servers; to make sure these events have time to impact the cluster. */
- private long waitForUnbalanceMilliSec;
- private long waitForKillsMilliSec;
- private long waitAfterBalanceMilliSec;
- private boolean killMetaRs;
+ private final long waitForUnbalanceMilliSec;
+ private final long waitForKillsMilliSec;
+ private final long waitAfterBalanceMilliSec;
+ private final boolean killMetaRs;
public UnbalanceKillAndRebalanceAction(long waitUnbalance, long waitKill, long waitAfterBalance,
boolean killMetaRs) {
@@ -56,6 +55,10 @@ public class UnbalanceKillAndRebalanceAction extends Action {
this.killMetaRs = killMetaRs;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
ClusterMetrics status = this.cluster.getClusterMetrics();
@@ -86,7 +89,7 @@ public class UnbalanceKillAndRebalanceAction extends Action {
}
if (!killMetaRs && targetServer.equals(metaServer)) {
- LOG.info("Not killing server because it holds hbase:meta.");
+ getLogger().info("Not killing server because it holds hbase:meta.");
} else {
killRs(targetServer);
killedServers.add(targetServer);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java
index 3f2a3a1..3d85e85 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceRegionsAction.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
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.chaos.actions;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
-
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ServerName;
@@ -32,10 +31,9 @@ import org.slf4j.LoggerFactory;
* Action that tries to unbalance the regions of a cluster.
*/
public class UnbalanceRegionsAction extends Action {
- private static final Logger LOG =
- LoggerFactory.getLogger(UnbalanceRegionsAction.class);
- private double fractionOfRegions;
- private double fractionOfServers;
+ private static final Logger LOG = LoggerFactory.getLogger(UnbalanceRegionsAction.class);
+ private final double fractionOfRegions;
+ private final double fractionOfServers;
/**
* Unbalances the regions on the cluster by choosing "target" servers, and moving
@@ -48,9 +46,13 @@ public class UnbalanceRegionsAction extends Action {
this.fractionOfServers = fractionOfServers;
}
+ @Override protected Logger getLogger() {
+ return LOG;
+ }
+
@Override
public void perform() throws Exception {
- LOG.info("Unbalancing regions");
+ getLogger().info("Unbalancing regions");
ClusterMetrics status = this.cluster.getClusterMetrics();
List<ServerName> victimServers = new LinkedList<>(status.getLiveServerMetrics().keySet());
int targetServerCount = (int)Math.ceil(fractionOfServers * victimServers.size());
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/CompositeSequentialPolicy.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/CompositeSequentialPolicy.java
index 4fefca8..bb7304d 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/CompositeSequentialPolicy.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/CompositeSequentialPolicy.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
@@ -23,7 +23,7 @@ import java.util.List;
/** A policy that runs multiple other policies one after the other */
public class CompositeSequentialPolicy extends Policy {
- private List<Policy> policies;
+ private final List<Policy> policies;
public CompositeSequentialPolicy(Policy... policies) {
this.policies = Arrays.asList(policies);
}