HDFS-15077. Fix intermittent failure of TestDFSClientRetries#testLeaseRenewSocketTimeout.
This commit is contained in:
parent
e9a7b30fa0
commit
40c755d809
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -50,6 +50,8 @@ import java.util.Random;
|
|||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -381,13 +383,18 @@ 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(new Supplier<Boolean>() {
|
||||
@Override
|
||||
public Boolean get() {
|
||||
return !(leaseRenewer1.isRunning());
|
||||
}}, 100, 10000);
|
||||
try {
|
||||
out1.write(new byte[256]);
|
||||
fail("existing output stream should be aborted");
|
||||
|
@ -400,14 +407,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();
|
||||
}
|
||||
|
@ -747,12 +754,13 @@ public class TestDFSClientRetries {
|
|||
return ret;
|
||||
}
|
||||
|
||||
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());
|
||||
private void verifyEmptyLease(final LeaseRenewer leaseRenewer)
|
||||
throws Exception {
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
@Override
|
||||
public Boolean get() {
|
||||
return leaseRenewer.isEmpty();
|
||||
}}, 100, 10000);
|
||||
}
|
||||
|
||||
class DFSClientReader implements Runnable {
|
||||
|
|
Loading…
Reference in New Issue