You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/08/03 22:57:08 UTC

[GitHub] [beam] TheNeuralBit commented on a change in pull request #12422: [BEAM-601] Run KinesisIOIT with localstack

TheNeuralBit commented on a change in pull request #12422:
URL: https://github.com/apache/beam/pull/12422#discussion_r464702259



##########
File path: sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java
##########
@@ -99,28 +116,101 @@ private void runRead() {
                 .withAWSClientsProvider(
                     options.getAwsAccessKey(),
                     options.getAwsSecretKey(),
-                    Regions.fromName(options.getAwsKinesisRegion()))
-                .withMaxNumRecords(numberOfRows)
+                    Regions.fromName(options.getAwsKinesisRegion()),
+                    options.getAwsServiceEndpoint(),
+                    options.getAwsVerifyCertificate())
+                .withMaxNumRecords(options.getNumberOfRecords())
                 // to prevent endless running in case of error
-                .withMaxReadTime(Duration.standardMinutes(10))
+                .withMaxReadTime(Duration.standardMinutes(10L))
                 .withInitialPositionInStream(InitialPositionInStream.AT_TIMESTAMP)
                 .withInitialTimestampInStream(now)
                 .withRequestRecordsLimit(1000));
 
     PAssert.thatSingleton(output.apply("Count All", Count.globally()))
-        .isEqualTo((long) numberOfRows);
+        .isEqualTo((long) options.getNumberOfRecords());
 
     PCollection<String> consolidatedHashcode =
         output
             .apply(ParDo.of(new ExtractDataValues()))
             .apply("Hash row contents", Combine.globally(new HashingFn()).withoutDefaults());
 
     PAssert.that(consolidatedHashcode)
-        .containsInAnyOrder(TestRow.getExpectedHashForRowCount(numberOfRows));
+        .containsInAnyOrder(TestRow.getExpectedHashForRowCount(options.getNumberOfRecords()));
 
     pipelineRead.run().waitUntilFinish();
   }
 
+  /** Necessary setup for localstack environment. */
+  private static void setupLocalstack() throws Exception {
+    System.setProperty(SDKGlobalConfiguration.DISABLE_CERT_CHECKING_SYSTEM_PROPERTY, "true");
+    System.setProperty(SDKGlobalConfiguration.AWS_CBOR_DISABLE_SYSTEM_PROPERTY, "true");
+
+    // For some unclear reason localstack requires the timestamp in seconds
+    now = Instant.ofEpochMilli(Long.divideUnsigned(Instant.now().getMillis(), 1000));
+
+    localstackContainer =
+        new LocalStackContainer("0.11.3")
+            .withServices(LocalStackContainer.Service.KINESIS)
+            .withEnv("USE_SSL", "true")
+            .withStartupAttempts(3);
+    localstackContainer.start();
+
+    options.setAwsServiceEndpoint(
+        localstackContainer
+            .getEndpointConfiguration(LocalStackContainer.Service.KINESIS)
+            .getServiceEndpoint()
+            .replace("http", "https"));
+    options.setAwsKinesisRegion(
+        localstackContainer
+            .getEndpointConfiguration(LocalStackContainer.Service.KINESIS)
+            .getSigningRegion());
+    options.setAwsAccessKey(
+        localstackContainer.getDefaultCredentialsProvider().getCredentials().getAWSAccessKeyId());
+    options.setAwsSecretKey(
+        localstackContainer.getDefaultCredentialsProvider().getCredentials().getAWSSecretKey());
+    options.setNumberOfRecords(1000);
+    options.setNumberOfShards(1);
+    options.setAwsKinesisStream("beam_kinesis_test");
+    options.setAwsVerifyCertificate(false);
+    createStream(options.getAwsKinesisStream());
+  }
+
+  private static void createStream(String streamName) throws Exception {
+    AmazonKinesisClientBuilder clientBuilder = AmazonKinesisClientBuilder.standard();
+
+    clientBuilder.setCredentials(localstackContainer.getDefaultCredentialsProvider());
+    clientBuilder.setEndpointConfiguration(
+        localstackContainer.getEndpointConfiguration(LocalStackContainer.Service.KINESIS));
+
+    AmazonKinesis client = clientBuilder.build();
+
+    client.createStream(streamName, 1);
+    int repeats = 10;
+    for (int i = 0; i <= repeats; ++i) {
+      String streamStatus =
+          client.describeStream(streamName).getStreamDescription().getStreamStatus();
+      if ("ACTIVE".equals(streamStatus)) {
+        break;
+      }
+      if (i == repeats) {
+        throw new RuntimeException("Unable to initialize stream");
+      }
+      Thread.sleep(1000L);
+    }
+  }
+
+  /** Check whether pipeline options were provided. If not, use localstack container. */
+  private static boolean doUseLocalstack() {
+    return "aws-access-key".equals(options.getAwsAccessKey())
+        && "aws-secret-key".equals(options.getAwsSecretKey())
+        && "aws-kinesis-stream".equals(options.getAwsKinesisStream())
+        && "aws-kinesis-region".equals(options.getAwsKinesisRegion())
+        && options.getNumberOfShards() == 2
+        && options.getNumberOfRecords() == 1000
+        && options.getAwsServiceEndpoint() == null
+        && options.getAwsVerifyCertificate();
+  }

Review comment:
       Rather than repeating the default values here, you could create a fresh instance using `PipelineOptionsFactory.fromArgs().as(KinesisTestOptions.class)` and compare each field against it.

##########
File path: sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java
##########
@@ -99,28 +116,101 @@ private void runRead() {
                 .withAWSClientsProvider(
                     options.getAwsAccessKey(),
                     options.getAwsSecretKey(),
-                    Regions.fromName(options.getAwsKinesisRegion()))
-                .withMaxNumRecords(numberOfRows)
+                    Regions.fromName(options.getAwsKinesisRegion()),
+                    options.getAwsServiceEndpoint(),
+                    options.getAwsVerifyCertificate())
+                .withMaxNumRecords(options.getNumberOfRecords())
                 // to prevent endless running in case of error
-                .withMaxReadTime(Duration.standardMinutes(10))
+                .withMaxReadTime(Duration.standardMinutes(10L))
                 .withInitialPositionInStream(InitialPositionInStream.AT_TIMESTAMP)
                 .withInitialTimestampInStream(now)
                 .withRequestRecordsLimit(1000));
 
     PAssert.thatSingleton(output.apply("Count All", Count.globally()))
-        .isEqualTo((long) numberOfRows);
+        .isEqualTo((long) options.getNumberOfRecords());
 
     PCollection<String> consolidatedHashcode =
         output
             .apply(ParDo.of(new ExtractDataValues()))
             .apply("Hash row contents", Combine.globally(new HashingFn()).withoutDefaults());
 
     PAssert.that(consolidatedHashcode)
-        .containsInAnyOrder(TestRow.getExpectedHashForRowCount(numberOfRows));
+        .containsInAnyOrder(TestRow.getExpectedHashForRowCount(options.getNumberOfRecords()));
 
     pipelineRead.run().waitUntilFinish();
   }
 
+  /** Necessary setup for localstack environment. */
+  private static void setupLocalstack() throws Exception {
+    System.setProperty(SDKGlobalConfiguration.DISABLE_CERT_CHECKING_SYSTEM_PROPERTY, "true");
+    System.setProperty(SDKGlobalConfiguration.AWS_CBOR_DISABLE_SYSTEM_PROPERTY, "true");
+
+    // For some unclear reason localstack requires the timestamp in seconds
+    now = Instant.ofEpochMilli(Long.divideUnsigned(Instant.now().getMillis(), 1000));
+
+    localstackContainer =
+        new LocalStackContainer("0.11.3")

Review comment:
       Is this something we're going to have to keep up-to-date? Is it possible to just omit it?

##########
File path: sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java
##########
@@ -99,28 +116,101 @@ private void runRead() {
                 .withAWSClientsProvider(
                     options.getAwsAccessKey(),
                     options.getAwsSecretKey(),
-                    Regions.fromName(options.getAwsKinesisRegion()))
-                .withMaxNumRecords(numberOfRows)
+                    Regions.fromName(options.getAwsKinesisRegion()),
+                    options.getAwsServiceEndpoint(),
+                    options.getAwsVerifyCertificate())
+                .withMaxNumRecords(options.getNumberOfRecords())
                 // to prevent endless running in case of error
-                .withMaxReadTime(Duration.standardMinutes(10))
+                .withMaxReadTime(Duration.standardMinutes(10L))
                 .withInitialPositionInStream(InitialPositionInStream.AT_TIMESTAMP)
                 .withInitialTimestampInStream(now)
                 .withRequestRecordsLimit(1000));
 
     PAssert.thatSingleton(output.apply("Count All", Count.globally()))
-        .isEqualTo((long) numberOfRows);
+        .isEqualTo((long) options.getNumberOfRecords());
 
     PCollection<String> consolidatedHashcode =
         output
             .apply(ParDo.of(new ExtractDataValues()))
             .apply("Hash row contents", Combine.globally(new HashingFn()).withoutDefaults());
 
     PAssert.that(consolidatedHashcode)
-        .containsInAnyOrder(TestRow.getExpectedHashForRowCount(numberOfRows));
+        .containsInAnyOrder(TestRow.getExpectedHashForRowCount(options.getNumberOfRecords()));
 
     pipelineRead.run().waitUntilFinish();
   }
 
+  /** Necessary setup for localstack environment. */
+  private static void setupLocalstack() throws Exception {
+    System.setProperty(SDKGlobalConfiguration.DISABLE_CERT_CHECKING_SYSTEM_PROPERTY, "true");
+    System.setProperty(SDKGlobalConfiguration.AWS_CBOR_DISABLE_SYSTEM_PROPERTY, "true");
+
+    // For some unclear reason localstack requires the timestamp in seconds
+    now = Instant.ofEpochMilli(Long.divideUnsigned(Instant.now().getMillis(), 1000));

Review comment:
       :(

##########
File path: sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java
##########
@@ -35,33 +38,45 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
+import org.testcontainers.containers.localstack.LocalStackContainer;
 
 /**
  * Integration test, that writes and reads data to and from real Kinesis. You need to provide {@link
- * KinesisTestOptions} in order to run this.
+ * KinesisTestOptions} in order to run this if you want to test it with production setup. By default
+ * when no options are provided an instance of localstack is used.
  */
 @RunWith(JUnit4.class)
 public class KinesisIOIT implements Serializable {
-  private static int numberOfShards;
-  private static int numberOfRows;
-
   @Rule public TestPipeline pipelineWrite = TestPipeline.create();
   @Rule public TestPipeline pipelineRead = TestPipeline.create();
 
+  private static LocalStackContainer localstackContainer;
+
   private static KinesisTestOptions options;
-  private static final Instant now = Instant.now();

Review comment:
       Does this need to be `Instant.now()`? Might be preferable to just make it some constant to make it more repeatable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org