Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-28413 Fix race condition in TestCleanerChore.retriesIOExceptionInStatus #5735

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,19 @@
import static org.hamcrest.Matchers.instanceOf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;

import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.PrintStream;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
Expand All @@ -42,10 +47,12 @@
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtil;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.FutureUtils;
import org.apache.hadoop.hbase.util.StoppableImplementation;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.AfterClass;
Expand Down Expand Up @@ -128,12 +135,15 @@ public void retriesIOExceptionInStatus() throws Exception {
fs.create(file).close();
assertTrue("test file didn't get created.", fs.exists(file));
final AtomicBoolean fails = new AtomicBoolean(true);
CountDownLatch successSignal = new CountDownLatch(1);
FilterFileSystem filtered = new FilterFileSystem(fs) {
public FileStatus[] listStatus(Path f) throws IOException {
if (fails.get()) {
throw new IOException("whomp whomp.");
}
return fs.listStatus(f);
FileStatus[] ret = fs.listStatus(f);
successSignal.countDown();
return ret;
}
};

Expand All @@ -144,7 +154,7 @@ public FileStatus[] listStatus(Path f) throws IOException {
// trouble talking to the filesystem
// and verify that it accurately reported the failure.
CompletableFuture<Boolean> errorFuture = chore.triggerCleanerNow();
ExecutionException e = assertThrows(ExecutionException.class, () -> errorFuture.get());
ExecutionException e = assertThrows(ExecutionException.class, errorFuture::get);
assertThat(e.getCause(), instanceOf(IOException.class));
assertThat(e.getCause().getMessage(), containsString("whomp"));

Expand All @@ -154,16 +164,60 @@ public FileStatus[] listStatus(Path f) throws IOException {

// filesystem is back
fails.set(false);
for (;;) {
CompletableFuture<Boolean> succFuture = chore.triggerCleanerNow();
// the reset of the future is async, so it is possible that we get the previous future
// again.
if (succFuture != errorFuture) {
// verify that it accurately reported success.
assertTrue("chore should claim it succeeded.", succFuture.get());
break;
}
}
CompletableFuture<Boolean> succFuture = chore.triggerCleanerNow();
// all invocations after the signal are expected to succeed. the future handle we have
// currently may or may succeed. an entire run might be scheduled between now and when we're
// signaled, so we cannot naively check equality of succFuture vs. errorFuture.
successSignal.await();
UTIL.waitFor(TimeUnit.MINUTES.toMillis(3), TimeUnit.SECONDS.toMillis(5),
new Waiter.ExplainingPredicate<Exception>() {
private int attemptCount = 1;
private CompletableFuture<Boolean> fut = succFuture;

@Override
public boolean evaluate() throws InterruptedException {
if (fut == null) {
fut = chore.triggerCleanerNow();
attemptCount++;
}
if (fut.isCompletedExceptionally()) {
fut = null;
return false;
} else {
return true;
}
}

@Override
public String explainFailure() throws Exception {
StringBuilder sb = new StringBuilder().append(String
.format("Unable to achieve a successful chore run after %s attempts.", attemptCount));
if (fut != null && fut.isCompletedExceptionally()) {
// call fut.exceptionNow() when we have JDK19+
Throwable cause = null;
try {
fut.get();
} catch (ExecutionException e) {
cause = e.getCause() != null ? e.getCause() : null;
} catch (RuntimeException e) {
cause = FutureUtils.unwrapCompletionException(e);
}
if (cause != null) {
sb.append(" Most recent failure: ");
try (ByteArrayOutputStream baos = new ByteArrayOutputStream(1024);
PrintStream stream = new PrintStream(baos)) {
cause.printStackTrace(stream);
sb.append(baos);
}
}
}
return sb.toString();
}
});

assertNotNull("chore future was null.", succFuture);
// verify that it accurately reported success.
assertTrue("chore should claim it succeeded.", succFuture.get());
// verify everything is gone.
assertFalse("file should have been destroyed.", fs.exists(file));
assertFalse("directory should have been destroyed.", fs.exists(child));
Expand Down