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 bh...@apache.org on 2018/09/25 05:10:54 UTC

[1/2] hadoop git commit: HDDS-444. Add rest service to the s3gateway. Contributed by Elek Marton.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 29dad7d25 -> 9c3fbbc4f


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java
new file mode 100644
index 0000000..0777856
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java
@@ -0,0 +1,110 @@
+/*
+ * 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.client;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * ObjectStore implementation with in-memory state.
+ */
+public class ObjectStoreStub extends ObjectStore {
+
+  public ObjectStoreStub() {
+    super();
+  }
+
+  private Map<String, OzoneVolumeStub> volumes = new HashMap<>();
+
+  @Override
+  public void createVolume(String volumeName) throws IOException {
+    createVolume(volumeName,
+        VolumeArgs.newBuilder()
+            .setAdmin("root")
+            .setOwner("root")
+            .setQuota("" + Integer.MAX_VALUE)
+            .setAcls(new ArrayList<>()).build());
+  }
+
+  @Override
+  public void createVolume(String volumeName, VolumeArgs volumeArgs)
+      throws IOException {
+    OzoneVolumeStub volume =
+        new OzoneVolumeStub(volumeName,
+            volumeArgs.getAdmin(),
+            volumeArgs.getOwner(),
+            Long.parseLong(volumeArgs.getQuota()),
+            System.currentTimeMillis(),
+            volumeArgs.getAcls());
+    volumes.put(volumeName, volume);
+  }
+
+  @Override
+  public OzoneVolume getVolume(String volumeName) throws IOException {
+    if (volumes.containsKey(volumeName)) {
+      return volumes.get(volumeName);
+    } else {
+      throw new IOException("VOLUME_NOT_FOUND");
+    }
+  }
+
+  @Override
+  public Iterator<? extends OzoneVolume> listVolumes(String volumePrefix)
+      throws IOException {
+    return volumes.values()
+        .stream()
+        .filter(volume -> volume.getName().startsWith(volumePrefix))
+        .collect(Collectors.toList())
+        .iterator();
+
+  }
+
+  @Override
+  public Iterator<? extends OzoneVolume> listVolumes(String volumePrefix,
+      String prevVolume) throws IOException {
+    return volumes.values()
+        .stream()
+        .filter(volume -> volume.getName().compareTo(prevVolume) > 0)
+        .filter(volume -> volume.getName().startsWith(volumePrefix))
+        .collect(Collectors.toList())
+        .iterator();
+  }
+
+  @Override
+  public Iterator<? extends OzoneVolume> listVolumesByUser(String user,
+      String volumePrefix, String prevVolume) throws IOException {
+    return volumes.values()
+        .stream()
+        .filter(volume -> volume.getOwner().equals(user))
+        .filter(volume -> volume.getName().compareTo(prevVolume) < 0)
+        .filter(volume -> volume.getName().startsWith(volumePrefix))
+        .collect(Collectors.toList())
+        .iterator();
+  }
+
+  @Override
+  public void deleteVolume(String volumeName) throws IOException {
+    volumes.remove(volumeName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
new file mode 100644
index 0000000..18dff71
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
@@ -0,0 +1,143 @@
+/*
+ * 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.client;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+
+/**
+ * In-memory ozone bucket for testing.
+ */
+public class OzoneBucketStub extends OzoneBucket {
+
+  private Map<String, OzoneKeyDetails> keyDetails = new HashMap<>();
+
+  private Map<String, byte[]> keyContents = new HashMap<>();
+
+  /**
+   * Constructs OzoneBucket instance.
+   *
+   * @param volumeName   Name of the volume the bucket belongs to.
+   * @param bucketName   Name of the bucket.
+   * @param acls         ACLs associated with the bucket.
+   * @param storageType  StorageType of the bucket.
+   * @param versioning   versioning status of the bucket.
+   * @param creationTime creation time of the bucket.
+   */
+  public OzoneBucketStub(
+      String volumeName,
+      String bucketName,
+      List<OzoneAcl> acls,
+      StorageType storageType, Boolean versioning,
+      long creationTime) {
+    super(volumeName,
+        bucketName,
+        ReplicationFactor.ONE,
+        ReplicationType.STAND_ALONE,
+        acls,
+        storageType,
+        versioning,
+        creationTime);
+  }
+
+  @Override
+  public OzoneOutputStream createKey(String key, long size) throws IOException {
+    return createKey(key, size, ReplicationType.STAND_ALONE,
+        ReplicationFactor.ONE);
+  }
+
+  @Override
+  public OzoneOutputStream createKey(String key, long size,
+      ReplicationType type, ReplicationFactor factor) throws IOException {
+    ByteArrayOutputStream byteArrayOutputStream =
+        new ByteArrayOutputStream((int) size) {
+          @Override
+          public void close() throws IOException {
+            keyContents.put(key, toByteArray());
+            keyDetails.put(key, new OzoneKeyDetails(
+                getVolumeName(),
+                getName(),
+                key,
+                size,
+                System.currentTimeMillis(),
+                System.currentTimeMillis(),
+                new ArrayList<>()
+            ));
+            super.close();
+          }
+        };
+    return new OzoneOutputStream(byteArrayOutputStream);
+  }
+
+  @Override
+  public OzoneInputStream readKey(String key) throws IOException {
+    return new OzoneInputStream(new ByteArrayInputStream(keyContents.get(key)));
+  }
+
+  @Override
+  public OzoneKeyDetails getKey(String key) throws IOException {
+    return keyDetails.get(key);
+  }
+
+  @Override
+  public Iterator<? extends OzoneKey> listKeys(String keyPrefix) {
+    return keyDetails.values()
+        .stream()
+        .filter(key -> key.getName().startsWith(keyPrefix))
+        .collect(Collectors.toList())
+        .iterator();
+  }
+
+  @Override
+  public Iterator<? extends OzoneKey> listKeys(String keyPrefix,
+      String prevKey) {
+    return keyDetails.values()
+        .stream()
+        .filter(key -> key.getName().compareTo(prevKey) > 0)
+        .filter(key -> key.getName().startsWith(keyPrefix))
+        .collect(Collectors.toList())
+        .iterator();
+  }
+
+  @Override
+  public void deleteKey(String key) throws IOException {
+    keyDetails.remove(key);
+  }
+
+  @Override
+  public void renameKey(String fromKeyName, String toKeyName)
+      throws IOException {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneClientStub.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneClientStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneClientStub.java
new file mode 100644
index 0000000..3c7a253
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneClientStub.java
@@ -0,0 +1,37 @@
+/*
+ * 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.client;
+
+import java.io.IOException;
+
+/**
+ * In-memory OzoneClient for testing.
+ */
+public class OzoneClientStub extends OzoneClient {
+
+  public OzoneClientStub() {
+    super(new ObjectStoreStub());
+  }
+
+  @Override
+  public void close() throws IOException {
+    //NOOP.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
new file mode 100644
index 0000000..eb866e0
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
@@ -0,0 +1,95 @@
+/*
+ * 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.client;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.OzoneAcl;
+
+/**
+ * Ozone volume with in-memory state for testing.
+ */
+public class OzoneVolumeStub extends OzoneVolume {
+
+  private Map<String, OzoneBucketStub> buckets = new HashMap<>();
+
+  public OzoneVolumeStub(String name, String admin, String owner,
+      long quotaInBytes,
+      long creationTime, List<OzoneAcl> acls) {
+    super(name, admin, owner, quotaInBytes, creationTime, acls);
+  }
+
+  @Override
+  public void createBucket(String bucketName) throws IOException {
+    createBucket(bucketName, new BucketArgs.Builder()
+        .setStorageType(StorageType.DEFAULT)
+        .setVersioning(false)
+        .build());
+  }
+
+  @Override
+  public void createBucket(String bucketName, BucketArgs bucketArgs)
+      throws IOException {
+    buckets.put(bucketName, new OzoneBucketStub(
+        getName(),
+        bucketName,
+        bucketArgs.getAcls(),
+        bucketArgs.getStorageType(),
+        bucketArgs.getVersioning(),
+        System.currentTimeMillis()));
+
+  }
+
+  @Override
+  public OzoneBucket getBucket(String bucketName) throws IOException {
+    return buckets.get(bucketName);
+  }
+
+  @Override
+  public Iterator<? extends OzoneBucket> listBuckets(String bucketPrefix) {
+    return buckets.values()
+        .stream()
+        .filter(bucket -> bucket.getName().startsWith(bucketPrefix))
+        .collect(Collectors.toList())
+        .iterator();
+  }
+
+  @Override
+  public Iterator<? extends OzoneBucket> listBuckets(String bucketPrefix,
+      String prevBucket) {
+    return buckets.values()
+        .stream()
+        .filter(bucket -> bucket.getName().compareTo(prevBucket) > 0)
+        .filter(bucket -> bucket.getName().startsWith(bucketPrefix))
+        .collect(Collectors.toList())
+        .iterator();
+  }
+
+  @Override
+  public void deleteBucket(String bucketName) throws IOException {
+    buckets.remove(bucketName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/package-info.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/package-info.java
new file mode 100644
index 0000000..10e4274
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * In-memory OzoneClient implementation to test REST endpoints.
+ */
+package org.apache.hadoop.ozone.client;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestBucketResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestBucketResponse.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestBucketResponse.java
new file mode 100644
index 0000000..efc69be
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestBucketResponse.java
@@ -0,0 +1,40 @@
+/*
+ * 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.s3.bucket;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+
+import org.apache.hadoop.ozone.s3.object.ListObjectResponse;
+
+import org.junit.Test;
+
+/**
+ * Testing JAXB serialization.
+ */
+public class TestBucketResponse {
+
+  @Test
+  public void serialize() throws JAXBException {
+    JAXBContext context = JAXBContext.newInstance(ListObjectResponse.class);
+    context.createMarshaller().marshal(new ListObjectResponse(), System.out);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestGetBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestGetBucket.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestGetBucket.java
new file mode 100644
index 0000000..f5a6f7e
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestGetBucket.java
@@ -0,0 +1,113 @@
+/*
+ * 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.s3.bucket;
+
+import java.io.IOException;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.apache.hadoop.ozone.s3.object.ListObject;
+import org.apache.hadoop.ozone.s3.object.ListObjectResponse;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Testing basic object list browsing.
+ */
+public class TestGetBucket {
+
+  @Test
+  public void listRoot() throws IOException {
+
+    ListObject getBucket = new ListObject();
+
+    OzoneClient client = createClientWithKeys("file1", "dir1/file2");
+
+    getBucket.setClient(client);
+
+    ListObjectResponse getBucketResponse =
+        getBucket.get("vol1", "b1", "/", null, null, 100, "", null);
+
+    Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
+    Assert.assertEquals("dir1/",
+        getBucketResponse.getCommonPrefixes().get(0).getPrefix());
+
+    Assert.assertEquals(1, getBucketResponse.getContents().size());
+    Assert.assertEquals("file1",
+        getBucketResponse.getContents().get(0).getKey());
+
+  }
+
+  @Test
+  public void listDir() throws IOException {
+
+    ListObject getBucket = new ListObject();
+
+    OzoneClient client = createClientWithKeys("dir1/file2", "dir1/dir2/file2");
+
+    getBucket.setClient(client);
+
+    ListObjectResponse getBucketResponse =
+        getBucket.get("vol1", "b1", "/", null, null, 100, "dir1", null);
+
+    Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
+    Assert.assertEquals("dir1/",
+        getBucketResponse.getCommonPrefixes().get(0).getPrefix());
+
+    Assert.assertEquals(0, getBucketResponse.getContents().size());
+
+  }
+
+  @Test
+  public void listSubDir() throws IOException {
+
+    ListObject getBucket = new ListObject();
+    OzoneClient ozoneClient =
+        createClientWithKeys("dir1/file2", "dir1/dir2/file2");
+
+    getBucket.setClient(ozoneClient);
+
+    ListObjectResponse getBucketResponse =
+        getBucket.get("vol1", "b1", "/", null, null, 100, "dir1/", null);
+
+    Assert.assertEquals(1, getBucketResponse.getCommonPrefixes().size());
+    Assert.assertEquals("dir1/dir2/",
+        getBucketResponse.getCommonPrefixes().get(0).getPrefix());
+
+    Assert.assertEquals(1, getBucketResponse.getContents().size());
+    Assert.assertEquals("dir1/file2",
+        getBucketResponse.getContents().get(0).getKey());
+
+  }
+
+  private OzoneClient createClientWithKeys(String... keys) throws IOException {
+    OzoneClient client = new OzoneClientStub();
+    client.getObjectStore().createVolume("vol1");
+    client.getObjectStore().getVolume("vol1").createBucket("b1");
+    OzoneBucket bucket =
+        client.getObjectStore().getVolume("vol1").getBucket("b1");
+    for (String key : keys) {
+      bucket.createKey(key, 0).close();
+    }
+    return client;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/package-info.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/package-info.java
new file mode 100644
index 0000000..de09dae
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Unit tests for the bucket related rest endpoints.
+ */
+package org.apache.hadoop.ozone.s3.bucket;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/TestDeleteObject.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/TestDeleteObject.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/TestDeleteObject.java
new file mode 100644
index 0000000..6c06a76
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/TestDeleteObject.java
@@ -0,0 +1,56 @@
+/*
+ * 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.s3.object;
+
+import java.io.IOException;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientStub;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test delete object.
+ */
+public class TestDeleteObject {
+
+  @Test
+  public void delete() throws IOException {
+    //GIVEN
+    OzoneClient client = new OzoneClientStub();
+    client.getObjectStore().createVolume("vol1");
+    client.getObjectStore().getVolume("vol1").createBucket("b1");
+    OzoneBucket bucket =
+        client.getObjectStore().getVolume("vol1").getBucket("b1");
+    bucket.createKey("key1", 0).close();
+
+    DeleteObject rest = new DeleteObject();
+    rest.setClient(client);
+
+    //WHEN
+    rest.delete("vol1", "b1", "key1");
+
+    //THEN
+    Assert.assertFalse("Bucket Should not contain any key after delete",
+        bucket.listKeys("").hasNext());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/package-info.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/package-info.java
new file mode 100644
index 0000000..897e209
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/object/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Unit tests for the object related rest endpoints.
+ */
+package org.apache.hadoop.ozone.s3.object;
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[2/2] hadoop git commit: HDDS-444. Add rest service to the s3gateway. Contributed by Elek Marton.

Posted by bh...@apache.org.
HDDS-444. Add rest service to the s3gateway. Contributed by Elek Marton.


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

Branch: refs/heads/trunk
Commit: 9c3fbbc4f65bb6268811bfb9ed54b89e8569c199
Parents: 29dad7d
Author: Bharat Viswanadham <bh...@apache.org>
Authored: Mon Sep 24 21:56:36 2018 -0700
Committer: Bharat Viswanadham <bh...@apache.org>
Committed: Mon Sep 24 22:10:16 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/cli/GenericCli.java  |   2 +-
 .../apache/hadoop/ozone/client/ObjectStore.java |  12 +-
 .../apache/hadoop/ozone/client/OzoneBucket.java |  24 ++-
 .../apache/hadoop/ozone/client/OzoneClient.java |   7 +
 .../apache/hadoop/ozone/client/OzoneVolume.java |  32 +++-
 .../main/compose/ozones3/docker-compose.yaml    |   2 +
 .../ozone/client/rpc/TestOzoneRpcClient.java    |  41 +++---
 .../hadoop/ozone/web/client/TestBuckets.java    |   4 +-
 .../web/ozShell/bucket/ListBucketHandler.java   |   3 +-
 .../ozone/web/ozShell/keys/ListKeyHandler.java  |   3 +-
 .../web/ozShell/volume/ListVolumeHandler.java   |   2 +-
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java |   2 +-
 hadoop-ozone/pom.xml                            |  11 ++
 .../CommonHeadersContainerResponseFilter.java   |  39 +++++
 .../apache/hadoop/ozone/s3/EndpointBase.java    |  76 ++++++++++
 .../org/apache/hadoop/ozone/s3/Gateway.java     |  17 ++-
 .../hadoop/ozone/s3/GatewayApplication.java     |  29 ++++
 .../hadoop/ozone/s3/OzoneClientProducer.java    |  48 ++++++
 .../ozone/s3/OzoneConfigurationHolder.java      |  43 ++++++
 .../hadoop/ozone/s3/S3GatewayConfigKeys.java    |  54 +++++++
 .../hadoop/ozone/s3/S3GatewayHttpServer.java    |  85 +++++++++++
 .../hadoop/ozone/s3/bucket/DeleteBucket.java    |  44 ++++++
 .../hadoop/ozone/s3/bucket/PutBucket.java       |  44 ++++++
 .../hadoop/ozone/s3/bucket/package-info.java    |  30 ++++
 .../ozone/s3/commontypes/CommonPrefix.java      |  47 ++++++
 .../ozone/s3/commontypes/IsoDateAdapter.java    |  47 ++++++
 .../ozone/s3/commontypes/KeyMetadata.java       |  87 +++++++++++
 .../ozone/s3/commontypes/package-info.java      |  29 ++++
 .../hadoop/ozone/s3/object/DeleteObject.java    |  51 +++++++
 .../hadoop/ozone/s3/object/HeadObject.java      |  58 ++++++++
 .../hadoop/ozone/s3/object/ListObject.java      | 118 +++++++++++++++
 .../ozone/s3/object/ListObjectResponse.java     | 147 +++++++++++++++++++
 .../hadoop/ozone/s3/object/PutObject.java       |  61 ++++++++
 .../hadoop/ozone/s3/object/package-info.java    |  29 ++++
 .../apache/hadoop/ozone/s3/package-info.java    |  22 +++
 .../src/main/resources/META-INF/beans.xml       |  20 +++
 .../webapps/s3gateway/WEB-INF/beans.xml         |  20 +++
 .../resources/webapps/s3gateway/WEB-INF/web.xml |  36 +++++
 .../hadoop/ozone/client/ObjectStoreStub.java    | 110 ++++++++++++++
 .../hadoop/ozone/client/OzoneBucketStub.java    | 143 ++++++++++++++++++
 .../hadoop/ozone/client/OzoneClientStub.java    |  37 +++++
 .../hadoop/ozone/client/OzoneVolumeStub.java    |  95 ++++++++++++
 .../hadoop/ozone/client/package-info.java       |  21 +++
 .../ozone/s3/bucket/TestBucketResponse.java     |  40 +++++
 .../hadoop/ozone/s3/bucket/TestGetBucket.java   | 113 ++++++++++++++
 .../hadoop/ozone/s3/bucket/package-info.java    |  21 +++
 .../ozone/s3/object/TestDeleteObject.java       |  56 +++++++
 .../hadoop/ozone/s3/object/package-info.java    |  21 +++
 48 files changed, 2041 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
index e56810c..e0c8150 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
@@ -82,7 +82,7 @@ public class GenericCli implements Callable<Void>, GenericParentCommand {
     if (configurationOverrides != null) {
       for (Entry<String, String> entry : configurationOverrides.entrySet()) {
         ozoneConf
-            .set(entry.getKey(), configurationOverrides.get(entry.getValue()));
+            .set(entry.getKey(), entry.getValue());
       }
     }
     return ozoneConf;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
index b6005d1..17d1938 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.client;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
@@ -56,6 +57,11 @@ public class ObjectStore {
     this.listCacheSize = HddsClientUtils.getListCacheSize(conf);
   }
 
+  @VisibleForTesting
+  protected ObjectStore() {
+    proxy = null;
+  }
+
   /**
    * Creates the volume with default values.
    * @param volumeName Name of the volume to be created.
@@ -96,7 +102,7 @@ public class ObjectStore {
    * @param volumePrefix Volume prefix to match
    * @return {@code Iterator<OzoneVolume>}
    */
-  public Iterator<OzoneVolume> listVolumes(String volumePrefix)
+  public Iterator<? extends OzoneVolume> listVolumes(String volumePrefix)
       throws IOException {
     return listVolumes(volumePrefix, null);
   }
@@ -111,7 +117,7 @@ public class ObjectStore {
    * @param prevVolume Volumes will be listed after this volume name
    * @return {@code Iterator<OzoneVolume>}
    */
-  public Iterator<OzoneVolume> listVolumes(String volumePrefix,
+  public Iterator<? extends OzoneVolume> listVolumes(String volumePrefix,
       String prevVolume) throws IOException {
     return new VolumeIterator(null, volumePrefix, prevVolume);
   }
@@ -127,7 +133,7 @@ public class ObjectStore {
    * @param prevVolume Volumes will be listed after this volume name
    * @return {@code Iterator<OzoneVolume>}
    */
-  public Iterator<OzoneVolume> listVolumesByUser(String user,
+  public Iterator<? extends OzoneVolume> listVolumesByUser(String user,
       String volumePrefix, String prevVolume)
       throws IOException {
     if(Strings.isNullOrEmpty(user)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index 97bd682..751992e 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.ozone.client;
 
-
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
@@ -121,6 +121,23 @@ public class OzoneBucket {
         OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT));
   }
 
+  @VisibleForTesting
+  OzoneBucket(String volumeName, String name,
+      ReplicationFactor defaultReplication,
+      ReplicationType defaultReplicationType,
+      List<OzoneAcl> acls, StorageType storageType, Boolean versioning,
+      long creationTime) {
+    this.proxy = null;
+    this.volumeName = volumeName;
+    this.name = name;
+    this.defaultReplication = defaultReplication;
+    this.defaultReplicationType = defaultReplicationType;
+    this.acls = acls;
+    this.storageType = storageType;
+    this.versioning = versioning;
+    this.creationTime = creationTime;
+  }
+
   /**
    * Returns Volume Name.
    *
@@ -273,7 +290,7 @@ public class OzoneBucket {
    * @param keyPrefix Bucket prefix to match
    * @return {@code Iterator<OzoneKey>}
    */
-  public Iterator<OzoneKey> listKeys(String keyPrefix) {
+  public Iterator<? extends OzoneKey> listKeys(String keyPrefix) {
     return listKeys(keyPrefix, null);
   }
 
@@ -287,7 +304,8 @@ public class OzoneBucket {
    * @param prevKey Keys will be listed after this key name
    * @return {@code Iterator<OzoneKey>}
    */
-  public Iterator<OzoneKey> listKeys(String keyPrefix, String prevKey) {
+  public Iterator<? extends OzoneKey> listKeys(String keyPrefix,
+      String prevKey) {
     return new KeyIterator(keyPrefix, prevKey);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java
index f191507..0d65d73 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import java.io.Closeable;
 import java.io.IOException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * OzoneClient connects to Ozone Cluster and
  * perform basic operations.
@@ -84,6 +86,11 @@ public class OzoneClient implements Closeable {
     this.objectStore = new ObjectStore(conf, this.proxy);
   }
 
+  @VisibleForTesting
+  protected OzoneClient(ObjectStore objectStore) {
+    this.objectStore = objectStore;
+    this.proxy = null;
+  }
   /**
    * Returns the object store associated with the Ozone Cluster.
    * @return ObjectStore

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
index 77f882a..e451b1a 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
@@ -18,17 +18,19 @@
 
 package org.apache.hadoop.ozone.client;
 
-import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 /**
  * A class that encapsulates OzoneVolume.
@@ -94,6 +96,19 @@ public class OzoneVolume {
     this.listCacheSize = HddsClientUtils.getListCacheSize(conf);
   }
 
+  @VisibleForTesting
+  protected OzoneVolume(String name, String admin, String owner,
+      long quotaInBytes,
+      long creationTime, List<OzoneAcl> acls) {
+    this.proxy = null;
+    this.name = name;
+    this.admin = admin;
+    this.owner = owner;
+    this.quotaInBytes = quotaInBytes;
+    this.creationTime = creationTime;
+    this.acls = acls;
+  }
+
   /**
    * Returns Volume name.
    *
@@ -208,12 +223,13 @@ public class OzoneVolume {
    * @param bucketPrefix Bucket prefix to match
    * @return {@code Iterator<OzoneBucket>}
    */
-  public Iterator<OzoneBucket> listBuckets(String bucketPrefix) {
+  public Iterator<? extends OzoneBucket> listBuckets(String bucketPrefix) {
     return listBuckets(bucketPrefix, null);
   }
 
   /**
-   * Returns Iterator to iterate over all buckets after prevBucket in the volume.
+   * Returns Iterator to iterate over all buckets after prevBucket in the
+   * volume.
    * If prevBucket is null it iterates from the first bucket in the volume.
    * The result can be restricted using bucket prefix, will return all
    * buckets if bucket prefix is null.
@@ -222,7 +238,7 @@ public class OzoneVolume {
    * @param prevBucket Buckets are listed after this bucket
    * @return {@code Iterator<OzoneBucket>}
    */
-  public Iterator<OzoneBucket> listBuckets(String bucketPrefix,
+  public Iterator<? extends OzoneBucket> listBuckets(String bucketPrefix,
       String prevBucket) {
     return new BucketIterator(bucketPrefix, prevBucket);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml
index 10dc61e..42d7d1d 100644
--- a/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml
+++ b/hadoop-ozone/dist/src/main/compose/ozones3/docker-compose.yaml
@@ -51,6 +51,8 @@ services:
       image: apache/hadoop-runner
       volumes:
          - ../..:/opt/hadoop
+      ports:
+         - 9878:9878
       env_file:
           - ./docker-config
       command: ["ozone","s3g"]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
index cc045d0..d644190 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
@@ -701,26 +701,29 @@ public class TestOzoneRpcClient {
       store.createVolume(
           volBaseNameB + i + "-" + RandomStringUtils.randomNumeric(5));
     }
-    Iterator<OzoneVolume> volIterator = store.listVolumes(volBase);
+    Iterator<? extends OzoneVolume> volIterator = store.listVolumes(volBase);
     int totalVolumeCount = 0;
     while(volIterator.hasNext()) {
       volIterator.next();
       totalVolumeCount++;
     }
     Assert.assertEquals(20, totalVolumeCount);
-    Iterator<OzoneVolume> volAIterator = store.listVolumes(volBaseNameA);
+    Iterator<? extends OzoneVolume> volAIterator = store.listVolumes(
+        volBaseNameA);
     for(int i = 0; i < 10; i++) {
       Assert.assertTrue(volAIterator.next().getName()
           .startsWith(volBaseNameA + i + "-"));
     }
     Assert.assertFalse(volAIterator.hasNext());
-    Iterator<OzoneVolume> volBIterator = store.listVolumes(volBaseNameB);
+    Iterator<? extends OzoneVolume> volBIterator = store.listVolumes(
+        volBaseNameB);
     for(int i = 0; i < 10; i++) {
       Assert.assertTrue(volBIterator.next().getName()
           .startsWith(volBaseNameB + i + "-"));
     }
     Assert.assertFalse(volBIterator.hasNext());
-    Iterator<OzoneVolume> iter = store.listVolumes(volBaseNameA + "1-");
+    Iterator<? extends OzoneVolume> iter = store.listVolumes(volBaseNameA +
+        "1-");
     Assert.assertTrue(iter.next().getName().startsWith(volBaseNameA + "1-"));
     Assert.assertFalse(iter.hasNext());
   }
@@ -751,7 +754,7 @@ public class TestOzoneRpcClient {
       volB.createBucket(
           bucketBaseNameB + i + "-" + RandomStringUtils.randomNumeric(5));
     }
-    Iterator<OzoneBucket> volABucketIter =
+    Iterator<? extends OzoneBucket> volABucketIter =
         volA.listBuckets("bucket-");
     int volABucketCount = 0;
     while(volABucketIter.hasNext()) {
@@ -759,7 +762,7 @@ public class TestOzoneRpcClient {
       volABucketCount++;
     }
     Assert.assertEquals(20, volABucketCount);
-    Iterator<OzoneBucket> volBBucketIter =
+    Iterator<? extends OzoneBucket> volBBucketIter =
         volA.listBuckets("bucket-");
     int volBBucketCount = 0;
     while(volBBucketIter.hasNext()) {
@@ -768,7 +771,7 @@ public class TestOzoneRpcClient {
     }
     Assert.assertEquals(20, volBBucketCount);
 
-    Iterator<OzoneBucket> volABucketAIter =
+    Iterator<? extends OzoneBucket> volABucketAIter =
         volA.listBuckets("bucket-a-");
     int volABucketACount = 0;
     while(volABucketAIter.hasNext()) {
@@ -776,7 +779,7 @@ public class TestOzoneRpcClient {
       volABucketACount++;
     }
     Assert.assertEquals(10, volABucketACount);
-    Iterator<OzoneBucket> volBBucketBIter =
+    Iterator<? extends OzoneBucket> volBBucketBIter =
         volA.listBuckets("bucket-b-");
     int volBBucketBCount = 0;
     while(volBBucketBIter.hasNext()) {
@@ -784,13 +787,15 @@ public class TestOzoneRpcClient {
       volBBucketBCount++;
     }
     Assert.assertEquals(10, volBBucketBCount);
-    Iterator<OzoneBucket> volABucketBIter = volA.listBuckets("bucket-b-");
+    Iterator<? extends OzoneBucket> volABucketBIter = volA.listBuckets(
+        "bucket-b-");
     for(int i = 0; i < 10; i++) {
       Assert.assertTrue(volABucketBIter.next().getName()
           .startsWith(bucketBaseNameB + i + "-"));
     }
     Assert.assertFalse(volABucketBIter.hasNext());
-    Iterator<OzoneBucket> volBBucketAIter = volB.listBuckets("bucket-a-");
+    Iterator<? extends OzoneBucket> volBBucketAIter = volB.listBuckets(
+        "bucket-a-");
     for(int i = 0; i < 10; i++) {
       Assert.assertTrue(volBBucketAIter.next().getName()
           .startsWith(bucketBaseNameA + i + "-"));
@@ -805,7 +810,7 @@ public class TestOzoneRpcClient {
     String volume = "vol-" + RandomStringUtils.randomNumeric(5);
     store.createVolume(volume);
     OzoneVolume vol = store.getVolume(volume);
-    Iterator<OzoneBucket> buckets = vol.listBuckets("");
+    Iterator<? extends OzoneBucket> buckets = vol.listBuckets("");
     while(buckets.hasNext()) {
       Assert.fail();
     }
@@ -889,7 +894,7 @@ public class TestOzoneRpcClient {
       four.write(value);
       four.close();
     }
-    Iterator<OzoneKey> volABucketAIter =
+    Iterator<? extends OzoneKey> volABucketAIter =
         volAbucketA.listKeys("key-");
     int volABucketAKeyCount = 0;
     while(volABucketAIter.hasNext()) {
@@ -897,7 +902,7 @@ public class TestOzoneRpcClient {
       volABucketAKeyCount++;
     }
     Assert.assertEquals(20, volABucketAKeyCount);
-    Iterator<OzoneKey> volABucketBIter =
+    Iterator<? extends OzoneKey> volABucketBIter =
         volAbucketB.listKeys("key-");
     int volABucketBKeyCount = 0;
     while(volABucketBIter.hasNext()) {
@@ -905,7 +910,7 @@ public class TestOzoneRpcClient {
       volABucketBKeyCount++;
     }
     Assert.assertEquals(20, volABucketBKeyCount);
-    Iterator<OzoneKey> volBBucketAIter =
+    Iterator<? extends OzoneKey> volBBucketAIter =
         volBbucketA.listKeys("key-");
     int volBBucketAKeyCount = 0;
     while(volBBucketAIter.hasNext()) {
@@ -913,7 +918,7 @@ public class TestOzoneRpcClient {
       volBBucketAKeyCount++;
     }
     Assert.assertEquals(20, volBBucketAKeyCount);
-    Iterator<OzoneKey> volBBucketBIter =
+    Iterator<? extends OzoneKey> volBBucketBIter =
         volBbucketB.listKeys("key-");
     int volBBucketBKeyCount = 0;
     while(volBBucketBIter.hasNext()) {
@@ -921,7 +926,7 @@ public class TestOzoneRpcClient {
       volBBucketBKeyCount++;
     }
     Assert.assertEquals(20, volBBucketBKeyCount);
-    Iterator<OzoneKey> volABucketAKeyAIter =
+    Iterator<? extends OzoneKey> volABucketAKeyAIter =
         volAbucketA.listKeys("key-a-");
     int volABucketAKeyACount = 0;
     while(volABucketAKeyAIter.hasNext()) {
@@ -929,7 +934,7 @@ public class TestOzoneRpcClient {
       volABucketAKeyACount++;
     }
     Assert.assertEquals(10, volABucketAKeyACount);
-    Iterator<OzoneKey> volABucketAKeyBIter =
+    Iterator<? extends OzoneKey> volABucketAKeyBIter =
         volAbucketA.listKeys("key-b-");
     for(int i = 0; i < 10; i++) {
       Assert.assertTrue(volABucketAKeyBIter.next().getName()
@@ -947,7 +952,7 @@ public class TestOzoneRpcClient {
     OzoneVolume vol = store.getVolume(volume);
     vol.createBucket(bucket);
     OzoneBucket buc = vol.getBucket(bucket);
-    Iterator<OzoneKey> keys = buc.listKeys("");
+    Iterator<? extends OzoneKey> keys = buc.listKeys("");
     while(keys.hasNext()) {
       Assert.fail();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
index e866d20..fe96198 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
@@ -305,7 +305,7 @@ public class TestBuckets {
           .build();
       vol.createBucket(bucketName, bucketArgs);
     }
-    Iterator<OzoneBucket> bucketIterator = vol.listBuckets(null);
+    Iterator<? extends OzoneBucket> bucketIterator = vol.listBuckets(null);
     int count = 0;
 
     while (bucketIterator.hasNext()) {
@@ -324,7 +324,7 @@ public class TestBuckets {
     client.close();
   }
 
-  private static int getSize(Iterator<OzoneBucket> bucketIterator) {
+  private static int getSize(Iterator<? extends OzoneBucket> bucketIterator) {
     int count = 0;
     while (bucketIterator.hasNext()) {
       count++;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
index 1d97bf5..d8c824b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/ListBucketHandler.java
@@ -88,7 +88,8 @@ public class ListBucketHandler extends Handler {
 
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
-    Iterator<OzoneBucket> bucketIterator = vol.listBuckets(prefix, startBucket);
+    Iterator<? extends OzoneBucket> bucketIterator =
+        vol.listBuckets(prefix, startBucket);
     List<BucketInfo> bucketList = new ArrayList<>();
     while (maxBuckets > 0 && bucketIterator.hasNext()) {
       BucketInfo bucketInfo =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
index 0a710ac..3c019a6 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/ListKeyHandler.java
@@ -91,7 +91,8 @@ public class ListKeyHandler extends Handler {
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     OzoneBucket bucket = vol.getBucket(bucketName);
-    Iterator<OzoneKey> keyIterator = bucket.listKeys(prefix, startKey);
+    Iterator<? extends OzoneKey> keyIterator = bucket.listKeys(prefix,
+        startKey);
     List<KeyInfo> keyInfos = new ArrayList<>();
 
     while (maxKeys > 0 && keyIterator.hasNext()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
index a54393c..948a45c 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/ListVolumeHandler.java
@@ -89,7 +89,7 @@ public class ListVolumeHandler extends Handler {
           "the length should be a positive number");
     }
 
-    Iterator<OzoneVolume> volumeIterator;
+    Iterator<? extends OzoneVolume> volumeIterator;
     if(userName != null) {
       volumeIterator = client.getObjectStore()
           .listVolumesByUser(userName, prefix, startVolume);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
index b876dc2..8fbd688 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
@@ -638,7 +638,7 @@ public class OzoneFileSystem extends FileSystem {
     private final Path path;
     private final FileStatus status;
     private String pathKey;
-    private Iterator<OzoneKey> keyIterator;
+    private Iterator<? extends OzoneKey> keyIterator;
 
     OzoneListingIterator(Path path)
         throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/pom.xml b/hadoop-ozone/pom.xml
index 075c674..bb96523 100644
--- a/hadoop-ozone/pom.xml
+++ b/hadoop-ozone/pom.xml
@@ -183,6 +183,17 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/CommonHeadersContainerResponseFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/CommonHeadersContainerResponseFilter.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/CommonHeadersContainerResponseFilter.java
new file mode 100644
index 0000000..d2f2d65
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/CommonHeadersContainerResponseFilter.java
@@ -0,0 +1,39 @@
+/*
+ * 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.s3;
+
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerResponseContext;
+import javax.ws.rs.container.ContainerResponseFilter;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+
+/**
+ * This class adds common header responses for all the requests.
+ */
+@Provider
+public class CommonHeadersContainerResponseFilter implements
+    ContainerResponseFilter {
+
+  @Override
+  public void filter(ContainerRequestContext containerRequestContext,
+      ContainerResponseContext containerResponseContext) throws IOException {
+    containerResponseContext.getHeaders().add("Server", "Ozone");
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/EndpointBase.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/EndpointBase.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/EndpointBase.java
new file mode 100644
index 0000000..bfbeeec
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/EndpointBase.java
@@ -0,0 +1,76 @@
+/**
+ * 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.s3;
+
+import javax.inject.Inject;
+import javax.ws.rs.NotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Basic helpers for all the REST endpoints.
+ */
+public class EndpointBase {
+
+  @Inject
+  private OzoneClient client;
+
+  protected OzoneBucket getBucket(String volumeName, String bucketName)
+      throws IOException {
+    return getVolume(volumeName).getBucket(bucketName);
+  }
+
+  protected OzoneBucket getBucket(OzoneVolume volume, String bucketName)
+      throws IOException {
+    OzoneBucket bucket = null;
+    try {
+      bucket = volume.getBucket(bucketName);
+    } catch (Exception ex) {
+      if (ex.getMessage().contains("NOT_FOUND")) {
+        throw new NotFoundException("Bucket" + bucketName + " is not found");
+      } else {
+        throw ex;
+      }
+    }
+    return bucket;
+  }
+
+  protected OzoneVolume getVolume(String volumeName) throws IOException {
+    OzoneVolume volume = null;
+    try {
+      volume = client.getObjectStore().getVolume(volumeName);
+    } catch (Exception ex) {
+      if (ex.getMessage().contains("NOT_FOUND")) {
+        throw new NotFoundException("Volume " + volumeName + " is not found");
+      } else {
+        throw ex;
+      }
+    }
+    return volume;
+  }
+
+  @VisibleForTesting
+  public void setClient(OzoneClient ozoneClient) {
+    this.client = ozoneClient;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
index 4e3e48e..061a2d7 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
@@ -17,8 +17,11 @@
  */
 package org.apache.hadoop.ozone.s3;
 
+import java.io.IOException;
+
 import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,21 +38,29 @@ public class Gateway extends GenericCli {
 
   private static final Logger LOG = LoggerFactory.getLogger(Gateway.class);
 
+  private S3GatewayHttpServer httpServer;
+
   public static void main(String[] args) throws Exception {
     new Gateway().run(args);
   }
 
   @Override
   public Void call() throws Exception {
+    OzoneConfiguration ozoneConfiguration = createOzoneConfiguration();
+    OzoneConfigurationHolder.setConfiguration(ozoneConfiguration);
+    httpServer = new S3GatewayHttpServer(ozoneConfiguration, "s3gateway");
     start();
     return null;
   }
 
-  public void start() {
+  public void start() throws IOException {
     LOG.info("Starting Ozone S3 gateway");
+    httpServer.start();
   }
 
-  public void stop() {
-    LOG.info("Stoping Ozone S3 gateway");
+  public void stop() throws Exception {
+    LOG.info("Stopping Ozone S3 gateway");
+    httpServer.stop();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/GatewayApplication.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/GatewayApplication.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/GatewayApplication.java
new file mode 100644
index 0000000..c5a291b
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/GatewayApplication.java
@@ -0,0 +1,29 @@
+/*
+ * 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.s3;
+
+import org.glassfish.jersey.server.ResourceConfig;
+
+/**
+ * JaxRS resource definition.
+ */
+public class GatewayApplication extends ResourceConfig {
+  public GatewayApplication() {
+    packages("org.apache.hadoop.ozone.s3");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java
new file mode 100644
index 0000000..e438157
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java
@@ -0,0 +1,48 @@
+/*
+ * 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.s3;
+
+import javax.enterprise.context.ApplicationScoped;
+import javax.enterprise.inject.Produces;
+import javax.inject.Inject;
+import java.io.IOException;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+
+/**
+ * This class creates the OzoneClient for the Rest endpoints.
+ */
+@ApplicationScoped
+public class OzoneClientProducer {
+
+  private OzoneConfiguration ozoneConfiguration;
+
+  @Inject
+  public OzoneClientProducer(
+      OzoneConfiguration ozoneConfiguration) {
+    this.ozoneConfiguration = ozoneConfiguration;
+  }
+
+  @Produces
+  public OzoneClient createClient() throws IOException {
+    return OzoneClientFactory.getClient(ozoneConfiguration);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneConfigurationHolder.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneConfigurationHolder.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneConfigurationHolder.java
new file mode 100644
index 0000000..4aeab1f
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneConfigurationHolder.java
@@ -0,0 +1,43 @@
+/*
+ * 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.s3;
+
+import javax.enterprise.inject.Produces;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+
+/**
+ * Ozone Configuration factory.
+ * <p>
+ * As the OzoneConfiguration is created by the CLI application here we inject
+ * it via a singleton instance to the Jax-RS/CDI instances.
+ */
+public class OzoneConfigurationHolder {
+
+  private static OzoneConfiguration configuration;
+
+  @Produces
+  public OzoneConfiguration configuration() {
+    return configuration;
+  }
+
+  public static void setConfiguration(
+      OzoneConfiguration conf) {
+    OzoneConfigurationHolder.configuration = conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayConfigKeys.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayConfigKeys.java
new file mode 100644
index 0000000..c340a50
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayConfigKeys.java
@@ -0,0 +1,54 @@
+/*
+ * 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.s3;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This class contains constants for configuration keys used in S3G.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public final class S3GatewayConfigKeys {
+
+  public static final String OZONE_S3G_HTTP_ENABLED_KEY =
+      "ozone.s3g.http.enabled";
+  public static final String OZONE_S3G_HTTP_BIND_HOST_KEY =
+      "ozone.s3g.http-bind-host";
+  public static final String OZONE_S3G_HTTPS_BIND_HOST_KEY =
+      "ozone.s3g.https-bind-host";
+  public static final String OZONE_S3G_HTTP_ADDRESS_KEY =
+      "ozone.s3g.http-address";
+  public static final String OZONE_S3G_HTTPS_ADDRESS_KEY =
+      "ozone.s3g.https-address";
+  public static final String OZONE_S3G_KEYTAB_FILE =
+      "ozone.s3g.keytab.file";
+  public static final String OZONE_S3G_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
+  public static final int OZONE_S3G_HTTP_BIND_PORT_DEFAULT = 9878;
+  public static final int OZONE_S3G_HTTPS_BIND_PORT_DEFAULT = 9879;
+  public static final String OZONE_S3G_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL =
+      "ozone.s3g.authentication.kerberos.principal";
+
+  /**
+   * Never constructed.
+   */
+  private S3GatewayConfigKeys() {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayHttpServer.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayHttpServer.java
new file mode 100644
index 0000000..f20b928
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/S3GatewayHttpServer.java
@@ -0,0 +1,85 @@
+/*
+ * 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.s3;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.server.BaseHttpServer;
+
+/**
+ * S3 Gateway specific configuration keys.
+ */
+public class S3GatewayHttpServer extends BaseHttpServer {
+
+  public S3GatewayHttpServer(Configuration conf,
+      String name) throws IOException {
+    super(conf, name);
+  }
+
+  @Override
+  protected String getHttpAddressKey() {
+    return S3GatewayConfigKeys.OZONE_S3G_HTTP_ADDRESS_KEY;
+  }
+
+  @Override
+  protected String getHttpBindHostKey() {
+    return S3GatewayConfigKeys.OZONE_S3G_HTTP_BIND_HOST_KEY;
+  }
+
+  @Override
+  protected String getHttpsAddressKey() {
+    return S3GatewayConfigKeys.OZONE_S3G_HTTPS_ADDRESS_KEY;
+  }
+
+  @Override
+  protected String getHttpsBindHostKey() {
+    return S3GatewayConfigKeys.OZONE_S3G_HTTPS_BIND_HOST_KEY;
+  }
+
+  @Override
+  protected String getBindHostDefault() {
+    return S3GatewayConfigKeys.OZONE_S3G_HTTP_BIND_HOST_DEFAULT;
+  }
+
+  @Override
+  protected int getHttpBindPortDefault() {
+    return S3GatewayConfigKeys.OZONE_S3G_HTTP_BIND_PORT_DEFAULT;
+  }
+
+  @Override
+  protected int getHttpsBindPortDefault() {
+    return S3GatewayConfigKeys.OZONE_S3G_HTTPS_BIND_PORT_DEFAULT;
+  }
+
+  @Override
+  protected String getKeytabFile() {
+    return S3GatewayConfigKeys.OZONE_S3G_KEYTAB_FILE;
+  }
+
+  @Override
+  protected String getSpnegoPrincipal() {
+    return S3GatewayConfigKeys.OZONE_S3G_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL;
+  }
+
+  @Override
+  protected String getEnabledKey() {
+    return S3GatewayConfigKeys.OZONE_S3G_HTTP_ENABLED_KEY;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/DeleteBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/DeleteBucket.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/DeleteBucket.java
new file mode 100644
index 0000000..bbeb7d5
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/DeleteBucket.java
@@ -0,0 +1,44 @@
+/*
+ * 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.s3.bucket;
+
+import javax.ws.rs.DELETE;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+
+import org.apache.hadoop.ozone.s3.EndpointBase;
+
+/**
+ * Delete a bucket.
+ */
+@Path("/{volume}/{bucket}")
+public class DeleteBucket extends EndpointBase {
+
+  @DELETE
+  @Produces(MediaType.APPLICATION_XML)
+  public void put(
+      @PathParam("volume") String volumeName,
+      @PathParam("bucket") String bucketName) throws IOException {
+
+    getVolume(volumeName).deleteBucket(bucketName);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/PutBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/PutBucket.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/PutBucket.java
new file mode 100644
index 0000000..a46b771
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/PutBucket.java
@@ -0,0 +1,44 @@
+/*
+ * 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.s3.bucket;
+
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+
+import org.apache.hadoop.ozone.s3.EndpointBase;
+
+/**
+ * Create new bucket.
+ */
+@Path("/{volume}/{bucket}")
+public class PutBucket extends EndpointBase {
+
+  @PUT
+  @Produces(MediaType.APPLICATION_XML)
+  public void put(
+      @PathParam("volume") String volumeName,
+      @PathParam("bucket") String bucketName) throws IOException {
+
+    getVolume(volumeName).createBucket(bucketName);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/package-info.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/package-info.java
new file mode 100644
index 0000000..c099c69
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/package-info.java
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/**
+ * Rest endpoint implementation for the bucket specific methods.
+ */
+@javax.xml.bind.annotation.XmlSchema(
+    namespace = "http://s3.amazonaws"
+        + ".com/doc/2006-03-01/", elementFormDefault =
+    javax.xml.bind.annotation.XmlNsForm.QUALIFIED,
+    xmlns = {
+        @javax.xml.bind.annotation.XmlNs(namespaceURI = "http://s3.amazonaws"
+            + ".com/doc/2006-03-01/", prefix = "")})
+
+package org.apache.hadoop.ozone.s3.bucket;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/CommonPrefix.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/CommonPrefix.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/CommonPrefix.java
new file mode 100644
index 0000000..83e6047
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/CommonPrefix.java
@@ -0,0 +1,47 @@
+/*
+ * 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.s3.commontypes;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+
+/**
+ * Directory name ("key prefix") in case of listing.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+public class CommonPrefix {
+
+  @XmlElement(name = "Prefix")
+  private String prefix;
+
+  public CommonPrefix(String prefix) {
+    this.prefix = prefix;
+  }
+
+  public CommonPrefix() {
+  }
+
+  public String getPrefix() {
+    return prefix;
+  }
+
+  public void setPrefix(String prefix) {
+    this.prefix = prefix;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/IsoDateAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/IsoDateAdapter.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/IsoDateAdapter.java
new file mode 100644
index 0000000..281e00b
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/IsoDateAdapter.java
@@ -0,0 +1,47 @@
+/*
+ * 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.s3.commontypes;
+
+import javax.xml.bind.annotation.adapters.XmlAdapter;
+import java.time.Instant;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+
+/**
+ * A converter to convert Instant to standard date string.
+ */
+public class IsoDateAdapter extends XmlAdapter<String, Instant> {
+
+  private DateTimeFormatter iso8861Formatter;
+
+  public IsoDateAdapter() {
+    iso8861Formatter =
+        DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mmX")
+            .withZone(ZoneOffset.UTC);
+  }
+
+  @Override
+  public Instant unmarshal(String v) throws Exception {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String marshal(Instant v) throws Exception {
+    return iso8861Formatter.format(v);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/KeyMetadata.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/KeyMetadata.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/KeyMetadata.java
new file mode 100644
index 0000000..34cea28
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/KeyMetadata.java
@@ -0,0 +1,87 @@
+/*
+ * 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.s3.commontypes;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import java.time.Instant;
+
+/**
+ * Metadata object represents one key in the object store.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+public class KeyMetadata {
+
+  @XmlElement(name = "Key")
+  private String key; // or the Object Name
+
+  @XmlJavaTypeAdapter(IsoDateAdapter.class)
+  @XmlElement(name = "LastModified")
+  private Instant lastModified;
+
+  @XmlElement(name = "ETag")
+  private String eTag;
+
+  @XmlElement(name = "Size")
+  private long size;
+
+  @XmlElement(name = "StorageClass")
+  private String storageClass;
+
+  public String getKey() {
+    return key;
+  }
+
+  public void setKey(String key) {
+    this.key = key;
+  }
+
+  public Instant getLastModified() {
+    return lastModified;
+  }
+
+  public void setLastModified(Instant lastModified) {
+    this.lastModified = lastModified;
+  }
+
+  public String getETag() {
+    return eTag;
+  }
+
+  public void setETag(String tag) {
+    this.eTag = tag;
+  }
+
+  public long getSize() {
+    return size;
+  }
+
+  public void setSize(long size) {
+    this.size = size;
+  }
+
+  public String getStorageClass() {
+    return storageClass;
+  }
+
+  public void setStorageClass(String storageClass) {
+    this.storageClass = storageClass;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/package-info.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/package-info.java
new file mode 100644
index 0000000..dd916e8
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/commontypes/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/**
+ * Common classes required for S3 rest API's.
+ */
+@javax.xml.bind.annotation.XmlSchema(
+    namespace = "http://s3.amazonaws"
+        + ".com/doc/2006-03-01/", elementFormDefault =
+    javax.xml.bind.annotation.XmlNsForm.QUALIFIED,
+    xmlns = {
+        @javax.xml.bind.annotation.XmlNs(namespaceURI = "http://s3.amazonaws"
+            + ".com/doc/2006-03-01/", prefix = "")})
+package org.apache.hadoop.ozone.s3.commontypes;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/DeleteObject.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/DeleteObject.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/DeleteObject.java
new file mode 100644
index 0000000..d5ef70e
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/DeleteObject.java
@@ -0,0 +1,51 @@
+/*
+ * 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.s3.object;
+
+import javax.ws.rs.DELETE;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.s3.EndpointBase;
+
+/**
+ * Delete Object rest endpoint.
+ */
+@Path("/{volume}/{bucket}/{path:.+}")
+public class DeleteObject extends EndpointBase {
+
+  @DELETE
+  @Produces(MediaType.APPLICATION_XML)
+  public Response delete(
+      @PathParam("volume") String volumeName,
+      @PathParam("bucket") String bucketName,
+      @PathParam("path") String keyPath) throws IOException {
+
+    OzoneBucket bucket = getBucket(volumeName, bucketName);
+    bucket.deleteKey(keyPath);
+    return Response.
+        ok()
+        .build();
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/HeadObject.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/HeadObject.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/HeadObject.java
new file mode 100644
index 0000000..e284100
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/HeadObject.java
@@ -0,0 +1,58 @@
+/*
+ * 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.s3.object;
+
+import javax.ws.rs.HEAD;
+import javax.ws.rs.HeaderParam;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.s3.EndpointBase;
+
+/**
+ * Get object info rest endpoint.
+ */
+@Path("/{volume}/{bucket}/{path:.+}")
+public class HeadObject extends EndpointBase {
+
+  @HEAD
+  @Produces(MediaType.APPLICATION_XML)
+  public Response head(
+      @PathParam("volume") String volumeName,
+      @PathParam("bucket") String bucketName,
+      @PathParam("path") String keyPath,
+      @HeaderParam("Content-Length") long length,
+      InputStream body) throws IOException {
+
+    OzoneBucket bucket = getBucket(volumeName, bucketName);
+    OzoneKeyDetails key = bucket.getKey(keyPath);
+
+    return Response.
+        ok()
+        .header("Content-Length", key.getDataSize())
+        .build();
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/ListObject.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/ListObject.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/ListObject.java
new file mode 100644
index 0000000..73f1343
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/ListObject.java
@@ -0,0 +1,118 @@
+/*
+ * 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.s3.object;
+
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.time.Instant;
+import java.util.Iterator;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.s3.EndpointBase;
+import org.apache.hadoop.ozone.s3.commontypes.KeyMetadata;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * List Object Rest endpoint.
+ */
+@Path("/{volume}/{bucket}")
+public class ListObject extends EndpointBase {
+
+
+  @GET
+  @Produces(MediaType.APPLICATION_XML)
+  public ListObjectResponse get(
+      @PathParam("volume") String volumeName,
+      @PathParam("bucket") String bucketName,
+      @QueryParam("delimiter") String delimiter,
+      @QueryParam("encoding-type") String encodingType,
+      @QueryParam("marker") String marker,
+      @DefaultValue("1000") @QueryParam("max-keys") int maxKeys,
+      @QueryParam("prefix") String prefix,
+      @Context HttpHeaders hh) throws IOException {
+
+    if (delimiter == null) {
+      delimiter = "/";
+    }
+    if (prefix == null) {
+      prefix = "";
+    }
+
+    OzoneVolume volume = getVolume(volumeName);
+    OzoneBucket bucket = getBucket(volume, bucketName);
+
+    Iterator<? extends OzoneKey> ozoneKeyIterator = bucket.listKeys(prefix);
+
+    ListObjectResponse response = new ListObjectResponse();
+    response.setDelimiter(delimiter);
+    response.setName(bucketName);
+    response.setPrefix(prefix);
+    response.setMarker("");
+    response.setMaxKeys(1000);
+    response.setEncodingType("url");
+    response.setTruncated(false);
+
+    String prevDir = null;
+    while (ozoneKeyIterator.hasNext()) {
+      OzoneKey next = ozoneKeyIterator.next();
+      String relativeKeyName = next.getName().substring(prefix.length());
+
+      int depth =
+          StringUtils.countMatches(relativeKeyName, delimiter);
+
+      if (prefix.length() > 0 && !prefix.endsWith(delimiter)
+          && relativeKeyName.length() > 0) {
+        response.addPrefix(prefix + "/");
+        break;
+      }
+      if (depth > 0) {
+        String dirName = relativeKeyName
+            .substring(0, relativeKeyName.indexOf(delimiter));
+        if (!dirName.equals(prevDir)) {
+          response.addPrefix(
+              prefix + dirName + delimiter);
+          prevDir = dirName;
+        }
+      } else if (relativeKeyName.endsWith(delimiter)) {
+        response.addPrefix(relativeKeyName);
+      } else if (relativeKeyName.length() > 0) {
+        KeyMetadata keyMetadata = new KeyMetadata();
+        keyMetadata.setKey(next.getName());
+        keyMetadata.setSize(next.getDataSize());
+        keyMetadata.setETag("" + next.getModificationTime());
+        keyMetadata.setStorageClass("STANDARD");
+        keyMetadata
+            .setLastModified(Instant.ofEpochMilli(next.getModificationTime()));
+        response.addKey(keyMetadata);
+      }
+    }
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/ListObjectResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/ListObjectResponse.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/ListObjectResponse.java
new file mode 100644
index 0000000..a32fb93
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/ListObjectResponse.java
@@ -0,0 +1,147 @@
+/*
+ * 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.s3.object;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.ozone.s3.commontypes.CommonPrefix;
+import org.apache.hadoop.ozone.s3.commontypes.KeyMetadata;
+
+/**
+ * Response from the ListObject RPC Call.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlRootElement(name = "ListBucketResult", namespace = "http://s3.amazonaws"
+    + ".com/doc/2006-03-01/")
+public class ListObjectResponse {
+
+  @XmlElement(name = "Name")
+  private String name;
+
+  @XmlElement(name = "Prefix")
+  private String prefix;
+
+  @XmlElement(name = "Marker")
+  private String marker;
+
+  @XmlElement(name = "MaxKeys")
+  private int maxKeys;
+
+  @XmlElement(name = "Delimiter")
+  private String delimiter = "/";
+
+  @XmlElement(name = "EncodingType")
+  private String encodingType = "url";
+
+  @XmlElement(name = "IsTruncated")
+  private boolean isTruncated;
+
+  @XmlElement(name = "Contents")
+  private List<KeyMetadata> contents = new ArrayList<>();
+
+  @XmlElement(name = "CommonPrefixes")
+  private List<CommonPrefix> commonPrefixes = new ArrayList<>();
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getPrefix() {
+    return prefix;
+  }
+
+  public void setPrefix(String prefix) {
+    this.prefix = prefix;
+  }
+
+  public String getMarker() {
+    return marker;
+  }
+
+  public void setMarker(String marker) {
+    this.marker = marker;
+  }
+
+  public int getMaxKeys() {
+    return maxKeys;
+  }
+
+  public void setMaxKeys(int maxKeys) {
+    this.maxKeys = maxKeys;
+  }
+
+  public String getDelimiter() {
+    return delimiter;
+  }
+
+  public void setDelimiter(String delimiter) {
+    this.delimiter = delimiter;
+  }
+
+  public String getEncodingType() {
+    return encodingType;
+  }
+
+  public void setEncodingType(String encodingType) {
+    this.encodingType = encodingType;
+  }
+
+  public boolean isTruncated() {
+    return isTruncated;
+  }
+
+  public void setTruncated(boolean truncated) {
+    isTruncated = truncated;
+  }
+
+  public List<KeyMetadata> getContents() {
+    return contents;
+  }
+
+  public void setContents(
+      List<KeyMetadata> contents) {
+    this.contents = contents;
+  }
+
+  public List<CommonPrefix> getCommonPrefixes() {
+    return commonPrefixes;
+  }
+
+  public void setCommonPrefixes(
+      List<CommonPrefix> commonPrefixes) {
+    this.commonPrefixes = commonPrefixes;
+  }
+
+  public void addKey(KeyMetadata keyMetadata) {
+    contents.add(keyMetadata);
+  }
+
+  public void addPrefix(String relativeKeyName) {
+    commonPrefixes.add(new CommonPrefix(relativeKeyName));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/PutObject.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/PutObject.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/PutObject.java
new file mode 100644
index 0000000..53e63d8
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/PutObject.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.s3.object;
+
+import javax.ws.rs.HeaderParam;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.ozone.s3.EndpointBase;
+
+/**
+ * File upload.
+ */
+@Path("/{volume}/{bucket}/{path:.+}")
+public class PutObject extends EndpointBase {
+
+  @PUT
+  @Produces(MediaType.APPLICATION_XML)
+  public void put(
+      @PathParam("volume") String volumeName,
+      @PathParam("bucket") String bucketName,
+      @PathParam("path") String keyPath,
+      @HeaderParam("Content-Length") long length,
+      InputStream body) throws IOException {
+
+    OzoneBucket bucket = getBucket(volumeName, bucketName);
+
+    OzoneOutputStream output = bucket
+        .createKey(keyPath, length, ReplicationType.STAND_ALONE,
+            ReplicationFactor.ONE);
+
+    IOUtils.copy(body, output);
+    output.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/package-info.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/package-info.java
new file mode 100644
index 0000000..e255991
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/object/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/**
+ * Rest endpoint implementation for the Object specific methods.
+ */
+@javax.xml.bind.annotation.XmlSchema(
+    namespace = "http://s3.amazonaws"
+        + ".com/doc/2006-03-01/", elementFormDefault =
+    javax.xml.bind.annotation.XmlNsForm.QUALIFIED,
+    xmlns = {
+        @javax.xml.bind.annotation.XmlNs(namespaceURI = "http://s3.amazonaws"
+            + ".com/doc/2006-03-01/", prefix = "")})
+package org.apache.hadoop.ozone.s3.object;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/package-info.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/package-info.java
new file mode 100644
index 0000000..9d41551
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains the top level generic classes of s3 gateway.
+ */
+package org.apache.hadoop.ozone.s3;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/resources/META-INF/beans.xml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/resources/META-INF/beans.xml b/hadoop-ozone/s3gateway/src/main/resources/META-INF/beans.xml
new file mode 100644
index 0000000..cf00d29
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/resources/META-INF/beans.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<beans xmlns="http://java.sun.com/xml/ns/javaee"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+      http://java.sun.com/xml/ns/javaee
+      http://java.sun.com/xml/ns/javaee/beans_1_0.xsd">
+</beans>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/resources/webapps/s3gateway/WEB-INF/beans.xml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/resources/webapps/s3gateway/WEB-INF/beans.xml b/hadoop-ozone/s3gateway/src/main/resources/webapps/s3gateway/WEB-INF/beans.xml
new file mode 100644
index 0000000..cf00d29
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/resources/webapps/s3gateway/WEB-INF/beans.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<beans xmlns="http://java.sun.com/xml/ns/javaee"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+      http://java.sun.com/xml/ns/javaee
+      http://java.sun.com/xml/ns/javaee/beans_1_0.xsd">
+</beans>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c3fbbc4/hadoop-ozone/s3gateway/src/main/resources/webapps/s3gateway/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/s3gateway/src/main/resources/webapps/s3gateway/WEB-INF/web.xml b/hadoop-ozone/s3gateway/src/main/resources/webapps/s3gateway/WEB-INF/web.xml
new file mode 100644
index 0000000..a3552f0
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/resources/webapps/s3gateway/WEB-INF/web.xml
@@ -0,0 +1,36 @@
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<web-app version="3.0" xmlns="http://java.sun.com/xml/ns/javaee"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://java.sun.com/xml/ns/javaee http://java.sun.com/xml/ns/javaee/web-app_3_0.xsd">
+  <servlet>
+    <servlet-name>jaxrs</servlet-name>
+    <servlet-class>org.glassfish.jersey.servlet.ServletContainer</servlet-class>
+    <init-param>
+      <param-name>javax.ws.rs.Application</param-name>
+      <param-value>org.apache.hadoop.ozone.s3.GatewayApplication</param-value>
+    </init-param>
+    <load-on-startup>1</load-on-startup>
+  </servlet>
+  <servlet-mapping>
+    <servlet-name>jaxrs</servlet-name>
+    <url-pattern>/*</url-pattern>
+  </servlet-mapping>
+
+  <listener>
+    <listener-class>org.jboss.weld.environment.servlet.Listener</listener-class>
+  </listener>
+
+
+</web-app>
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org