浏览代码

HADOOP-13609. Refine credential provider related codes for AliyunOss integration. Contributed by Genmao Yu

Kai Zheng 8 年之前
父节点
当前提交
9cd4760257

+ 87 - 0
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunCredentialsProvider.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.fs.aliyun.oss;
+
+import com.aliyun.oss.common.auth.Credentials;
+import com.aliyun.oss.common.auth.CredentialsProvider;
+import com.aliyun.oss.common.auth.DefaultCredentials;
+import com.aliyun.oss.common.auth.InvalidCredentialsException;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
+/**
+ * Support session credentials for authenticating with Aliyun.
+ */
+public class AliyunCredentialsProvider implements CredentialsProvider {
+  private Credentials credentials = null;
+
+  public AliyunCredentialsProvider(Configuration conf)
+      throws IOException {
+    String accessKeyId;
+    String accessKeySecret;
+    String securityToken;
+    try {
+      accessKeyId = AliyunOSSUtils.getValueWithKey(conf, ACCESS_KEY_ID);
+      accessKeySecret = AliyunOSSUtils.getValueWithKey(conf, ACCESS_KEY_SECRET);
+    } catch (IOException e) {
+      throw new InvalidCredentialsException(e);
+    }
+
+    try {
+      securityToken = AliyunOSSUtils.getValueWithKey(conf, SECURITY_TOKEN);
+    } catch (IOException e) {
+      securityToken = null;
+    }
+
+    if (StringUtils.isEmpty(accessKeyId)
+        || StringUtils.isEmpty(accessKeySecret)) {
+      throw new InvalidCredentialsException(
+          "AccessKeyId and AccessKeySecret should not be null or empty.");
+    }
+
+    if (StringUtils.isNotEmpty(securityToken)) {
+      credentials = new DefaultCredentials(accessKeyId, accessKeySecret,
+          securityToken);
+    } else {
+      credentials = new DefaultCredentials(accessKeyId, accessKeySecret);
+    }
+  }
+
+  @Override
+  public void setCredentials(Credentials creds) {
+    if (creds == null) {
+      throw new InvalidCredentialsException("Credentials should not be null.");
+    }
+
+    credentials = creds;
+  }
+
+  @Override
+  public Credentials getCredentials() {
+    if (credentials == null) {
+      throw new InvalidCredentialsException("Invalid credentials");
+    }
+
+    return credentials;
+  }
+}

+ 2 - 2
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.fs.aliyun.oss;
 
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
@@ -43,6 +41,8 @@ import com.aliyun.oss.model.ObjectMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
 /**
  * Implementation of {@link FileSystem} for <a href="https://oss.aliyun.com">
  * Aliyun OSS</a>, used to access OSS blob system in a filesystem style.

+ 19 - 1
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java

@@ -23,7 +23,25 @@ import com.aliyun.oss.OSSClient;
 import com.aliyun.oss.OSSException;
 import com.aliyun.oss.common.auth.CredentialsProvider;
 import com.aliyun.oss.common.comm.Protocol;
-import com.aliyun.oss.model.*;
+import com.aliyun.oss.model.AbortMultipartUploadRequest;
+import com.aliyun.oss.model.CannedAccessControlList;
+import com.aliyun.oss.model.CompleteMultipartUploadRequest;
+import com.aliyun.oss.model.CompleteMultipartUploadResult;
+import com.aliyun.oss.model.CopyObjectResult;
+import com.aliyun.oss.model.DeleteObjectsRequest;
+import com.aliyun.oss.model.GetObjectRequest;
+import com.aliyun.oss.model.InitiateMultipartUploadRequest;
+import com.aliyun.oss.model.InitiateMultipartUploadResult;
+import com.aliyun.oss.model.ListObjectsRequest;
+import com.aliyun.oss.model.ObjectMetadata;
+import com.aliyun.oss.model.ObjectListing;
+import com.aliyun.oss.model.OSSObjectSummary;
+import com.aliyun.oss.model.PartETag;
+import com.aliyun.oss.model.PutObjectResult;
+import com.aliyun.oss.model.UploadPartCopyRequest;
+import com.aliyun.oss.model.UploadPartCopyResult;
+import com.aliyun.oss.model.UploadPartRequest;
+import com.aliyun.oss.model.UploadPartResult;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.slf4j.Logger;

+ 2 - 2
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.fs.aliyun.oss;
 
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
@@ -31,6 +29,8 @@ import org.apache.hadoop.fs.FSExceptionMessages;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
 /**
  * The input stream for OSS blob system.
  * The class uses multi-part downloading to read data from the object content

+ 2 - 2
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSOutputStream.java

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.fs.aliyun.oss;
 
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
 import java.io.BufferedOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
@@ -33,6 +31,8 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.util.Progressable;
 
+import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
+
 /**
  * The output stream for OSS blob system.
  * Data will be buffered on local disk, then uploaded to OSS in

+ 3 - 11
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java

@@ -23,8 +23,6 @@ import java.io.InputStream;
 import java.net.URI;
 
 import com.aliyun.oss.common.auth.CredentialsProvider;
-import com.aliyun.oss.common.auth.DefaultCredentialProvider;
-import com.aliyun.oss.common.auth.DefaultCredentials;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.ProviderUtils;
@@ -32,7 +30,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-import static org.apache.hadoop.fs.aliyun.oss.Constants.ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY;
 
 /**
  * Utility methods for Aliyun OSS code.
@@ -52,7 +49,7 @@ final public class AliyunOSSUtils {
    * @return the value for the key
    * @throws IOException if failed to get password from configuration
    */
-  static public String getPassword(Configuration conf, String key)
+  public static String getValueWithKey(Configuration conf, String key)
       throws IOException {
     try {
       final char[] pass = conf.getPassword(key);
@@ -126,12 +123,7 @@ final public class AliyunOSSUtils {
       Configuration newConf =
           ProviderUtils.excludeIncompatibleCredentialProviders(conf,
               AliyunOSSFileSystem.class);
-      String accessKey =
-          AliyunOSSUtils.getPassword(newConf, ACCESS_KEY);
-      String secretKey =
-          AliyunOSSUtils.getPassword(newConf, SECRET_KEY);
-      credentials = new DefaultCredentialProvider(
-          new DefaultCredentials(accessKey, secretKey));
+      credentials = new AliyunCredentialsProvider(newConf);
     } else {
       try {
         LOG.debug("Credential provider class is:" + className);
@@ -139,7 +131,7 @@ final public class AliyunOSSUtils {
         try {
           credentials =
               (CredentialsProvider)credClass.getDeclaredConstructor(
-                  URI.class, Configuration.class).newInstance(uri, conf);
+                  Configuration.class).newInstance(conf);
         } catch (NoSuchMethodException | SecurityException e) {
           credentials =
               (CredentialsProvider)credClass.getDeclaredConstructor()

+ 2 - 2
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java

@@ -31,8 +31,8 @@ public final class Constants {
       "fs.oss.credentials.provider";
 
   // OSS access verification
-  public static final String ACCESS_KEY = "fs.oss.accessKeyId";
-  public static final String SECRET_KEY = "fs.oss.accessKeySecret";
+  public static final String ACCESS_KEY_ID = "fs.oss.accessKeyId";
+  public static final String ACCESS_KEY_SECRET = "fs.oss.accessKeySecret";
   public static final String SECURITY_TOKEN = "fs.oss.securityToken";
 
   // Number of simultaneous connections to oss

+ 0 - 64
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/TemporaryAliyunCredentialsProvider.java

@@ -1,64 +0,0 @@
-/**
- * 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.fs.aliyun.oss;
-
-import com.aliyun.oss.common.auth.Credentials;
-import com.aliyun.oss.common.auth.CredentialsProvider;
-import com.aliyun.oss.common.auth.DefaultCredentials;
-import com.aliyun.oss.common.auth.InvalidCredentialsException;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-
-import java.net.URI;
-
-import static org.apache.hadoop.fs.aliyun.oss.Constants.*;
-
-/**
- * Support session credentials for authenticating with ALiyun.
- */
-public class TemporaryAliyunCredentialsProvider implements CredentialsProvider {
-  public static final String NAME
-      = "org.apache.hadoop.fs.aliyun.oss.TemporaryAliyunCredentialsProvider";
-  private final String accessKeyId;
-  private final String accessKeySecret;
-  private final String securityToken;
-
-  public TemporaryAliyunCredentialsProvider(URI uri, Configuration conf) {
-    this.accessKeyId = conf.get(ACCESS_KEY, null);
-    this.accessKeySecret = conf.get(SECRET_KEY, null);
-    this.securityToken = conf.get(SECURITY_TOKEN, null);
-  }
-
-  @Override
-  public void setCredentials(Credentials creds) {
-
-  }
-
-  @Override
-  public Credentials getCredentials() {
-    if (!StringUtils.isEmpty(accessKeyId)
-        && !StringUtils.isEmpty(accessKeySecret)
-        && !StringUtils.isEmpty(securityToken)) {
-      return new DefaultCredentials(accessKeyId, accessKeySecret,
-          securityToken);
-    }
-    throw new InvalidCredentialsException(
-        "AccessKeyId, AccessKeySecret or SecurityToken is unset");
-  }
-}

+ 25 - 12
hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSTemporaryCredentials.java → hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunCredentials.java

@@ -26,10 +26,10 @@ import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
 import org.junit.Test;
 
-import java.net.URI;
+import java.io.IOException;
 
-import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY;
-import static org.apache.hadoop.fs.aliyun.oss.Constants.SECRET_KEY;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY_ID;
+import static org.apache.hadoop.fs.aliyun.oss.Constants.ACCESS_KEY_SECRET;
 import static org.apache.hadoop.fs.aliyun.oss.Constants.SECURITY_TOKEN;
 
 /**
@@ -38,8 +38,7 @@ import static org.apache.hadoop.fs.aliyun.oss.Constants.SECURITY_TOKEN;
  * should only be used against transient filesystems where you don't care about
  * the data.
  */
-public class TestAliyunOSSTemporaryCredentials
-    extends AbstractFSContractTestBase {
+public class TestAliyunCredentials extends AbstractFSContractTestBase {
 
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
@@ -47,19 +46,33 @@ public class TestAliyunOSSTemporaryCredentials
   }
 
   @Test
-  public void testTemporaryCredentialValidation() throws Throwable {
+  public void testCredentialMissingAccessKeyId() throws Throwable {
     Configuration conf = new Configuration();
-    conf.set(ACCESS_KEY, "accessKeyId");
-    conf.set(SECRET_KEY, "accessKeySecret");
-    conf.set(SECURITY_TOKEN, "");
-    URI uri = getFileSystem().getUri();
-    TemporaryAliyunCredentialsProvider provider
-        = new TemporaryAliyunCredentialsProvider(uri, conf);
+    conf.set(ACCESS_KEY_ID, "");
+    conf.set(ACCESS_KEY_SECRET, "accessKeySecret");
+    conf.set(SECURITY_TOKEN, "token");
+    validateCredential(conf);
+  }
+
+  @Test
+  public void testCredentialMissingAccessKeySecret() throws Throwable {
+    Configuration conf = new Configuration();
+    conf.set(ACCESS_KEY_ID, "accessKeyId");
+    conf.set(ACCESS_KEY_SECRET, "");
+    conf.set(SECURITY_TOKEN, "token");
+    validateCredential(conf);
+  }
+
+  private void validateCredential(Configuration conf) {
     try {
+      AliyunCredentialsProvider provider
+          = new AliyunCredentialsProvider(conf);
       Credentials credentials = provider.getCredentials();
       fail("Expected a CredentialInitializationException, got " + credentials);
     } catch (InvalidCredentialsException expected) {
       // expected
+    } catch (IOException e) {
+      fail("Unexpected exception.");
     }
   }
 }

+ 2 - 2
hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSFileSystemStore.java

@@ -67,8 +67,8 @@ public class TestAliyunOSSFileSystemStore {
   @BeforeClass
   public static void checkSettings() throws Exception {
     Configuration conf = new Configuration();
-    assumeNotNull(conf.get(Constants.ACCESS_KEY));
-    assumeNotNull(conf.get(Constants.SECRET_KEY));
+    assumeNotNull(conf.get(Constants.ACCESS_KEY_ID));
+    assumeNotNull(conf.get(Constants.ACCESS_KEY_SECRET));
     assumeNotNull(conf.get("test.fs.oss.name"));
   }