You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/12/20 08:11:34 UTC

[GitHub] [pinot] navina commented on a diff in pull request #10009: Support the cross-account access using IAM role for S3 PinotFS

navina commented on code in PR #10009:
URL: https://github.com/apache/pinot/pull/10009#discussion_r1053013267


##########
pinot-plugins/pinot-file-system/pinot-s3/src/main/java/org/apache/pinot/plugin/filesystem/S3Config.java:
##########
@@ -0,0 +1,156 @@
+/**
+ * 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.pinot.plugin.filesystem;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import java.util.UUID;
+import org.apache.pinot.spi.env.PinotConfiguration;
+
+
+/**
+ * S3 related config
+ */
+public class S3Config {
+
+  private static final boolean DEFAULT_DISABLE_ACL = true;
+
+  public static final String ACCESS_KEY = "accessKey";
+  public static final String SECRET_KEY = "secretKey";
+  public static final String REGION = "region";
+  public static final String ENDPOINT = "endpoint";
+  public static final String DISABLE_ACL_CONFIG_KEY = "disableAcl";
+
+  // Encryption related configurations
+  public static final String SERVER_SIDE_ENCRYPTION_CONFIG_KEY = "serverSideEncryption";
+  public static final String SSE_KMS_KEY_ID_CONFIG_KEY = "ssekmsKeyId";
+  public static final String SSE_KMS_ENCRYPTION_CONTEXT_CONFIG_KEY = "ssekmsEncryptionContext";
+
+  // IAM Role related configurations
+  public static final String IAM_ROLE_BASED_ACCESS_ENABLED = "iamRoleBasedAccessEnabled";
+  public static final String ROLE_ARN = "roleArn";
+  public static final String ROLE_SESSION_NAME = "roleSessionName";
+  public static final String EXTERNAL_ID = "externalId";
+  public static final String SESSION_DURATION_SECONDS = "sessionDurationSeconds";
+  public static final String ASYNC_SESSION_UPDATED_ENABLED = "asyncSessionUpdateEnabled";
+  public static final String DEFAULT_IAM_ROLE_BASED_ACCESS_ENABLED = "false";
+  public static final String DEFAULT_SESSION_DURATION_SECONDS = "900";
+  public static final String DEFAULT_ASYNC_SESSION_UPDATED_ENABLED = "true";
+
+  private final String _accessKey;
+  private final String _secretKey;
+  private final String _region;
+  private final boolean _disableAcl;
+  private final String _endpoint;
+
+  private final String _serverSideEncryption;
+  private String _ssekmsKeyId;
+  private String _ssekmsEncryptionContext;
+
+  private boolean _iamRoleBasedAccess;
+  private String _roleArn;
+  private String _roleSessionName;
+  private String _externalId;
+  private int _sessionDurationSeconds;
+  private boolean _asyncSessionUpdateEnabled;
+
+  public S3Config(PinotConfiguration pinotConfig) {
+    _disableAcl = pinotConfig.getProperty(DISABLE_ACL_CONFIG_KEY, DEFAULT_DISABLE_ACL);
+    _accessKey = pinotConfig.getProperty(ACCESS_KEY);
+    _secretKey = pinotConfig.getProperty(SECRET_KEY);
+    _region = pinotConfig.getProperty(REGION);
+    _endpoint = pinotConfig.getProperty(ENDPOINT);
+
+    _serverSideEncryption = pinotConfig.getProperty(SERVER_SIDE_ENCRYPTION_CONFIG_KEY);
+    _ssekmsKeyId = pinotConfig.getProperty(SSE_KMS_KEY_ID_CONFIG_KEY);
+    _ssekmsEncryptionContext = pinotConfig.getProperty(SSE_KMS_ENCRYPTION_CONTEXT_CONFIG_KEY);
+
+    _iamRoleBasedAccess = Boolean.parseBoolean(
+        pinotConfig.getProperty(IAM_ROLE_BASED_ACCESS_ENABLED, DEFAULT_IAM_ROLE_BASED_ACCESS_ENABLED));
+    _roleArn = pinotConfig.getProperty(ROLE_ARN);
+    _roleSessionName =
+        pinotConfig.getProperty(ROLE_SESSION_NAME, Joiner.on("-").join("pinot", "s3", UUID.randomUUID()));
+    _externalId = pinotConfig.getProperty(EXTERNAL_ID);
+    _sessionDurationSeconds =
+        Integer.parseInt(pinotConfig.getProperty(SESSION_DURATION_SECONDS, DEFAULT_SESSION_DURATION_SECONDS));
+    _asyncSessionUpdateEnabled = Boolean.parseBoolean(
+        pinotConfig.getProperty(ASYNC_SESSION_UPDATED_ENABLED, DEFAULT_ASYNC_SESSION_UPDATED_ENABLED));
+
+    if (_iamRoleBasedAccess) {
+      Preconditions.checkNotNull(_roleArn, "Must provide 'roleArn' for table %s if iamRoleBasedAccess is enabled");

Review Comment:
   is this `table %s` a placeholder for some variable? 



##########
pinot-plugins/pinot-file-system/pinot-s3/src/main/java/org/apache/pinot/plugin/filesystem/S3PinotFS.java:
##########
@@ -68,56 +69,68 @@
 import software.amazon.awssdk.services.s3.model.S3Exception;
 import software.amazon.awssdk.services.s3.model.S3Object;
 import software.amazon.awssdk.services.s3.model.ServerSideEncryption;
+import software.amazon.awssdk.services.sts.StsClient;
+import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider;
+import software.amazon.awssdk.services.sts.model.AssumeRoleRequest;
 
 
 /**
  * Implementation of PinotFS for AWS S3 file system
  */
 public class S3PinotFS extends BasePinotFS {
-  public static final String ACCESS_KEY = "accessKey";
-  public static final String SECRET_KEY = "secretKey";
-  public static final String REGION = "region";
-  public static final String ENDPOINT = "endpoint";
-  public static final String DISABLE_ACL_CONFIG_KEY = "disableAcl";
-  public static final String SERVER_SIDE_ENCRYPTION_CONFIG_KEY = "serverSideEncryption";
-  public static final String SSE_KMS_KEY_ID_CONFIG_KEY = "ssekmsKeyId";
-  public static final String SSE_KMS_ENCRYPTION_CONTEXT_CONFIG_KEY = "ssekmsEncryptionContext";
-
   private static final Logger LOGGER = LoggerFactory.getLogger(S3PinotFS.class);
+
   private static final String DELIMITER = "/";
   public static final String S3_SCHEME = "s3://";
-  private static final boolean DEFAULT_DISABLE_ACL = true;
   private S3Client _s3Client;
-  private boolean _disableAcl = DEFAULT_DISABLE_ACL;
+  private boolean _disableAcl;
   private ServerSideEncryption _serverSideEncryption = null;
   private String _ssekmsKeyId;
   private String _ssekmsEncryptionContext;
 
   @Override
   public void init(PinotConfiguration config) {
-    Preconditions.checkArgument(!isNullOrEmpty(config.getProperty(REGION)), "Region can't be null or empty");
-    String region = config.getProperty(REGION);
-    _disableAcl = config.getProperty(DISABLE_ACL_CONFIG_KEY, DEFAULT_DISABLE_ACL);
-    String serverSideEncryption = config.getProperty(SERVER_SIDE_ENCRYPTION_CONFIG_KEY);
-    setServerSideEncryption(serverSideEncryption, config);
+    S3Config s3Config = new S3Config(config);
+    Preconditions.checkArgument(StringUtils.isNotEmpty(s3Config.getRegion()), "Region can't be null or empty");
+
+    _disableAcl = s3Config.getDisableAcl();
+    setServerSideEncryption(s3Config.getServerSideEncryption(), s3Config);
 
     AwsCredentialsProvider awsCredentialsProvider;
     try {
-      if (!isNullOrEmpty(config.getProperty(ACCESS_KEY)) && !isNullOrEmpty(config.getProperty(SECRET_KEY))) {
-        String accessKey = config.getProperty(ACCESS_KEY);
-        String secretKey = config.getProperty(SECRET_KEY);
+      if (StringUtils.isNotEmpty(s3Config.getAccessKey()) && StringUtils.isNotEmpty(s3Config.getSecretKey())) {
+        String accessKey = s3Config.getAccessKey();

Review Comment:
   nit: remove local variables `accessKey` and `secretKey` ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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