Skip to content

Commit

Permalink
Merge pull request #16595З: [BEAM-8807] Add integration test for SnsI…
Browse files Browse the repository at this point in the history
…O.write (Sdk v1 & v2)
  • Loading branch information
aromanenko-dev authored Jan 26, 2022
2 parents 81006dd + 7b06fa7 commit 41d585f
Show file tree
Hide file tree
Showing 11 changed files with 359 additions and 91 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,13 @@
* ensure it can be sent to worker machines.
*/
public interface AwsClientsProvider extends Serializable {
AmazonCloudWatch getCloudWatchClient();

/** @deprecated DynamoDBIO doesn't require a CloudWatch client */
@Deprecated
@SuppressWarnings("return.type.incompatible")
default AmazonCloudWatch getCloudWatchClient() {
return null;
}

AmazonDynamoDB createDynamoDB();
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,6 @@
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.cloudwatch.AmazonCloudWatch;
import com.amazonaws.services.cloudwatch.AmazonCloudWatchClientBuilder;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClientBuilder;
import org.checkerframework.checker.nullness.qual.Nullable;
Expand All @@ -52,19 +50,6 @@ private AWSCredentialsProvider getCredentialsProvider() {
return new AWSStaticCredentialsProvider(new BasicAWSCredentials(accessKey, secretKey));
}

@Override
public AmazonCloudWatch getCloudWatchClient() {
AmazonCloudWatchClientBuilder clientBuilder =
AmazonCloudWatchClientBuilder.standard().withCredentials(getCredentialsProvider());
if (serviceEndpoint == null) {
clientBuilder.withRegion(region);
} else {
clientBuilder.withEndpointConfiguration(
new EndpointConfiguration(serviceEndpoint, region.getName()));
}
return clientBuilder.build();
}

@Override
public AmazonDynamoDB createDynamoDB() {
AmazonDynamoDBClientBuilder clientBuilder =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,13 @@
* ensure it can be sent to worker machines.
*/
public interface AwsClientsProvider extends Serializable {
AmazonCloudWatch getCloudWatchClient();

/** @deprecated SnsIO doesn't require a CloudWatch client */
@Deprecated
@SuppressWarnings("return.type.incompatible")
default AmazonCloudWatch getCloudWatchClient() {
return null;
}

AmazonSNS createSnsPublisher();
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,6 @@
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.cloudwatch.AmazonCloudWatch;
import com.amazonaws.services.cloudwatch.AmazonCloudWatchClientBuilder;
import com.amazonaws.services.sns.AmazonSNS;
import com.amazonaws.services.sns.AmazonSNSClientBuilder;
import org.checkerframework.checker.nullness.qual.Nullable;
Expand Down Expand Up @@ -54,9 +52,9 @@ private AWSCredentialsProvider getCredentialsProvider() {
}

@Override
public AmazonCloudWatch getCloudWatchClient() {
AmazonCloudWatchClientBuilder clientBuilder =
AmazonCloudWatchClientBuilder.standard().withCredentials(getCredentialsProvider());
public AmazonSNS createSnsPublisher() {
AmazonSNSClientBuilder clientBuilder =
AmazonSNSClientBuilder.standard().withCredentials(getCredentialsProvider());
if (serviceEndpoint == null) {
clientBuilder.withRegion(region);
} else {
Expand All @@ -65,12 +63,4 @@ public AmazonCloudWatch getCloudWatchClient() {
}
return clientBuilder.build();
}

@Override
public AmazonSNS createSnsPublisher() {
return AmazonSNSClientBuilder.standard()
.withCredentials(getCredentialsProvider())
.withRegion(region)
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,17 +18,18 @@
package org.apache.beam.sdk.io.aws;

import static org.apache.beam.sdk.testing.TestPipeline.testingPipelineOptions;
import static org.testcontainers.containers.localstack.LocalStackContainer.Service.S3;

import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration;
import java.util.function.Consumer;
import org.apache.beam.sdk.io.aws.options.AwsOptions;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestPipelineOptions;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
import org.apache.commons.lang3.StringUtils;
import org.junit.rules.ExternalResource;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.localstack.LocalStackContainer;
Expand Down Expand Up @@ -76,43 +77,28 @@ public interface ITOptions extends AwsOptions, TestPipelineOptions {
void setLocalstackLogLevel(String level);
}

private final Service service;
private final OptionsT options;
private final LocalStackContainer localstack;

public ITEnvironment(Service service, Class<OptionsT> optionsClass) {
this(service, optionsClass, o -> {});
}

public ITEnvironment(
Service service, Class<OptionsT> optionsClass, Consumer<OptionsT> optionsMutator) {
this(service, optionsClass, optionsMutator, new String[0]);
}

public ITEnvironment(Service service, Class<OptionsT> optionsClass, String... env) {
this(service, optionsClass, o -> {}, env);
this(new Service[] {service}, optionsClass, env);
}

public ITEnvironment(
Service service,
Class<OptionsT> optionsClass,
Consumer<OptionsT> optionsMutator,
String... env) {
this.service = service;
public ITEnvironment(Service[] services, Class<OptionsT> optionsClass, String... env) {
localstack =
new LocalStackContainer(DockerImageName.parse(LOCALSTACK).withTag(LOCALSTACK_VERSION))
.withServices(service)
.withServices(services)
.withStartupAttempts(3);

PipelineOptionsFactory.register(optionsClass);
options = testingPipelineOptions().as(optionsClass);
optionsMutator.accept(options);

localstack.setEnv(ImmutableList.copyOf(env));
if (options.getLocalstackLogLevel() != null) {
localstack
.withEnv("LS_LOG", options.getLocalstackLogLevel())
.withLogConsumer(new Slf4jLogConsumer(LoggerFactory.getLogger(service.name())));
.withLogConsumer(
new Slf4jLogConsumer(LoggerFactory.getLogger(StringUtils.join(services))));
}
}

Expand Down Expand Up @@ -150,7 +136,8 @@ protected void after() {
/** Necessary setup for localstack environment. */
private void startLocalstack() {
localstack.start();
options.setAwsServiceEndpoint(localstack.getEndpointOverride(service).toString());
options.setAwsServiceEndpoint(
localstack.getEndpointOverride(S3).toString()); // service irrelevant
options.setAwsRegion(localstack.getRegion());
options.setAwsCredentialsProvider(localstack.getDefaultCredentialsProvider());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

import static java.util.Collections.synchronizedMap;

import com.amazonaws.services.cloudwatch.AmazonCloudWatch;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
import java.util.HashMap;
import java.util.Map;
Expand All @@ -42,11 +41,6 @@ static AwsClientsProvider of(AmazonDynamoDB client) {
return provider;
}

@Override
public AmazonCloudWatch getCloudWatchClient() {
return null; // never used
}

@Override
public AmazonDynamoDB createDynamoDB() {
return clients.get(id);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,13 @@ public interface S3ITOptions extends ITEnvironment.ITOptions, S3Options {}

@ClassRule
public static ITEnvironment<S3ITOptions> env =
new ITEnvironment<>(S3, S3ITOptions.class, S3ClientFixFix::set);
new ITEnvironment<S3ITOptions>(S3, S3ITOptions.class) {
@Override
protected void before() {
super.before();
options().setS3ClientFactoryClass(S3ClientFixFix.class);
}
};

@Rule public TestPipeline pipelineWrite = env.createTestPipeline();
@Rule public TestPipeline pipelineRead = env.createTestPipeline();
Expand Down Expand Up @@ -111,10 +117,6 @@ protected void before() {
// Fix duplicated Content-Length header due to case-sensitive handling of header names
// https://github.com/aws/aws-sdk-java/issues/2503
private static class S3ClientFixFix extends DefaultS3ClientBuilderFactory {
private static void set(S3Options s3Options) {
s3Options.setS3ClientFactoryClass(S3ClientFixFix.class);
}

@Override
public AmazonS3ClientBuilder createBuilder(S3Options s3Options) {
return super.createBuilder(s3Options)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,159 @@
/*
* 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.beam.sdk.io.aws.sns;

import static org.apache.beam.sdk.io.common.IOITHelper.executeWithRetry;
import static org.apache.beam.sdk.io.common.TestRow.getExpectedHashForRowCount;
import static org.apache.beam.sdk.values.TypeDescriptors.strings;
import static org.testcontainers.containers.localstack.LocalStackContainer.Service.SNS;
import static org.testcontainers.containers.localstack.LocalStackContainer.Service.SQS;

import com.amazonaws.regions.Regions;
import com.amazonaws.services.sns.AmazonSNS;
import com.amazonaws.services.sns.AmazonSNSClientBuilder;
import com.amazonaws.services.sns.model.PublishRequest;
import com.amazonaws.services.sqs.AmazonSQS;
import com.amazonaws.services.sqs.AmazonSQSClientBuilder;
import com.amazonaws.services.sqs.model.Message;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.Serializable;
import org.apache.beam.sdk.io.GenerateSequence;
import org.apache.beam.sdk.io.aws.ITEnvironment;
import org.apache.beam.sdk.io.aws.sqs.SqsIO;
import org.apache.beam.sdk.io.common.HashingFn;
import org.apache.beam.sdk.io.common.TestRow;
import org.apache.beam.sdk.io.common.TestRow.DeterministicallyConstructTestRowFn;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Combine;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExternalResource;
import org.junit.rules.Timeout;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
import org.testcontainers.containers.localstack.LocalStackContainer.Service;

@RunWith(JUnit4.class)
public class SnsIOIT {
public interface ITOptions extends ITEnvironment.ITOptions {}

private static final ObjectMapper MAPPER = new ObjectMapper();
private static final TypeDescriptor<PublishRequest> publishRequests =
TypeDescriptor.of(PublishRequest.class);

@ClassRule
public static ITEnvironment<ITOptions> env =
new ITEnvironment<>(new Service[] {SQS, SNS}, ITOptions.class, "SQS_PROVIDER=elasticmq");

@Rule public Timeout globalTimeout = Timeout.seconds(600);

@Rule public TestPipeline pipelineWrite = env.createTestPipeline();
@Rule public TestPipeline pipelineRead = env.createTestPipeline();
@Rule public AwsResources resources = new AwsResources();

@Test
public void testWriteThenRead() {
ITOptions opts = env.options();
int rows = opts.getNumberOfRows();

// Write test dataset to SNS

pipelineWrite
.apply("Generate Sequence", GenerateSequence.from(0).to(rows))
.apply("Prepare TestRows", ParDo.of(new DeterministicallyConstructTestRowFn()))
.apply("SNS request", MapElements.into(publishRequests).via(resources::publishRequest))
.apply(
"Write to SNS",
SnsIO.write()
.withTopicName(resources.snsTopic)
.withResultOutputTag(new TupleTag<>())
.withAWSClientsProvider(
opts.getAwsCredentialsProvider().getCredentials().getAWSAccessKeyId(),
opts.getAwsCredentialsProvider().getCredentials().getAWSSecretKey(),
Regions.fromName(opts.getAwsRegion()),
opts.getAwsServiceEndpoint()));

// Read test dataset from SQS.
PCollection<String> output =
pipelineRead
.apply(
"Read from SQS",
SqsIO.read().withQueueUrl(resources.sqsQueue).withMaxNumRecords(rows))
.apply("Extract message", MapElements.into(strings()).via(SnsIOIT::extractMessage));

PAssert.thatSingleton(output.apply("Count All", Count.globally())).isEqualTo((long) rows);

PAssert.that(output.apply(Combine.globally(new HashingFn()).withoutDefaults()))
.containsInAnyOrder(getExpectedHashForRowCount(rows));

pipelineWrite.run();
pipelineRead.run();
}

private static String extractMessage(Message msg) {
try {
return MAPPER.readTree(msg.getBody()).get("Message").asText();
} catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
}

private static class AwsResources extends ExternalResource implements Serializable {
private transient AmazonSQS sqs = env.buildClient(AmazonSQSClientBuilder.standard());
private transient AmazonSNS sns = env.buildClient(AmazonSNSClientBuilder.standard());

private String sqsQueue;
private String snsTopic;
private String sns2Sqs;

PublishRequest publishRequest(TestRow r) {
return new PublishRequest(snsTopic, r.name());
}

@Override
protected void before() throws Throwable {
snsTopic = sns.createTopic("beam-snsio-it").getTopicArn();
// add SQS subscription so we can read the messages again
sqsQueue = sqs.createQueue("beam-snsio-it").getQueueUrl();
sns2Sqs = sns.subscribe(snsTopic, "sqs", sqsQueue).getSubscriptionArn();
}

@Override
protected void after() {
try {
executeWithRetry(() -> sns.unsubscribe(sns2Sqs));
executeWithRetry(() -> sns.deleteTopic(snsTopic));
executeWithRetry(() -> sqs.deleteQueue(sqsQueue));
} catch (Exception e) {
throw new RuntimeException(e);
} finally {
sns.shutdown();
sqs.shutdown();
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import static org.joda.time.Duration.standardSeconds;

import com.amazonaws.http.SdkHttpMetadata;
import com.amazonaws.services.cloudwatch.AmazonCloudWatch;
import com.amazonaws.services.sns.AmazonSNS;
import com.amazonaws.services.sns.model.GetTopicAttributesResult;
import com.amazonaws.services.sns.model.InternalErrorException;
Expand Down Expand Up @@ -79,11 +78,6 @@ public Provider(AmazonSNS pub) {
publisher = pub;
}

@Override
public AmazonCloudWatch getCloudWatchClient() {
return Mockito.mock(AmazonCloudWatch.class);
}

@Override
public AmazonSNS createSnsPublisher() {
return publisher;
Expand Down
Loading

0 comments on commit 41d585f

Please sign in to comment.