瀏覽代碼

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

Alejandro Abdelnur 10 年之前
父節點
當前提交
70b218748b
共有 15 個文件被更改,包括 423 次插入227 次删除
  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-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
   OPTIMIZATIONS
 
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)
     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>
 
 
   <Match>
   <Match>
-    <Class name="org.apache.hadoop.crypto.key.kms.KMSClientProvider"/>
+    <Class name="org.apache.hadoop.util.HttpExceptionUtils"/>
     <Method name="validateResponse"/>
     <Method name="validateResponse"/>
     <Bug pattern="REC_CATCH_EXCEPTION"/>
     <Bug pattern="REC_CATCH_EXCEPTION"/>
   </Match>
   </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.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.client.utils.URIBuilder;
 import org.apache.http.client.utils.URIBuilder;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
 
 
@@ -44,7 +45,6 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.io.Writer;
-import java.lang.reflect.Constructor;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
 import java.net.SocketTimeoutException;
 import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.net.URI;
@@ -54,7 +54,6 @@ import java.net.URLEncoder;
 import java.security.GeneralSecurityException;
 import java.security.GeneralSecurityException;
 import java.security.NoSuchAlgorithmException;
 import java.security.NoSuchAlgorithmException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
-import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -413,58 +412,6 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
     return conn;
     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,
   private static <T> T call(HttpURLConnection conn, Map jsonOutput,
       int expectedResponse, Class<T> klass)
       int expectedResponse, Class<T> klass)
       throws IOException {
       throws IOException {
@@ -477,7 +424,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       conn.getInputStream().close();
       conn.getInputStream().close();
       throw ex;
       throw ex;
     }
     }
-    validateResponse(conn, expectedResponse);
+    HttpExceptionUtils.validateResponse(conn, expectedResponse);
     if (APPLICATION_JSON_MIME.equalsIgnoreCase(conn.getContentType())
     if (APPLICATION_JSON_MIME.equalsIgnoreCase(conn.getContentType())
         && klass != null) {
         && klass != null) {
       ObjectMapper mapper = new ObjectMapper();
       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.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.http.NameValuePair;
 import org.apache.http.NameValuePair;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.apache.http.client.utils.URLEncodedUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -221,18 +222,8 @@ public class DelegationTokenAuthenticationFilter
           try {
           try {
             ProxyUsers.authorize(ugi, request.getRemoteHost());
             ProxyUsers.authorize(ugi, request.getRemoteHost());
           } catch (AuthorizationException ex) {
           } 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;
             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.authentication.server.KerberosAuthenticationHandler;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
 
 
 import javax.servlet.ServletException;
 import javax.servlet.ServletException;
@@ -346,8 +347,9 @@ public abstract class DelegationTokenAuthenticationHandler
         token.setExpires(0);
         token.setExpires(0);
         request.setAttribute(DELEGATION_TOKEN_UGI_ATTRIBUTE, ugi);
         request.setAttribute(DELEGATION_TOKEN_UGI_ATTRIBUTE, ugi);
       } catch (Throwable ex) {
       } 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 {
     } else {
       token = authHandler.authenticate(request, response);
       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.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -217,7 +218,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     AuthenticatedURL aUrl = new AuthenticatedURL(this);
     AuthenticatedURL aUrl = new AuthenticatedURL(this);
     HttpURLConnection conn = aUrl.openConnection(url, token);
     HttpURLConnection conn = aUrl.openConnection(url, token);
     conn.setRequestMethod(operation.getHttpMethod());
     conn.setRequestMethod(operation.getHttpMethod());
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     if (hasResponse) {
     if (hasResponse) {
       String contentType = conn.getHeaderField(CONTENT_TYPE);
       String contentType = conn.getHeaderField(CONTENT_TYPE);
       contentType = (contentType != null) ? contentType.toLowerCase()
       contentType = (contentType != null) ? contentType.toLowerCase()
@@ -241,21 +242,4 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     return ret;
     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.authentication.server.AuthenticationToken;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -224,7 +225,8 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     Mockito.when(request.getQueryString()).thenReturn(
     Mockito.when(request.getQueryString()).thenReturn(
         DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" +
         DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() + "&" +
             DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
             DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
-            token.encodeToUrlString());
+            token.encodeToUrlString()
+    );
     Assert.assertFalse(handler.managementOperation(null, request, response));
     Assert.assertFalse(handler.managementOperation(null, request, response));
     Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
     Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
     try {
     try {
@@ -273,8 +275,8 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
             UserGroupInformation.getCurrentUser(), "user");
             UserGroupInformation.getCurrentUser(), "user");
     Mockito.when(request.getQueryString()).
     Mockito.when(request.getQueryString()).
         thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() +
         thenReturn(DelegationTokenAuthenticator.OP_PARAM + "=" + op.toString() +
-        "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
-        dToken.encodeToUrlString());
+            "&" + DelegationTokenAuthenticator.TOKEN_PARAM + "=" +
+            dToken.encodeToUrlString());
     Assert.assertFalse(handler.managementOperation(token, request, response));
     Assert.assertFalse(handler.managementOperation(token, request, response));
     Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
     Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
     pwriter.close();
     pwriter.close();
@@ -333,15 +335,11 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
     Mockito.when(request.getQueryString()).thenReturn(
     Mockito.when(request.getQueryString()).thenReturn(
         DelegationTokenAuthenticator.DELEGATION_PARAM + "=invalid");
         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 {
   private void testInvalidDelegationTokenHeader() throws Exception {
@@ -350,15 +348,10 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     Mockito.when(request.getHeader(Mockito.eq(
     Mockito.when(request.getHeader(Mockito.eq(
         DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER))).thenReturn(
         DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER))).thenReturn(
         "invalid");
         "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 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.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.util.HttpExceptionUtils;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.Provider;
 import javax.ws.rs.ext.Provider;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.LinkedHashMap;
-import java.util.Map;
 
 
 /**
 /**
  * Jersey provider that converts KMS exceptions into detailed HTTP errors.
  * 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");
   private static final String ENTER = System.getProperty("line.separator");
 
 
   protected Response createResponse(Response.Status status, Throwable ex) {
   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) {
   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.hdfs.DFSConfigKeys;
 import org.apache.hadoop.lib.wsrs.EnumSetParam;
 import org.apache.hadoop.lib.wsrs.EnumSetParam;
 import org.apache.hadoop.security.UserGroupInformation;
 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.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 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.DelegationTokenAuthenticatedURL;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
 import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
 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.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 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_ENTRIES_JSON = "entries";
   public static final String ACL_BIT_JSON = "aclBit";
   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;
   public static final int HTTP_TEMPORARY_REDIRECT = 307;
 
 
   private static final String HTTP_GET = "GET";
   private static final String HTTP_GET = "GET";
@@ -223,7 +217,6 @@ public class HttpFSFileSystem extends FileSystem
   private URI uri;
   private URI uri;
   private Path workingDir;
   private Path workingDir;
   private UserGroupInformation realUser;
   private UserGroupInformation realUser;
-  private String doAs;
 
 
 
 
 
 
@@ -336,7 +329,6 @@ public class HttpFSFileSystem extends FileSystem
     if (realUser == null) {
     if (realUser == null) {
       realUser = UserGroupInformation.getLoginUser();
       realUser = UserGroupInformation.getLoginUser();
     }
     }
-    doAs = ugi.getShortUserName();
     super.initialize(name, conf);
     super.initialize(name, conf);
     try {
     try {
       uri = new URI(name.getScheme() + "://" + name.getAuthority());
       uri = new URI(name.getScheme() + "://" + name.getAuthority());
@@ -435,7 +427,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.OPEN.toString());
     params.put(OP_PARAM, Operation.OPEN.toString());
     HttpURLConnection conn = getConnection(Operation.OPEN.getMethod(), params,
     HttpURLConnection conn = getConnection(Operation.OPEN.getMethod(), params,
                                            f, true);
                                            f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     return new FSDataInputStream(
     return new FSDataInputStream(
       new HttpFSDataInputStream(conn.getInputStream(), bufferSize));
       new HttpFSDataInputStream(conn.getInputStream(), bufferSize));
   }
   }
@@ -462,7 +454,7 @@ public class HttpFSFileSystem extends FileSystem
       try {
       try {
         super.close();
         super.close();
       } finally {
       } 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);
             OutputStream os = new BufferedOutputStream(conn.getOutputStream(), bufferSize);
             return new HttpFSDataOutputStream(conn, os, expectedStatus, statistics);
             return new HttpFSDataOutputStream(conn, os, expectedStatus, statistics);
           } catch (IOException ex) {
           } catch (IOException ex) {
-            HttpFSUtils.validateResponse(conn, expectedStatus);
+            HttpExceptionUtils.validateResponse(conn, expectedStatus);
             throw ex;
             throw ex;
           }
           }
         } else {
         } else {
-          HttpFSUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
+          HttpExceptionUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
           throw new IOException("Missing HTTP 'Location' header for [" + conn.getURL() + "]");
           throw new IOException("Missing HTTP 'Location' header for [" + conn.getURL() + "]");
         }
         }
       } else {
       } else {
@@ -514,7 +506,7 @@ public class HttpFSFileSystem extends FileSystem
       if (exceptionAlreadyHandled) {
       if (exceptionAlreadyHandled) {
         throw ex;
         throw ex;
       } else {
       } else {
-        HttpFSUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
+        HttpExceptionUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
         throw ex;
         throw ex;
       }
       }
     }
     }
@@ -595,7 +587,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(SOURCES_PARAM, srcs);
     params.put(SOURCES_PARAM, srcs);
     HttpURLConnection conn = getConnection(Operation.CONCAT.getMethod(),
     HttpURLConnection conn = getConnection(Operation.CONCAT.getMethod(),
         params, f, true);
         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());
     params.put(DESTINATION_PARAM, dst.toString());
     HttpURLConnection conn = getConnection(Operation.RENAME.getMethod(),
     HttpURLConnection conn = getConnection(Operation.RENAME.getMethod(),
                                            params, src, true);
                                            params, src, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(RENAME_JSON);
     return (Boolean) json.get(RENAME_JSON);
   }
   }
@@ -644,7 +636,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(RECURSIVE_PARAM, Boolean.toString(recursive));
     params.put(RECURSIVE_PARAM, Boolean.toString(recursive));
     HttpURLConnection conn = getConnection(Operation.DELETE.getMethod(),
     HttpURLConnection conn = getConnection(Operation.DELETE.getMethod(),
                                            params, f, true);
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(DELETE_JSON);
     return (Boolean) json.get(DELETE_JSON);
   }
   }
@@ -665,7 +657,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.LISTSTATUS.toString());
     params.put(OP_PARAM, Operation.LISTSTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.LISTSTATUS.getMethod(),
     HttpURLConnection conn = getConnection(Operation.LISTSTATUS.getMethod(),
                                            params, f, true);
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(FILE_STATUSES_JSON);
     json = (JSONObject) json.get(FILE_STATUSES_JSON);
     JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
     JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
@@ -713,7 +705,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(PERMISSION_PARAM, permissionToString(permission));
     params.put(PERMISSION_PARAM, permissionToString(permission));
     HttpURLConnection conn = getConnection(Operation.MKDIRS.getMethod(),
     HttpURLConnection conn = getConnection(Operation.MKDIRS.getMethod(),
                                            params, f, true);
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(MKDIRS_JSON);
     return (Boolean) json.get(MKDIRS_JSON);
   }
   }
@@ -734,7 +726,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETFILESTATUS.toString());
     params.put(OP_PARAM, Operation.GETFILESTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.GETFILESTATUS.getMethod(),
     HttpURLConnection conn = getConnection(Operation.GETFILESTATUS.getMethod(),
                                            params, f, true);
                                            params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(FILE_STATUS_JSON);
     json = (JSONObject) json.get(FILE_STATUS_JSON);
     f = makeQualified(f);
     f = makeQualified(f);
@@ -753,7 +745,7 @@ public class HttpFSFileSystem extends FileSystem
       HttpURLConnection conn =
       HttpURLConnection conn =
         getConnection(Operation.GETHOMEDIRECTORY.getMethod(), params,
         getConnection(Operation.GETHOMEDIRECTORY.getMethod(), params,
                       new Path(getUri().toString(), "/"), false);
                       new Path(getUri().toString(), "/"), false);
-      HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
       JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
       JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
       return new Path((String) json.get(HOME_DIR_JSON));
       return new Path((String) json.get(HOME_DIR_JSON));
     } catch (IOException ex) {
     } catch (IOException ex) {
@@ -778,7 +770,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(GROUP_PARAM, groupname);
     params.put(GROUP_PARAM, groupname);
     HttpURLConnection conn = getConnection(Operation.SETOWNER.getMethod(),
     HttpURLConnection conn = getConnection(Operation.SETOWNER.getMethod(),
                                            params, p, true);
                                            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(OP_PARAM, Operation.SETPERMISSION.toString());
     params.put(PERMISSION_PARAM, permissionToString(permission));
     params.put(PERMISSION_PARAM, permissionToString(permission));
     HttpURLConnection conn = getConnection(Operation.SETPERMISSION.getMethod(), params, p, true);
     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));
     params.put(ACCESS_TIME_PARAM, Long.toString(atime));
     HttpURLConnection conn = getConnection(Operation.SETTIMES.getMethod(),
     HttpURLConnection conn = getConnection(Operation.SETTIMES.getMethod(),
                                            params, p, true);
                                            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));
     params.put(REPLICATION_PARAM, Short.toString(replication));
     HttpURLConnection conn =
     HttpURLConnection conn =
       getConnection(Operation.SETREPLICATION.getMethod(), params, src, true);
       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);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(SET_REPLICATION_JSON);
     return (Boolean) json.get(SET_REPLICATION_JSON);
   }
   }
@@ -857,7 +849,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(
     HttpURLConnection conn = getConnection(
             Operation.MODIFYACLENTRIES.getMethod(), params, path, true);
             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));
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(
     HttpURLConnection conn = getConnection(
             Operation.REMOVEACLENTRIES.getMethod(), params, path, true);
             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());
     params.put(OP_PARAM, Operation.REMOVEDEFAULTACL.toString());
     HttpURLConnection conn = getConnection(
     HttpURLConnection conn = getConnection(
             Operation.REMOVEDEFAULTACL.getMethod(), params, path, true);
             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());
     params.put(OP_PARAM, Operation.REMOVEACL.toString());
     HttpURLConnection conn = getConnection(Operation.REMOVEACL.getMethod(),
     HttpURLConnection conn = getConnection(Operation.REMOVEACL.getMethod(),
             params, path, true);
             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));
     params.put(ACLSPEC_PARAM, AclEntry.aclSpecToString(aclSpec));
     HttpURLConnection conn = getConnection(Operation.SETACL.getMethod(),
     HttpURLConnection conn = getConnection(Operation.SETACL.getMethod(),
                                            params, path, true);
                                            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());
     params.put(OP_PARAM, Operation.GETACLSTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.GETACLSTATUS.getMethod(),
     HttpURLConnection conn = getConnection(Operation.GETACLSTATUS.getMethod(),
             params, path, true);
             params, path, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(ACL_STATUS_JSON);
     json = (JSONObject) json.get(ACL_STATUS_JSON);
     return createAclStatus(json);
     return createAclStatus(json);
@@ -996,7 +988,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETCONTENTSUMMARY.toString());
     params.put(OP_PARAM, Operation.GETCONTENTSUMMARY.toString());
     HttpURLConnection conn =
     HttpURLConnection conn =
       getConnection(Operation.GETCONTENTSUMMARY.getMethod(), params, f, true);
       getConnection(Operation.GETCONTENTSUMMARY.getMethod(), params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) ((JSONObject)
     JSONObject json = (JSONObject) ((JSONObject)
       HttpFSUtils.jsonParse(conn)).get(CONTENT_SUMMARY_JSON);
       HttpFSUtils.jsonParse(conn)).get(CONTENT_SUMMARY_JSON);
     return new ContentSummary((Long) json.get(CONTENT_SUMMARY_LENGTH_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());
     params.put(OP_PARAM, Operation.GETFILECHECKSUM.toString());
     HttpURLConnection conn =
     HttpURLConnection conn =
       getConnection(Operation.GETFILECHECKSUM.getMethod(), params, f, true);
       getConnection(Operation.GETFILECHECKSUM.getMethod(), params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     final JSONObject json = (JSONObject) ((JSONObject)
     final JSONObject json = (JSONObject) ((JSONObject)
       HttpFSUtils.jsonParse(conn)).get(FILE_CHECKSUM_JSON);
       HttpFSUtils.jsonParse(conn)).get(FILE_CHECKSUM_JSON);
     return new FileChecksum() {
     return new FileChecksum() {
@@ -1115,7 +1107,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_SET_FLAG_PARAM, EnumSetParam.toString(flag));
     params.put(XATTR_SET_FLAG_PARAM, EnumSetParam.toString(flag));
     HttpURLConnection conn = getConnection(Operation.SETXATTR.getMethod(),
     HttpURLConnection conn = getConnection(Operation.SETXATTR.getMethod(),
         params, f, true);
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
   }
 
 
   @Override
   @Override
@@ -1125,7 +1117,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_NAME_PARAM, name);
     params.put(XATTR_NAME_PARAM, name);
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, f, true);
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     Map<String, byte[]> xAttrs = createXAttrMap(
     Map<String, byte[]> xAttrs = createXAttrMap(
         (JSONArray) json.get(XATTRS_JSON));
         (JSONArray) json.get(XATTRS_JSON));
@@ -1169,7 +1161,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.GETXATTRS.toString());
     params.put(OP_PARAM, Operation.GETXATTRS.toString());
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, f, true);
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
     return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
   }
   }
@@ -1185,7 +1177,7 @@ public class HttpFSFileSystem extends FileSystem
     multiValuedParams.put(XATTR_NAME_PARAM, names);
     multiValuedParams.put(XATTR_NAME_PARAM, names);
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
     HttpURLConnection conn = getConnection(Operation.GETXATTRS.getMethod(),
         params, multiValuedParams, f, true);
         params, multiValuedParams, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
     return createXAttrMap((JSONArray) json.get(XATTRS_JSON));
   }
   }
@@ -1196,7 +1188,7 @@ public class HttpFSFileSystem extends FileSystem
     params.put(OP_PARAM, Operation.LISTXATTRS.toString());
     params.put(OP_PARAM, Operation.LISTXATTRS.toString());
     HttpURLConnection conn = getConnection(Operation.LISTXATTRS.getMethod(),
     HttpURLConnection conn = getConnection(Operation.LISTXATTRS.getMethod(),
         params, f, true);
         params, f, true);
-    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return createXAttrNames((String) json.get(XATTRNAMES_JSON));
     return createXAttrNames((String) json.get(XATTRNAMES_JSON));
   }
   }
@@ -1208,6 +1200,6 @@ public class HttpFSFileSystem extends FileSystem
     params.put(XATTR_NAME_PARAM, name);
     params.put(XATTR_NAME_PARAM, name);
     HttpURLConnection conn = getConnection(Operation.REMOVEXATTR.getMethod(),
     HttpURLConnection conn = getConnection(Operation.REMOVEXATTR.getMethod(),
         params, f, true);
         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.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
 import org.json.simple.parser.JSONParser;
 import org.json.simple.parser.ParseException;
 import org.json.simple.parser.ParseException;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
-import java.lang.reflect.Constructor;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URI;
 import java.net.URL;
 import java.net.URL;
@@ -115,54 +113,6 @@ public class HttpFSUtils {
     return new URL(sb.toString());
     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
    * Convenience method that JSON Parses the <code>InputStream</code> of a
    * <code>HttpURLConnection</code>.
    * <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;
 package org.apache.hadoop.lib.wsrs;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 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.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
-import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.ExceptionMapper;
-import java.util.LinkedHashMap;
-import java.util.Map;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class ExceptionProvider implements ExceptionMapper<Throwable> {
 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");
   private static final String ENTER = System.getProperty("line.separator");
 
 
   protected Response createResponse(Response.Status status, Throwable throwable) {
   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) {
   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 {
   private void testCreate() throws Exception {
     Path path = new Path(getProxiedFSTestDir(), "foo.txt");
     Path path = new Path(getProxiedFSTestDir(), "foo.txt");
+    FileSystem fs = FileSystem.get(getProxiedFSConf());
+    fs.delete(path, true);
     testCreate(path, false);
     testCreate(path, false);
     testCreate(path, true);
     testCreate(path, true);
     try {
     try {
@@ -190,7 +192,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       Assert.fail("the create should have failed because the file exists " +
       Assert.fail("the create should have failed because the file exists " +
                   "and override is FALSE");
                   "and override is FALSE");
     } catch (IOException ex) {
     } catch (IOException ex) {
-
+System.out.println("#");
     } catch (Exception ex) {
     } catch (Exception ex) {
       Assert.fail(ex.toString());
       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);
       Assert.assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, resp);
       reader = new BufferedReader(new InputStreamReader(conn.getErrorStream()));
       reader = new BufferedReader(new InputStreamReader(conn.getErrorStream()));
       String res = reader.readLine();
       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;
       BufferedReader reader;
       reader = new BufferedReader(new InputStreamReader(conn.getErrorStream()));
       reader = new BufferedReader(new InputStreamReader(conn.getErrorStream()));
       String err = reader.readLine();
       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"));
     }
     }
   }
   }