Skip to content

Commit

Permalink
HBASE-24220 Allow that zk NOTEMPTY multi exception is retryable by ru…
Browse files Browse the repository at this point in the history
…nning in-series

hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
 Cleanup checkstyle warnings. Don't depend on hbase-client
 ScannerCallable.

hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
 Cut down on cluster resource usage.

hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
 Debug

hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java
 Debug

hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
 Debug

hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MiniZooKeeperCluster.java
 Debug

hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
 Allow that NONEMPTY is retryable by running in series.
  • Loading branch information
saintstack committed Apr 20, 2020
1 parent bcacc4c commit 2d2e1d9
Show file tree
Hide file tree
Showing 8 changed files with 43 additions and 47 deletions.
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
/**
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
Expand All @@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.mapred;

import static org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl.LOG_PER_ROW_COUNT;

import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.DoNotRetryIOException;
Expand Down Expand Up @@ -58,9 +57,6 @@ public class TableRecordReaderImpl {

/**
* Restart from survivable exceptions by creating a new scanner.
*
* @param firstRow
* @throws IOException
*/
public void restart(byte[] firstRow) throws IOException {
Scan currentScan;
Expand Down Expand Up @@ -100,8 +96,6 @@ public void restart(byte[] firstRow) throws IOException {

/**
* Build the scanner. Not done in constructor to allow for extension.
*
* @throws IOException
*/
public void init() throws IOException {
restart(startRow);
Expand Down Expand Up @@ -194,10 +188,8 @@ public float getProgress() {
* @param key HStoreKey as input key.
* @param value MapWritable as input value
* @return true if there was more data
* @throws IOException
*/
public boolean next(ImmutableBytesWritable key, Result value)
throws IOException {
public boolean next(ImmutableBytesWritable key, Result value) throws IOException {
Result result;
try {
try {
Expand Down
Original file line number Diff line number Diff line change
@@ -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
Expand Down Expand Up @@ -37,7 +37,6 @@
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;

/**
Expand Down Expand Up @@ -102,10 +101,9 @@ public void restart(byte[] firstRow) throws IOException {
* In new mapreduce APIs, TaskAttemptContext has two getCounter methods
* Check if getCounter(String, String) method is available.
* @return The getCounter method or null if not available.
* @throws IOException
*/
protected static Method retrieveGetCounterWithStringsParams(TaskAttemptContext context)
throws IOException {
throws IOException {
Method m = null;
try {
m = context.getClass().getMethod("getCounter",
Expand Down Expand Up @@ -142,9 +140,6 @@ public void setScan(Scan scan) {

/**
* Build the scanner. Not done in constructor to allow for extension.
*
* @throws IOException
* @throws InterruptedException
*/
public void initialize(InputSplit inputsplit,
TaskAttemptContext context) throws IOException,
Expand Down Expand Up @@ -176,7 +171,6 @@ public void close() {
* Returns the current key.
*
* @return The current key.
* @throws IOException
* @throws InterruptedException When the job is aborted.
*/
public ImmutableBytesWritable getCurrentKey() throws IOException,
Expand Down Expand Up @@ -204,12 +198,18 @@ public Result getCurrentValue() throws IOException, InterruptedException {
* @throws InterruptedException When the job was aborted.
*/
public boolean nextKeyValue() throws IOException, InterruptedException {
if (key == null) key = new ImmutableBytesWritable();
if (value == null) value = new Result();
if (key == null) {
key = new ImmutableBytesWritable();
}
if (value == null) {
value = new Result();
}
try {
try {
value = this.scanner.next();
if (value != null && value.isStale()) numStale++;
if (value != null && value.isStale()) {
numStale++;
}
if (logScannerActivity) {
rowcount ++;
if (rowcount >= logPerRowCount) {
Expand Down Expand Up @@ -242,7 +242,9 @@ public boolean nextKeyValue() throws IOException, InterruptedException {
scanner.next(); // skip presumed already mapped row
}
value = scanner.next();
if (value != null && value.isStale()) numStale++;
if (value != null && value.isStale()) {
numStale++;
}
numRestarts++;
}

Expand Down Expand Up @@ -281,7 +283,6 @@ public boolean nextKeyValue() throws IOException, InterruptedException {
* counters thus can update counters based on scanMetrics.
* If hbase runs on old version of mapreduce, it won't be able to get
* access to counters and TableRecorderReader can't update counter values.
* @throws IOException
*/
private void updateCounters() throws IOException {
ScanMetrics scanMetrics = scanner.getScanMetrics();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ public static void setUpBaseConf(Configuration conf) {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
setUpBaseConf(TEST_UTIL.getConfiguration());
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster(1);
TEST_UTIL.startMiniMapReduceCluster();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
/**
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
Expand Down Expand Up @@ -30,7 +30,6 @@
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
Expand All @@ -43,6 +42,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.LongAdder;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
Expand Down Expand Up @@ -92,7 +92,6 @@
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;

/**
Expand Down Expand Up @@ -225,7 +224,7 @@ public void testRecoveredEdits() throws Exception {
Path editsdir = WALSplitUtil
.getRegionDirRecoveredEditsDir(FSUtils.getWALRegionDir(conf,
tableName, hri.getEncodedName()));
LOG.debug("checking edits dir " + editsdir);
LOG.debug("Checking edits dir " + editsdir);
FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
@Override
public boolean accept(Path p) {
Expand All @@ -235,9 +234,9 @@ public boolean accept(Path p) {
return true;
}
});
assertTrue(
"edits dir should have more than a single file in it. instead has " + files.length,
files.length > 1);
LOG.info("Files {}", Arrays.stream(files).map(f -> f.getPath().toString()).
collect(Collectors.joining(",")));
assertTrue("Edits dir should have more than a one file", files.length > 1);
for (int i = 0; i < files.length; i++) {
int c = countWAL(files[i].getPath(), fs, conf);
count += c;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
Expand Down Expand Up @@ -195,14 +194,13 @@ public static void setupBeforeClass() throws Exception {

@AfterClass
public static void tearDownAfterClass() throws Exception {
HRegionServer rs = null;
for (JVMClusterUtil.RegionServerThread thread:
TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
rs = thread.getRegionServer();
}
assertEquals(1, TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().size());
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().get(0).
getRegionServer();
// Strange place for an assert.
assertFalse("RegionServer should have ABORTED (FaultyAccessController)", rs.isAborted());
cleanUp();
TEST_UTIL.shutdownMiniCluster();
assertFalse("region server should have aborted due to FaultyAccessController", rs.isAborted());
}

private static void setUpTableAndUserPermissions() throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,8 @@ private void checkHttpMethods(String url) throws Exception {
HttpURLConnection conn = (HttpURLConnection) new URL(url).openConnection();
conn.setRequestMethod("TRACE");
conn.connect();
Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode());
Assert.assertEquals(conn.getResponseMessage(),
HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode());
}

protected static volatile boolean tableCreated = false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -323,19 +323,18 @@ private void createDir(File dir) throws IOException {
public void shutdown() throws IOException {
// shut down all the zk servers
for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
NIOServerCnxnFactory standaloneServerFactory =
standaloneServerFactoryList.get(i);
NIOServerCnxnFactory standaloneServerFactory = standaloneServerFactoryList.get(i);
int clientPort = clientPortList.get(i);

standaloneServerFactory.shutdown();
if (!waitForServerDown(clientPort, connectionTimeout)) {
throw new IOException("Waiting for shutdown of standalone server");
throw new IOException("Waiting for shutdown of standalone server at port=" + clientPort +
", timeout=" + this.connectionTimeout);
}
}
standaloneServerFactoryList.clear();

for (ZooKeeperServer zkServer: zooKeeperServers) {
//explicitly close ZKDatabase since ZookeeperServer does not close them
// Explicitly close ZKDatabase since ZookeeperServer does not close them
zkServer.getZKDatabase().close();
}
zooKeeperServers.clear();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
Expand Down Expand Up @@ -1535,6 +1536,10 @@ private static List<String> listChildrenBFSAndWatchThem(ZKWatcher zkw, final Str
public abstract static class ZKUtilOp {
private String path;

@Override public String toString() {
return this.getClass().getSimpleName() + ", path=" + this.path;
}

private ZKUtilOp(String path) {
this.path = path;
}
Expand Down Expand Up @@ -1750,12 +1755,13 @@ public static void multiOrSequential(ZKWatcher zkw, List<ZKUtilOp> ops,
case NONODE:
case BADVERSION:
case NOAUTH:
case NOTEMPTY:
// if we get an exception that could be solved by running sequentially
// (and the client asked us to), then break out and run sequentially
if (runSequentialOnMultiFailure) {
LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
+ " Attempting to run operations sequentially because"
+ " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
LOG.info("multi exception: {}; running operations sequentially " +
"(runSequentialOnMultiFailure=true); {}", ke.toString(),
ops.stream().map(o -> o.toString()).collect(Collectors.joining(",")));
processSequentially(zkw, ops);
break;
}
Expand Down

0 comments on commit 2d2e1d9

Please sign in to comment.