You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cn...@apache.org on 2016/02/23 23:48:17 UTC

hadoop git commit: HDFS-9845. OzoneHandler : Support List and Info Volumes. Contributed by Anu Engineer.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 eeacb24ab -> fac463323


HDFS-9845. OzoneHandler : Support List and Info Volumes. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fac46332
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fac46332
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fac46332

Branch: refs/heads/HDFS-7240
Commit: fac4633234ac33aa3b47c0d419e70f0275c58344
Parents: eeacb24
Author: cnauroth <cn...@apache.org>
Authored: Tue Feb 23 14:46:24 2016 -0800
Committer: cnauroth <cn...@apache.org>
Committed: Tue Feb 23 14:46:24 2016 -0800

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-7240.txt           |   3 +
 .../server/datanode/ObjectStoreHandler.java     |  11 +-
 .../ozone/web/ObjectStoreApplication.java       |   2 +
 .../ozone/web/handlers/ServiceFilter.java       |  61 ++++++++++
 .../hadoop/ozone/web/TestOzoneVolumes.java      | 119 ++++++++++++++++++-
 5 files changed, 192 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac46332/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
index 18237d5..83cfef8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
@@ -41,3 +41,6 @@
 
     HDFS-9834. OzoneHandler : Enable MiniDFSCluster based testing for Ozone.
     (Anu Engineer via cnauroth)
+
+    HDFS-9845. OzoneHandler : Support List and Info Volumes.
+    (Anu Engineer via cnauroth)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac46332/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
index aaa1e44..5b751b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
@@ -17,15 +17,18 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_STORAGE_HANDLER_TYPE_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_STORAGE_HANDLER_TYPE_KEY;
 
 import java.io.IOException;
+import java.util.Collections;
 
 import com.sun.jersey.api.container.ContainerFactory;
 import com.sun.jersey.api.core.ApplicationAdapter;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ozone.web.handlers.ServiceFilter;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.ObjectStoreApplication;
 import org.apache.hadoop.ozone.web.netty.ObjectStoreJerseyContainer;
@@ -62,9 +65,13 @@ public final class ObjectStoreHandler {
                 DFS_STORAGE_HANDLER_TYPE_KEY, shType));
       }
     }
+    ApplicationAdapter aa =
+        new ApplicationAdapter(new ObjectStoreApplication());
+    aa.setPropertiesAndFeatures(Collections.<String, Object>singletonMap(
+        PROPERTY_CONTAINER_REQUEST_FILTERS,
+        ServiceFilter.class.getCanonicalName()));
     this.objectStoreJerseyContainer = ContainerFactory.createContainer(
-        ObjectStoreJerseyContainer.class, new ApplicationAdapter(
-            new ObjectStoreApplication()));
+        ObjectStoreJerseyContainer.class, aa);
     this.objectStoreJerseyContainer.setStorageHandler(storageHandler);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac46332/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java
index 0257e06..900b98c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/ObjectStoreApplication.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.ozone.web;
 
 import org.apache.hadoop.ozone.web.exceptions.OzoneExceptionMapper;
 import org.apache.hadoop.ozone.web.handlers.BucketHandler;
+import org.apache.hadoop.ozone.web.handlers.ServiceFilter;
 import org.apache.hadoop.ozone.web.handlers.VolumeHandler;
 
 import javax.ws.rs.core.Application;
@@ -46,6 +47,7 @@ public class ObjectStoreApplication extends Application {
   @Override
   public Set<Object> getSingletons() {
     HashSet<Object> set = new HashSet<>();
+    set.add(ServiceFilter.class);
     return set;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac46332/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java
new file mode 100644
index 0000000..c9f20c2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/ServiceFilter.java
@@ -0,0 +1,61 @@
+/*
+ * 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.ozone.web.handlers;
+
+import com.sun.jersey.spi.container.ContainerRequest;
+import com.sun.jersey.spi.container.ContainerRequestFilter;
+import org.apache.hadoop.ozone.web.headers.Header;
+
+import javax.ws.rs.core.UriBuilder;
+import javax.ws.rs.ext.Provider;
+
+/**
+ * This class is used to intercept root URL requests and route it to
+ * Volume List functionality.
+ */
+@Provider
+public class ServiceFilter implements ContainerRequestFilter {
+  /**
+   * Filter the request.
+   * <p>
+   * An implementation may modify the state of the request or
+   * create a new instance.
+   *
+   * @param request the request.
+   *
+   * @return the request.
+   */
+  @Override
+  public ContainerRequest filter(ContainerRequest request) {
+    if (request.getRequestUri().getPath().length() > 1) {
+      return request;
+    }
+
+    // Just re-route it to volume handler with some hypothetical volume name.
+    // volume name is ignored.
+
+    request.setUris(request.getBaseUri(),
+        UriBuilder.fromUri(request.getRequestUri())
+        .path("/service")
+        .queryParam("info", Header.OZONE_LIST_QUERY_SERVICE)
+        .build());
+
+    return request;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fac46332/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
index 5a2e63c..aa07484 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.impl.client.DefaultHttpClient;
 import org.junit.AfterClass;
@@ -42,6 +43,7 @@ import java.util.Date;
 import java.util.Locale;
 
 import static java.net.HttpURLConnection.HTTP_CREATED;
+import static java.net.HttpURLConnection.HTTP_OK;
 import static org.junit.Assert.assertEquals;
 
 public class TestOzoneVolumes {
@@ -241,7 +243,7 @@ public class TestOzoneVolumes {
           format.format(new Date(Time.monotonicNow())));
       httppost.addHeader(HttpHeaders.AUTHORIZATION,
           Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " " +
-              "bilbo"); // This is not a root user in Simple
+              "bilbo"); // This is not a root user in Simple Auth
       httppost.addHeader(Header.OZONE_USER, OzoneConsts.OZONE_SIMPLE_HDFS_USER);
 
       HttpResponse response = client.execute(httppost);
@@ -253,7 +255,7 @@ public class TestOzoneVolumes {
   }
 
   /**
-   * Create a bunch of volumes in a loop
+   * Create a bunch of volumes in a loop.
    *
    * @throws IOException
    */
@@ -285,4 +287,117 @@ public class TestOzoneVolumes {
       client.getConnectionManager().shutdown();
     }
   }
+  /**
+   * Get volumes owned by the user.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testGetVolumesByUser() throws IOException {
+    SimpleDateFormat format =
+        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
+
+    HttpClient client = new DefaultHttpClient();
+    try {
+      HttpGet httpget =
+          new HttpGet(String.format("http://localhost:%d/", port));
+
+      httpget.addHeader(Header.OZONE_VERSION_HEADER,
+          Header.OZONE_V1_VERSION_HEADER);
+
+      httpget.addHeader(HttpHeaders.DATE,
+          format.format(new Date(Time.monotonicNow())));
+
+      httpget.addHeader(HttpHeaders.AUTHORIZATION,
+          Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " " +
+              OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+
+      httpget.addHeader(Header.OZONE_USER,
+          OzoneConsts.OZONE_SIMPLE_HDFS_USER );
+
+      HttpResponse response = client.execute(httpget);
+      assertEquals(response.toString(), HTTP_OK,
+          response.getStatusLine().getStatusCode());
+
+    } finally {
+      client.getConnectionManager().shutdown();
+    }
+  }
+
+  /**
+   * Admins can read volumes belonging to other users.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testGetVolumesOfAnotherUser() throws IOException {
+    SimpleDateFormat format =
+        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
+
+    HttpClient client = new DefaultHttpClient();
+    try {
+      HttpGet httpget =
+          new HttpGet(String.format("http://localhost:%d/", port));
+
+      httpget.addHeader(Header.OZONE_VERSION_HEADER,
+          Header.OZONE_V1_VERSION_HEADER);
+      httpget.addHeader(HttpHeaders.DATE,
+          format.format(new Date(Time.monotonicNow())));
+
+      httpget.addHeader(HttpHeaders.AUTHORIZATION,
+          Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " " +
+              OzoneConsts.OZONE_SIMPLE_ROOT_USER);
+
+      // User Root is getting volumes belonging to user HDFS
+      httpget.addHeader(Header.OZONE_USER, OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+
+      HttpResponse response = client.execute(httpget);
+      assertEquals(response.toString(), HTTP_OK,
+          response.getStatusLine().getStatusCode());
+
+    } finally {
+      client.getConnectionManager().shutdown();
+    }
+  }
+
+  /**
+   * if you try to read volumes belonging to another user,
+   * then server always ignores it.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testGetVolumesOfAnotherUserShouldFail() throws IOException {
+    SimpleDateFormat format =
+        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
+
+    HttpClient client = new DefaultHttpClient();
+    String userName = OzoneUtils.getRequestID().toLowerCase();
+    try {
+      HttpGet httpget =
+          new HttpGet(String.format("http://localhost:%d/", port));
+
+      httpget.addHeader(Header.OZONE_VERSION_HEADER,
+          Header.OZONE_V1_VERSION_HEADER);
+      httpget.addHeader(HttpHeaders.DATE,
+          format.format(new Date(Time.monotonicNow())));
+
+      httpget.addHeader(HttpHeaders.AUTHORIZATION,
+          Header.OZONE_SIMPLE_AUTHENTICATION_SCHEME + " " +
+              userName);
+
+      // userName is NOT a root user, hence he should NOT be able to read the
+      // volumes of user HDFS
+      httpget.addHeader(Header.OZONE_USER, OzoneConsts.OZONE_SIMPLE_HDFS_USER);
+
+      HttpResponse response = client.execute(httpget);
+      // We will get an Error called userNotFound when using Simple Auth Scheme
+      assertEquals(response.toString(), ErrorTable.USER_NOT_FOUND.getHttpCode(),
+          response.getStatusLine().getStatusCode());
+
+    } finally {
+      client.getConnectionManager().shutdown();
+    }
+  }
+
 }