[
https://issues.apache.org/jira/browse/BEAM-601?focusedWorklogId=464805&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-464805
]
ASF GitHub Bot logged work on BEAM-601:
---------------------------------------
Author: ASF GitHub Bot
Created on: 30/Jul/20 23:29
Start Date: 30/Jul/20 23:29
Worklog Time Spent: 10m
Work Description: TheNeuralBit commented on a change in pull request
#12422:
URL: https://github.com/apache/beam/pull/12422#discussion_r463325571
##########
File path:
sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java
##########
@@ -34,34 +35,65 @@
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.PCollection;
import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
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.
*/
@RunWith(JUnit4.class)
public class KinesisIOIT implements Serializable {
- private static int numberOfShards;
- private static int numberOfRows;
+ private static final String STREAM_NAME = "beam_kinesis";
+ private static final int NUM_RECORDS = 1000;
@Rule public TestPipeline pipelineWrite = TestPipeline.create();
@Rule public TestPipeline pipelineRead = TestPipeline.create();
- private static KinesisTestOptions options;
- private static final Instant now = Instant.now();
+ static {
+
System.setProperty(SDKGlobalConfiguration.DISABLE_CERT_CHECKING_SYSTEM_PROPERTY,
"true");
+
System.setProperty(SDKGlobalConfiguration.AWS_CBOR_DISABLE_SYSTEM_PROPERTY,
"true");
+ }
+
+ private final LocalStackContainer localstackContainer =
+ new LocalStackContainer("0.11.3")
+ .withServices(LocalStackContainer.Service.KINESIS)
+ .withEnv("USE_SSL", "true")
+ .withStartupAttempts(3);
+
+ private String endpoint;
+ private String region;
+ private String accessKey;
+ private String secretKey;
+
+ @Before
+ public void setup() throws Exception {
+ localstackContainer.start();
+ endpoint =
+ localstackContainer
+ .getEndpointConfiguration(LocalStackContainer.Service.KINESIS)
+ .getServiceEndpoint()
+ .replace("http", "https");
+ region =
+ localstackContainer
+ .getEndpointConfiguration(LocalStackContainer.Service.KINESIS)
+ .getSigningRegion();
+ accessKey =
+
localstackContainer.getDefaultCredentialsProvider().getCredentials().getAWSAccessKeyId();
+ secretKey =
+
localstackContainer.getDefaultCredentialsProvider().getCredentials().getAWSSecretKey();
+
+ createStream();
+ }
- @BeforeClass
- public static void setup() {
- PipelineOptionsFactory.register(KinesisTestOptions.class);
- options =
TestPipeline.testingPipelineOptions().as(KinesisTestOptions.class);
- numberOfShards = options.getNumberOfShards();
- numberOfRows = options.getNumberOfRecords();
Review comment:
I think we should maintain the ability to test against production AWS.
Someday maybe we'll get some AWS credits to run this continuously against prod,
and it could still be useful for local testing. Could you make it so we only
start up a localstack container if nothing in `KinesisTestOptions` is modified?
When starting the localstack you could just set the relevant fields in the
PipelineOptions and let the rest of the test read them as it does now.
----------------------------------------------------------------
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 464805)
Time Spent: 50m (was: 40m)
> Enable Kinesis integration tests
> --------------------------------
>
> Key: BEAM-601
> URL: https://issues.apache.org/jira/browse/BEAM-601
> Project: Beam
> Issue Type: Improvement
> Components: testing
> Affects Versions: 0.3.0-incubating
> Reporter: Przemyslaw Pastuszka
> Assignee: Piotr Szuberski
> Priority: P3
> Labels: starter
> Time Spent: 50m
> Remaining Estimate: 0h
>
> There's an integration test for KinesisIO called KinesisReaderIT, but it is
> currently ignored, because it needs real Kinesis instance setup.
> As part of this task please:
> * setup real Kinesis environment on AWS for testing purposes
> * enable KinesisReaderIT test
> * setup jenkins, so that it passes all KinesisTestOptions when running
> integration tests
> This is a follow up to BEAM-461 requested by [[email protected]] in
> https://github.com/apache/incubator-beam/pull/687/
--
This message was sent by Atlassian Jira
(v8.3.4#803005)