asfgit closed pull request #6789: [FLINK-10316][kinesis] bug was preventing 
FlinkKinesisProducer to connect to Kinesalite
URL: https://github.com/apache/flink/pull/6789
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
 
b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
index a4d60ed496d..75c84cdca5a 100644
--- 
a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
+++ 
b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java
@@ -87,6 +87,12 @@ public static void validateConsumerConfiguration(Properties 
config) {
 
                validateAwsConfiguration(config);
 
+               if (!(config.containsKey(AWSConfigConstants.AWS_REGION) ^ 
config.containsKey(ConsumerConfigConstants.AWS_ENDPOINT))) {
+                       // per validation in AwsClientBuilder
+                       throw new IllegalArgumentException(String.format("For 
FlinkKinesisConsumer either AWS region ('%s') or AWS endpoint ('%s') must be 
set in the config.",
+                                       AWSConfigConstants.AWS_REGION, 
AWSConfigConstants.AWS_ENDPOINT));
+               }
+
                if 
(config.containsKey(ConsumerConfigConstants.STREAM_INITIAL_POSITION)) {
                        String initPosType = 
config.getProperty(ConsumerConfigConstants.STREAM_INITIAL_POSITION);
 
@@ -213,6 +219,11 @@ public static KinesisProducerConfiguration 
getValidatedProducerConfiguration(Pro
 
                validateAwsConfiguration(config);
 
+               if (!config.containsKey(AWSConfigConstants.AWS_REGION)) {
+                       // per requirement in Amazon Kinesis Producer Library
+                       throw new IllegalArgumentException(String.format("For 
FlinkKinesisProducer AWS region ('%s') must be set in the config.", 
AWSConfigConstants.AWS_REGION));
+               }
+
                KinesisProducerConfiguration kpc = 
KinesisProducerConfiguration.fromProperties(config);
                
kpc.setRegion(config.getProperty(AWSConfigConstants.AWS_REGION));
 
@@ -266,12 +277,6 @@ public static void validateAwsConfiguration(Properties 
config) {
                        }
                }
 
-               if (!(config.containsKey(AWSConfigConstants.AWS_REGION) ^ 
config.containsKey(ConsumerConfigConstants.AWS_ENDPOINT))) {
-                       // per validation in AwsClientBuilder
-                       throw new 
IllegalArgumentException(String.format("Either AWS region ('%s') or AWS 
endpoint ('%s') must be set in the config.",
-                               AWSConfigConstants.AWS_REGION, 
AWSConfigConstants.AWS_REGION));
-               }
-
                if (config.containsKey(AWSConfigConstants.AWS_REGION)) {
                        // specified AWS Region name must be recognizable
                        if 
(!AWSUtil.isValidRegion(config.getProperty(AWSConfigConstants.AWS_REGION))) {
diff --git 
a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java
 
b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java
index c4bfa17675c..ae2d21e9493 100644
--- 
a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java
+++ 
b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java
@@ -125,14 +125,10 @@ public void 
testCorrectlySetRegionInProducerConfiguration() {
                assertEquals("incorrect region", region, kpc.getRegion());
        }
 
-       // 
----------------------------------------------------------------------
-       // validateAwsConfiguration() tests
-       // 
----------------------------------------------------------------------
-
        @Test
-       public void testMissingAwsRegionInConfig() {
-               String expectedMessage = String.format("Either AWS region 
('%s') or AWS endpoint ('%s') must be set in the config.",
-                       AWSConfigConstants.AWS_REGION, 
AWSConfigConstants.AWS_REGION);
+       public void testMissingAwsRegionInProducerConfig() {
+               String expectedMessage = String.format("For 
FlinkKinesisProducer AWS region ('%s') must be set in the config.",
+                               AWSConfigConstants.AWS_REGION);
                exception.expect(IllegalArgumentException.class);
                exception.expectMessage(expectedMessage);
 
@@ -140,9 +136,13 @@ public void testMissingAwsRegionInConfig() {
                testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, 
"accessKey");
                
testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
 
-               KinesisConfigUtil.validateAwsConfiguration(testConfig);
+               KinesisConfigUtil.getValidatedProducerConfiguration(testConfig);
        }
 
+       // 
----------------------------------------------------------------------
+       // validateAwsConfiguration() tests
+       // 
----------------------------------------------------------------------
+
        @Test
        public void testUnrecognizableAwsRegionInConfig() {
                exception.expect(IllegalArgumentException.class);
@@ -156,22 +156,6 @@ public void testUnrecognizableAwsRegionInConfig() {
                KinesisConfigUtil.validateAwsConfiguration(testConfig);
        }
 
-       @Test
-       public void testAwsRegionOrEndpointInConfig() {
-               String expectedMessage = String.format("Either AWS region 
('%s') or AWS endpoint ('%s') must be set in the config.",
-                       AWSConfigConstants.AWS_REGION, 
AWSConfigConstants.AWS_REGION);
-               exception.expect(IllegalArgumentException.class);
-               exception.expectMessage(expectedMessage);
-
-               Properties testConfig = new Properties();
-               testConfig.setProperty(AWSConfigConstants.AWS_REGION, 
"us-east");
-               testConfig.setProperty(AWSConfigConstants.AWS_ENDPOINT, "fake");
-               testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, 
"accessKey");
-               
testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
-
-               KinesisConfigUtil.validateAwsConfiguration(testConfig);
-       }
-
        @Test
        public void 
testCredentialProviderTypeSetToBasicButNoCredentialSetInConfig() {
                exception.expect(IllegalArgumentException.class);
@@ -200,6 +184,22 @@ public void 
testUnrecognizableCredentialProviderTypeInConfig() {
        // validateConsumerConfiguration() tests
        // 
----------------------------------------------------------------------
 
+       @Test
+       public void testAwsRegionOrEndpointInConsumerConfig() {
+               String expectedMessage = String.format("For 
FlinkKinesisConsumer either AWS region ('%s') or AWS endpoint ('%s') must be 
set in the config.",
+                               AWSConfigConstants.AWS_REGION, 
AWSConfigConstants.AWS_ENDPOINT);
+               exception.expect(IllegalArgumentException.class);
+               exception.expectMessage(expectedMessage);
+
+               Properties testConfig = new Properties();
+               testConfig.setProperty(AWSConfigConstants.AWS_REGION, 
"us-east-1");
+               testConfig.setProperty(AWSConfigConstants.AWS_ENDPOINT, "fake");
+               testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, 
"accessKey");
+               
testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+
+               KinesisConfigUtil.validateConsumerConfiguration(testConfig);
+       }
+
        @Test
        public void testUnrecognizableStreamInitPositionTypeInConfig() {
                exception.expect(IllegalArgumentException.class);


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to