HDFS-15077. Fix intermittent failure of TestDFSClientRetries#testLeaseRenewSocketTimeout. (#1797)
This commit is contained in:
parent
571795cd18
commit
aba3f6c3e1
@ -270,8 +270,9 @@ private void unsyncSetGraceSleepPeriod(final long gracePeriod) {
|
|||||||
half: LEASE_RENEWER_SLEEP_DEFAULT;
|
half: LEASE_RENEWER_SLEEP_DEFAULT;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
/** Is the daemon running? */
|
/** Is the daemon running? */
|
||||||
synchronized boolean isRunning() {
|
public synchronized boolean isRunning() {
|
||||||
return daemon != null && daemon.isAlive();
|
return daemon != null && daemon.isAlive();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -387,13 +387,14 @@ public void testLeaseRenewSocketTimeout() throws Exception
|
|||||||
Mockito.anyString());
|
Mockito.anyString());
|
||||||
DFSClient client = new DFSClient(null, spyNN, conf, null);
|
DFSClient client = new DFSClient(null, spyNN, conf, null);
|
||||||
// Get hold of the lease renewer instance used by the client
|
// Get hold of the lease renewer instance used by the client
|
||||||
LeaseRenewer leaseRenewer = client.getLeaseRenewer();
|
final LeaseRenewer leaseRenewer1 = client.getLeaseRenewer();
|
||||||
leaseRenewer.setRenewalTime(100);
|
leaseRenewer1.setRenewalTime(100);
|
||||||
OutputStream out1 = client.create(file1, false);
|
OutputStream out1 = client.create(file1, false);
|
||||||
|
|
||||||
Mockito.verify(spyNN, timeout(10000).times(1)).renewLease(
|
Mockito.verify(spyNN, timeout(10000).times(1)).renewLease(
|
||||||
Mockito.anyString());
|
Mockito.anyString());
|
||||||
verifyEmptyLease(leaseRenewer);
|
verifyEmptyLease(leaseRenewer1);
|
||||||
|
GenericTestUtils.waitFor(() -> !(leaseRenewer1.isRunning()), 100, 10000);
|
||||||
try {
|
try {
|
||||||
out1.write(new byte[256]);
|
out1.write(new byte[256]);
|
||||||
fail("existing output stream should be aborted");
|
fail("existing output stream should be aborted");
|
||||||
@ -406,14 +407,14 @@ public void testLeaseRenewSocketTimeout() throws Exception
|
|||||||
// throws SocketTimeoutException.
|
// throws SocketTimeoutException.
|
||||||
Mockito.doNothing().when(spyNN).renewLease(
|
Mockito.doNothing().when(spyNN).renewLease(
|
||||||
Mockito.anyString());
|
Mockito.anyString());
|
||||||
leaseRenewer = client.getLeaseRenewer();
|
final LeaseRenewer leaseRenewer2 = client.getLeaseRenewer();
|
||||||
leaseRenewer.setRenewalTime(100);
|
leaseRenewer2.setRenewalTime(100);
|
||||||
OutputStream out2 = client.create(file2, false);
|
OutputStream out2 = client.create(file2, false);
|
||||||
Mockito.verify(spyNN, timeout(10000).times(2)).renewLease(
|
Mockito.verify(spyNN, timeout(10000).times(2)).renewLease(
|
||||||
Mockito.anyString());
|
Mockito.anyString());
|
||||||
out2.write(new byte[256]);
|
out2.write(new byte[256]);
|
||||||
out2.close();
|
out2.close();
|
||||||
verifyEmptyLease(leaseRenewer);
|
verifyEmptyLease(leaseRenewer2);
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
@ -758,11 +759,7 @@ private boolean busyTest(int xcievers, int threads, int fileLen, int timeWin, in
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void verifyEmptyLease(LeaseRenewer leaseRenewer) throws Exception {
|
private void verifyEmptyLease(LeaseRenewer leaseRenewer) throws Exception {
|
||||||
int sleepCount = 0;
|
GenericTestUtils.waitFor(() -> leaseRenewer.isEmpty(), 100, 10000);
|
||||||
while (!leaseRenewer.isEmpty() && sleepCount++ < 20) {
|
|
||||||
Thread.sleep(500);
|
|
||||||
}
|
|
||||||
assertTrue("Lease should be empty.", leaseRenewer.isEmpty());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class DFSClientReader implements Runnable {
|
class DFSClientReader implements Runnable {
|
||||||
|
Loading…
Reference in New Issue
Block a user