[ 
https://issues.apache.org/jira/browse/BEAM-4137?focusedWorklogId=108941&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-108941
 ]

ASF GitHub Bot logged work on BEAM-4137:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 05/Jun/18 06:16
            Start Date: 05/Jun/18 06:16
    Worklog Time Spent: 10m 
      Work Description: iemejia closed pull request #5548: [BEAM-4137] Remove 
MongoDB specific options from common/IOTestPipelin…
URL: https://github.com/apache/beam/pull/5548
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
 
b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
index 1260167eca1..89b7ae81bc5 100644
--- 
a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
+++ 
b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
@@ -114,23 +114,4 @@
   String getCharset();
 
   void setCharset(String charset);
-
-  /* MongoDB */
-  @Description("MongoDB host (host name/ip address)")
-  @Default.String("mongodb-host")
-  String getMongoDBHostName();
-
-  void setMongoDBHostName(String host);
-
-  @Description("Port for MongoDB")
-  @Default.Integer(27017)
-  Integer getMongoDBPort();
-
-  void setMongoDBPort(Integer port);
-
-  @Description("Mongo database name")
-  @Default.String("beam")
-  String getMongoDBDatabaseName();
-
-  void setMongoDBDatabaseName(String name);
 }
diff --git 
a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBIOIT.java
 
b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBIOIT.java
index 23f5fe40fed..1753ad5489c 100644
--- 
a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBIOIT.java
+++ 
b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBIOIT.java
@@ -26,6 +26,8 @@
 import org.apache.beam.sdk.io.GenerateSequence;
 import org.apache.beam.sdk.io.common.HashingFn;
 import org.apache.beam.sdk.io.common.IOTestPipelineOptions;
+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.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -34,19 +36,19 @@
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.values.PCollection;
 import org.bson.Document;
-import org.junit.After;
+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;
 
-
 /**
  * A test of {@link org.apache.beam.sdk.io.mongodb.MongoDbIO} on an 
independent Mongo instance.
  *
  * <p>This test requires a running instance of MongoDB. Pass in connection 
information using
  * PipelineOptions:
+ *
  * <pre>
  *  ./gradlew integrationTest -p sdks/java/io/mongodb 
-DintegrationTestPipelineOptions='[
  *  "--mongoDBHostName=1.2.3.4",
@@ -57,75 +59,87 @@
  *  -DintegrationTestRunner=direct
  * </pre>
  *
- * <p>Please see 'build_rules.gradle' file for instructions regarding
- * running this test using Beam performance testing framework.</p>
+ * <p>Please see 'build_rules.gradle' file for instructions regarding running 
this test using Beam
+ * performance testing framework.
  */
 @RunWith(JUnit4.class)
 public class MongoDBIOIT {
 
-  private static final Map<Integer, String> EXPECTED_HASHES = ImmutableMap.of(
-    1000, "75a0d5803418444e76ae5b421662764c",
-    100_000, "3bc762dc1c291904e3c7f577774c6276",
-    10_000_000, "e5e0503902018c83e8c8977ef437feba"
-  );
+  /** MongoDBIOIT options. */
+  public interface MongoDBPipelineOptions extends IOTestPipelineOptions {
+    @Description("MongoDB host (host name/ip address)")
+    @Default.String("mongodb-host")
+    String getMongoDBHostName();
 
-  private static int numberOfRecords;
+    void setMongoDBHostName(String host);
 
-  private static String host;
+    @Description("Port for MongoDB")
+    @Default.Integer(27017)
+    Integer getMongoDBPort();
 
-  private static Integer port;
+    void setMongoDBPort(Integer port);
 
-  private static String database;
+    @Description("Mongo database name")
+    @Default.String("beam")
+    String getMongoDBDatabaseName();
 
-  private static String collection;
+    void setMongoDBDatabaseName(String name);
+  }
 
-  @Rule
-  public final TestPipeline writePipeline = TestPipeline.create();
+  private static final Map<Integer, String> EXPECTED_HASHES =
+      ImmutableMap.of(
+          1000, "75a0d5803418444e76ae5b421662764c",
+          100_000, "3bc762dc1c291904e3c7f577774c6276",
+          10_000_000, "e5e0503902018c83e8c8977ef437feba");
 
-  @Rule
-  public final TestPipeline readPipeline = TestPipeline.create();
+  private static MongoDBPipelineOptions options;
+  private static String collection;
+
+  @Rule public final TestPipeline writePipeline = TestPipeline.create();
+  @Rule public final TestPipeline readPipeline = TestPipeline.create();
 
   @BeforeClass
   public static void setUp() {
-    PipelineOptionsFactory.register(IOTestPipelineOptions.class);
-    IOTestPipelineOptions options = TestPipeline.testingPipelineOptions()
-      .as(IOTestPipelineOptions.class);
-
-    numberOfRecords = options.getNumberOfRecords();
-    host = options.getMongoDBHostName();
-    port = options.getMongoDBPort();
-    database = options.getMongoDBDatabaseName();
+    PipelineOptionsFactory.register(MongoDBPipelineOptions.class);
+    options = 
TestPipeline.testingPipelineOptions().as(MongoDBPipelineOptions.class);
     collection = String.format("test_%s", new Date().getTime());
   }
 
-  @After
-  public void tearDown() {
-    new MongoClient(host).getDatabase(database).drop();
+  @AfterClass
+  public static void tearDown() {
+    new MongoClient(options.getMongoDBHostName())
+        .getDatabase(options.getMongoDBDatabaseName())
+        .drop();
   }
 
   @Test
   public void testWriteAndRead() {
-    String mongoUrl = String.format("mongodb://%s:%s", host, port);
+    final String mongoUrl =
+        String.format("mongodb://%s:%s", options.getMongoDBHostName(), 
options.getMongoDBPort());
 
     writePipeline
-      .apply("Generate sequence", GenerateSequence.from(0).to(numberOfRecords))
-      .apply("Produce documents", MapElements.via(new LongToDocumentFn()))
-      .apply("Write documents to MongoDB", MongoDbIO.write()
-        .withUri(mongoUrl)
-        .withDatabase(database)
-        .withCollection(collection));
-
+        .apply("Generate sequence", 
GenerateSequence.from(0).to(options.getNumberOfRecords()))
+        .apply("Produce documents", MapElements.via(new LongToDocumentFn()))
+        .apply(
+            "Write documents to MongoDB",
+            MongoDbIO.write()
+                .withUri(mongoUrl)
+                .withDatabase(options.getMongoDBDatabaseName())
+                .withCollection(collection));
     writePipeline.run().waitUntilFinish();
 
-    PCollection<String> consolidatedHashcode  = readPipeline
-      .apply("Read all documents", MongoDbIO.read()
-        .withUri(mongoUrl)
-        .withDatabase(database)
-        .withCollection(collection))
-      .apply("Map documents to Strings", MapElements.via(new 
DocumentToStringFn()))
-      .apply("Calculate hashcode", Combine.globally(new HashingFn()));
-
-    String expectedHash = getHashForRecordCount(numberOfRecords, 
EXPECTED_HASHES);
+    PCollection<String> consolidatedHashcode =
+        readPipeline
+            .apply(
+                "Read all documents",
+                MongoDbIO.read()
+                    .withUri(mongoUrl)
+                    .withDatabase(options.getMongoDBDatabaseName())
+                    .withCollection(collection))
+            .apply("Map documents to Strings", MapElements.via(new 
DocumentToStringFn()))
+            .apply("Calculate hashcode", Combine.globally(new HashingFn()));
+
+    String expectedHash = getHashForRecordCount(options.getNumberOfRecords(), 
EXPECTED_HASHES);
     PAssert.thatSingleton(consolidatedHashcode).isEqualTo(expectedHash);
 
     readPipeline.run().waitUntilFinish();


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 108941)
    Time Spent: 2h 10m  (was: 2h)

> Split IOTestPipelineOptions to multiple, test-specific files
> ------------------------------------------------------------
>
>                 Key: BEAM-4137
>                 URL: https://issues.apache.org/jira/browse/BEAM-4137
>             Project: Beam
>          Issue Type: Improvement
>          Components: testing
>            Reporter: Łukasz Gajowy
>            Assignee: Łukasz Gajowy
>            Priority: Minor
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> Currently we have one big IOTestPipelineOptions interface that is used in 
> many IOITs. It contains test specific options that should rather be located 
> next to testing classes, not in a generic file. Let's split this. 
> Additionally, besides separation of concerns,  this will allow adding 
> test-specific @Default and @Required annotations and validate the options 
> better. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to