This is an automated email from the ASF dual-hosted git repository.
adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git
The following commit(s) were added to refs/heads/master by this push:
new 7acbdb3 KUDU-2687: fix ITClient retries (take two)
7acbdb3 is described below
commit 7acbdb30c45e9253cdbb86193b5cb65f4f3c95a4
Author: Adar Dembo <adar@cloudera.com>
AuthorDate: Wed Mar 20 18:07:17 2019 -0700
KUDU-2687: fix ITClient retries (take two)
After adding randomly injected failures, I finally figured out why ITClient
retries were broken: because, surprisingly, the RetryRule does _not_
reinitialize non-static test class fields. As such, the latches' counts were
never reset, and so every test retry would immediately fail.
Along the way I improved the test in a few ways:
- Switched from two latches to just one.
- Ensured that the test cleaned up its threads on failure.
- Made the test throw the failing exception directly rather than logging it
and forcing you to scroll around looking for the logged message.
- Switched from Random to RandomUtils.
- Relatedly, fixed the reuse of CapturingLogAppender in TestSecurity, which
caused every test to capture the events of all tests that ran before it.
Change-Id: I31ec2d708d0ed41f25abcf09011967062c9a56c6
Reviewed-on: http://gerrit.cloudera.org:8080/12820
Reviewed-by: Grant Henke <granthenke@apache.org>
Tested-by: Kudu Jenkins
---
.../test/java/org/apache/kudu/client/ITClient.java | 64 +++++++++++++---------
.../java/org/apache/kudu/client/TestSecurity.java | 3 +-
2 files changed, 41 insertions(+), 26 deletions(-)
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java b/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java
index 48ddec5..09ce6c6 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/ITClient.java
@@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit;
import com.google.common.collect.ImmutableList;
import org.apache.kudu.test.KuduTestHarness;
+import org.apache.kudu.test.RandomUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -51,18 +52,21 @@ public class ITClient {
private static final String RUNTIME_PROPERTY_NAME = "itclient.runtime.seconds";
private static final long DEFAULT_RUNTIME_SECONDS = 60;
- // Time we'll spend waiting at the end of the test for things to settle. Also the minimum
this
- // test can run for.
+
+ // Time we'll spend waiting at the end of the test for things to settle. Also
+ // the minimum this test can run for.
private static final long TEST_MIN_RUNTIME_SECONDS = 2;
+
private static final long TEST_TIMEOUT_SECONDS = 600000;
private static final String TABLE_NAME =
ITClient.class.getName() + "-" + System.currentTimeMillis();
- // One error and we stop the test.
- private final CountDownLatch keepRunningLatch = new CountDownLatch(1);
- // Latch used to track if an error occurred and we need to stop the test early.
- private final CountDownLatch errorLatch = new CountDownLatch(1);
+ // Tracks whether it's time for the test to end or not.
+ private CountDownLatch keepRunningLatch;
+
+ // If the test fails, will contain an exception that describes the failure.
+ private Exception failureException;
private KuduTable table;
private long runtimeInSeconds;
@@ -74,17 +78,21 @@ public class ITClient {
@Before
public void setUp() throws Exception {
+ // Set (or reset, in the event of a retry) test state.
+ keepRunningLatch = new CountDownLatch(1);
+ failureException = null;
+ sharedWriteTimestamp = 0;
+ // Extract and verify the test's running time.
String runtimeProp = System.getProperty(RUNTIME_PROPERTY_NAME);
runtimeInSeconds = runtimeProp == null ? DEFAULT_RUNTIME_SECONDS : Long.parseLong(runtimeProp);
-
if (runtimeInSeconds < TEST_MIN_RUNTIME_SECONDS || runtimeInSeconds > TEST_TIMEOUT_SECONDS)
{
Assert.fail("This test needs to run more than " + TEST_MIN_RUNTIME_SECONDS + " seconds"
+
" and less than " + TEST_TIMEOUT_SECONDS + " seconds");
}
+ LOG.info("Test will run for {} seconds", runtimeInSeconds);
- LOG.info ("Test running for {} seconds", runtimeInSeconds);
-
+ // Create the test table.
CreateTableOptions builder = new CreateTableOptions().setNumReplicas(3);
builder.setRangePartitionColumns(ImmutableList.of("key"));
table = harness.getClient().createTable(TABLE_NAME, getBasicSchema(), builder);
@@ -102,31 +110,39 @@ public class ITClient {
thread.start();
}
- // await() returns yes if the latch reaches 0, we don't want that.
- Assert.assertFalse("Look for the last ERROR line in the log that comes from ITClient",
- errorLatch.await(runtimeInSeconds, TimeUnit.SECONDS));
-
- // Indicate we want to stop, then wait a little bit for it to happen.
- keepRunningLatch.countDown();
+ // If we time out here, the test ran to completion and passed. Otherwise, a
+ // count down was triggered from an error and the test failed.
+ boolean failure = keepRunningLatch.await(runtimeInSeconds, TimeUnit.SECONDS);
+ if (!failure) {
+ // The test passed but the threads are still running; tell them to stop.
+ keepRunningLatch.countDown();
+ }
for (Thread thread : threads) {
// Give plenty of time for threads to stop.
thread.join(DEFAULT_SLEEP);
}
- AsyncKuduScanner scannerBuilder = harness.getAsyncClient().newScannerBuilder(table).build();
+ if (failure) {
+ throw failureException;
+ }
+
+ // If the test passed, do some extra validation at the end.
+ AsyncKuduScanner scannerBuilder = harness.getAsyncClient()
+ .newScannerBuilder(table)
+ .build();
int rowCount = countRowsInScan(scannerBuilder);
Assert.assertTrue(rowCount + " should be higher than 0", rowCount > 0);
}
/**
- * Logs an error message and triggers the error count down latch, stopping this test.
+ * Logs an error message and triggers the count down latch, stopping this test.
* @param message error message to print
* @param exception optional exception to print
*/
private void reportError(String message, Exception exception) {
- LOG.error(message, exception);
- errorLatch.countDown();
+ failureException = new Exception(message, exception);
+ keepRunningLatch.countDown();
}
/**
@@ -134,7 +150,7 @@ public class ITClient {
*/
class ChaosThread implements Runnable {
- private final Random random = new Random();
+ private final Random random = RandomUtils.getRandom();
@Override
public void run() {
@@ -179,7 +195,6 @@ public class ITClient {
return true;
}
connections.get(random.nextInt(connections.size())).disconnect();
-
} catch (Exception e) {
if (keepRunningLatch.getCount() == 0) {
// Likely shutdown() related.
@@ -228,7 +243,7 @@ public class ITClient {
class WriterThread implements Runnable {
private final KuduSession session = harness.getClient().newSession();
- private final Random random = new Random();
+ private final Random random = RandomUtils.getRandom();
private int currentRowKey = 0;
@Override
@@ -294,7 +309,7 @@ public class ITClient {
*/
class ScannerThread implements Runnable {
- private final Random random = new Random();
+ private final Random random = RandomUtils.getRandom();
// Updated by calling a full scan.
private int lastRowCount = 0;
@@ -302,14 +317,13 @@ public class ITClient {
@Override
public void run() {
while (keepRunningLatch.getCount() > 0) {
-
boolean shouldContinue;
// First check if we've written at least one row.
if (sharedWriteTimestamp == 0) {
shouldContinue = true;
} else if (lastRowCount == 0 || // Need to full scan once before random reading
- random.nextBoolean()) {
+ random.nextBoolean()) {
shouldContinue = fullScan();
} else {
shouldContinue = randomGet();
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
index 210fa2a..10d942d 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
@@ -55,7 +55,7 @@ public class TestSecurity {
private static final int TICKET_LIFETIME_SECS = 10;
private static final int RENEWABLE_LIFETIME_SECS = 20;
- private final CapturingLogAppender cla = new CapturingLogAppender();
+ private CapturingLogAppender cla;
private MiniKuduCluster miniCluster;
private KuduClient client;
@@ -91,6 +91,7 @@ public class TestSecurity {
@Before
public void setUp() {
FakeDNS.getInstance().install();
+ cla = new CapturingLogAppender();
}
@After
|