HDFS-9905. WebHdfsFileSystem#runWithRetry should display original stack trace on error. (Wei-Chiu Chuang via iwasakims)
(cherry picked from commit 6fcde2e38d
)
This commit is contained in:
parent
5949436010
commit
96111caca9
|
@ -737,8 +737,10 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
node = url.getAuthority();
|
||||
}
|
||||
try {
|
||||
ioe = ioe.getClass().getConstructor(String.class)
|
||||
.newInstance(node + ": " + ioe.getMessage());
|
||||
IOException newIoe = ioe.getClass().getConstructor(String.class)
|
||||
.newInstance(node + ": " + ioe.getMessage());
|
||||
newIoe.setStackTrace(ioe.getStackTrace());
|
||||
ioe = newIoe;
|
||||
} catch (NoSuchMethodException | SecurityException
|
||||
| InstantiationException | IllegalAccessException
|
||||
| IllegalArgumentException | InvocationTargetException e) {
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hdfs.web;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
|
@ -46,6 +45,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.runner.RunWith;
|
||||
|
@ -150,7 +150,8 @@ public class TestWebHdfsTimeouts {
|
|||
fs.listFiles(new Path("/"), false);
|
||||
fail("expected timeout");
|
||||
} catch (SocketTimeoutException e) {
|
||||
assertEquals(fs.getUri().getAuthority() + ": connect timed out", e.getMessage());
|
||||
GenericTestUtils.assertExceptionContains(fs.getUri().getAuthority()
|
||||
+ ": connect timed out",e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -163,7 +164,8 @@ public class TestWebHdfsTimeouts {
|
|||
fs.listFiles(new Path("/"), false);
|
||||
fail("expected timeout");
|
||||
} catch (SocketTimeoutException e) {
|
||||
assertEquals(fs.getUri().getAuthority() + ": Read timed out", e.getMessage());
|
||||
GenericTestUtils.assertExceptionContains(fs.getUri().getAuthority() +
|
||||
": Read timed out", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -178,7 +180,8 @@ public class TestWebHdfsTimeouts {
|
|||
fs.getDelegationToken("renewer");
|
||||
fail("expected timeout");
|
||||
} catch (SocketTimeoutException e) {
|
||||
assertEquals(fs.getUri().getAuthority() + ": connect timed out", e.getMessage());
|
||||
GenericTestUtils.assertExceptionContains(fs.getUri().getAuthority() +
|
||||
": connect timed out", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -192,7 +195,8 @@ public class TestWebHdfsTimeouts {
|
|||
fs.getDelegationToken("renewer");
|
||||
fail("expected timeout");
|
||||
} catch (SocketTimeoutException e) {
|
||||
assertEquals(fs.getUri().getAuthority() + ": Read timed out", e.getMessage());
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
fs.getUri().getAuthority() + ": Read timed out", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -207,7 +211,8 @@ public class TestWebHdfsTimeouts {
|
|||
fs.getFileChecksum(new Path("/file"));
|
||||
fail("expected timeout");
|
||||
} catch (SocketTimeoutException e) {
|
||||
assertEquals(fs.getUri().getAuthority() + ": connect timed out", e.getMessage());
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
fs.getUri().getAuthority() + ": connect timed out", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -222,7 +227,8 @@ public class TestWebHdfsTimeouts {
|
|||
fs.getFileChecksum(new Path("/file"));
|
||||
fail("expected timeout");
|
||||
} catch (SocketTimeoutException e) {
|
||||
assertEquals(fs.getUri().getAuthority() + ": Read timed out", e.getMessage());
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
fs.getUri().getAuthority() + ": Read timed out", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -238,7 +244,8 @@ public class TestWebHdfsTimeouts {
|
|||
os = fs.create(new Path("/file"));
|
||||
fail("expected timeout");
|
||||
} catch (SocketTimeoutException e) {
|
||||
assertEquals(fs.getUri().getAuthority() + ": connect timed out", e.getMessage());
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
fs.getUri().getAuthority() + ": connect timed out", e);
|
||||
} finally {
|
||||
IOUtils.cleanup(LOG, os);
|
||||
}
|
||||
|
@ -258,7 +265,7 @@ public class TestWebHdfsTimeouts {
|
|||
os = null;
|
||||
fail("expected timeout");
|
||||
} catch (SocketTimeoutException e) {
|
||||
assertEquals("Read timed out", e.getMessage());
|
||||
GenericTestUtils.assertExceptionContains("Read timed out", e);
|
||||
} finally {
|
||||
IOUtils.cleanup(LOG, os);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue