Browse Source

HADOOP-11015. Http server/client utils to propagate and recreate Exceptions from server to client. (tucu)

Alejandro Abdelnur 10 năm trước cách đây
mục cha
commit
70b218748b
15 tập tin đã thay đổi với 423 bổ sung227 xóa
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 1 1
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  3. 2 55
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  4. 3 12
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java
  5. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
  6. 2 18
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
  7. 185 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HttpExceptionUtils.java
  8. 14 21
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java
  9. 167 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestHttpExceptionUtils.java
  10. 2 10
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java
  11. 31 39
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  12. 0 50
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
  13. 2 12
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java
  14. 3 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  15. 4 6
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -498,6 +498,9 @@ Release 2.6.0 - UNRELEASED
 
     HADOOP-11054. Add a KeyProvider instantiation based on a URI. (tucu)
 
+    HADOOP-11015. Http server/client utils to propagate and recreate 
+    Exceptions from server to client. (tucu)
+
   OPTIMIZATIONS
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)

+ 1 - 1
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -367,7 +367,7 @@
      </Match>
 
   <Match>
-    <Class name="org.apache.hadoop.crypto.key.kms.KMSClientProvider"/>
+    <Class name="org.apache.hadoop.util.HttpExceptionUtils"/>
     <Method name="validateResponse"/>
     <Bug pattern="REC_CATCH_EXCEPTION"/>
   </Match>

+ 2 - 55
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.client.utils.URIBuilder;
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -44,7 +45,6 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
-import java.lang.reflect.Constructor;
 import java.net.HttpURLConnection;
 import java.net.SocketTimeoutException;
 import java.net.URI;
@@ -54,7 +54,6 @@ import java.net.URLEncoder;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 import java.security.PrivilegedExceptionAction;
-import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.HashMap;
@@ -413,58 +412,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return conn;
   }
 
-  // trick, riding on generics to throw an undeclared exception
-
-  private static void throwEx(Throwable ex) {
-    KMSClientProvider.<RuntimeException>throwException(ex);
-  }
-
-  @SuppressWarnings("unchecked")
-  private static <E extends Throwable> void throwException(Throwable ex)
-      throws E {
-    throw (E) ex;
-  }
-
-  @SuppressWarnings("unchecked")
-  private static void validateResponse(HttpURLConnection conn, int expected)
-      throws IOException {
-    int status = conn.getResponseCode();
-    if (status != expected) {
-      InputStream es = null;
-      try {
-        Exception toThrow;
-        String contentType = conn.getHeaderField(CONTENT_TYPE);
-        if (contentType != null &&
-            contentType.toLowerCase().startsWith(APPLICATION_JSON_MIME)) {
-          es = conn.getErrorStream();
-          ObjectMapper mapper = new ObjectMapper();
-          Map json = mapper.readValue(es, Map.class);
-          String exClass = (String) json.get(
-              KMSRESTConstants.ERROR_EXCEPTION_JSON);
-          String exMsg = (String)
-              json.get(KMSRESTConstants.ERROR_MESSAGE_JSON);
-          try {
-            ClassLoader cl = KMSClientProvider.class.getClassLoader();
-            Class klass = cl.loadClass(exClass);
-            Constructor constr = klass.getConstructor(String.class);
-            toThrow = (Exception) constr.newInstance(exMsg);
-          } catch (Exception ex) {
-            toThrow = new IOException(MessageFormat.format(
-                "HTTP status [{0}], {1}", status, conn.getResponseMessage()));
-          }
-        } else {
-          toThrow = new IOException(MessageFormat.format(
-              "HTTP status [{0}], {1}", status, conn.getResponseMessage()));
-        }
-        throwEx(toThrow);
-      } finally {
-        if (es != null) {
-          es.close();
-        }
-      }
-    }
-  }
-
   private static <T> T call(HttpURLConnection conn, Map jsonOutput,
       int expectedResponse, Class<T> klass)
       throws IOException {
@@ -477,7 +424,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       conn.getInputStream().close();
       throw ex;
     }
-    validateResponse(conn, expectedResponse);
+    HttpExceptionUtils.validateResponse(conn, expectedResponse);
     if (APPLICATION_JSON_MIME.equalsIgnoreCase(conn.getContentType())
         && klass != null) {
       ObjectMapper mapper = new ObjectMapper();

+ 3 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHand
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -221,18 +222,8 @@ public class DelegationTokenAuthenticationFilter
           try {
             ProxyUsers.authorize(ugi, request.getRemoteHost());
           } catch (AuthorizationException ex) {
-            String msg = String.format(
-                "User '%s' from host '%s' not allowed to impersonate user '%s'",
-                realUser, request.getRemoteHost(), doAsUser);
-            response.setStatus(HttpServletResponse.SC_FORBIDDEN);
-            response.setContentType(APPLICATION_JSON_MIME);
-            Map<String, String> json = new HashMap<String, String>();
-            json.put(ERROR_EXCEPTION_JSON,
-                AuthorizationException.class.getName());
-            json.put(ERROR_MESSAGE_JSON, msg);
-            Writer writer = response.getWriter();
-            ObjectMapper jsonMapper = new ObjectMapper();
-            jsonMapper.writeValue(writer, json);
+            HttpExceptionUtils.createServletExceptionResponse(response,
+                HttpServletResponse.SC_FORBIDDEN, ex);
             requestCompleted = true;
           }
         }

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import javax.servlet.ServletException;
@@ -346,8 +347,9 @@ public abstract class DelegationTokenAuthenticationHandler
         token.setExpires(0);
         request.setAttribute(DELEGATION_TOKEN_UGI_ATTRIBUTE, ugi);
       } catch (Throwable ex) {
-        throw new AuthenticationException("Could not verify DelegationToken, " +
-            ex.toString(), ex);
+        token = null;
+        HttpExceptionUtils.createServletExceptionResponse(response,
+            HttpServletResponse.SC_FORBIDDEN, new AuthenticationException(ex));
       }
     } else {
       token = authHandler.authenticate(request, response);

+ 2 - 18
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.security.authentication.client.Authenticator;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -217,7 +218,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     AuthenticatedURL aUrl = new AuthenticatedURL(this);
     HttpURLConnection conn = aUrl.openConnection(url, token);
     conn.setRequestMethod(operation.getHttpMethod());
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     if (hasResponse) {
       String contentType = conn.getHeaderField(CONTENT_TYPE);
       contentType = (contentType != null) ? contentType.toLowerCase()
@@ -241,21 +242,4 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     return ret;
   }
 
-  @SuppressWarnings("unchecked")
-  private static void validateResponse(HttpURLConnection conn, int expected)
-      throws IOException {
-    int status = conn.getResponseCode();
-    if (status != expected) {
-      try {
-        conn.getInputStream().close();
-      } catch (IOException ex) {
-        //NOP
-      }
-      String msg = String.format("HTTP status, expected [%d], got [%d]: %s", 
-          expected, status, conn.getResponseMessage());
-      LOG.debug(msg);
-      throw new IOException(msg);
-    }
-  }
-
 }

+ 185 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HttpExceptionUtils.java

@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Writer;
+import java.lang.reflect.Constructor;
+import java.net.HttpURLConnection;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * HTTP utility class to help propagate server side exception to the client
+ * over HTTP as a JSON payload.
+ * <p/>
+ * It creates HTTP Servlet and JAX-RPC error responses including details of the
+ * exception that allows a client to recreate the remote exception.
+ * <p/>
+ * It parses HTTP client connections and recreates the exception.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HttpExceptionUtils {
+
+  public static final String ERROR_JSON = "RemoteException";
+  public static final String ERROR_EXCEPTION_JSON = "exception";
+  public static final String ERROR_CLASSNAME_JSON = "javaClassName";
+  public static final String ERROR_MESSAGE_JSON = "message";
+
+  private static final String APPLICATION_JSON_MIME = "application/json";
+
+  private static final String ENTER = System.getProperty("line.separator");
+
+  /**
+   * Creates a HTTP servlet response serializing the exception in it as JSON.
+   *
+   * @param response the servlet response
+   * @param status the error code to set in the response
+   * @param ex the exception to serialize in the response
+   * @throws IOException thrown if there was an error while creating the
+   * response
+   */
+  public static void createServletExceptionResponse(
+      HttpServletResponse response, int status, Throwable ex)
+      throws IOException {
+    response.setStatus(status);
+    response.setContentType(APPLICATION_JSON_MIME);
+    Map<String, Object> json = new LinkedHashMap<String, Object>();
+    json.put(ERROR_MESSAGE_JSON, getOneLineMessage(ex));
+    json.put(ERROR_EXCEPTION_JSON, ex.getClass().getSimpleName());
+    json.put(ERROR_CLASSNAME_JSON, ex.getClass().getName());
+    Map<String, Object> jsonResponse = new LinkedHashMap<String, Object>();
+    jsonResponse.put(ERROR_JSON, json);
+    ObjectMapper jsonMapper = new ObjectMapper();
+    Writer writer = response.getWriter();
+    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, jsonResponse);
+    writer.flush();
+  }
+
+  /**
+   * Creates a HTTP JAX-RPC response serializing the exception in it as JSON.
+   *
+   * @param status the error code to set in the response
+   * @param ex the exception to serialize in the response
+   * @return the JAX-RPC response with the set error and JSON encoded exception
+   */
+  public static Response createJerseyExceptionResponse(Response.Status status,
+      Throwable ex) {
+    Map<String, Object> json = new LinkedHashMap<String, Object>();
+    json.put(ERROR_MESSAGE_JSON, getOneLineMessage(ex));
+    json.put(ERROR_EXCEPTION_JSON, ex.getClass().getSimpleName());
+    json.put(ERROR_CLASSNAME_JSON, ex.getClass().getName());
+    Map<String, Object> response = new LinkedHashMap<String, Object>();
+    response.put(ERROR_JSON, json);
+    return Response.status(status).type(MediaType.APPLICATION_JSON).
+        entity(response).build();
+  }
+
+  private static String getOneLineMessage(Throwable exception) {
+    String message = exception.getMessage();
+    if (message != null) {
+      int i = message.indexOf(ENTER);
+      if (i > -1) {
+        message = message.substring(0, i);
+      }
+    }
+    return message;
+  }
+
+  // trick, riding on generics to throw an undeclared exception
+
+  private static void throwEx(Throwable ex) {
+    HttpExceptionUtils.<RuntimeException>throwException(ex);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <E extends Throwable> void throwException(Throwable ex)
+      throws E {
+    throw (E) ex;
+  }
+
+  /**
+   * Validates the status of an <code>HttpURLConnection</code> against an
+   * expected HTTP status code. If the current status code is not the expected
+   * one it throws an exception with a detail message using Server side error
+   * messages if available.
+   * <p/>
+   * <b>NOTE:</b> this method will throw the deserialized exception even if not
+   * declared in the <code>throws</code> of the method signature.
+   *
+   * @param conn the <code>HttpURLConnection</code>.
+   * @param expectedStatus the expected HTTP status code.
+   * @throws IOException thrown if the current status code does not match the
+   * expected one.
+   */
+  @SuppressWarnings("unchecked")
+  public static void validateResponse(HttpURLConnection conn,
+      int expectedStatus) throws IOException {
+    if (conn.getResponseCode() != expectedStatus) {
+      Exception toThrow;
+      InputStream es = null;
+      try {
+        es = conn.getErrorStream();
+        ObjectMapper mapper = new ObjectMapper();
+        Map json = mapper.readValue(es, Map.class);
+        json = (Map) json.get(ERROR_JSON);
+        String exClass = (String) json.get(ERROR_CLASSNAME_JSON);
+        String exMsg = (String) json.get(ERROR_MESSAGE_JSON);
+        if (exClass != null) {
+          try {
+            ClassLoader cl = HttpExceptionUtils.class.getClassLoader();
+            Class klass = cl.loadClass(exClass);
+            Constructor constr = klass.getConstructor(String.class);
+            toThrow = (Exception) constr.newInstance(exMsg);
+          } catch (Exception ex) {
+            toThrow = new IOException(String.format(
+                "HTTP status [%d], exception [%s], message [%s] ",
+                conn.getResponseCode(), exClass, exMsg));
+          }
+        } else {
+          String msg = (exMsg != null) ? exMsg : conn.getResponseMessage();
+          toThrow = new IOException(String.format(
+              "HTTP status [%d], message [%s]", conn.getResponseCode(), msg));
+        }
+      } catch (Exception ex) {
+        toThrow = new IOException(String.format(
+            "HTTP status [%d], message [%s]", conn.getResponseCode(),
+            conn.getResponseMessage()));
+      } finally {
+        if (es != null) {
+          try {
+            es.close();
+          } catch (IOException ex) {
+            //ignore
+          }
+        }
+      }
+      throwEx(toThrow);
+    }
+  }
+
+}

+ 14 - 21
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.AuthenticationToken;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.After;
 import org.junit.Assert;
@@ -224,7 +225,8 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     Mockito.when(request.getQueryString()).thenReturn(
         DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" +
             DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
-            token.encodeToUrlString());
+            token.encodeToUrlString()
+    );
     Assert.assertFalse(handler.managementOperation(null, request, response));
     Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
     try {
@@ -273,8 +275,8 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
             UserGroupInformation.getCurrentUser(), "user");
     Mockito.when(request.getQueryString()).
         thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() +
-        "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
-        dToken.encodeToUrlString());
+            "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
+            dToken.encodeToUrlString());
     Assert.assertFalse(handler.managementOperation(token, request, response));
     Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
     pwriter.close();
@@ -333,15 +335,11 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
     Mockito.when(request.getQueryString()).thenReturn(
         DelegationTokenAuthenticator.DELEGATION_PARAM + "=invalid");
-
-    try {
-      handler.authenticate(request, response);
-      Assert.fail();
-    } catch (AuthenticationException ex) {
-      //NOP
-    } catch (Exception ex) {
-      Assert.fail();
-    }
+    StringWriter writer = new StringWriter();
+    Mockito.when(response.getWriter()).thenReturn(new PrintWriter(writer));
+    Assert.assertNull(handler.authenticate(request, response));
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_FORBIDDEN);
+    Assert.assertTrue(writer.toString().contains("AuthenticationException"));
   }
 
   private void testInvalidDelegationTokenHeader() throws Exception {
@@ -350,15 +348,10 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     Mockito.when(request.getHeader(Mockito.eq(
         DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER))).thenReturn(
         "invalid");
-
-    try {
-      handler.authenticate(request, response);
-      Assert.fail();
-    } catch (AuthenticationException ex) {
-      //NOP
-    } catch (Exception ex) {
-      Assert.fail();
-    }
+    StringWriter writer = new StringWriter();
+    Mockito.when(response.getWriter()).thenReturn(new PrintWriter(writer));
+    Assert.assertNull(handler.authenticate(request, response));
+    Assert.assertTrue(writer.toString().contains("AuthenticationException"));
   }
 
 }

+ 167 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestHttpExceptionUtils.java

@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.HttpURLConnection;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestHttpExceptionUtils {
+
+  @Test
+  public void testCreateServletException() throws IOException {
+    StringWriter writer = new StringWriter();
+    PrintWriter printWriter = new PrintWriter(writer);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(response.getWriter()).thenReturn(printWriter);
+    int status = HttpServletResponse.SC_INTERNAL_SERVER_ERROR;
+    Exception ex = new IOException("Hello IOEX");
+    HttpExceptionUtils.createServletExceptionResponse(response, status, ex);
+    Mockito.verify(response).setStatus(status);
+    Mockito.verify(response).setContentType(Mockito.eq("application/json"));
+    ObjectMapper mapper = new ObjectMapper();
+    Map json = mapper.readValue(writer.toString(), Map.class);
+    json = (Map) json.get(HttpExceptionUtils.ERROR_JSON);
+    Assert.assertEquals(IOException.class.getName(),
+        json.get(HttpExceptionUtils.ERROR_CLASSNAME_JSON));
+    Assert.assertEquals(IOException.class.getSimpleName(),
+        json.get(HttpExceptionUtils.ERROR_EXCEPTION_JSON));
+    Assert.assertEquals("Hello IOEX",
+        json.get(HttpExceptionUtils.ERROR_MESSAGE_JSON));
+  }
+
+  @Test
+  public void testCreateJerseyException() throws IOException {
+    Exception ex = new IOException("Hello IOEX");
+    Response response = HttpExceptionUtils.createJerseyExceptionResponse(
+        Response.Status.INTERNAL_SERVER_ERROR, ex);
+    Assert.assertEquals(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(),
+        response.getStatus());
+    Assert.assertArrayEquals(
+        Arrays.asList(MediaType.APPLICATION_JSON_TYPE).toArray(),
+        response.getMetadata().get("Content-Type").toArray());
+    Map entity = (Map) response.getEntity();
+    entity = (Map) entity.get(HttpExceptionUtils.ERROR_JSON);
+    Assert.assertEquals(IOException.class.getName(),
+        entity.get(HttpExceptionUtils.ERROR_CLASSNAME_JSON));
+    Assert.assertEquals(IOException.class.getSimpleName(),
+        entity.get(HttpExceptionUtils.ERROR_EXCEPTION_JSON));
+    Assert.assertEquals("Hello IOEX",
+        entity.get(HttpExceptionUtils.ERROR_MESSAGE_JSON));
+  }
+
+  @Test
+  public void testValidateResponseOK() throws IOException {
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_CREATED);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+  }
+
+  @Test(expected = IOException.class)
+  public void testValidateResponseFailNoErrorMessage() throws IOException {
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_BAD_REQUEST);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+  }
+
+  @Test
+  public void testValidateResponseNonJsonErrorMessage() throws IOException {
+    String msg = "stream";
+    InputStream is = new ByteArrayInputStream(msg.getBytes());
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getErrorStream()).thenReturn(is);
+    Mockito.when(conn.getResponseMessage()).thenReturn("msg");
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_BAD_REQUEST);
+    try {
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+      Assert.fail();
+    } catch (IOException ex) {
+      Assert.assertTrue(ex.getMessage().contains("msg"));
+      Assert.assertTrue(ex.getMessage().contains("" +
+          HttpURLConnection.HTTP_BAD_REQUEST));
+    }
+  }
+
+  @Test
+  public void testValidateResponseJsonErrorKnownException() throws IOException {
+    Map<String, Object> json = new HashMap<String, Object>();
+    json.put(HttpExceptionUtils.ERROR_EXCEPTION_JSON, IllegalStateException.class.getSimpleName());
+    json.put(HttpExceptionUtils.ERROR_CLASSNAME_JSON, IllegalStateException.class.getName());
+    json.put(HttpExceptionUtils.ERROR_MESSAGE_JSON, "EX");
+    Map<String, Object> response = new HashMap<String, Object>();
+    response.put(HttpExceptionUtils.ERROR_JSON, json);
+    ObjectMapper jsonMapper = new ObjectMapper();
+    String msg = jsonMapper.writeValueAsString(response);
+    InputStream is = new ByteArrayInputStream(msg.getBytes());
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getErrorStream()).thenReturn(is);
+    Mockito.when(conn.getResponseMessage()).thenReturn("msg");
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_BAD_REQUEST);
+    try {
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+      Assert.fail();
+    } catch (IllegalStateException ex) {
+      Assert.assertEquals("EX", ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testValidateResponseJsonErrorUnknownException()
+      throws IOException {
+    Map<String, Object> json = new HashMap<String, Object>();
+    json.put(HttpExceptionUtils.ERROR_EXCEPTION_JSON, "FooException");
+    json.put(HttpExceptionUtils.ERROR_CLASSNAME_JSON, "foo.FooException");
+    json.put(HttpExceptionUtils.ERROR_MESSAGE_JSON, "EX");
+    Map<String, Object> response = new HashMap<String, Object>();
+    response.put(HttpExceptionUtils.ERROR_JSON, json);
+    ObjectMapper jsonMapper = new ObjectMapper();
+    String msg = jsonMapper.writeValueAsString(response);
+    InputStream is = new ByteArrayInputStream(msg.getBytes());
+    HttpURLConnection conn = Mockito.mock(HttpURLConnection.class);
+    Mockito.when(conn.getErrorStream()).thenReturn(is);
+    Mockito.when(conn.getResponseMessage()).thenReturn("msg");
+    Mockito.when(conn.getResponseCode()).thenReturn(
+        HttpURLConnection.HTTP_BAD_REQUEST);
+    try {
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_CREATED);
+      Assert.fail();
+    } catch (IOException ex) {
+      Assert.assertTrue(ex.getMessage().contains("EX"));
+      Assert.assertTrue(ex.getMessage().contains("foo.FooException"));
+    }
+  }
+
+}

+ 2 - 10
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java

@@ -21,22 +21,19 @@ import org.apache.hadoop.classification.InterfaceAudience;
 
 import com.sun.jersey.api.container.ContainerException;
 
-import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.Provider;
 
 import java.io.IOException;
-import java.util.LinkedHashMap;
-import java.util.Map;
 
 /**
  * Jersey provider that converts KMS exceptions into detailed HTTP errors.
@@ -50,12 +47,7 @@ public class KMSExceptionsProvider implements ExceptionMapper<Exception> {
   private static final String ENTER = System.getProperty("line.separator");
 
   protected Response createResponse(Response.Status status, Throwable ex) {
-    Map<String, Object> json = new LinkedHashMap<String, Object>();
-    json.put(KMSRESTConstants.ERROR_EXCEPTION_JSON, ex.getClass().getName());
-    json.put(KMSRESTConstants.ERROR_MESSAGE_JSON, getOneLineMessage(ex));
-    log(status, ex);
-    return Response.status(status).type(MediaType.APPLICATION_JSON).
-        entity(json).build();
+    return HttpExceptionUtils.createJerseyExceptionResponse(status, ex);
   }
 
   protected String getOneLineMessage(Throwable exception) {

+ 31 - 39
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -40,13 +40,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.lib.wsrs.EnumSetParam;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
 import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -179,11 +178,6 @@ public class HttpFSFileSystem extends FileSystem
   public static final String ACL_ENTRIES_JSON = "entries";
   public static final String ACL_BIT_JSON = "aclBit";
 
-  public static final String ERROR_JSON = "RemoteException";
-  public static final String ERROR_EXCEPTION_JSON = "exception";
-  public static final String ERROR_CLASSNAME_JSON = "javaClassName";
-  public static final String ERROR_MESSAGE_JSON = "message";
-
   public static final int HTTP_TEMPORARY_REDIRECT = 307;
 
   private static final String HTTP_GET = "GET";
@@ -223,7 +217,6 @@ public class HttpFSFileSystem extends FileSystem
   private URI uri;
   private Path workingDir;
   private UserGroupInformation realUser;
-  private String doAs;
 
 
 
@@ -336,7 +329,6 @@ public class HttpFSFileSystem extends FileSystem
     if (realUser == null) {
       realUser = UserGroupInformation.getLoginUser();
     }
-    doAs = ugi.getShortUserName();
     super.initialize(name, conf);
     try {
       uri = new URI(name.getScheme() + "://" + name.getAuthority());
@@ -435,7 +427,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.OPEN.toString());
     HttpURLConnection conn = getConnection(Operation.OPEN.getMethod(), params,
                                            f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     return new FSDataInputStream(
       new HttpFSDataInputStream(conn.getInputStream(), bufferSize));
   }
@@ -462,7 +454,7 @@ public class HttpFSFileSystem extends FileSystem
       try {
         super.close();
       } finally {
-        HttpFSUtils.validateResponse(conn, closeStatus);
+        HttpExceptionUtils.validateResponse(conn, closeStatus);
       }
     }
 
@@ -498,11 +490,11 @@ public class HttpFSFileSystem extends FileSystem
             OutputStream os = new BufferedOutputStream(conn.getOutputStream(), bufferSize);
             return new HttpFSDataOutputStream(conn, os, expectedStatus, statistics);
           } catch (IOException ex) {
-            HttpFSUtils.validateResponse(conn, expectedStatus);
+            HttpExceptionUtils.validateResponse(conn, expectedStatus);
             throw ex;
           }
         } else {
-          HttpFSUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
+          HttpExceptionUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
           throw new IOException("Missing HTTP 'Location' header for [" + conn.getURL() + "]");
         }
       } else {
@@ -514,7 +506,7 @@ public class HttpFSFileSystem extends FileSystem
       if (exceptionAlreadyHandled) {
         throw ex;
       } else {
-        HttpFSUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
+        HttpExceptionUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
         throw ex;
       }
     }
@@ -595,7 +587,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(SOURCES_PARAM, srcs);
     HttpURLConnection conn = getConnection(Operation.CONCAT.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -609,7 +601,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(DESTINATION_PARAM, dst.toString());
     HttpURLConnection conn = getConnection(Operation.RENAME.getMethod(),
                                            params, src, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(RENAME_JSON);
   }
@@ -644,7 +636,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(RECURSIVE_PARAM, Boolean.toString(recursive));
     HttpURLConnection conn = getConnection(Operation.DELETE.getMethod(),
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(DELETE_JSON);
   }
@@ -665,7 +657,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.LISTSTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.LISTSTATUS.getMethod(),
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(FILE_STATUSES_JSON);
     JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
@@ -713,7 +705,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(PERMISSION_PARAM, permissionToString(permission));
     HttpURLConnection conn = getConnection(Operation.MKDIRS.getMethod(),
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(MKDIRS_JSON);
   }
@@ -734,7 +726,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETFILESTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.GETFILESTATUS.getMethod(),
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(FILE_STATUS_JSON);
     f = makeQualified(f);
@@ -753,7 +745,7 @@ public class HttpFSFileSystem extends FileSystem
       HttpURLConnection conn =
         getConnection(Operation.GETHOMEDIRECTORY.getMethod(), params,
                       new Path(getUri().toString(), "/"), false);
-      HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
       JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
       return new Path((String) json.get(HOME_DIR_JSON));
     } catch (IOException ex) {
@@ -778,7 +770,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(GROUP_PARAM, groupname);
     HttpURLConnection conn = getConnection(Operation.SETOWNER.getMethod(),
                                            params, p, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -793,7 +785,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.SETPERMISSION.toString());
     params.put(PERMISSION_PARAM, permissionToString(permission));
     HttpURLConnection conn = getConnection(Operation.SETPERMISSION.getMethod(), params, p, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -815,7 +807,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACCESS_TIME_PARAM, Long.toString(atime));
     HttpURLConnection conn = getConnection(Operation.SETTIMES.getMethod(),
                                            params, p, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -837,7 +829,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(REPLICATION_PARAM, Short.toString(replication));
     HttpURLConnection conn =
       getConnection(Operation.SETREPLICATION.getMethod(), params, src, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(SET_REPLICATION_JSON);
   }
@@ -857,7 +849,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(
             Operation.MODIFYACLENTRIES.getMethod(), params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -874,7 +866,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(
             Operation.REMOVEACLENTRIES.getMethod(), params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -888,7 +880,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.REMOVEDEFAULTACL.toString());
     HttpURLConnection conn = getConnection(
             Operation.REMOVEDEFAULTACL.getMethod(), params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -902,7 +894,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.REMOVEACL.toString());
     HttpURLConnection conn = getConnection(Operation.REMOVEACL.getMethod(),
             params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -920,7 +912,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(Operation.SETACL.getMethod(),
                                            params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -935,7 +927,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETACLSTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.GETACLSTATUS.getMethod(),
             params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(ACL_STATUS_JSON);
     return createAclStatus(json);
@@ -996,7 +988,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETCONTENTSUMMARY.toString());
     HttpURLConnection conn =
       getConnection(Operation.GETCONTENTSUMMARY.getMethod(), params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) ((JSONObject)
       HttpFSUtils.jsonParse(conn)).get(CONTENT_SUMMARY_JSON);
     return new ContentSummary((Long) json.get(CONTENT_SUMMARY_LENGTH_JSON),
@@ -1014,7 +1006,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETFILECHECKSUM.toString());
     HttpURLConnection conn =
       getConnection(Operation.GETFILECHECKSUM.getMethod(), params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     final JSONObject json = (JSONObject) ((JSONObject)
       HttpFSUtils.jsonParse(conn)).get(FILE_CHECKSUM_JSON);
     return new FileChecksum() {
@@ -1115,7 +1107,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_SET_FLAG_PARAM, EnumSetParam.toString(flag));
     HttpURLConnection conn = getConnection(Operation.SETXATTR.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   @Override
@@ -1125,7 +1117,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_NAME_PARAM, name);
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     Map<String, byte[]> xAttrs = createXAttrMap(
         (JSONArray) json.get(XATTRS_JSON));
@@ -1169,7 +1161,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETXATTRS.toString());
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
   }
@@ -1185,7 +1177,7 @@ public class HttpFSFileSystem extends FileSystem
     multiValuedParams.put(XATTR_NAME_PARAM, names);
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, multiValuedParams, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
   }
@@ -1196,7 +1188,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.LISTXATTRS.toString());
     HttpURLConnection conn = getConnection(Operation.LISTXATTRS.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrNames((String) json.get(XATTRNAMES_JSON));
   }
@@ -1208,6 +1200,6 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_NAME_PARAM, name);
     HttpURLConnection conn = getConnection(Operation.REMOVEXATTR.getMethod(),
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 }

+ 0 - 50
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java

@@ -19,13 +19,11 @@ package org.apache.hadoop.fs.http.client;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
-import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
 import org.json.simple.parser.ParseException;
 
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.lang.reflect.Constructor;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
@@ -115,54 +113,6 @@ public class HttpFSUtils {
     return new URL(sb.toString());
   }
 
-  /**
-   * Validates the status of an <code>HttpURLConnection</code> against an
-   * expected HTTP status code. If the current status code is not the expected
-   * one it throws an exception with a detail message using Server side error
-   * messages if available.
-   *
-   * @param conn the <code>HttpURLConnection</code>.
-   * @param expected the expected HTTP status code.
-   *
-   * @throws IOException thrown if the current status code does not match the
-   * expected one.
-   */
-  @SuppressWarnings({"unchecked"})
-  static void validateResponse(HttpURLConnection conn, int expected)
-    throws IOException {
-    int status = conn.getResponseCode();
-    if (status != expected) {
-      try {
-        JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
-        json = (JSONObject) json.get(HttpFSFileSystem.ERROR_JSON);
-        String message = (String) json.get(HttpFSFileSystem.ERROR_MESSAGE_JSON);
-        String exception = (String)
-          json.get(HttpFSFileSystem.ERROR_EXCEPTION_JSON);
-        String className = (String)
-          json.get(HttpFSFileSystem.ERROR_CLASSNAME_JSON);
-
-        try {
-          ClassLoader cl = HttpFSFileSystem.class.getClassLoader();
-          Class klass = cl.loadClass(className);
-          Constructor constr = klass.getConstructor(String.class);
-          throw (IOException) constr.newInstance(message);
-        } catch (IOException ex) {
-          throw ex;
-        } catch (Exception ex) {
-          throw new IOException(MessageFormat.format("{0} - {1}", exception,
-                                                     message));
-        }
-      } catch (IOException ex) {
-        if (ex.getCause() instanceof IOException) {
-          throw (IOException) ex.getCause();
-        }
-        throw new IOException(
-          MessageFormat.format("HTTP status [{0}], {1}",
-                               status, conn.getResponseMessage()));
-      }
-    }
-  }
-
   /**
    * Convenience method that JSON Parses the <code>InputStream</code> of a
    * <code>HttpURLConnection</code>.

+ 2 - 12
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java

@@ -19,15 +19,12 @@
 package org.apache.hadoop.lib.wsrs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
-import java.util.LinkedHashMap;
-import java.util.Map;
 
 @InterfaceAudience.Private
 public class ExceptionProvider implements ExceptionMapper<Throwable> {
@@ -36,14 +33,7 @@ public class ExceptionProvider implements ExceptionMapper<Throwable> {
   private static final String ENTER = System.getProperty("line.separator");
 
   protected Response createResponse(Response.Status status, Throwable throwable) {
-    Map<String, Object> json = new LinkedHashMap<String, Object>();
-    json.put(HttpFSFileSystem.ERROR_MESSAGE_JSON, getOneLineMessage(throwable));
-    json.put(HttpFSFileSystem.ERROR_EXCEPTION_JSON, throwable.getClass().getSimpleName());
-    json.put(HttpFSFileSystem.ERROR_CLASSNAME_JSON, throwable.getClass().getName());
-    Map<String, Object> response = new LinkedHashMap<String, Object>();
-    response.put(HttpFSFileSystem.ERROR_JSON, json);
-    log(status, throwable);
-    return Response.status(status).type(MediaType.APPLICATION_JSON).entity(response).build();
+    return HttpExceptionUtils.createJerseyExceptionResponse(status, throwable);
   }
 
   protected String getOneLineMessage(Throwable throwable) {

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -183,6 +183,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
 
   private void testCreate() throws Exception {
     Path path = new Path(getProxiedFSTestDir(), "foo.txt");
+    FileSystem fs = FileSystem.get(getProxiedFSConf());
+    fs.delete(path, true);
     testCreate(path, false);
     testCreate(path, true);
     try {
@@ -190,7 +192,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       Assert.fail("the create should have failed because the file exists " +
                   "and override is FALSE");
     } catch (IOException ex) {
-
+System.out.println("#");
     } catch (Exception ex) {
       Assert.fail(ex.toString());
     }

+ 4 - 6
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServerNoACLs.java

@@ -189,9 +189,8 @@ public class TestHttpFSServerNoACLs extends HTestCase {
       Assert.assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp);
       reader = new BufferedReader(new InputStreamReader(conn.getErrorStream()));
       String res = reader.readLine();
-      Assert.assertTrue(res.contains("RemoteException"));
-      Assert.assertTrue(res.contains("ACL"));
-      Assert.assertTrue(res.contains("rejected"));
+      Assert.assertTrue(res.contains("AclException"));
+      Assert.assertTrue(res.contains("Support for ACLs has been disabled"));
     }
   }
 
@@ -224,9 +223,8 @@ public class TestHttpFSServerNoACLs extends HTestCase {
       BufferedReader reader;
       reader = new BufferedReader(new InputStreamReader(conn.getErrorStream()));
       String err = reader.readLine();
-      Assert.assertTrue(err.contains("RemoteException"));
-      Assert.assertTrue(err.contains("ACL"));
-      Assert.assertTrue(err.contains("rejected"));
+      Assert.assertTrue(err.contains("AclException"));
+      Assert.assertTrue(err.contains("Support for ACLs has been disabled"));
     }
   }