HADOOP-14788. Credentials readTokenStorageFile to stop wrapping IOEs in IOEs.

Contributed by Ajay Kumar.
This commit is contained in:
Steve Loughran 2018-01-19 14:48:45 +00:00
parent c5bbd6418e
commit e5a1ad6e24
3 changed files with 82 additions and 2 deletions

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.io;
import java.io.*;
import java.lang.reflect.Constructor;
import java.net.Socket;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
@ -35,6 +36,7 @@ import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.util.Shell;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -450,4 +452,58 @@ public class IOUtils {
throw ioe;
}
}
/**
* Takes an IOException, file/directory path, and method name and returns an
* IOException with the input exception as the cause and also include the
* file,method details. The new exception provides the stack trace of the
* place where the exception is thrown and some extra diagnostics
* information.
*
* Return instance of same exception if exception class has a public string
* constructor; Otherwise return an PathIOException.
* InterruptedIOException and PathIOException are returned unwrapped.
*
* @param path file/directory path
* @param methodName method name
* @param exception the caught exception.
* @return an exception to throw
*/
public static IOException wrapException(final String path,
final String methodName, final IOException exception) {
if (exception instanceof InterruptedIOException
|| exception instanceof PathIOException) {
return exception;
} else {
String msg = String
.format("Failed with %s while processing file/directory :[%s] in "
+ "method:[%s]",
exception.getClass().getName(), path, methodName);
try {
return wrapWithMessage(exception, msg);
} catch (Exception ex) {
// For subclasses which have no (String) constructor throw IOException
// with wrapped message
return new PathIOException(path, exception);
}
}
}
@SuppressWarnings("unchecked")
private static <T extends IOException> T wrapWithMessage(
final T exception, final String msg) throws T {
Class<? extends Throwable> clazz = exception.getClass();
try {
Constructor<? extends Throwable> ctor = clazz
.getConstructor(String.class);
Throwable t = ctor.newInstance(msg);
return (T) (t.initCause(exception));
} catch (Throwable e) {
LOG.warn("Unable to wrap exception of type " +
clazz + ": it has no (String) constructor", e);
throw exception;
}
}
}

View File

@ -207,7 +207,8 @@ public class Credentials implements Writable {
in.close();
return credentials;
} catch(IOException ioe) {
throw new IOException("Exception reading " + filename, ioe);
throw IOUtils.wrapException(filename.toString(), "Credentials"
+ ".readTokenStorageFile", ioe);
} finally {
IOUtils.cleanupWithLogger(LOG, in);
}

View File

@ -33,13 +33,16 @@ import java.io.OutputStream;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.charset.CharacterCodingException;
import java.nio.file.Files;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.FileUtils;;
import org.apache.hadoop.fs.PathIOException;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.LambdaTestUtils;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
@ -328,4 +331,24 @@ public class TestIOUtils {
}
}
@Test
public void testWrapException() throws Exception {
// Test for IOException with valid (String) constructor
LambdaTestUtils.intercept(EOFException.class,
"Failed with java.io.EOFException while processing file/directory "
+ ":[/tmp/abc.txt] in method:[testWrapException]", () -> {
throw IOUtils.wrapException("/tmp/abc.txt", "testWrapException",
new EOFException("EOFException "));
});
// Test for IOException with no (String) constructor
PathIOException returnedEx = LambdaTestUtils
.intercept(PathIOException.class, "Input/output error:",
() -> {
throw IOUtils.wrapException("/tmp/abc.txt", "testWrapEx",
new CharacterCodingException());
});
assertEquals("/tmp/abc.txt", returnedEx.getPath().toString());
}
}