|
@@ -19,6 +19,7 @@ package org.apache.hadoop.net;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
|
|
+import java.io.InterruptedIOException;
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
import java.net.SocketTimeoutException;
|
|
import java.net.SocketTimeoutException;
|
|
import java.nio.channels.Pipe;
|
|
import java.nio.channels.Pipe;
|
|
@@ -26,8 +27,13 @@ import java.util.Arrays;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
|
+import org.apache.hadoop.test.MultithreadedTestUtil;
|
|
|
|
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
|
|
|
|
+import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
|
|
|
|
|
|
-import junit.framework.TestCase;
|
|
|
|
|
|
+import org.junit.Test;
|
|
|
|
+import static org.junit.Assert.*;
|
|
|
|
|
|
/**
|
|
/**
|
|
* This tests timout out from SocketInputStream and
|
|
* This tests timout out from SocketInputStream and
|
|
@@ -36,14 +42,17 @@ import junit.framework.TestCase;
|
|
* Normal read and write using these streams are tested by pretty much
|
|
* Normal read and write using these streams are tested by pretty much
|
|
* every DFS unit test.
|
|
* every DFS unit test.
|
|
*/
|
|
*/
|
|
-public class TestSocketIOWithTimeout extends TestCase {
|
|
|
|
|
|
+public class TestSocketIOWithTimeout {
|
|
|
|
|
|
static Log LOG = LogFactory.getLog(TestSocketIOWithTimeout.class);
|
|
static Log LOG = LogFactory.getLog(TestSocketIOWithTimeout.class);
|
|
|
|
|
|
private static int TIMEOUT = 1*1000;
|
|
private static int TIMEOUT = 1*1000;
|
|
private static String TEST_STRING = "1234567890";
|
|
private static String TEST_STRING = "1234567890";
|
|
|
|
+
|
|
|
|
+ private MultithreadedTestUtil.TestContext ctx = new TestContext();
|
|
|
|
|
|
- private void doIO(InputStream in, OutputStream out) throws IOException {
|
|
|
|
|
|
+ private void doIO(InputStream in, OutputStream out,
|
|
|
|
+ int expectedTimeout) throws IOException {
|
|
/* Keep on writing or reading until we get SocketTimeoutException.
|
|
/* Keep on writing or reading until we get SocketTimeoutException.
|
|
* It expects this exception to occur within 100 millis of TIMEOUT.
|
|
* It expects this exception to occur within 100 millis of TIMEOUT.
|
|
*/
|
|
*/
|
|
@@ -61,34 +70,15 @@ public class TestSocketIOWithTimeout extends TestCase {
|
|
long diff = System.currentTimeMillis() - start;
|
|
long diff = System.currentTimeMillis() - start;
|
|
LOG.info("Got SocketTimeoutException as expected after " +
|
|
LOG.info("Got SocketTimeoutException as expected after " +
|
|
diff + " millis : " + e.getMessage());
|
|
diff + " millis : " + e.getMessage());
|
|
- assertTrue(Math.abs(TIMEOUT - diff) <= 200);
|
|
|
|
|
|
+ assertTrue(Math.abs(expectedTimeout - diff) <=
|
|
|
|
+ TestNetUtils.TIME_FUDGE_MILLIS);
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Just reads one byte from the input stream.
|
|
|
|
- */
|
|
|
|
- static class ReadRunnable implements Runnable {
|
|
|
|
- private InputStream in;
|
|
|
|
-
|
|
|
|
- public ReadRunnable(InputStream in) {
|
|
|
|
- this.in = in;
|
|
|
|
- }
|
|
|
|
- public void run() {
|
|
|
|
- try {
|
|
|
|
- in.read();
|
|
|
|
- } catch (IOException e) {
|
|
|
|
- LOG.info("Got expection while reading as expected : " +
|
|
|
|
- e.getMessage());
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
- assertTrue(false);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- public void testSocketIOWithTimeout() throws IOException {
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testSocketIOWithTimeout() throws Exception {
|
|
|
|
|
|
// first open pipe:
|
|
// first open pipe:
|
|
Pipe pipe = Pipe.open();
|
|
Pipe pipe = Pipe.open();
|
|
@@ -96,7 +86,7 @@ public class TestSocketIOWithTimeout extends TestCase {
|
|
Pipe.SinkChannel sink = pipe.sink();
|
|
Pipe.SinkChannel sink = pipe.sink();
|
|
|
|
|
|
try {
|
|
try {
|
|
- InputStream in = new SocketInputStream(source, TIMEOUT);
|
|
|
|
|
|
+ final InputStream in = new SocketInputStream(source, TIMEOUT);
|
|
OutputStream out = new SocketOutputStream(sink, TIMEOUT);
|
|
OutputStream out = new SocketOutputStream(sink, TIMEOUT);
|
|
|
|
|
|
byte[] writeBytes = TEST_STRING.getBytes();
|
|
byte[] writeBytes = TEST_STRING.getBytes();
|
|
@@ -105,37 +95,62 @@ public class TestSocketIOWithTimeout extends TestCase {
|
|
|
|
|
|
out.write(writeBytes);
|
|
out.write(writeBytes);
|
|
out.write(byteWithHighBit);
|
|
out.write(byteWithHighBit);
|
|
- doIO(null, out);
|
|
|
|
|
|
+ doIO(null, out, TIMEOUT);
|
|
|
|
|
|
in.read(readBytes);
|
|
in.read(readBytes);
|
|
assertTrue(Arrays.equals(writeBytes, readBytes));
|
|
assertTrue(Arrays.equals(writeBytes, readBytes));
|
|
assertEquals(byteWithHighBit & 0xff, in.read());
|
|
assertEquals(byteWithHighBit & 0xff, in.read());
|
|
- doIO(in, null);
|
|
|
|
|
|
+ doIO(in, null, TIMEOUT);
|
|
|
|
+
|
|
|
|
+ // Change timeout on the read side.
|
|
|
|
+ ((SocketInputStream)in).setTimeout(TIMEOUT * 2);
|
|
|
|
+ doIO(in, null, TIMEOUT * 2);
|
|
|
|
+
|
|
|
|
|
|
/*
|
|
/*
|
|
* Verify that it handles interrupted threads properly.
|
|
* Verify that it handles interrupted threads properly.
|
|
- * Use a large timeout and expect the thread to return quickly.
|
|
|
|
|
|
+ * Use a large timeout and expect the thread to return quickly
|
|
|
|
+ * upon interruption.
|
|
*/
|
|
*/
|
|
- in = new SocketInputStream(source, 0);
|
|
|
|
- Thread thread = new Thread(new ReadRunnable(in));
|
|
|
|
- thread.start();
|
|
|
|
-
|
|
|
|
- try {
|
|
|
|
- Thread.sleep(1000);
|
|
|
|
- } catch (InterruptedException ignored) {}
|
|
|
|
-
|
|
|
|
|
|
+ ((SocketInputStream)in).setTimeout(0);
|
|
|
|
+ TestingThread thread = new TestingThread(ctx) {
|
|
|
|
+ @Override
|
|
|
|
+ public void doWork() throws Exception {
|
|
|
|
+ try {
|
|
|
|
+ in.read();
|
|
|
|
+ fail("Did not fail with interrupt");
|
|
|
|
+ } catch (InterruptedIOException ste) {
|
|
|
|
+ LOG.info("Got expection while reading as expected : " +
|
|
|
|
+ ste.getMessage());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+ ctx.addThread(thread);
|
|
|
|
+ ctx.startThreads();
|
|
|
|
+ // If the thread is interrupted before it calls read()
|
|
|
|
+ // then it throws ClosedByInterruptException due to
|
|
|
|
+ // some Java quirk. Waiting for it to call read()
|
|
|
|
+ // gets it into select(), so we get the expected
|
|
|
|
+ // InterruptedIOException.
|
|
|
|
+ Thread.sleep(1000);
|
|
thread.interrupt();
|
|
thread.interrupt();
|
|
|
|
+ ctx.stop();
|
|
|
|
+
|
|
|
|
+ //make sure the channels are still open
|
|
|
|
+ assertTrue(source.isOpen());
|
|
|
|
+ assertTrue(sink.isOpen());
|
|
|
|
|
|
|
|
+ // Nevertheless, the output stream is closed, because
|
|
|
|
+ // a partial write may have succeeded (see comment in
|
|
|
|
+ // SocketOutputStream#write(byte[]), int, int)
|
|
try {
|
|
try {
|
|
- thread.join();
|
|
|
|
- } catch (InterruptedException e) {
|
|
|
|
- throw new IOException("Unexpected InterruptedException : " + e);
|
|
|
|
|
|
+ out.write(1);
|
|
|
|
+ fail("Did not throw");
|
|
|
|
+ } catch (IOException ioe) {
|
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
|
+ "stream is closed", ioe);
|
|
}
|
|
}
|
|
|
|
|
|
- //make sure the channels are still open
|
|
|
|
- assertTrue(source.isOpen());
|
|
|
|
- assertTrue(sink.isOpen());
|
|
|
|
-
|
|
|
|
out.close();
|
|
out.close();
|
|
assertFalse(sink.isOpen());
|
|
assertFalse(sink.isOpen());
|
|
|
|
|