HDFS-15077. Fix intermittent failure of TestDFSClientRetries#testLeaseRenewSocketTimeout. (#1797)

(cherry picked from commit aba3f6c3e1)
This commit is contained in:
Masatake Iwasaki 2020-01-08 16:45:39 +09:00
parent 3e32f5476a
commit 3e17b471b6
2 changed files with 10 additions and 12 deletions

View File

@ -270,8 +270,9 @@ public class LeaseRenewer {
half: LEASE_RENEWER_SLEEP_DEFAULT;
}
@VisibleForTesting
/** Is the daemon running? */
synchronized boolean isRunning() {
public synchronized boolean isRunning() {
return daemon != null && daemon.isAlive();
}

View File

@ -394,13 +394,14 @@ public class TestDFSClientRetries {
Mockito.anyString());
DFSClient client = new DFSClient(null, spyNN, conf, null);
// Get hold of the lease renewer instance used by the client
LeaseRenewer leaseRenewer = client.getLeaseRenewer();
leaseRenewer.setRenewalTime(100);
final LeaseRenewer leaseRenewer1 = client.getLeaseRenewer();
leaseRenewer1.setRenewalTime(100);
OutputStream out1 = client.create(file1, false);
Mockito.verify(spyNN, timeout(10000).times(1)).renewLease(
Mockito.anyString());
verifyEmptyLease(leaseRenewer);
verifyEmptyLease(leaseRenewer1);
GenericTestUtils.waitFor(() -> !(leaseRenewer1.isRunning()), 100, 10000);
try {
out1.write(new byte[256]);
fail("existing output stream should be aborted");
@ -413,14 +414,14 @@ public class TestDFSClientRetries {
// throws SocketTimeoutException.
Mockito.doNothing().when(spyNN).renewLease(
Mockito.anyString());
leaseRenewer = client.getLeaseRenewer();
leaseRenewer.setRenewalTime(100);
final LeaseRenewer leaseRenewer2 = client.getLeaseRenewer();
leaseRenewer2.setRenewalTime(100);
OutputStream out2 = client.create(file2, false);
Mockito.verify(spyNN, timeout(10000).times(2)).renewLease(
Mockito.anyString());
out2.write(new byte[256]);
out2.close();
verifyEmptyLease(leaseRenewer);
verifyEmptyLease(leaseRenewer2);
} finally {
cluster.shutdown();
}
@ -765,11 +766,7 @@ public class TestDFSClientRetries {
}
private void verifyEmptyLease(LeaseRenewer leaseRenewer) throws Exception {
int sleepCount = 0;
while (!leaseRenewer.isEmpty() && sleepCount++ < 20) {
Thread.sleep(500);
}
assertTrue("Lease should be empty.", leaseRenewer.isEmpty());
GenericTestUtils.waitFor(() -> leaseRenewer.isEmpty(), 100, 10000);
}
class DFSClientReader implements Runnable {