Skip to content
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

Kafka Source - Cleanup and Enhancements for MSK #3029

Merged
merged 5 commits into from
Jul 18, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
6 changes: 3 additions & 3 deletions data-prepper-plugins/kafka-plugins/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ dependencies {
implementation 'io.confluent:kafka-schema-registry-client:7.3.3'
implementation 'io.confluent:kafka-schema-registry:7.3.3:tests'
implementation 'software.amazon.msk:aws-msk-iam-auth:1.1.6'
implementation 'com.amazonaws:aws-java-sdk-kafka:1.12.169'
implementation 'com.amazonaws:aws-java-sdk:1.0.12'
implementation 'com.amazonaws:aws-java-sdk-sts:1.12.505'
implementation 'software.amazon.awssdk:sts:2.20.103'
Copy link
Collaborator

Choose a reason for hiding this comment

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

minor: it's better to not set sdk version here but you can change this later

implementation 'software.amazon.awssdk:auth:2.20.103'
implementation 'software.amazon.awssdk:kafka:2.20.103'
testImplementation 'org.mockito:mockito-inline:4.1.0'
testImplementation 'org.yaml:snakeyaml:2.0'
testImplementation testLibs.spring.test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ public static class AwsMskConfig {
private String arn;

@JsonProperty("broker_connection_type")
private MskBrokerConnectionType brokerConnectionType;
private MskBrokerConnectionType brokerConnectionType = MskBrokerConnectionType.PUBLIC;
Copy link
Contributor

Choose a reason for hiding this comment

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

single VPC connection type should be default. but it can be addressed in next commit


public String getArn() {
return arn;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager;
import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionType;
import org.opensearch.dataprepper.plugins.kafka.configuration.MskBrokerConnectionType;
import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.AwsIamAuthConfig;
Expand All @@ -33,16 +34,17 @@
import org.opensearch.dataprepper.plugins.kafka.util.KafkaSourceJsonDeserializer;
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;

import com.amazonaws.services.kafka.model.GetBootstrapBrokersRequest;
import com.amazonaws.services.kafka.AWSKafka;
import com.amazonaws.services.kafka.AWSKafkaClientBuilder;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.services.kafka.model.GetBootstrapBrokersResult;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
import com.amazonaws.services.kafka.model.InternalServerErrorException;
import com.amazonaws.services.kafka.model.ConflictException;
import software.amazon.awssdk.services.kafka.KafkaClient;
import software.amazon.awssdk.services.kafka.model.GetBootstrapBrokersRequest;
import software.amazon.awssdk.services.kafka.model.GetBootstrapBrokersResponse;
import software.amazon.awssdk.services.kafka.model.InternalServerErrorException;
import software.amazon.awssdk.services.kafka.model.ConflictException;
import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider;
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
import software.amazon.awssdk.services.sts.model.AssumeRoleRequest;
import software.amazon.awssdk.services.sts.StsClient;
import software.amazon.awssdk.regions.Region;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -170,27 +172,40 @@ private long calculateLongestThreadWaitingTime() {
}

public String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuthConfig, final AwsConfig awsConfig) {
ClientConfiguration clientConfiguration = new ClientConfiguration();
AWSCredentialsProvider credentialProvider;
if (awsIamAuthConfig == AwsIamAuthConfig.DEFAULT) {
credentialProvider = new DefaultAWSCredentialsProviderChain();
} else if (awsIamAuthConfig == AwsIamAuthConfig.ROLE) {
AwsCredentialsProvider credentialsProvider = DefaultCredentialsProvider.create();
if (awsIamAuthConfig == AwsIamAuthConfig.ROLE) {
String sessionName = "data-prepper-kafka-session"+UUID.randomUUID();
credentialProvider = new STSAssumeRoleSessionCredentialsProvider.Builder(awsConfig.getStsRoleArn(), sessionName).build();
StsClient stsClient = StsClient.builder()
.region(Region.of(awsConfig.getRegion()))
.credentialsProvider(credentialsProvider)
.build();
credentialsProvider = StsAssumeRoleCredentialsProvider
.builder()
.stsClient(stsClient)
.refreshRequest(
AssumeRoleRequest
.builder()
.roleArn(awsConfig.getStsRoleArn())
.roleSessionName(sessionName)
.build()
).build();
} else {
throw new RuntimeException("Unknown AWS IAM auth mode");
}
AWSKafka kafkaClient = AWSKafkaClientBuilder.standard()
.withClientConfiguration(clientConfiguration)
.withCredentials(credentialProvider)
.withRegion(awsConfig.getRegion())
final AwsConfig.AwsMskConfig awsMskConfig = awsConfig.getAwsMskConfig();
KafkaClient kafkaClient = KafkaClient.builder()
.credentialsProvider(credentialsProvider)
.region(Region.of(awsConfig.getRegion()))
.build();
final GetBootstrapBrokersRequest request =
GetBootstrapBrokersRequest
.builder()
.clusterArn(awsMskConfig.getArn())
.build();
GetBootstrapBrokersRequest request = new GetBootstrapBrokersRequest();
String clusterArn = awsConfig.getAwsMskConfig().getArn();
request.setClusterArn(clusterArn);

int numRetries = 0;
boolean retryable;
GetBootstrapBrokersResult result = null;
GetBootstrapBrokersResponse result = null;
do {
retryable = false;
try {
Expand All @@ -205,8 +220,11 @@ public String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuthConfig,
LOG.info("Failed to get bootstrap server information from MSK, using user configured bootstrap servers");
return sourceConfig.getBootStrapServers();
}
// TODO return this based on the broker_connection_mode
return result.getBootstrapBrokerStringSaslIam();
if (awsMskConfig.getBrokerConnectionType() == MskBrokerConnectionType.PUBLIC) {
return result.bootstrapBrokerStringPublicSaslIam();
} else {
return result.bootstrapBrokerStringVpcConnectivitySaslIam();
Copy link
Contributor

Choose a reason for hiding this comment

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

There are 3 connection types. single-vpc, public and multi-Vpc

}
}

private Properties getConsumerProperties(final TopicConfig topicConfig) {
Expand Down
Loading