You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dm...@apache.org on 2017/01/18 19:39:55 UTC

ignite git commit: IGNITE-4530: Add support to provide AWSCredentialsProvider to TcpDiscoveryS3IpFinder in addition to just AWSCredentials Reviewed and merged by Denis Magda (dmagda@apache.org)

Repository: ignite
Updated Branches:
  refs/heads/master 177d7f499 -> 80bcf2739


IGNITE-4530: Add support to provide AWSCredentialsProvider to TcpDiscoveryS3IpFinder in addition to just AWSCredentials
Reviewed and merged by Denis Magda (dmagda@apache.org)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/80bcf273
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/80bcf273
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/80bcf273

Branch: refs/heads/master
Commit: 80bcf2739c3f84769805005d7c0acb5769606e7d
Parents: 177d7f4
Author: Aliaksandr Kazlou <al...@gmail.com>
Authored: Wed Jan 18 11:39:36 2017 -0800
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jan 18 11:39:36 2017 -0800

----------------------------------------------------------------------
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java | 37 ++++++++-
 .../TcpDiscoveryS3IpFinderAbstractSelfTest.java | 84 ++++++++++++++++++++
 ...3IpFinderAwsCredentialsProviderSelfTest.java | 46 +++++++++++
 ...scoveryS3IpFinderAwsCredentialsSelfTest.java | 45 +++++++++++
 .../s3/TcpDiscoveryS3IpFinderSelfTest.java      | 79 ------------------
 .../ignite/testsuites/IgniteS3TestSuite.java    | 26 +++---
 6 files changed, 225 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/80bcf273/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java b/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
index 86e4eb2..53d6532 100644
--- a/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
+++ b/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinder.java
@@ -20,6 +20,7 @@ package org.apache.ignite.spi.discovery.tcp.ipfinder.s3;
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.AmazonS3Client;
 import com.amazonaws.services.s3.model.ObjectListing;
@@ -51,7 +52,8 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinderAdapter;
  * <h1 class="header">Configuration</h1>
  * <h2 class="header">Mandatory</h2>
  * <ul>
- *      <li>AWS credentials (see {@link #setAwsCredentials(AWSCredentials)})</li>
+ *      <li>AWS credentials (see {@link #setAwsCredentials(AWSCredentials)} and
+ *      {@link #setAwsCredentials(AWSCredentialsProvider)})</li>
  *      <li>Bucket name (see {@link #setBucketName(String)})</li>
  * </ul>
  * <h2 class="header">Optional</h2>
@@ -111,6 +113,10 @@ public class TcpDiscoveryS3IpFinder extends TcpDiscoveryIpFinderAdapter {
     @GridToStringExclude
     private AWSCredentials cred;
 
+    /** AWS Credentials. */
+    @GridToStringExclude
+    private AWSCredentialsProvider credProvider;
+
     /**
      * Constructor.
      */
@@ -236,7 +242,7 @@ public class TcpDiscoveryS3IpFinder extends TcpDiscoveryIpFinderAdapter {
     private void initClient() throws IgniteSpiException {
         if (initGuard.compareAndSet(false, true))
             try {
-                if (cred == null)
+                if (cred == null && credProvider == null)
                     throw new IgniteSpiException("AWS credentials are not set.");
 
                 if (cfg == null)
@@ -245,7 +251,7 @@ public class TcpDiscoveryS3IpFinder extends TcpDiscoveryIpFinderAdapter {
                 if (F.isEmpty(bucketName))
                     throw new IgniteSpiException("Bucket name is null or empty (provide bucket name and restart).");
 
-                s3 = cfg != null ? new AmazonS3Client(cred, cfg) : new AmazonS3Client(cred);
+                s3 = createAmazonS3Client();
 
                 if (!s3.doesBucketExist(bucketName)) {
                     try {
@@ -288,6 +294,17 @@ public class TcpDiscoveryS3IpFinder extends TcpDiscoveryIpFinderAdapter {
     }
 
     /**
+     * Instantiates {@code AmazonS3Client} instance.
+     *
+     * @return Client instance to use to connect to AWS.
+     */
+    private AmazonS3Client createAmazonS3Client() {
+        return cfg != null
+            ? (cred != null ? new AmazonS3Client(cred, cfg) : new AmazonS3Client(credProvider, cfg))
+            : (cred != null ? new AmazonS3Client(cred) : new AmazonS3Client(credProvider));
+    }
+
+    /**
      * Sets bucket name for IP finder.
      *
      * @param bucketName Bucket name.
@@ -310,7 +327,7 @@ public class TcpDiscoveryS3IpFinder extends TcpDiscoveryIpFinderAdapter {
     }
 
     /**
-     * Sets AWS credentials.
+     * Sets AWS credentials. Either use {@link #setAwsCredentials(AWSCredentialsProvider)} or this one.
      * <p>
      * For details refer to Amazon S3 API reference.
      *
@@ -321,6 +338,18 @@ public class TcpDiscoveryS3IpFinder extends TcpDiscoveryIpFinderAdapter {
         this.cred = cred;
     }
 
+    /**
+     * Sets AWS credentials provider. Either use {@link #setAwsCredentials(AWSCredentials)} or this one.
+     * <p>
+     * For details refer to Amazon S3 API reference.
+     *
+     * @param credProvider AWS credentials provider.
+     */
+    @IgniteSpiConfiguration(optional = false)
+    public void setAwsCredentials(AWSCredentialsProvider credProvider) {
+        this.credProvider = credProvider;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(TcpDiscoveryS3IpFinder.class, this, "super", super.toString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/80bcf273/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAbstractSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAbstractSelfTest.java
new file mode 100644
index 0000000..d17416f
--- /dev/null
+++ b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAbstractSelfTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.ignite.spi.discovery.tcp.ipfinder.s3;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinderAbstractSelfTest;
+import org.apache.ignite.testsuites.IgniteIgnore;
+import org.apache.ignite.testsuites.IgniteS3TestSuite;
+
+/**
+ * Abstract TcpDiscoveryS3IpFinder to test with different ways of setting AWS credentials.
+ */
+abstract class TcpDiscoveryS3IpFinderAbstractSelfTest
+    extends TcpDiscoveryIpFinderAbstractSelfTest<TcpDiscoveryS3IpFinder> {
+    /**
+     * Constructor.
+     *
+     * @throws Exception If any error occurs.
+     */
+    protected TcpDiscoveryS3IpFinderAbstractSelfTest() throws Exception {
+    }
+
+    /** {@inheritDoc} */
+    @Override protected TcpDiscoveryS3IpFinder ipFinder() throws Exception {
+        TcpDiscoveryS3IpFinder finder = new TcpDiscoveryS3IpFinder();
+
+        injectLogger(finder);
+
+        assert finder.isShared() : "Ip finder should be shared by default.";
+
+        setAwsCredentials(finder);
+
+        // Bucket name should be unique for the host to parallel test run on one bucket.
+        String bucketName = IgniteS3TestSuite.getBucketName(
+            "ip-finder-unit-test-bucket-" + InetAddress.getLocalHost().getAddress()[3]);
+        finder.setBucketName(bucketName);
+
+        for (int i = 0; i < 5; i++) {
+            Collection<InetSocketAddress> addrs = finder.getRegisteredAddresses();
+
+            if (!addrs.isEmpty())
+                finder.unregisterAddresses(addrs);
+            else
+                return finder;
+
+            U.sleep(1000);
+        }
+
+        if (!finder.getRegisteredAddresses().isEmpty())
+            throw new Exception("Failed to initialize IP finder.");
+
+        return finder;
+    }
+
+    /** {@inheritDoc} */
+    @IgniteIgnore("https://issues.apache.org/jira/browse/IGNITE-2420")
+    @Override public void testIpFinder() throws Exception {
+        super.testIpFinder();
+    }
+
+    /**
+     * Set AWS credentials into the provided {@code finder}.
+     * @param finder finder credentials to set into
+     */
+    protected abstract void setAwsCredentials(TcpDiscoveryS3IpFinder finder);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/80bcf273/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAwsCredentialsProviderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAwsCredentialsProviderSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAwsCredentialsProviderSelfTest.java
new file mode 100644
index 0000000..6952b54
--- /dev/null
+++ b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAwsCredentialsProviderSelfTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.ignite.spi.discovery.tcp.ipfinder.s3;
+
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.BasicAWSCredentials;
+import org.apache.ignite.testsuites.IgniteS3TestSuite;
+
+/**
+ * TcpDiscoveryS3IpFinder test using AWS credentials provider.
+ */
+public class TcpDiscoveryS3IpFinderAwsCredentialsProviderSelfTest extends TcpDiscoveryS3IpFinderAbstractSelfTest {
+    /**
+     * Constructor.
+     *
+     * @throws Exception If any error occurs.
+     */
+    public TcpDiscoveryS3IpFinderAwsCredentialsProviderSelfTest() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void setAwsCredentials(TcpDiscoveryS3IpFinder finder) {
+        finder.setAwsCredentials(new AWSStaticCredentialsProvider(
+            new BasicAWSCredentials(IgniteS3TestSuite.getAccessKey(), IgniteS3TestSuite.getSecretKey())));
+    }
+
+    @Override public void testIpFinder() throws Exception {
+        super.testIpFinder();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/80bcf273/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAwsCredentialsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAwsCredentialsSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAwsCredentialsSelfTest.java
new file mode 100644
index 0000000..7447378
--- /dev/null
+++ b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderAwsCredentialsSelfTest.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.ignite.spi.discovery.tcp.ipfinder.s3;
+
+import com.amazonaws.auth.BasicAWSCredentials;
+import org.apache.ignite.testsuites.IgniteS3TestSuite;
+
+/**
+ * TcpDiscoveryS3IpFinder test using AWS credentials.
+ */
+public class TcpDiscoveryS3IpFinderAwsCredentialsSelfTest extends TcpDiscoveryS3IpFinderAbstractSelfTest {
+    /**
+     * Constructor.
+     *
+     * @throws Exception If any error occurs.
+     */
+    public TcpDiscoveryS3IpFinderAwsCredentialsSelfTest() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void setAwsCredentials(TcpDiscoveryS3IpFinder finder) {
+        finder.setAwsCredentials(new BasicAWSCredentials(IgniteS3TestSuite.getAccessKey(),
+            IgniteS3TestSuite.getSecretKey()));
+    }
+
+    @Override public void testIpFinder() throws Exception {
+        super.testIpFinder();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/80bcf273/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java
deleted file mode 100644
index a1ebc8f..0000000
--- a/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java
+++ /dev/null
@@ -1,79 +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.ignite.spi.discovery.tcp.ipfinder.s3;
-
-import com.amazonaws.auth.BasicAWSCredentials;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.Collection;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinderAbstractSelfTest;
-import org.apache.ignite.testsuites.IgniteIgnore;
-import org.apache.ignite.testsuites.IgniteS3TestSuite;
-
-/**
- * TcpDiscoveryS3IpFinder test.
- */
-public class TcpDiscoveryS3IpFinderSelfTest
-    extends TcpDiscoveryIpFinderAbstractSelfTest<TcpDiscoveryS3IpFinder> {
-    /**
-     * Constructor.
-     *
-     * @throws Exception If any error occurs.
-     */
-    public TcpDiscoveryS3IpFinderSelfTest() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override protected TcpDiscoveryS3IpFinder ipFinder() throws Exception {
-        TcpDiscoveryS3IpFinder finder = new TcpDiscoveryS3IpFinder();
-
-        injectLogger(finder);
-
-        assert finder.isShared() : "Ip finder should be shared by default.";
-
-        finder.setAwsCredentials(new BasicAWSCredentials(IgniteS3TestSuite.getAccessKey(),
-            IgniteS3TestSuite.getSecretKey()));
-
-        // Bucket name should be unique for the host to parallel test run on one bucket.
-        finder.setBucketName("ip-finder-unit-test-bucket-" + InetAddress.getLocalHost().getAddress()[3]);
-
-        for (int i = 0; i < 5; i++) {
-            Collection<InetSocketAddress> addrs = finder.getRegisteredAddresses();
-
-            if (!addrs.isEmpty())
-                finder.unregisterAddresses(addrs);
-            else
-                return finder;
-
-            U.sleep(1000);
-        }
-
-        if (!finder.getRegisteredAddresses().isEmpty())
-            throw new Exception("Failed to initialize IP finder.");
-
-        return finder;
-    }
-
-    /** {@inheritDoc} */
-    @IgniteIgnore("https://issues.apache.org/jira/browse/IGNITE-2420")
-    @Override public void testIpFinder() throws Exception {
-        super.testIpFinder();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/80bcf273/modules/aws/src/test/java/org/apache/ignite/testsuites/IgniteS3TestSuite.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/testsuites/IgniteS3TestSuite.java b/modules/aws/src/test/java/org/apache/ignite/testsuites/IgniteS3TestSuite.java
index 86322e6..a703c66 100644
--- a/modules/aws/src/test/java/org/apache/ignite/testsuites/IgniteS3TestSuite.java
+++ b/modules/aws/src/test/java/org/apache/ignite/testsuites/IgniteS3TestSuite.java
@@ -23,7 +23,8 @@ import org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpiConfigSelfTest;
 import org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpiSelfTest;
 import org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpiStartStopSelfTest;
 import org.apache.ignite.spi.checkpoint.s3.S3SessionCheckpointSelfTest;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinderSelfTest;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinderAwsCredentialsProviderSelfTest;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinderAwsCredentialsSelfTest;
 import org.apache.ignite.testframework.IgniteTestSuite;
 
 /**
@@ -45,7 +46,8 @@ public class IgniteS3TestSuite extends TestSuite {
         suite.addTestSuite(S3SessionCheckpointSelfTest.class);
 
         // S3 IP finder.
-        suite.addTestSuite(TcpDiscoveryS3IpFinderSelfTest.class);
+        suite.addTestSuite(TcpDiscoveryS3IpFinderAwsCredentialsSelfTest.class);
+        suite.addTestSuite(TcpDiscoveryS3IpFinderAwsCredentialsProviderSelfTest.class);
 
         return suite;
     }
@@ -54,20 +56,26 @@ public class IgniteS3TestSuite extends TestSuite {
      * @return Access key.
      */
     public static String getAccessKey() {
-        String key = System.getenv("test.amazon.access.key");
-
-        assert key != null : "Environment variable 'test.amazon.access.key' is not set";
-
-        return key;
+        return getRequiredEnvVar("test.amazon.access.key");
     }
 
     /**
      * @return Access key.
      */
     public static String getSecretKey() {
-        String key = System.getenv("test.amazon.secret.key");
+        return getRequiredEnvVar("test.amazon.secret.key");
+    }
+
+    public static String getBucketName(final String defaultBucketName) {
+        String value = System.getenv("test.s3.bucket.name");
+
+        return value == null ? defaultBucketName : value;
+    }
+
+    private static String getRequiredEnvVar(String name) {
+        String key = System.getenv(name);
 
-        assert key != null : "Environment variable 'test.amazon.secret.key' is not set";
+        assert key != null : String.format("Environment variable '%s' is not set", name);
 
         return key;
     }