Skip to content

Commit

Permalink
HBASE-27252 Clean up error-prone findings in hbase-it
Browse files Browse the repository at this point in the history
Close #4662

Co-authored-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Viraj Jasani <vjasani@apache.org>
(cherry picked from commit 1004876)

Conflicts:
	hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBackupRestore.java
	hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngest.java
	hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java
  • Loading branch information
apurtell authored and Apache9 committed Aug 20, 2022
1 parent 46ffdab commit 7c3fc45
Show file tree
Hide file tree
Showing 45 changed files with 240 additions and 204 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import java.io.Closeable;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
Expand Down Expand Up @@ -224,10 +225,11 @@ public void processResult(int rc, String path, Object ctx, byte[] data, Stat sta
} catch (Exception e) {
break;
}
zk.getData(path, false, getTaskForExecutionCallback, new String(data));
zk.getData(path, false, getTaskForExecutionCallback,
new String(data, StandardCharsets.UTF_8));
break;
case OK:
String cmd = new String(data);
String cmd = new String(data, StandardCharsets.UTF_8);
LOG.info("Executing command : " + cmd);
String status = ChaosConstants.TASK_COMPLETION_STRING;
try {
Expand Down Expand Up @@ -368,7 +370,8 @@ private void createIfZNodeNotExists(String path) {
*/
public void setStatusOfTaskZNode(String taskZNode, String status) {
LOG.info("Setting status of Task ZNode: " + taskZNode + " status : " + status);
zk.setData(taskZNode, status.getBytes(), -1, setStatusOfTaskZNodeCallback, null);
zk.setData(taskZNode, status.getBytes(StandardCharsets.UTF_8), -1, setStatusOfTaskZNodeCallback,
null);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.hadoop.hbase;

import java.io.IOException;
import java.nio.charset.StandardCharsets;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
Expand Down Expand Up @@ -111,7 +112,7 @@ public String submitTask(final TaskObject taskObject) {
zk.create(
CHAOS_AGENT_STATUS_ZNODE + ZNODE_PATH_SEPARATOR + taskObject.getTaskHostname()
+ ZNODE_PATH_SEPARATOR + TASK_PREFIX,
taskObject.getCommand().getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE,
taskObject.getCommand().getBytes(StandardCharsets.UTF_8), ZooDefs.Ids.OPEN_ACL_UNSAFE,
CreateMode.EPHEMERAL_SEQUENTIAL, submitTaskCallback, taskObject);
long start = EnvironmentEdgeManager.currentTime();

Expand Down Expand Up @@ -189,7 +190,7 @@ public void process(WatchedEvent watchedEvent) {
case OK:
if (ctx != null) {

String status = new String(data);
String status = new String(data, StandardCharsets.UTF_8);
taskStatus = status;
switch (status) {
case TASK_COMPLETION_STRING:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public static enum ServiceType {
HBASE_MASTER("master"),
HBASE_REGIONSERVER("regionserver");

private String name;
private final String name;

ServiceType(String name) {
this.name = name;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -360,9 +360,9 @@ public boolean restoreClusterMetrics(ClusterMetrics initial) throws IOException

// do a best effort restore
boolean success = true;
success = restoreMasters(initial, current) & success;
success = restoreRegionServers(initial, current) & success;
success = restoreAdmin() & success;
success = restoreMasters(initial, current) && success;
success = restoreRegionServers(initial, current) && success;
success = restoreAdmin() && success;

LOG.info("Restoring cluster - done");
return success;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -280,22 +280,14 @@ public String getCommand(ServiceType service, Operation op) {
*/
static class ZookeeperShellCommandProvider extends CommandProvider {
private final String zookeeperHome;
private final String confDir;

ZookeeperShellCommandProvider(Configuration conf) throws IOException {
zookeeperHome =
conf.get("hbase.it.clustermanager.zookeeper.home", System.getenv("ZOOBINDIR"));
String tmp =
conf.get("hbase.it.clustermanager.zookeeper.conf.dir", System.getenv("ZOOCFGDIR"));
if (zookeeperHome == null) {
throw new IOException("ZooKeeper home configuration parameter i.e. "
+ "'hbase.it.clustermanager.zookeeper.home' is not configured properly.");
}
if (tmp != null) {
confDir = String.format("--config %s", tmp);
} else {
confDir = "";
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;

/**
Expand Down Expand Up @@ -63,6 +64,7 @@ public class IntegrationTestIngest extends IntegrationTestBase {

// Log is being used in IntegrationTestIngestWithEncryption, hence it is protected
protected static final Logger LOG = LoggerFactory.getLogger(IntegrationTestIngest.class);

protected IntegrationTestingUtility util;
protected HBaseCluster cluster;
protected LoadTestTool loadTool;
Expand Down Expand Up @@ -137,7 +139,7 @@ protected Set<String> getColumnFamilies() {
families.add(Bytes.toString(family));
}
} else {
for (String family : familiesString.split(",")) {
for (String family : Splitter.on(',').split(familiesString)) {
families.add(family);
}
}
Expand Down Expand Up @@ -168,8 +170,7 @@ protected void runIngestTest(long defaultRunTime, long keysPerServerPerIter, int
LOG.info("Intended run time: " + (runtime / 60000) + " min, left:"
+ ((runtime - (EnvironmentEdgeManager.currentTime() - start)) / 60000) + " min");

int ret = -1;
ret = loadTool.run(getArgsForLoadTestTool("-write",
int ret = loadTool.run(getArgsForLoadTestTool("-write",
String.format("%d:%d:%d", colsPerKey, recordSize, writeThreads), startKey, numKeys));
if (0 != ret) {
String errorMsg = "Load failed with error code " + ret;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ protected void processOptions(CommandLine cmd) {
}

@Test
@Override
public void testIngest() throws Exception {
runIngestTest(JUNIT_RUN_TIME, 100, 10, 1024, 10, 20);
};
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,9 @@
public class IntegrationTestLazyCfLoading {
private static final TableName TABLE_NAME =
TableName.valueOf(IntegrationTestLazyCfLoading.class.getSimpleName());
@SuppressWarnings("InlineFormatString")
private static final String TIMEOUT_KEY = "hbase.%s.timeout";
@SuppressWarnings("InlineFormatString")
private static final String ENCODING_KEY = "hbase.%s.datablock.encoding";

/** A soft test timeout; duration of the test, as such, depends on number of keys to put. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static org.junit.Assert.assertTrue;

import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
Expand Down Expand Up @@ -225,7 +226,7 @@ private static void initConf(Configuration conf) {

}

class MajorCompaction implements Runnable {
static class MajorCompaction implements Runnable {

@Override
public void run() {
Expand All @@ -241,7 +242,7 @@ public void run() {
}
}

class CleanMobAndArchive implements Runnable {
static class CleanMobAndArchive implements Runnable {

@Override
public void run() {
Expand All @@ -256,7 +257,7 @@ public void run() {

Thread.sleep(130000);
} catch (Exception e) {
e.printStackTrace();
LOG.warn("Exception in CleanMobAndArchive", e);
}
}
}
Expand Down Expand Up @@ -287,7 +288,8 @@ public void run() {
try {
Thread.sleep(500);
} catch (InterruptedException ee) {

// Restore interrupt status
Thread.currentThread().interrupt();
}
}
if (i % 100000 == 0) {
Expand Down Expand Up @@ -322,7 +324,7 @@ public void testMobCompaction() throws InterruptedException, IOException {
Thread.sleep(1000);
}

getNumberOfMobFiles(conf, table.getName(), new String(fam));
getNumberOfMobFiles(conf, table.getName(), new String(fam, StandardCharsets.UTF_8));
LOG.info("Waiting for write thread to finish ...");
writeData.join();
// Cleanup again
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,9 @@
import static org.junit.Assert.assertTrue;

import com.codahale.metrics.Histogram;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
import java.util.Set;
Expand Down Expand Up @@ -104,7 +104,7 @@ private enum Stat {
* Wraps the invocation of {@link PerformanceEvaluation} in a {@code Callable}.
*/
static class PerfEvalCallable implements Callable<TimingResult> {
private final Queue<String> argv = new LinkedList<>();
private final Queue<String> argv = new ArrayDeque<>();
private final Admin admin;

public PerfEvalCallable(Admin admin, String argv) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,7 +175,6 @@ protected void runIngestTest(long defaultRunTime, long keysPerServerPerIter, int

int verifyPercent = 100;
int updatePercent = 20;
int ret = -1;
int regionReplicaId =
conf.getInt(String.format("%s.%s", TEST_NAME, LoadTestTool.OPT_REGION_REPLICA_ID), 1);

Expand All @@ -191,7 +190,7 @@ protected void runIngestTest(long defaultRunTime, long keysPerServerPerIter, int
args.add("-" + LoadTestTool.OPT_REGION_REPLICA_ID);
args.add(String.valueOf(regionReplicaId));

ret = loadTool.run(args.toArray(new String[args.size()]));
int ret = loadTool.run(args.toArray(new String[args.size()]));
if (0 != ret) {
String errorMsg = "Load failed with error code " + ret;
LOG.error(errorMsg);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,14 +116,13 @@ public static void setUseDistributedCluster(Configuration conf) {
}

/**
* @return whether we are interacting with a distributed cluster as opposed to and in-process mini
* cluster or a local cluster.
* Returns whether we are interacting with a distributed cluster as opposed to and in-process mini
* cluster or a local cluster.
* @see IntegrationTestingUtility#setUseDistributedCluster(Configuration)
*/
public boolean isDistributedCluster() {
Configuration conf = getConfiguration();
boolean isDistributedCluster = false;
isDistributedCluster =
boolean isDistributedCluster =
Boolean.parseBoolean(System.getProperty(IS_DISTRIBUTED_CLUSTER, "false"));
if (!isDistributedCluster) {
isDistributedCluster = conf.getBoolean(IS_DISTRIBUTED_CLUSTER, false);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ public static void main(String[] args) throws Exception {
System.exit(ret);
}

private class IntegrationTestFilter extends ClassTestFinder.TestClassFilter {
private static class IntegrationTestFilter extends ClassTestFinder.TestClassFilter {
private Pattern testFilterRe = Pattern.compile(".*\\.IntegrationTest.*");

public IntegrationTestFilter() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.hadoop.hbase;

import java.io.IOException;
import java.util.List;
import java.util.Locale;
import java.util.Set;
import org.apache.commons.lang3.StringUtils;
Expand All @@ -41,6 +42,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;

/**
Expand Down Expand Up @@ -106,10 +109,12 @@ protected void processOptions(CommandLine cmd) {
int minValueSize = 0, maxValueSize = 0;
String valueSize = cmd.getOptionValue(VALUE_SIZE_KEY, VALUE_SIZE_DEFAULT);
if (valueSize.contains(":")) {
String[] valueSizes = valueSize.split(":");
if (valueSize.length() != 2) throw new RuntimeException("Invalid value size: " + valueSize);
minValueSize = Integer.parseInt(valueSizes[0]);
maxValueSize = Integer.parseInt(valueSizes[1]);
List<String> valueSizes = Splitter.on(':').splitToList(valueSize);
if (valueSizes.size() != 2) {
throw new RuntimeException("Invalid value size: " + valueSize);
}
minValueSize = Integer.parseInt(Iterables.get(valueSizes, 0));
maxValueSize = Integer.parseInt(Iterables.get(valueSizes, 1));
} else {
minValueSize = maxValueSize = Integer.parseInt(valueSize);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@

import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
Expand Down Expand Up @@ -106,7 +107,8 @@ admin.<ShellExecService.Stub, ShellExecResponse> coprocessorService(ShellExecSer
assertFalse("the response from a background task should have no stderr", resp.hasStderr());

Waiter.waitFor(conn.getConfiguration(), 5_000, () -> testFile.length() > 0);
final String content = new String(Files.readAllBytes(testFile.toPath())).trim();
final String content =
new String(Files.readAllBytes(testFile.toPath()), StandardCharsets.UTF_8).trim();
assertEquals("hello world", content);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Properties;
Expand Down Expand Up @@ -277,13 +276,13 @@ protected void startNameNode(ServerName server) throws IOException {

protected void unbalanceRegions(ClusterMetrics clusterStatus, List<ServerName> fromServers,
List<ServerName> toServers, double fractionOfRegions) throws Exception {
List<byte[]> victimRegions = new LinkedList<>();
List<byte[]> victimRegions = new ArrayList<>();
for (Map.Entry<ServerName, ServerMetrics> entry : clusterStatus.getLiveServerMetrics()
.entrySet()) {
ServerName sn = entry.getKey();
ServerMetrics serverLoad = entry.getValue();
// Ugh.
List<byte[]> regions = new LinkedList<>(serverLoad.getRegionMetrics().keySet());
List<byte[]> regions = new ArrayList<>(serverLoad.getRegionMetrics().keySet());
int victimRegionCount = (int) Math.ceil(fractionOfRegions * regions.size());
getLogger().debug("Removing {} regions from {}", victimRegionCount, sn);
Random rand = ThreadLocalRandom.current();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
*/
public class AddCPULoadAction extends SudoCommandAction {
private static final Logger LOG = LoggerFactory.getLogger(AddCPULoadAction.class);
@SuppressWarnings("InlineFormatString")
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";
Expand All @@ -51,6 +52,7 @@ protected Logger getLogger() {
return LOG;
}

@Override
protected void localPerform() throws IOException {
getLogger().info("Starting to execute AddCPULoadAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,16 +28,10 @@
* Action that tries to adjust the bloom filter setting on all the columns of a table
*/
public class ChangeBloomFilterAction extends Action {
private final long sleepTime;
private final TableName tableName;
private static final Logger LOG = LoggerFactory.getLogger(ChangeBloomFilterAction.class);
private final TableName tableName;

public ChangeBloomFilterAction(TableName tableName) {
this(-1, tableName);
}

public ChangeBloomFilterAction(int sleepTime, TableName tableName) {
this.sleepTime = sleepTime;
this.tableName = tableName;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ protected Logger getLogger() {
return LOG;
}

@Override
protected void localPerform() throws IOException {
getLogger().info("Starting to execute CorruptPacketsCommandAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ protected Logger getLogger() {
return LOG;
}

@Override
protected void localPerform() throws IOException {
getLogger().info("Starting to execute DelayPacketsCommandAction");
ServerName server = PolicyBasedChaosMonkey.selectRandomItem(getCurrentServers());
Expand Down
Loading

0 comments on commit 7c3fc45

Please sign in to comment.