You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by bh...@apache.org on 2021/11/02 19:53:19 UTC

[ozone] branch master updated: HDDS-5883 Change S3G client to set S3 Auth per req (#2775)

This is an automated email from the ASF dual-hosted git repository.

bharat pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new e427b71  HDDS-5883 Change S3G client to set S3 Auth per req (#2775)
e427b71 is described below

commit e427b71d08cfda01bdf079985426725a702fd555
Author: Ritesh H Shukla <ke...@gmail.com>
AuthorDate: Tue Nov 2 12:53:06 2021 -0700

    HDDS-5883 Change S3G client to set S3 Auth per req (#2775)
---
 .../ozone/client/protocol/ClientProtocol.java      | 19 ++++++
 .../apache/hadoop/ozone/client/rpc/RpcClient.java  | 33 +++++++++--
 .../apache/hadoop/ozone/om/protocol/S3Auth.java    | 45 ++++++++++++++
 .../om/protocolPB/OzoneManagerClientProtocol.java  | 38 ++++++++++++
 ...OzoneManagerProtocolClientSideTranslatorPB.java | 45 +++++++++++---
 .../src/main/proto/OmClientProtocol.proto          |  2 +-
 .../hadoop/ozone/s3/OzoneClientProducer.java       | 69 +++++-----------------
 .../hadoop/ozone/s3/endpoint/BucketEndpoint.java   |  5 ++
 .../hadoop/ozone/s3/endpoint/EndpointBase.java     | 29 ++++++++-
 .../hadoop/ozone/s3/endpoint/ObjectEndpoint.java   |  1 +
 .../hadoop/ozone/s3/endpoint/RootEndpoint.java     |  5 ++
 .../hadoop/ozone/s3/TestOzoneClientProducer.java   |  4 +-
 12 files changed, 225 insertions(+), 70 deletions(-)

diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
index c4f98e8..2feb577 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.protocol.S3Auth;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRoleInfo;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
@@ -752,4 +753,22 @@ public interface ClientProtocol {
    */
   OzoneKey headObject(String volumeName, String bucketName,
       String keyName) throws IOException;
+
+  /**
+   * Sets the S3 Authentication information for the requests executed on behalf
+   * of the S3 API implementation within Ozone.
+   * @param s3Auth authentication information for each S3 API call.
+   */
+  void setTheadLocalS3Auth(S3Auth s3Auth);
+
+  /**
+   * Gets the S3 Authentication information that is attached to the thread.
+   * @return S3 Authentication information.
+   */
+  S3Auth getThreadLocalS3Auth();
+
+  /**
+   * Clears the S3 Authentication information attached to the thread.
+   */
+  void clearTheadLocalS3Auth();
 }
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index f67d1dd..60fecc2 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -105,8 +105,10 @@ import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfoEx;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.protocol.S3Auth;
 import org.apache.hadoop.ozone.om.protocolPB.OmTransport;
 import org.apache.hadoop.ozone.om.protocolPB.OmTransportFactory;
+import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerClientProtocol;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRoleInfo;
 import org.apache.hadoop.ozone.security.GDPRSymmetricKey;
@@ -147,7 +149,7 @@ public class RpcClient implements ClientProtocol {
       LoggerFactory.getLogger(RpcClient.class);
 
   private final ConfigurationSource conf;
-  private final OzoneManagerProtocol ozoneManagerClient;
+  private final OzoneManagerClientProtocol ozoneManagerClient;
   private final XceiverClientFactory xceiverClientManager;
   private final int chunkSize;
   private final UserGroupInformation ugi;
@@ -183,11 +185,10 @@ public class RpcClient implements ClientProtocol {
     this.clientConfig = conf.getObject(OzoneClientConfig.class);
 
     OmTransport omTransport = createOmTransport(omServiceId);
-
     this.ozoneManagerClient = TracingUtil.createProxy(
         new OzoneManagerProtocolClientSideTranslatorPB(omTransport,
             clientId.toString()),
-        OzoneManagerProtocol.class, conf
+        OzoneManagerClientProtocol.class, conf
     );
     dtService = omTransport.getDelegationTokenService();
     ServiceInfoEx serviceInfoEx = ozoneManagerClient.getServiceInfo();
@@ -543,6 +544,15 @@ public class RpcClient implements ClientProtocol {
    * @return listOfAcls
    * */
   private List<OzoneAcl> getAclList() {
+    if (ozoneManagerClient.getThreadLocalS3Auth() != null) {
+      UserGroupInformation aclUgi =
+          UserGroupInformation.createRemoteUser(
+             ozoneManagerClient.getThreadLocalS3Auth().getAccessID());
+      return OzoneAclUtil.getAclList(
+          aclUgi.getUserName(),
+          aclUgi.getGroupNames(),
+         userRights, groupRights);
+    }
     return OzoneAclUtil.getAclList(ugi.getUserName(), ugi.getGroupNames(),
         userRights, groupRights);
   }
@@ -782,7 +792,6 @@ public class RpcClient implements ClientProtocol {
         .addAllMetadata(metadata)
         .setAcls(getAclList())
         .setLatestVersionLocation(getLatestVersionLocation);
-
     if (Boolean.parseBoolean(metadata.get(OzoneConsts.GDPR_FLAG))) {
       try{
         GDPRSymmetricKey gKey = new GDPRSymmetricKey(new SecureRandom());
@@ -1495,4 +1504,20 @@ public class RpcClient implements ClientProtocol {
         keyInfo.getModificationTime(), keyInfo.getReplicationConfig());
 
   }
+
+  @Override
+  public void setTheadLocalS3Auth(
+      S3Auth ozoneSharedSecretAuth) {
+    ozoneManagerClient.setThreadLocalS3Auth(ozoneSharedSecretAuth);
+  }
+
+  @Override
+  public S3Auth getThreadLocalS3Auth() {
+    return ozoneManagerClient.getThreadLocalS3Auth();
+  }
+
+  @Override
+  public void clearTheadLocalS3Auth() {
+    ozoneManagerClient.clearThreadLocalS3Auth();
+  }
 }
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/S3Auth.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/S3Auth.java
new file mode 100644
index 0000000..d86f343
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/S3Auth.java
@@ -0,0 +1,45 @@
+/**
+ * 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.om.protocol;
+
+/**
+ * S3Auth wraps the data needed for S3 Authentication.
+ */
+public class S3Auth {
+  private String stringToSign;
+  private String signature;
+  private String accessID;
+
+  public S3Auth(final String stringToSign,
+                final String signature, final String accessID) {
+    this.accessID = accessID;
+    this.stringToSign = stringToSign;
+    this.signature = signature;
+  }
+  public String getStringTosSign() {
+    return stringToSign;
+  }
+
+  public String getSignature() {
+    return signature;
+  }
+
+  public String getAccessID() {
+    return accessID;
+  }
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerClientProtocol.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerClientProtocol.java
new file mode 100644
index 0000000..fca6c14
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerClientProtocol.java
@@ -0,0 +1,38 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.protocolPB;
+
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.protocol.S3Auth;
+
+/**
+ * OzoneManagerClientProtocol defines interfaces needed on the client side
+ * when communicating with Ozone Manager.
+ */
+public interface OzoneManagerClientProtocol extends OzoneManagerProtocol {
+  /**
+   * Sets the S3 Authentication information when OM request is generated as
+   * part of the S3 API implementation from Ozone. S3 Gateway needs to add
+   * authentication information on a per-request basis which is attached as
+   * a thread local variable.
+   */
+  void setThreadLocalS3Auth(S3Auth s3Auth);
+  S3Auth getThreadLocalS3Auth();
+  void clearThreadLocalS3Auth();
+}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
index 7c8b785..b093919 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
@@ -25,9 +25,9 @@ import java.util.Map;
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos
     .UpgradeFinalizationStatus;
-import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
 import org.apache.hadoop.io.Text;
@@ -56,7 +56,7 @@ import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfoEx;
-import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.protocol.S3Auth;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AddAclRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AddAclResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AllocateBlockRequest;
@@ -172,12 +172,13 @@ import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.
 
 @InterfaceAudience.Private
 public final class OzoneManagerProtocolClientSideTranslatorPB
-    implements OzoneManagerProtocol {
+    implements OzoneManagerClientProtocol {
 
   private final String clientID;
 
   private OmTransport transport;
-
+  private ThreadLocal<S3Auth> threadLocalS3Auth
+      = new ThreadLocal<>();
   public OzoneManagerProtocolClientSideTranslatorPB(OmTransport omTransport,
       String clientId) {
     this.clientID = clientId;
@@ -223,11 +224,23 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
    */
   private OMResponse submitRequest(OMRequest omRequest)
       throws IOException {
-    OMRequest payload = OMRequest.newBuilder(omRequest)
-        .setTraceID(TracingUtil.exportCurrentSpan())
-        .build();
-
-    return transport.submitRequest(payload);
+    OMRequest.Builder  builder = OMRequest.newBuilder(omRequest);
+    // Insert S3 Authentication information for each request.
+    if (getThreadLocalS3Auth() != null) {
+      builder.setS3Authentication(
+          S3Authentication.newBuilder()
+              .setSignature(
+                  threadLocalS3Auth.get().getSignature())
+              .setStringToSign(
+                  threadLocalS3Auth.get().getStringTosSign())
+              .setAccessId(
+                  threadLocalS3Auth.get().getAccessID())
+              .build());
+    }
+    OMResponse response =
+        transport.submitRequest(
+            builder.setTraceID(TracingUtil.exportCurrentSpan()).build());
+    return response;
   }
 
   /**
@@ -1259,6 +1272,20 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
     }
   }
 
+  @Override
+  public void setThreadLocalS3Auth(
+      S3Auth s3Auth) {
+    this.threadLocalS3Auth.set(s3Auth);
+  }
+  @Override
+  public void clearThreadLocalS3Auth() {
+    this.threadLocalS3Auth.remove();
+  }
+  @Override
+  public S3Auth getThreadLocalS3Auth() {
+    return this.threadLocalS3Auth.get();
+  }
+
   /**
    * Get File Status for an Ozone key.
    *
diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
index dda8a1c..f53bc84 100644
--- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
+++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
@@ -1346,7 +1346,7 @@ message UpdateGetS3SecretRequest {
 }
 
 /**
-  This will be used by OM to authenicate S3 gateway requests on a per request basis.
+  This will be used by OM to authenticate S3 gateway requests on a per request basis.
 */
 message S3Authentication {
     optional string stringToSign = 1;
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
index e7d3ed0..8f48d8a 100644
--- 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
@@ -25,31 +25,25 @@ import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.container.ContainerRequestContext;
 import javax.ws.rs.core.Context;
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.security.PrivilegedExceptionAction;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ozone.OzoneSecurityUtil;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.om.protocol.S3Auth;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.signature.SignatureInfo;
 import org.apache.hadoop.ozone.s3.signature.SignatureInfo.Version;
 import org.apache.hadoop.ozone.s3.signature.SignatureProcessor;
 import org.apache.hadoop.ozone.s3.signature.StringToSignProducer;
-import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-
-import com.google.common.annotations.VisibleForTesting;
-import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMTokenProto.Type.S3AUTHINFO;
-import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.INTERNAL_ERROR;
-import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_HEADER;
 import org.jetbrains.annotations.NotNull;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.INTERNAL_ERROR;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_HEADER;
+
 /**
  * This class creates the OzoneClient for the Rest endpoints.
  */
@@ -59,7 +53,7 @@ public class OzoneClientProducer {
   private static final Logger LOG =
       LoggerFactory.getLogger(OzoneClientProducer.class);
 
-  private OzoneClient client;
+  private static OzoneClient client;
 
   @Inject
   private SignatureProcessor signatureProcessor;
@@ -77,23 +71,22 @@ public class OzoneClientProducer {
   private ContainerRequestContext context;
 
   @Produces
-  public OzoneClient createClient() throws WebApplicationException,
+  public synchronized OzoneClient createClient() throws WebApplicationException,
       IOException {
-    client = getClient(ozoneConfiguration);
+    if (client == null) {
+      client = createOzoneClient();
+    }
     return client;
   }
 
   @PreDestroy
   public void destroy() throws IOException {
-    client.close();
+    client.getObjectStore().getClientProxy().clearTheadLocalS3Auth();
   }
-
-  private OzoneClient getClient(OzoneConfiguration config)
-      throws WebApplicationException {
-    OzoneClient ozoneClient = null;
+  @Produces
+  public S3Auth getSignature() {
     try {
       SignatureInfo signatureInfo = signatureProcessor.parseSignature();
-
       String stringToSign = "";
       if (signatureInfo.getVersion() == Version.V4) {
         stringToSign =
@@ -102,48 +95,18 @@ public class OzoneClientProducer {
 
       String awsAccessId = signatureInfo.getAwsAccessId();
       validateAccessId(awsAccessId);
-
-      UserGroupInformation remoteUser =
-          UserGroupInformation.createRemoteUser(awsAccessId);
-      if (OzoneSecurityUtil.isSecurityEnabled(config)) {
-        LOG.debug("Creating s3 auth info for client.");
-
-        if (signatureInfo.getVersion() == Version.NONE) {
-          throw MALFORMED_HEADER;
-        }
-
-        OzoneTokenIdentifier identifier = new OzoneTokenIdentifier();
-        identifier.setTokenType(S3AUTHINFO);
-        identifier.setStrToSign(stringToSign);
-        identifier.setSignature(signatureInfo.getSignature());
-        identifier.setAwsAccessId(awsAccessId);
-        identifier.setOwner(new Text(awsAccessId));
-        LOG.trace("Adding token for service:{}", omService);
-
-        Token<OzoneTokenIdentifier> token = new Token(identifier.getBytes(),
-            identifier.getSignature().getBytes(StandardCharsets.UTF_8),
-            identifier.getKind(),
-            omService);
-        remoteUser.addToken(token);
-
-      }
-      ozoneClient =
-          remoteUser.doAs((PrivilegedExceptionAction<OzoneClient>) () -> {
-            return createOzoneClient();
-          });
+      return new S3Auth(stringToSign,
+          signatureInfo.getSignature(),
+          awsAccessId);
     } catch (OS3Exception ex) {
       LOG.debug("Error during Client Creation: ", ex);
       throw wrapOS3Exception(ex);
-    } catch(InterruptedException ex){
-      LOG.debug("Error during Client Creation: ", ex);
-      Thread.currentThread().interrupt();
     } catch (Exception e) {
       // For any other critical errors during object creation throw Internal
       // error.
       LOG.debug("Error during Client Creation: ", e);
       throw wrapOS3Exception(INTERNAL_ERROR);
     }
-    return ozoneClient;
   }
 
   @NotNull
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
index 72a95ce..dbf8cf3 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
@@ -602,4 +602,9 @@ public class BucketEndpoint extends EndpointBase {
     keyMetadata.setLastModified(next.getModificationTime());
     response.addKey(keyMetadata);
   }
+
+  @Override
+  public void init() {
+
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
index 137b8ea..6a88075 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.s3.endpoint;
 
+import javax.annotation.PostConstruct;
 import javax.inject.Inject;
 import java.io.IOException;
 import java.util.Collections;
@@ -29,18 +30,25 @@ import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
+import org.apache.hadoop.ozone.om.protocol.S3Auth;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Basic helpers for all the REST endpoints.
  */
-public class EndpointBase {
+public abstract class EndpointBase {
 
   @Inject
   private OzoneClient client;
+  @Inject
+  private S3Auth s3Auth;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(EndpointBase.class);
 
   protected OzoneBucket getBucket(OzoneVolume volume, String bucketName)
       throws OS3Exception, IOException {
@@ -57,6 +65,21 @@ public class EndpointBase {
     return bucket;
   }
 
+  /**
+   * Initializes the object post construction. Calls init() from any
+   * child classes to work around the issue of only one method can be annotated.
+   */
+  @PostConstruct
+  public void initialization() {
+    LOG.debug("S3 access id: {}", s3Auth.getAccessID());
+    getClient().getObjectStore().
+        getClientProxy().
+        setTheadLocalS3Auth(s3Auth);
+    init();
+  }
+
+  public abstract void init();
+
   protected OzoneBucket getBucket(String bucketName)
       throws OS3Exception, IOException {
     OzoneBucket bucket;
@@ -171,4 +194,8 @@ public class EndpointBase {
   public void setClient(OzoneClient ozoneClient) {
     this.client = ozoneClient;
   }
+
+  public OzoneClient getClient() {
+    return client;
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
index 6d61426..64af127 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
@@ -119,6 +119,7 @@ public class ObjectEndpoint extends EndpointBase {
   @Context
   private HttpHeaders headers;
 
+
   private List<String> customizableGetHeaders = new ArrayList<>();
   private int bufferSize;
 
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java
index 0826276..814d87b 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java
@@ -62,4 +62,9 @@ public class RootEndpoint extends EndpointBase {
 
     return Response.ok(response).build();
   }
+
+  @Override
+  public void init() {
+
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestOzoneClientProducer.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestOzoneClientProducer.java
index 38776f1..e7e3371 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestOzoneClientProducer.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestOzoneClientProducer.java
@@ -17,11 +17,11 @@
  */
 package org.apache.hadoop.ozone.s3;
 
-import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.container.ContainerRequestContext;
 import javax.ws.rs.core.MultivaluedHashMap;
 import javax.ws.rs.core.MultivaluedMap;
 import javax.ws.rs.core.UriInfo;
+import java.io.IOException;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.Collection;
@@ -128,7 +128,7 @@ public class TestOzoneClientProducer {
       producer.createClient();
       fail("testGetClientFailure");
     } catch (Exception ex) {
-      Assert.assertTrue(ex instanceof WebApplicationException);
+      Assert.assertTrue(ex instanceof IOException);
     }
   }
 

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