diff --git a/hadoop-cloud-storage-project/hadoop-cos/pom.xml b/hadoop-cloud-storage-project/hadoop-cos/pom.xml
index 2c2b9a55315f9..7eb111ba6fde8 100644
--- a/hadoop-cloud-storage-project/hadoop-cos/pom.xml
+++ b/hadoop-cloud-storage-project/hadoop-cos/pom.xml
@@ -34,6 +34,7 @@
UTF-8
true
+ 3.1.1322
@@ -113,6 +114,19 @@
compile
+
+ org.assertj
+ assertj-core
+ test
+
+
+
+ com.tencentcloudapi
+ tencentcloud-sdk-java
+ ${tencentcloud.verion}
+ test
+
+
org.apache.hadoop
hadoop-common
diff --git a/hadoop-cloud-storage-project/hadoop-cos/src/main/java/org/apache/hadoop/fs/cosn/CosNativeFileSystemStore.java b/hadoop-cloud-storage-project/hadoop-cos/src/main/java/org/apache/hadoop/fs/cosn/CosNativeFileSystemStore.java
index d2484c0e47b3c..c145ce2ff9c79 100644
--- a/hadoop-cloud-storage-project/hadoop-cos/src/main/java/org/apache/hadoop/fs/cosn/CosNativeFileSystemStore.java
+++ b/hadoop-cloud-storage-project/hadoop-cos/src/main/java/org/apache/hadoop/fs/cosn/CosNativeFileSystemStore.java
@@ -32,8 +32,6 @@
import com.qcloud.cos.COSClient;
import com.qcloud.cos.ClientConfig;
-import com.qcloud.cos.auth.BasicCOSCredentials;
-import com.qcloud.cos.auth.COSCredentials;
import com.qcloud.cos.endpoint.SuffixEndpointBuilder;
import com.qcloud.cos.exception.CosClientException;
import com.qcloud.cos.exception.CosServiceException;
@@ -103,11 +101,6 @@ private void initCOSClient(URI uri, Configuration conf) throws IOException {
throw new IOException(exceptionMsg);
}
- COSCredentials cosCred;
- cosCred = new BasicCOSCredentials(
- credentialProviderList.getCredentials().getCOSAccessKeyId(),
- credentialProviderList.getCredentials().getCOSSecretKey());
-
boolean useHttps = conf.getBoolean(CosNConfigKeys.COSN_USE_HTTPS_KEY,
CosNConfigKeys.DEFAULT_USE_HTTPS);
@@ -133,7 +126,7 @@ private void initCOSClient(URI uri, Configuration conf) throws IOException {
conf.getInt(CosNConfigKeys.MAX_CONNECTION_NUM,
CosNConfigKeys.DEFAULT_MAX_CONNECTION_NUM));
- this.cosClient = new COSClient(cosCred, config);
+ this.cosClient = new COSClient(credentialProviderList.getCredentials(), config);
}
/**
diff --git a/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/TestCosCredentials.java b/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/TestCosCredentials.java
index 8b74f3639ddbd..88ede2fa5c5e1 100644
--- a/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/TestCosCredentials.java
+++ b/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/TestCosCredentials.java
@@ -17,9 +17,13 @@
*/
package org.apache.hadoop.fs.cosn;
+import com.qcloud.cos.auth.BasicSessionCredentials;
import com.qcloud.cos.auth.COSCredentials;
import com.qcloud.cos.auth.COSCredentialsProvider;
+
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
+
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -28,12 +32,14 @@
import java.net.URI;
import java.net.URISyntaxException;
+import static org.apache.hadoop.fs.cosn.auth.DynamicTemporaryCosnCredentialsProvider.STS_SECRET_ID_KEY;
+import static org.apache.hadoop.fs.cosn.auth.DynamicTemporaryCosnCredentialsProvider.STS_SECRET_KEY_KEY;
import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class TestCosCredentials {
- private static final Logger LOG =
- LoggerFactory.getLogger(TestCosCredentials.class);
+ private static final Logger LOG = LoggerFactory.getLogger(TestCosCredentials.class);
private final URI fsUri;
@@ -50,10 +56,8 @@ public TestCosCredentials() throws URISyntaxException {
@Test
public void testSimpleCredentialsProvider() throws Throwable {
Configuration configuration = new Configuration();
- configuration.set(CosNConfigKeys.COSN_SECRET_ID_KEY,
- testCosNSecretId);
- configuration.set(CosNConfigKeys.COSN_SECRET_KEY_KEY,
- testCosNSecretKey);
+ configuration.set(CosNConfigKeys.COSN_SECRET_ID_KEY, testCosNSecretId);
+ configuration.set(CosNConfigKeys.COSN_SECRET_KEY_KEY, testCosNSecretKey);
validateCredentials(this.fsUri, configuration);
}
@@ -63,15 +67,14 @@ public void testEnvironmentCredentialsProvider() throws Throwable {
// Set EnvironmentVariableCredentialsProvider as the CosCredentials
// Provider.
configuration.set(CosNConfigKeys.COSN_CREDENTIALS_PROVIDER,
- "org.apache.hadoop.fs.cosn.EnvironmentVariableCredentialsProvider");
+ "org.apache.hadoop.fs.cosn.auth.EnvironmentVariableCredentialsProvider");
// Set the environment variables storing the secret id and secret key.
System.setProperty(Constants.COSN_SECRET_ID_ENV, testCosNEnvSecretId);
System.setProperty(Constants.COSN_SECRET_KEY_ENV, testCosNEnvSecretKey);
validateCredentials(this.fsUri, configuration);
}
- private void validateCredentials(URI uri, Configuration configuration)
- throws IOException {
+ private void validateCredentials(URI uri, Configuration configuration) throws IOException {
if (null != configuration) {
COSCredentialsProvider credentialsProvider =
CosNUtils.createCosCredentialsProviderSet(uri, configuration);
@@ -79,7 +82,7 @@ private void validateCredentials(URI uri, Configuration configuration)
assertNotNull("The cos credentials obtained is null.", cosCredentials);
if (configuration.get(
CosNConfigKeys.COSN_CREDENTIALS_PROVIDER).compareToIgnoreCase(
- "org.apache.hadoop.fs.cosn.EnvironmentVariableCredentialsProvider")
+ "org.apache.hadoop.fs.cosn.auth.EnvironmentVariableCredentialsProvider")
== 0) {
if (null == cosCredentials.getCOSAccessKeyId()
|| cosCredentials.getCOSAccessKeyId().isEmpty()
@@ -131,4 +134,36 @@ private void validateCredentials(URI uri, Configuration configuration)
}
}
}
+
+ @Test
+ public void testTmpTokenCredentialsProvider() throws Throwable {
+ Configuration configuration = new Configuration();
+ // Set DynamicTemporaryCosnCredentialsProvider as the CosCredentials
+ // Provider.
+ configuration.set(CosNConfigKeys.COSN_CREDENTIALS_PROVIDER,
+ "org.apache.hadoop.fs.cosn.auth.DynamicTemporaryCosnCredentialsProvider");
+
+ configuration.set(STS_SECRET_ID_KEY, System.getProperty(STS_SECRET_ID_KEY));
+ configuration.set(STS_SECRET_KEY_KEY, System.getProperty(STS_SECRET_KEY_KEY));
+ validateTmpTokenCredentials(this.fsUri, configuration);
+ }
+
+ private void validateTmpTokenCredentials(URI uri, Configuration configuration)
+ throws IOException {
+ COSCredentialsProvider credentialsProvider =
+ CosNUtils.createCosCredentialsProviderSet(uri, configuration);
+ COSCredentials cosCredentials = credentialsProvider.getCredentials();
+ assertNotNull("The cos credentials obtained is null.", cosCredentials);
+ assertTrue("CredentialsProvider must be DynamicTemporaryCosnCredentialsProvider",
+ StringUtils.equalsIgnoreCase(configuration.get(CosNConfigKeys.COSN_CREDENTIALS_PROVIDER),
+ "org.apache.hadoop.fs.cosn.auth.DynamicTemporaryCosnCredentialsProvider"));
+
+ assertTrue("cosCredentials must be instanceof BasicSessionCredentials",
+ cosCredentials instanceof BasicSessionCredentials);
+
+ assertNotNull("session access key id is null", cosCredentials.getCOSAccessKeyId());
+ assertNotNull("session access key is null", cosCredentials.getCOSSecretKey());
+ assertNotNull("access token is null",
+ ((BasicSessionCredentials) cosCredentials).getSessionToken());
+ }
}
diff --git a/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/auth/DynamicTemporaryCosnCredentialsProvider.java b/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/auth/DynamicTemporaryCosnCredentialsProvider.java
new file mode 100644
index 0000000000000..8b46fa1592944
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/auth/DynamicTemporaryCosnCredentialsProvider.java
@@ -0,0 +1,162 @@
+/*
+ * 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.cosn.auth;
+
+import com.qcloud.cos.auth.BasicSessionCredentials;
+import com.qcloud.cos.auth.COSCredentials;
+import com.qcloud.cos.auth.COSCredentialsProvider;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.cosn.CosNConfigKeys;
+
+import com.tencentcloudapi.common.Credential;
+import com.tencentcloudapi.common.profile.ClientProfile;
+import com.tencentcloudapi.common.profile.HttpProfile;
+import com.tencentcloudapi.sts.v20180813.StsClient;
+import com.tencentcloudapi.sts.v20180813.models.GetFederationTokenRequest;
+import com.tencentcloudapi.sts.v20180813.models.GetFederationTokenResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * A COSCredentialsProvider that generates temporary credentials from Tencent Cloud STS.
+ * This provider requires a long-term secret ID and key with permission to call
+ * the STS GetFederationToken action.
+ */
+public class DynamicTemporaryCosnCredentialsProvider implements COSCredentialsProvider {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(DynamicTemporaryCosnCredentialsProvider.class);
+
+ public static final String STS_SECRET_ID_KEY = "fs.cosn.auth.sts.secret.id";
+ public static final String STS_SECRET_KEY_KEY = "fs.cosn.auth.sts.secret.key";
+ public static final String STS_ENDPOINT_KEY = "fs.cosn.auth.sts.endpoint";
+ public static final String DEFAULT_STS_ENDPOINT = "sts.tencentcloudapi.com";
+ public static final String TOKEN_DURATION_SECONDS_KEY = "fs.cosn.auth.sts.token.duration.seconds";
+ public static final int DEFAULT_TOKEN_DURATION_SECONDS = 900; // 15 minutes
+
+ private final String longTermSecretId;
+ private final String longTermSecretKey;
+ private final String stsEndpoint;
+ private final String region;
+ private final String bucketName;
+ private final long durationSeconds;
+
+ private final AtomicReference expiringCredentialsRef =
+ new AtomicReference<>();
+
+ public DynamicTemporaryCosnCredentialsProvider(Configuration conf) throws IOException {
+ this.longTermSecretId = conf.get(STS_SECRET_ID_KEY);
+ this.longTermSecretKey = conf.get(STS_SECRET_KEY_KEY);
+ this.stsEndpoint = conf.get(STS_ENDPOINT_KEY, DEFAULT_STS_ENDPOINT);
+ this.region = conf.get(CosNConfigKeys.COSN_REGION_KEY);
+ this.bucketName = conf.get("fs.defaultFS").replace("cosn://", "");
+ this.durationSeconds = conf.getLong(TOKEN_DURATION_SECONDS_KEY, DEFAULT_TOKEN_DURATION_SECONDS);
+
+ if (this.longTermSecretId == null || this.longTermSecretKey == null) {
+ throw new IOException(
+ "Long-term STS credentials not provided in configuration. Please set " + STS_SECRET_ID_KEY
+ + " and " + STS_SECRET_KEY_KEY);
+ }
+ if (this.region == null || this.bucketName == null) {
+ throw new IOException("Bucket region or name not configured.");
+ }
+ }
+
+ @Override
+ public COSCredentials getCredentials() {
+ ExpiringCredentials current = expiringCredentialsRef.get();
+ // Refresh if credentials are not present, or are within 60 seconds of expiry.
+ if (current == null
+ || System.currentTimeMillis() >= current.getExpirationTimeMillis() - 60000) {
+ LOG.info("STS credentials expired or not found, requesting new token.");
+ refresh();
+ }
+ return expiringCredentialsRef.get().getCredentials();
+ }
+
+ @Override
+ public void refresh() {
+ try {
+ Credential cred = new Credential(this.longTermSecretId, this.longTermSecretKey);
+ HttpProfile httpProfile = new HttpProfile();
+ httpProfile.setEndpoint(this.stsEndpoint);
+ ClientProfile clientProfile = new ClientProfile();
+ clientProfile.setHttpProfile(httpProfile);
+
+ StsClient client = new StsClient(cred, this.region, clientProfile);
+ GetFederationTokenRequest req = new GetFederationTokenRequest();
+
+ String policyTemplate = "{\"version\":\"2.0\",\"statement\":[{\"action\":[\"cos:*\"],"
+ + "\"effect\":\"allow\",\"resource\":[\"qcs::cos:%s:uid/%s:%s/*\"]}]}";
+ String policy =
+ String.format(policyTemplate, this.region, getAppIdFromBucket(this.bucketName),
+ this.bucketName);
+ req.setPolicy(policy);
+
+ req.setDurationSeconds(this.durationSeconds);
+ req.setName("HadoopCosNContractTest");
+
+ GetFederationTokenResponse resp = client.GetFederationToken(req);
+
+ long expirationTimeMillis = (resp.getExpiredTime() * 1000);
+ BasicSessionCredentials credentials =
+ new BasicSessionCredentials(resp.getCredentials().getTmpSecretId(),
+ resp.getCredentials().getTmpSecretKey(), resp.getCredentials().getToken());
+
+ expiringCredentialsRef.set(new ExpiringCredentials(credentials, expirationTimeMillis));
+ LOG.info("Successfully refreshed STS credentials. Expiration: {}",
+ new java.util.Date(expirationTimeMillis));
+
+ } catch (Exception e) {
+ LOG.error("Failed to get token from STS: {}", e.toString());
+ throw new RuntimeException("Failed to get token from STS", e);
+ }
+ }
+
+ private String getAppIdFromBucket(String bucket) {
+ int lastDash = bucket.lastIndexOf('-');
+ if (lastDash != -1 && lastDash < bucket.length() - 1) {
+ return bucket.substring(lastDash + 1);
+ }
+ throw new IllegalArgumentException("Could not determine AppID from bucket name: " + bucket);
+ }
+
+ /**
+ * Helper class to hold credentials and their expiration time.
+ */
+ private static class ExpiringCredentials {
+ private final BasicSessionCredentials credentials;
+ private final long expirationTimeMillis;
+
+ ExpiringCredentials(BasicSessionCredentials credentials, long expirationTimeMillis) {
+ this.credentials = credentials;
+ this.expirationTimeMillis = expirationTimeMillis;
+ }
+
+ BasicSessionCredentials getCredentials() {
+ return credentials;
+ }
+
+ long getExpirationTimeMillis() {
+ return expirationTimeMillis;
+ }
+ }
+}
\ No newline at end of file
diff --git a/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/contract/TestCosNContractDynamicToken.java b/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/contract/TestCosNContractDynamicToken.java
new file mode 100644
index 0000000000000..3d53359ceff23
--- /dev/null
+++ b/hadoop-cloud-storage-project/hadoop-cos/src/test/java/org/apache/hadoop/fs/cosn/contract/TestCosNContractDynamicToken.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cosn.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.cosn.CosNConfigKeys;
+import org.apache.hadoop.fs.cosn.auth.DynamicTemporaryCosnCredentialsProvider;
+
+import static org.apache.hadoop.fs.cosn.auth.DynamicTemporaryCosnCredentialsProvider.STS_SECRET_ID_KEY;
+import static org.apache.hadoop.fs.cosn.auth.DynamicTemporaryCosnCredentialsProvider.STS_SECRET_KEY_KEY;
+
+/**
+ * Contract tests for CosN using a dynamic temporary token provider (STS).
+ * This test requires long-term credentials with STS access to be configured.
+ */
+public class TestCosNContractDynamicToken extends AbstractContractCreateTest {
+ @Override
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new CosNContract(conf);
+ }
+
+ @Override
+ protected Configuration createConfiguration() {
+ Configuration newConf = super.createConfiguration();
+
+ newConf.set(CosNConfigKeys.COSN_CREDENTIALS_PROVIDER,
+ DynamicTemporaryCosnCredentialsProvider.class.getName());
+ newConf.setBoolean("fs.cosn.impl.disable.cache", true);
+
+ String stsSecretId = System.getProperty(STS_SECRET_ID_KEY);
+ String stsSecretKey = System.getProperty(STS_SECRET_KEY_KEY);
+
+ if (stsSecretId == null || stsSecretKey == null) {
+ // Fallback to configuration for convenience, but log a warning.
+ System.err.println("WARN: Reading long-term STS credentials from configuration file. "
+ + "It is recommended to use system properties for security.");
+ stsSecretId = newConf.get(STS_SECRET_ID_KEY);
+ stsSecretKey = newConf.get(STS_SECRET_KEY_KEY);
+ }
+
+ if (stsSecretId == null || stsSecretKey == null) {
+ throw new RuntimeException("STS credentials for tests are not provided. "
+ + "Please set them via system properties (-Dfs.cosn.auth.sts.secret.id=... and "
+ + "-Dfs.cosn.auth.sts.secret.key=...)");
+ }
+
+ newConf.set(STS_SECRET_ID_KEY, stsSecretId);
+ newConf.set(STS_SECRET_KEY_KEY, stsSecretKey);
+
+ return newConf;
+ }
+}
\ No newline at end of file
diff --git a/hadoop-cloud-storage-project/hadoop-cos/src/test/resources/core-site.xml b/hadoop-cloud-storage-project/hadoop-cos/src/test/resources/core-site.xml
index fbd23bb326eb4..99f2c2db6d77e 100644
--- a/hadoop-cloud-storage-project/hadoop-cos/src/test/resources/core-site.xml
+++ b/hadoop-cloud-storage-project/hadoop-cos/src/test/resources/core-site.xml
@@ -20,7 +20,7 @@
fs.cosn.credentials.provider
- org.apache.hadoop.fs.cosn.auth.SimpleCredentialProvider
+ org.apache.hadoop.fs.cosn.auth.SimpleCredentialsProvider
This option allows the user to specify how to get the credentials.
Comma-separated class names of credential provider classes which
diff --git a/hadoop-cloud-storage-project/hadoop-cos/src/test/resources/log4j.properties b/hadoop-cloud-storage-project/hadoop-cos/src/test/resources/log4j.properties
index 1a6baaec65ae7..ced0687caad45 100644
--- a/hadoop-cloud-storage-project/hadoop-cos/src/test/resources/log4j.properties
+++ b/hadoop-cloud-storage-project/hadoop-cos/src/test/resources/log4j.properties
@@ -12,7 +12,7 @@
# log4j configuration used during build and unit tests
log4j.rootLogger=info,stdout
-log4j.threshhold=ALL
+log4j.threshold=ALL
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n