|
@@ -18,7 +18,6 @@
|
|
|
|
|
|
package org.apache.hadoop.ipc;
|
|
|
|
|
|
-import java.lang.reflect.Field;
|
|
|
import java.lang.reflect.Proxy;
|
|
|
import java.lang.reflect.Method;
|
|
|
import java.lang.reflect.Array;
|
|
@@ -27,18 +26,14 @@ import java.lang.reflect.InvocationTargetException;
|
|
|
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.io.*;
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Arrays;
|
|
|
-import java.util.List;
|
|
|
import java.io.Closeable;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.HashMap;
|
|
|
|
|
|
import javax.net.SocketFactory;
|
|
|
|
|
|
import org.apache.commons.logging.*;
|
|
|
|
|
|
import org.apache.hadoop.io.*;
|
|
|
+import org.apache.hadoop.ipc.RPC.RpcInvoker;
|
|
|
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
|
|
|
import org.apache.hadoop.ipc.VersionedProtocol;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
@@ -53,36 +48,9 @@ import org.apache.hadoop.conf.*;
|
|
|
public class WritableRpcEngine implements RpcEngine {
|
|
|
private static final Log LOG = LogFactory.getLog(RPC.class);
|
|
|
|
|
|
-
|
|
|
- /**
|
|
|
- * Get all superInterfaces that extend VersionedProtocol
|
|
|
- * @param childInterfaces
|
|
|
- * @return the super interfaces that extend VersionedProtocol
|
|
|
- */
|
|
|
- private static Class<?>[] getSuperInterfaces(Class<?>[] childInterfaces) {
|
|
|
- List<Class<?>> allInterfaces = new ArrayList<Class<?>>();
|
|
|
-
|
|
|
- for (Class<?> childInterface : childInterfaces) {
|
|
|
- if (VersionedProtocol.class.isAssignableFrom(childInterface)) {
|
|
|
- allInterfaces.add(childInterface);
|
|
|
- allInterfaces.addAll(
|
|
|
- Arrays.asList(
|
|
|
- getSuperInterfaces(childInterface.getInterfaces())));
|
|
|
- } else {
|
|
|
- LOG.warn("Interface " + childInterface +
|
|
|
- " ignored because it does not extend VersionedProtocol");
|
|
|
- }
|
|
|
- }
|
|
|
- return (Class<?>[]) allInterfaces.toArray(new Class[allInterfaces.size()]);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Get all interfaces that the given protocol implements or extends
|
|
|
- * which are assignable from VersionedProtocol.
|
|
|
- */
|
|
|
- private static Class<?>[] getProtocolInterfaces(Class<?> protocol) {
|
|
|
- Class<?>[] interfaces = protocol.getInterfaces();
|
|
|
- return getSuperInterfaces(interfaces);
|
|
|
+ static { // Register the rpcRequest deserializer for WritableRpcEngine
|
|
|
+ org.apache.hadoop.ipc.Server.registerProtocolEngine(RpcKind.RPC_WRITABLE,
|
|
|
+ Invocation.class, new Server.WritableRpcInvoker());
|
|
|
}
|
|
|
|
|
|
|
|
@@ -120,15 +88,7 @@ public class WritableRpcEngine implements RpcEngine {
|
|
|
clientVersion = 0;
|
|
|
clientMethodsHash = 0;
|
|
|
} else {
|
|
|
- try {
|
|
|
- Field versionField = method.getDeclaringClass().getField("versionID");
|
|
|
- versionField.setAccessible(true);
|
|
|
- this.clientVersion = versionField.getLong(method.getDeclaringClass());
|
|
|
- } catch (NoSuchFieldException ex) {
|
|
|
- throw new RuntimeException(ex);
|
|
|
- } catch (IllegalAccessException ex) {
|
|
|
- throw new RuntimeException(ex);
|
|
|
- }
|
|
|
+ this.clientVersion = RPC.getProtocolVersion(method.getDeclaringClass());
|
|
|
this.clientMethodsHash = ProtocolSignature.getFingerprint(method
|
|
|
.getDeclaringClass().getMethods());
|
|
|
}
|
|
@@ -329,140 +289,25 @@ public class WritableRpcEngine implements RpcEngine {
|
|
|
|
|
|
/** An RPC Server. */
|
|
|
public static class Server extends RPC.Server {
|
|
|
- private boolean verbose;
|
|
|
-
|
|
|
/**
|
|
|
- * The key in Map
|
|
|
- */
|
|
|
- static class ProtoNameVer {
|
|
|
- final String protocol;
|
|
|
- final long version;
|
|
|
- ProtoNameVer(String protocol, long ver) {
|
|
|
- this.protocol = protocol;
|
|
|
- this.version = ver;
|
|
|
- }
|
|
|
- @Override
|
|
|
- public boolean equals(Object o) {
|
|
|
- if (o == null)
|
|
|
- return false;
|
|
|
- if (this == o)
|
|
|
- return true;
|
|
|
- if (! (o instanceof ProtoNameVer))
|
|
|
- return false;
|
|
|
- ProtoNameVer pv = (ProtoNameVer) o;
|
|
|
- return ((pv.protocol.equals(this.protocol)) &&
|
|
|
- (pv.version == this.version));
|
|
|
- }
|
|
|
- @Override
|
|
|
- public int hashCode() {
|
|
|
- return protocol.hashCode() * 37 + (int) version;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * The value in map
|
|
|
- */
|
|
|
- static class ProtoClassProtoImpl {
|
|
|
- final Class<?> protocolClass;
|
|
|
- final Object protocolImpl;
|
|
|
- ProtoClassProtoImpl(Class<?> protocolClass, Object protocolImpl) {
|
|
|
- this.protocolClass = protocolClass;
|
|
|
- this.protocolImpl = protocolImpl;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private Map<ProtoNameVer, ProtoClassProtoImpl> protocolImplMap =
|
|
|
- new HashMap<ProtoNameVer, ProtoClassProtoImpl>(10);
|
|
|
-
|
|
|
- // Register protocol and its impl for rpc calls
|
|
|
- private void registerProtocolAndImpl(Class<?> protocolClass,
|
|
|
- Object protocolImpl) throws IOException {
|
|
|
- String protocolName = RPC.getProtocolName(protocolClass);
|
|
|
- VersionedProtocol vp = (VersionedProtocol) protocolImpl;
|
|
|
- long version;
|
|
|
- try {
|
|
|
- version = vp.getProtocolVersion(protocolName, 0);
|
|
|
- } catch (Exception ex) {
|
|
|
- LOG.warn("Protocol " + protocolClass +
|
|
|
- " NOT registered as getProtocolVersion throws exception ");
|
|
|
- return;
|
|
|
- }
|
|
|
- protocolImplMap.put(new ProtoNameVer(protocolName, version),
|
|
|
- new ProtoClassProtoImpl(protocolClass, protocolImpl));
|
|
|
- LOG.info("Protocol Name = " + protocolName + " version=" + version +
|
|
|
- " ProtocolImpl=" + protocolImpl.getClass().getName() +
|
|
|
- " protocolClass=" + protocolClass.getName());
|
|
|
- }
|
|
|
-
|
|
|
- private static class VerProtocolImpl {
|
|
|
- final long version;
|
|
|
- final ProtoClassProtoImpl protocolTarget;
|
|
|
- VerProtocolImpl(long ver, ProtoClassProtoImpl protocolTarget) {
|
|
|
- this.version = ver;
|
|
|
- this.protocolTarget = protocolTarget;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- @SuppressWarnings("unused") // will be useful later.
|
|
|
- private VerProtocolImpl[] getSupportedProtocolVersions(
|
|
|
- String protocolName) {
|
|
|
- VerProtocolImpl[] resultk = new VerProtocolImpl[protocolImplMap.size()];
|
|
|
- int i = 0;
|
|
|
- for (Map.Entry<ProtoNameVer, ProtoClassProtoImpl> pv :
|
|
|
- protocolImplMap.entrySet()) {
|
|
|
- if (pv.getKey().protocol.equals(protocolName)) {
|
|
|
- resultk[i++] =
|
|
|
- new VerProtocolImpl(pv.getKey().version, pv.getValue());
|
|
|
- }
|
|
|
- }
|
|
|
- if (i == 0) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- VerProtocolImpl[] result = new VerProtocolImpl[i];
|
|
|
- System.arraycopy(resultk, 0, result, 0, i);
|
|
|
- return result;
|
|
|
- }
|
|
|
-
|
|
|
- private VerProtocolImpl getHighestSupportedProtocol(String protocolName) {
|
|
|
- Long highestVersion = 0L;
|
|
|
- ProtoClassProtoImpl highest = null;
|
|
|
- for (Map.Entry<ProtoNameVer, ProtoClassProtoImpl> pv : protocolImplMap
|
|
|
- .entrySet()) {
|
|
|
- if (pv.getKey().protocol.equals(protocolName)) {
|
|
|
- if ((highest == null) || (pv.getKey().version > highestVersion)) {
|
|
|
- highest = pv.getValue();
|
|
|
- highestVersion = pv.getKey().version;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- if (highest == null) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- return new VerProtocolImpl(highestVersion, highest);
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- /** Construct an RPC server.
|
|
|
+ * Construct an RPC server.
|
|
|
* @param instance the instance whose methods will be called
|
|
|
* @param conf the configuration to use
|
|
|
* @param bindAddress the address to bind on to listen for connection
|
|
|
* @param port the port to listen for connections on
|
|
|
*
|
|
|
- * @deprecated Use #Server(Class, Object, Configuration, String, int)
|
|
|
- *
|
|
|
+ * @deprecated Use #Server(Class, Object, Configuration, String, int)
|
|
|
*/
|
|
|
@Deprecated
|
|
|
public Server(Object instance, Configuration conf, String bindAddress,
|
|
|
- int port)
|
|
|
- throws IOException {
|
|
|
+ int port) throws IOException {
|
|
|
this(null, instance, conf, bindAddress, port);
|
|
|
}
|
|
|
|
|
|
|
|
|
/** Construct an RPC server.
|
|
|
- * @param protocol class
|
|
|
- * @param instance the instance whose methods will be called
|
|
|
+ * @param protocolClass class
|
|
|
+ * @param protocolImpl the instance whose methods will be called
|
|
|
* @param conf the configuration to use
|
|
|
* @param bindAddress the address to bind on to listen for connection
|
|
|
* @param port the port to listen for connections on
|
|
@@ -474,16 +319,8 @@ public class WritableRpcEngine implements RpcEngine {
|
|
|
false, null);
|
|
|
}
|
|
|
|
|
|
- private static String classNameBase(String className) {
|
|
|
- String[] names = className.split("\\.", -1);
|
|
|
- if (names == null || names.length == 0) {
|
|
|
- return className;
|
|
|
- }
|
|
|
- return names[names.length-1];
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- /** Construct an RPC server.
|
|
|
+ /**
|
|
|
+ * Construct an RPC server.
|
|
|
* @param protocolImpl the instance whose methods will be called
|
|
|
* @param conf the configuration to use
|
|
|
* @param bindAddress the address to bind on to listen for connection
|
|
@@ -505,7 +342,8 @@ public class WritableRpcEngine implements RpcEngine {
|
|
|
|
|
|
}
|
|
|
|
|
|
- /** Construct an RPC server.
|
|
|
+ /**
|
|
|
+ * Construct an RPC server.
|
|
|
* @param protocolClass - the protocol being registered
|
|
|
* can be null for compatibility with old usage (see below for details)
|
|
|
* @param protocolImpl the protocol impl that will be called
|
|
@@ -520,7 +358,7 @@ public class WritableRpcEngine implements RpcEngine {
|
|
|
int numHandlers, int numReaders, int queueSizePerHandler,
|
|
|
boolean verbose, SecretManager<? extends TokenIdentifier> secretManager)
|
|
|
throws IOException {
|
|
|
- super(bindAddress, port, Invocation.class, numHandlers, numReaders,
|
|
|
+ super(bindAddress, port, null, numHandlers, numReaders,
|
|
|
queueSizePerHandler, conf,
|
|
|
classNameBase(protocolImpl.getClass().getName()), secretManager);
|
|
|
|
|
@@ -535,7 +373,7 @@ public class WritableRpcEngine implements RpcEngine {
|
|
|
* the protocolImpl is derived from the protocolClass(es)
|
|
|
* we register all interfaces extended by the protocolImpl
|
|
|
*/
|
|
|
- protocols = getProtocolInterfaces(protocolImpl.getClass());
|
|
|
+ protocols = RPC.getProtocolInterfaces(protocolImpl.getClass());
|
|
|
|
|
|
} else {
|
|
|
if (!protocolClass.isAssignableFrom(protocolImpl.getClass())) {
|
|
@@ -544,132 +382,125 @@ public class WritableRpcEngine implements RpcEngine {
|
|
|
protocolImpl.getClass());
|
|
|
}
|
|
|
// register protocol class and its super interfaces
|
|
|
- registerProtocolAndImpl(protocolClass, protocolImpl);
|
|
|
- protocols = getProtocolInterfaces(protocolClass);
|
|
|
+ registerProtocolAndImpl(RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
|
|
|
+ protocols = RPC.getProtocolInterfaces(protocolClass);
|
|
|
}
|
|
|
for (Class<?> p : protocols) {
|
|
|
if (!p.equals(VersionedProtocol.class)) {
|
|
|
- registerProtocolAndImpl(p, protocolImpl);
|
|
|
+ registerProtocolAndImpl(RpcKind.RPC_WRITABLE, p, protocolImpl);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
-
|
|
|
- @Override
|
|
|
- public <PROTO, IMPL extends PROTO> Server
|
|
|
- addProtocol(
|
|
|
- Class<PROTO> protocolClass, IMPL protocolImpl) throws IOException {
|
|
|
- registerProtocolAndImpl(protocolClass, protocolImpl);
|
|
|
- return this;
|
|
|
+ private static void log(String value) {
|
|
|
+ if (value!= null && value.length() > 55)
|
|
|
+ value = value.substring(0, 55)+"...";
|
|
|
+ LOG.info(value);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Process a client call
|
|
|
- * @param protocolName - the protocol name (the class of the client proxy
|
|
|
- * used to make calls to the rpc server.
|
|
|
- * @param param parameters
|
|
|
- * @param receivedTime time at which the call receoved (for metrics)
|
|
|
- * @return the call's return
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- public Writable call(String protocolName, Writable param, long receivedTime)
|
|
|
- throws IOException {
|
|
|
- try {
|
|
|
- Invocation call = (Invocation)param;
|
|
|
- if (verbose) log("Call: " + call);
|
|
|
-
|
|
|
- // Verify rpc version
|
|
|
- if (call.getRpcVersion() != writableRpcVersion) {
|
|
|
- // Client is using a different version of WritableRpc
|
|
|
- throw new IOException(
|
|
|
- "WritableRpc version mismatch, client side version="
|
|
|
- + call.getRpcVersion() + ", server side version="
|
|
|
- + writableRpcVersion);
|
|
|
- }
|
|
|
+ static class WritableRpcInvoker implements RpcInvoker {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Writable call(org.apache.hadoop.ipc.RPC.Server server,
|
|
|
+ String protocolName, Writable rpcRequest, long receivedTime)
|
|
|
+ throws IOException {
|
|
|
+ try {
|
|
|
+ Invocation call = (Invocation)rpcRequest;
|
|
|
+ if (server.verbose) log("Call: " + call);
|
|
|
+
|
|
|
+ // Verify rpc version
|
|
|
+ if (call.getRpcVersion() != writableRpcVersion) {
|
|
|
+ // Client is using a different version of WritableRpc
|
|
|
+ throw new IOException(
|
|
|
+ "WritableRpc version mismatch, client side version="
|
|
|
+ + call.getRpcVersion() + ", server side version="
|
|
|
+ + writableRpcVersion);
|
|
|
+ }
|
|
|
|
|
|
- long clientVersion = call.getProtocolVersion();
|
|
|
- final String protoName;
|
|
|
- ProtoClassProtoImpl protocolImpl;
|
|
|
- if (call.declaringClassProtocolName.equals(VersionedProtocol.class.getName())) {
|
|
|
- // VersionProtocol methods are often used by client to figure out
|
|
|
- // which version of protocol to use.
|
|
|
- //
|
|
|
- // Versioned protocol methods should go the protocolName protocol
|
|
|
- // rather than the declaring class of the method since the
|
|
|
- // the declaring class is VersionedProtocol which is not
|
|
|
- // registered directly.
|
|
|
- // Send the call to the highest protocol version
|
|
|
- protocolImpl =
|
|
|
- getHighestSupportedProtocol(protocolName).protocolTarget;
|
|
|
- } else {
|
|
|
- protoName = call.declaringClassProtocolName;
|
|
|
-
|
|
|
- // Find the right impl for the protocol based on client version.
|
|
|
- ProtoNameVer pv =
|
|
|
- new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
|
|
|
- protocolImpl = protocolImplMap.get(pv);
|
|
|
- if (protocolImpl == null) { // no match for Protocol AND Version
|
|
|
- VerProtocolImpl highest =
|
|
|
- getHighestSupportedProtocol(protoName);
|
|
|
+ long clientVersion = call.getProtocolVersion();
|
|
|
+ final String protoName;
|
|
|
+ ProtoClassProtoImpl protocolImpl;
|
|
|
+ if (call.declaringClassProtocolName.equals(VersionedProtocol.class.getName())) {
|
|
|
+ // VersionProtocol methods are often used by client to figure out
|
|
|
+ // which version of protocol to use.
|
|
|
+ //
|
|
|
+ // Versioned protocol methods should go the protocolName protocol
|
|
|
+ // rather than the declaring class of the method since the
|
|
|
+ // the declaring class is VersionedProtocol which is not
|
|
|
+ // registered directly.
|
|
|
+ // Send the call to the highest protocol version
|
|
|
+ VerProtocolImpl highest = server.getHighestSupportedProtocol(
|
|
|
+ RpcKind.RPC_WRITABLE, protocolName);
|
|
|
if (highest == null) {
|
|
|
- throw new IOException("Unknown protocol: " + protoName);
|
|
|
- } else { // protocol supported but not the version that client wants
|
|
|
- throw new RPC.VersionMismatch(protoName, clientVersion,
|
|
|
- highest.version);
|
|
|
+ throw new IOException("Unknown protocol: " + protocolName);
|
|
|
+ }
|
|
|
+ protocolImpl = highest.protocolTarget;
|
|
|
+ } else {
|
|
|
+ protoName = call.declaringClassProtocolName;
|
|
|
+
|
|
|
+ // Find the right impl for the protocol based on client version.
|
|
|
+ ProtoNameVer pv =
|
|
|
+ new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
|
|
|
+ protocolImpl =
|
|
|
+ server.getProtocolImplMap(RpcKind.RPC_WRITABLE).get(pv);
|
|
|
+ if (protocolImpl == null) { // no match for Protocol AND Version
|
|
|
+ VerProtocolImpl highest =
|
|
|
+ server.getHighestSupportedProtocol(RpcKind.RPC_WRITABLE,
|
|
|
+ protoName);
|
|
|
+ if (highest == null) {
|
|
|
+ throw new IOException("Unknown protocol: " + protoName);
|
|
|
+ } else { // protocol supported but not the version that client wants
|
|
|
+ throw new RPC.VersionMismatch(protoName, clientVersion,
|
|
|
+ highest.version);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- // Invoke the protocol method
|
|
|
-
|
|
|
- long startTime = System.currentTimeMillis();
|
|
|
- Method method =
|
|
|
- protocolImpl.protocolClass.getMethod(call.getMethodName(),
|
|
|
- call.getParameterClasses());
|
|
|
- method.setAccessible(true);
|
|
|
- rpcDetailedMetrics.init(protocolImpl.protocolClass);
|
|
|
- Object value =
|
|
|
- method.invoke(protocolImpl.protocolImpl, call.getParameters());
|
|
|
- int processingTime = (int) (System.currentTimeMillis() - startTime);
|
|
|
- int qTime = (int) (startTime-receivedTime);
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Served: " + call.getMethodName() +
|
|
|
- " queueTime= " + qTime +
|
|
|
- " procesingTime= " + processingTime);
|
|
|
- }
|
|
|
- rpcMetrics.addRpcQueueTime(qTime);
|
|
|
- rpcMetrics.addRpcProcessingTime(processingTime);
|
|
|
- rpcDetailedMetrics.addProcessingTime(call.getMethodName(),
|
|
|
- processingTime);
|
|
|
- if (verbose) log("Return: "+value);
|
|
|
-
|
|
|
- return new ObjectWritable(method.getReturnType(), value);
|
|
|
-
|
|
|
- } catch (InvocationTargetException e) {
|
|
|
- Throwable target = e.getTargetException();
|
|
|
- if (target instanceof IOException) {
|
|
|
- throw (IOException)target;
|
|
|
- } else {
|
|
|
- IOException ioe = new IOException(target.toString());
|
|
|
- ioe.setStackTrace(target.getStackTrace());
|
|
|
+
|
|
|
+
|
|
|
+ // Invoke the protocol method
|
|
|
+
|
|
|
+ long startTime = System.currentTimeMillis();
|
|
|
+ Method method =
|
|
|
+ protocolImpl.protocolClass.getMethod(call.getMethodName(),
|
|
|
+ call.getParameterClasses());
|
|
|
+ method.setAccessible(true);
|
|
|
+ server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
|
|
|
+ Object value =
|
|
|
+ method.invoke(protocolImpl.protocolImpl, call.getParameters());
|
|
|
+ int processingTime = (int) (System.currentTimeMillis() - startTime);
|
|
|
+ int qTime = (int) (startTime-receivedTime);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Served: " + call.getMethodName() +
|
|
|
+ " queueTime= " + qTime +
|
|
|
+ " procesingTime= " + processingTime);
|
|
|
+ }
|
|
|
+ server.rpcMetrics.addRpcQueueTime(qTime);
|
|
|
+ server.rpcMetrics.addRpcProcessingTime(processingTime);
|
|
|
+ server.rpcDetailedMetrics.addProcessingTime(call.getMethodName(),
|
|
|
+ processingTime);
|
|
|
+ if (server.verbose) log("Return: "+value);
|
|
|
+
|
|
|
+ return new ObjectWritable(method.getReturnType(), value);
|
|
|
+
|
|
|
+ } catch (InvocationTargetException e) {
|
|
|
+ Throwable target = e.getTargetException();
|
|
|
+ if (target instanceof IOException) {
|
|
|
+ throw (IOException)target;
|
|
|
+ } else {
|
|
|
+ IOException ioe = new IOException(target.toString());
|
|
|
+ ioe.setStackTrace(target.getStackTrace());
|
|
|
+ throw ioe;
|
|
|
+ }
|
|
|
+ } catch (Throwable e) {
|
|
|
+ if (!(e instanceof IOException)) {
|
|
|
+ LOG.error("Unexpected throwable object ", e);
|
|
|
+ }
|
|
|
+ IOException ioe = new IOException(e.toString());
|
|
|
+ ioe.setStackTrace(e.getStackTrace());
|
|
|
throw ioe;
|
|
|
}
|
|
|
- } catch (Throwable e) {
|
|
|
- if (!(e instanceof IOException)) {
|
|
|
- LOG.error("Unexpected throwable object ", e);
|
|
|
- }
|
|
|
- IOException ioe = new IOException(e.toString());
|
|
|
- ioe.setStackTrace(e.getStackTrace());
|
|
|
- throw ioe;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private static void log(String value) {
|
|
|
- if (value!= null && value.length() > 55)
|
|
|
- value = value.substring(0, 55)+"...";
|
|
|
- LOG.info(value);
|
|
|
- }
|
|
|
}
|