Skip to content

[FLINK-36975][Connectors/AWS] Fix the AWS Config Option AWS_ROLE_CREDENTIALS_PROVIDER_OPTION to pick the right key #196

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import static org.apache.flink.connector.aws.config.AWSConfigConstants.profileName;
import static org.apache.flink.connector.aws.config.AWSConfigConstants.profilePath;
import static org.apache.flink.connector.aws.config.AWSConfigConstants.roleArn;
import static org.apache.flink.connector.aws.config.AWSConfigConstants.roleCredentialsProvider;
import static org.apache.flink.connector.aws.config.AWSConfigConstants.roleSessionName;
import static org.apache.flink.connector.aws.config.AWSConfigConstants.roleStsEndpoint;
import static org.apache.flink.connector.aws.config.AWSConfigConstants.secretKey;
Expand Down Expand Up @@ -129,7 +130,7 @@ public class AWSConfigOptions {
+ " type is set to WEB_IDENTITY_TOKEN.");

public static final ConfigOption<String> AWS_ROLE_CREDENTIALS_PROVIDER_OPTION =
ConfigOptions.key(webIdentityTokenFile(AWS_CREDENTIALS_PROVIDER))
ConfigOptions.key(roleCredentialsProvider(AWS_CREDENTIALS_PROVIDER))
.stringType()
.noDefaultValue()
.withDescription(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* 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.flink.connector.aws.config;

import org.junit.jupiter.api.Test;

import static org.assertj.core.api.Assertions.assertThat;

/** Unit tests for {@link AWSConfigOptions}. */
class AWSConfigOptionsTest {

@Test
void testConfigOptionKeys() {
// Test AWS_REGION_OPTION
assertThat(AWSConfigOptions.AWS_REGION_OPTION.key())
.isEqualTo(AWSConfigConstants.AWS_REGION);

// Test AWS_CREDENTIALS_PROVIDER_OPTION
assertThat(AWSConfigOptions.AWS_CREDENTIALS_PROVIDER_OPTION.key())
.isEqualTo(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER);

// Test AWS_ACCESS_KEY_ID_OPTION
assertThat(AWSConfigOptions.AWS_ACCESS_KEY_ID_OPTION.key())
.isEqualTo(
AWSConfigConstants.accessKeyId(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_SECRET_ACCESS_KEY_OPTION
assertThat(AWSConfigOptions.AWS_SECRET_ACCESS_KEY_OPTION.key())
.isEqualTo(
AWSConfigConstants.secretKey(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_PROFILE_PATH_OPTION
assertThat(AWSConfigOptions.AWS_PROFILE_PATH_OPTION.key())
.isEqualTo(
AWSConfigConstants.profilePath(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_PROFILE_NAME_OPTION
assertThat(AWSConfigOptions.AWS_PROFILE_NAME_OPTION.key())
.isEqualTo(
AWSConfigConstants.profileName(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_ROLE_STS_ENDPOINT_OPTION
assertThat(AWSConfigOptions.AWS_ROLE_STS_ENDPOINT_OPTION.key())
.isEqualTo(
AWSConfigConstants.roleStsEndpoint(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test CUSTOM_CREDENTIALS_PROVIDER_CLASS_OPTION
assertThat(AWSConfigOptions.CUSTOM_CREDENTIALS_PROVIDER_CLASS_OPTION.key())
.isEqualTo(
AWSConfigConstants.customCredentialsProviderClass(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_ROLE_ARN_OPTION
assertThat(AWSConfigOptions.AWS_ROLE_ARN_OPTION.key())
.isEqualTo(AWSConfigConstants.roleArn(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_ROLE_SESSION_NAME
assertThat(AWSConfigOptions.AWS_ROLE_SESSION_NAME.key())
.isEqualTo(
AWSConfigConstants.roleSessionName(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_ROLE_EXTERNAL_ID_OPTION
assertThat(AWSConfigOptions.AWS_ROLE_EXTERNAL_ID_OPTION.key())
.isEqualTo(
AWSConfigConstants.externalId(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_WEB_IDENTITY_TOKEN_FILE
assertThat(AWSConfigOptions.AWS_WEB_IDENTITY_TOKEN_FILE.key())
.isEqualTo(
AWSConfigConstants.webIdentityTokenFile(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_ROLE_CREDENTIALS_PROVIDER_OPTION - this was the one with the bug
assertThat(AWSConfigOptions.AWS_ROLE_CREDENTIALS_PROVIDER_OPTION.key())
.isEqualTo(
AWSConfigConstants.roleCredentialsProvider(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER))
.isNotEqualTo(
AWSConfigConstants.webIdentityTokenFile(
AWSConfigConstants.AWS_CREDENTIALS_PROVIDER));

// Test AWS_ENDPOINT_OPTION
assertThat(AWSConfigOptions.AWS_ENDPOINT_OPTION.key())
.isEqualTo(AWSConfigConstants.AWS_ENDPOINT);

// Test TRUST_ALL_CERTIFICATES_OPTION
assertThat(AWSConfigOptions.TRUST_ALL_CERTIFICATES_OPTION.key())
.isEqualTo(AWSConfigConstants.TRUST_ALL_CERTIFICATES);

// Test HTTP_PROTOCOL_VERSION_OPTION
assertThat(AWSConfigOptions.HTTP_PROTOCOL_VERSION_OPTION.key())
.isEqualTo(AWSConfigConstants.HTTP_PROTOCOL_VERSION);

// Test HTTP_CLIENT_MAX_CONCURRENCY_OPTION
assertThat(AWSConfigOptions.HTTP_CLIENT_MAX_CONCURRENCY_OPTION.key())
.isEqualTo(AWSConfigConstants.HTTP_CLIENT_MAX_CONCURRENCY);

// Test HTTP_CLIENT_READ_TIMEOUT_MILLIS_OPTION
assertThat(AWSConfigOptions.HTTP_CLIENT_READ_TIMEOUT_MILLIS_OPTION.key())
.isEqualTo(AWSConfigConstants.HTTP_CLIENT_READ_TIMEOUT_MILLIS);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* 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.flink.connector.kinesis.sink;

import org.apache.flink.annotation.Internal;

import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;

/**
* Provider interface for KinesisAsyncClient instances. This is primarily used for testing to inject
* mock clients.
*/
@Internal
interface KinesisClientProvider {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this also extend SdkAutoCloseable so it's inline with the other resource impls?

/**
* Returns a KinesisAsyncClient instance.
*
* @return The KinesisAsyncClient instance
*/
KinesisAsyncClient get();

/** Closes any resources held by this provider. */
void close();
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.connector.base.sink.AsyncSinkBase;
import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
import org.apache.flink.connector.base.sink.writer.ElementConverter;
Expand Down Expand Up @@ -72,6 +73,8 @@ public class KinesisStreamsSink<InputT> extends AsyncSinkBase<InputT, PutRecords
private final String streamArn;
private final Properties kinesisClientProperties;

@VisibleForTesting private KinesisClientProvider kinesisClientProvider;

KinesisStreamsSink(
ElementConverter<InputT, PutRecordsRequestEntry> elementConverter,
Integer maxBatchSize,
Expand Down Expand Up @@ -124,6 +127,19 @@ public static <InputT> KinesisStreamsSinkBuilder<InputT> builder() {
return new KinesisStreamsSinkBuilder<>();
}

/**
* Set a custom KinesisAsyncClient provider for testing purposes. This method is only intended
* to be used in tests.
*
* @param kinesisClientProvider The provider that supplies the KinesisAsyncClient
*/
@VisibleForTesting
void setKinesisClientProvider(KinesisClientProvider kinesisClientProvider) {
this.kinesisClientProvider =
Preconditions.checkNotNull(
kinesisClientProvider, "The kinesisClientProvider must not be null.");
}

@Internal
@Override
public StatefulSinkWriter<InputT, BufferedRequestState<PutRecordsRequestEntry>> createWriter(
Expand All @@ -141,7 +157,8 @@ public StatefulSinkWriter<InputT, BufferedRequestState<PutRecordsRequestEntry>>
streamName,
streamArn,
kinesisClientProperties,
Collections.emptyList());
Collections.emptyList(),
kinesisClientProvider);
}

@Internal
Expand Down Expand Up @@ -170,6 +187,7 @@ public StatefulSinkWriter<InputT, BufferedRequestState<PutRecordsRequestEntry>>
streamName,
streamArn,
kinesisClientProperties,
recoveredState);
recoveredState,
kinesisClientProvider);
}
}
Loading
Loading