You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by jb...@apache.org on 2017/03/20 13:28:56 UTC

[1/2] beam git commit: [BEAM-1644] Move PipelineOptions for IO ITs into shared location.

Repository: beam
Updated Branches:
  refs/heads/master faa9645d2 -> 53d7682d0


[BEAM-1644] Move PipelineOptions for IO ITs into shared location.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/3b52fcd8
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/3b52fcd8
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/3b52fcd8

Branch: refs/heads/master
Commit: 3b52fcd89767f496b0825932d276f8f65d6342cb
Parents: faa9645
Author: Stephen Sisk <si...@google.com>
Authored: Wed Mar 15 14:31:49 2017 -0700
Committer: Jean-Baptiste Onofr� <jb...@apache.org>
Committed: Mon Mar 20 14:10:41 2017 +0100

----------------------------------------------------------------------
 pom.xml                                         | 13 ++++
 sdks/java/io/common/pom.xml                     | 38 ++++++++++
 .../sdk/io/common/IOTestPipelineOptions.java    | 79 ++++++++++++++++++++
 sdks/java/io/elasticsearch/pom.xml              |  7 ++
 .../sdk/io/elasticsearch/ElasticsearchIOIT.java |  7 +-
 .../elasticsearch/ElasticsearchTestDataSet.java | 11 +--
 .../elasticsearch/ElasticsearchTestOptions.java | 46 ------------
 sdks/java/io/jdbc/pom.xml                       |  6 ++
 .../org/apache/beam/sdk/io/jdbc/JdbcIOIT.java   | 11 +--
 .../beam/sdk/io/jdbc/JdbcTestDataSet.java       | 12 +--
 .../beam/sdk/io/jdbc/PostgresTestOptions.java   | 60 ---------------
 sdks/java/io/pom.xml                            |  1 +
 12 files changed, 167 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index caaaaec..65c5012 100644
--- a/pom.xml
+++ b/pom.xml
@@ -368,6 +368,19 @@
 
       <dependency>
         <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-io-common</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-io-common</artifactId>
+        <classifier>tests</classifier>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.beam</groupId>
         <artifactId>beam-sdks-java-io-elasticsearch</artifactId>
         <version>${project.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/common/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/common/pom.xml b/sdks/java/io/common/pom.xml
new file mode 100644
index 0000000..fa51b47
--- /dev/null
+++ b/sdks/java/io/common/pom.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-io-parent</artifactId>
+        <version>0.7.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>beam-sdks-java-io-common</artifactId>
+    <name>Apache Beam :: SDKs :: Java :: IO :: Common</name>
+    <description>Code used by all Beam IOs</description>
+    <dependencies>
+      <dependency>
+        <groupId>org.apache.beam</groupId>
+        <artifactId>beam-sdks-java-core</artifactId>
+      </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..d3915c9
--- /dev/null
+++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java
@@ -0,0 +1,79 @@
+/*
+ * 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.common;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+
+/**
+ * This shared set of options is used so that the full suite of IO tests can be run in one pass -
+ * if a test tries to read TestPipelineOptions, it must be able to understand all the options
+ * that were passed on the command line.
+ */
+public interface IOTestPipelineOptions extends TestPipelineOptions {
+  /* Postgres */
+  @Description("Server name for postgres server (host name/ip address)")
+  @Default.String("postgres-server-name")
+  String getPostgresServerName();
+  void setPostgresServerName(String value);
+
+  @Description("Username for postgres server")
+  @Default.String("postgres-username")
+  String getPostgresUsername();
+  void setPostgresUsername(String value);
+
+  // Note that passwords are not as secure an authentication as other methods, and used here for
+  // a test environment only.
+  @Description("Password for postgres server")
+  @Default.String("postgres-password")
+  String getPostgresPassword();
+  void setPostgresPassword(String value);
+
+  @Description("Database name for postgres server")
+  @Default.String("postgres-database-name")
+  String getPostgresDatabaseName();
+  void setPostgresDatabaseName(String value);
+
+  @Description("Port for postgres server")
+  @Default.Integer(0)
+  Integer getPostgresPort();
+  void setPostgresPort(Integer value);
+
+  @Description("Whether the postgres server uses SSL")
+  @Default.Boolean(true)
+  Boolean getPostgresSsl();
+  void setPostgresSsl(Boolean value);
+
+  /* Elasticsearch */
+  @Description("Server name for Elasticsearch server (host name/ip address)")
+  @Default.String("elasticsearch-server-name")
+  String getElasticsearchServer();
+  void setElasticsearchServer(String value);
+
+  @Description("Http port for elasticsearch server")
+  @Default.Integer(9200)
+  Integer getElasticsearchHttpPort();
+  void setElasticsearchHttpPort(Integer value);
+
+  @Description("Tcp port for elasticsearch server")
+  @Default.Integer(9300)
+  Integer getElasticsearchTcpPort();
+  void setElasticsearchTcpPort(Integer value);
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/elasticsearch/pom.xml b/sdks/java/io/elasticsearch/pom.xml
index d09b660..bca8b22 100644
--- a/sdks/java/io/elasticsearch/pom.xml
+++ b/sdks/java/io/elasticsearch/pom.xml
@@ -144,6 +144,13 @@
       <scope>test</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-io-common</artifactId>
+      <scope>test</scope>
+      <classifier>tests</classifier>
+    </dependency>
+
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
index bd6c503..b5fec17 100644
--- a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
+++ b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.List;
 import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.common.IOTestPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.SourceTestUtils;
@@ -57,14 +58,14 @@ import org.slf4j.LoggerFactory;
 public class ElasticsearchIOIT {
   private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIOIT.class);
   private static TransportClient client;
-  private static ElasticsearchTestOptions options;
+  private static IOTestPipelineOptions options;
   private static ElasticsearchIO.ConnectionConfiguration readConnectionConfiguration;
   @Rule public TestPipeline pipeline = TestPipeline.create();
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    PipelineOptionsFactory.register(ElasticsearchTestOptions.class);
-    options = TestPipeline.testingPipelineOptions().as(ElasticsearchTestOptions.class);
+    PipelineOptionsFactory.register(IOTestPipelineOptions.class);
+    options = TestPipeline.testingPipelineOptions().as(IOTestPipelineOptions.class);
     client = ElasticsearchTestDataSet.getClient(options);
     readConnectionConfiguration =
         ElasticsearchTestDataSet.getConnectionConfiguration(

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java
index bd8141f..3a9aae6 100644
--- a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java
+++ b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.io.elasticsearch;
 import static java.net.InetAddress.getByName;
 
 import java.io.IOException;
+import org.apache.beam.sdk.io.common.IOTestPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.elasticsearch.client.transport.TransportClient;
 import org.elasticsearch.common.transport.InetSocketTransportAddress;
@@ -58,9 +59,9 @@ public class ElasticsearchTestDataSet {
    *     Elasticsearch as shown above.
    */
   public static void main(String[] args) throws Exception {
-    PipelineOptionsFactory.register(ElasticsearchTestOptions.class);
-    ElasticsearchTestOptions options =
-        PipelineOptionsFactory.fromArgs(args).as(ElasticsearchTestOptions.class);
+    PipelineOptionsFactory.register(IOTestPipelineOptions.class);
+    IOTestPipelineOptions options =
+        PipelineOptionsFactory.fromArgs(args).as(IOTestPipelineOptions.class);
 
     createAndPopulateIndex(getClient(options), ReadOrWrite.READ);
   }
@@ -72,7 +73,7 @@ public class ElasticsearchTestDataSet {
         (rOw == ReadOrWrite.READ) ? ES_INDEX : writeIndex, ES_TYPE, NUM_DOCS, client);
   }
 
-  public static TransportClient getClient(ElasticsearchTestOptions options) throws Exception {
+  public static TransportClient getClient(IOTestPipelineOptions options) throws Exception {
     TransportClient client =
         TransportClient.builder()
             .build()
@@ -84,7 +85,7 @@ public class ElasticsearchTestDataSet {
   }
 
   public static ElasticsearchIO.ConnectionConfiguration getConnectionConfiguration(
-      ElasticsearchTestOptions options, ReadOrWrite rOw) throws IOException {
+      IOTestPipelineOptions options, ReadOrWrite rOw) throws IOException {
     ElasticsearchIO.ConnectionConfiguration connectionConfiguration =
         ElasticsearchIO.ConnectionConfiguration.create(
             new String[] {

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestOptions.java b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestOptions.java
deleted file mode 100644
index df7c797..0000000
--- a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestOptions.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.elasticsearch;
-
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.testing.TestPipelineOptions;
-
-/**
- * These options can be used by a test connecting to an Elasticsearch instance to configure their
- * connection.
- */
-
-public interface ElasticsearchTestOptions extends TestPipelineOptions {
-    @Description("Server name for Elasticsearch server (host name/ip address)")
-    @Default.String("elasticsearch-server-name")
-    String getElasticsearchServer();
-    void setElasticsearchServer(String value);
-
-
-    @Description("Http port for elasticsearch server")
-    @Default.Integer(9200)
-    Integer getElasticsearchHttpPort();
-    void setElasticsearchHttpPort(Integer value);
-
-    @Description("Tcp port for elasticsearch server")
-    @Default.Integer(9300)
-    Integer getElasticsearchTcpPort();
-    void setElasticsearchTcpPort(Integer value);
-
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml
index f6c946f..6212c2f 100644
--- a/sdks/java/io/jdbc/pom.xml
+++ b/sdks/java/io/jdbc/pom.xml
@@ -164,5 +164,11 @@
       <version>9.4.1212.jre7</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-io-common</artifactId>
+      <scope>test</scope>
+      <classifier>tests</classifier>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
index a09db48..3eaa5b7 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
@@ -28,6 +28,7 @@ import java.util.List;
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.common.IOTestPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -72,9 +73,9 @@ public class JdbcIOIT {
 
   @BeforeClass
   public static void setup() throws SQLException {
-    PipelineOptionsFactory.register(PostgresTestOptions.class);
-    PostgresTestOptions options = TestPipeline.testingPipelineOptions()
-        .as(PostgresTestOptions.class);
+    PipelineOptionsFactory.register(IOTestPipelineOptions.class);
+    IOTestPipelineOptions options = TestPipeline.testingPipelineOptions()
+        .as(IOTestPipelineOptions.class);
 
     // We do dataSource set up in BeforeClass rather than Before since we don't need to create a new
     // dataSource for each test.
@@ -118,11 +119,11 @@ public class JdbcIOIT {
    */
   @Test
   public void testRead() throws SQLException {
-    String tableName = JdbcTestDataSet.READ_TABLE_NAME;
+    String writeTableName = JdbcTestDataSet.READ_TABLE_NAME;
 
     PCollection<KV<String, Integer>> output = pipeline.apply(JdbcIO.<KV<String, Integer>>read()
             .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(dataSource))
-            .withQuery("select name,id from " + tableName)
+            .withQuery("select name,id from " + writeTableName)
             .withRowMapper(new CreateKVOfNameAndId())
             .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())));
 

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestDataSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestDataSet.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestDataSet.java
index 2a5d6f4..0b88be2 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestDataSet.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestDataSet.java
@@ -22,6 +22,7 @@ import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Statement;
 import javax.sql.DataSource;
+import org.apache.beam.sdk.io.common.IOTestPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.postgresql.ds.PGSimpleDataSource;
 import org.slf4j.Logger;
@@ -47,18 +48,19 @@ public class JdbcTestDataSet {
    *   --postgresDatabaseName=myfancydb \
    *   --postgresPassword=yourpassword --postgresSsl=false" \
    *   -Dexec.classpathScope=test
-   * @param args Please pass options from PostgresTestOptions used for connection to postgres as
+   * @param args Please pass options from IOTestPipelineOptions used for connection to postgres as
    * shown above.
    */
   public static void main(String[] args) throws SQLException {
-    PipelineOptionsFactory.register(PostgresTestOptions.class);
-    PostgresTestOptions options =
-        PipelineOptionsFactory.fromArgs(args).as(PostgresTestOptions.class);
+    PipelineOptionsFactory.register(IOTestPipelineOptions.class);
+    IOTestPipelineOptions options =
+        PipelineOptionsFactory.fromArgs(args).as(IOTestPipelineOptions.class);
 
     createReadDataTable(getDataSource(options));
   }
 
-  public static PGSimpleDataSource getDataSource(PostgresTestOptions options) throws SQLException {
+  public static PGSimpleDataSource getDataSource(IOTestPipelineOptions options)
+      throws SQLException {
     PGSimpleDataSource dataSource = new PGSimpleDataSource();
 
     // Tests must receive parameters for connections from PipelineOptions

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/PostgresTestOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/PostgresTestOptions.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/PostgresTestOptions.java
deleted file mode 100644
index 5612d19..0000000
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/PostgresTestOptions.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.jdbc;
-
-import org.apache.beam.sdk.options.Default;
-import org.apache.beam.sdk.options.Description;
-import org.apache.beam.sdk.testing.TestPipelineOptions;
-
-/**
- * These options can be used by a test connecting to a postgres database to configure their
- * connection.
- */
-public interface PostgresTestOptions extends TestPipelineOptions {
-    @Description("Server name for postgres server (host name/ip address)")
-    @Default.String("postgres-server-name")
-    String getPostgresServerName();
-    void setPostgresServerName(String value);
-
-    @Description("Username for postgres server")
-    @Default.String("postgres-username")
-    String getPostgresUsername();
-    void setPostgresUsername(String value);
-
-    // Note that passwords are not as secure an authentication as other methods, and used here for
-    // a test environment only.
-    @Description("Password for postgres server")
-    @Default.String("postgres-password")
-    String getPostgresPassword();
-    void setPostgresPassword(String value);
-
-    @Description("Database name for postgres server")
-    @Default.String("postgres-database-name")
-    String getPostgresDatabaseName();
-    void setPostgresDatabaseName(String value);
-
-    @Description("Port for postgres server")
-    @Default.Integer(0)
-    Integer getPostgresPort();
-    void setPostgresPort(Integer value);
-
-    @Description("Whether the postgres server uses SSL")
-    @Default.Boolean(true)
-    Boolean getPostgresSsl();
-    void setPostgresSsl(Boolean value);
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/3b52fcd8/sdks/java/io/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml
index 64424c5..73fbba1 100644
--- a/sdks/java/io/pom.xml
+++ b/sdks/java/io/pom.xml
@@ -64,6 +64,7 @@
   </dependencyManagement>
 
   <modules>
+    <module>common</module>
     <module>elasticsearch</module>
     <module>google-cloud-platform</module>
     <module>hadoop-common</module>


[2/2] beam git commit: [BEAM-1644] This closes #2253

Posted by jb...@apache.org.
[BEAM-1644] This closes #2253


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/53d7682d
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/53d7682d
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/53d7682d

Branch: refs/heads/master
Commit: 53d7682d05794fd412d1c9eb46eab64e9353bd12
Parents: faa9645 3b52fcd
Author: Jean-Baptiste Onofr� <jb...@apache.org>
Authored: Mon Mar 20 14:28:45 2017 +0100
Committer: Jean-Baptiste Onofr� <jb...@apache.org>
Committed: Mon Mar 20 14:28:45 2017 +0100

----------------------------------------------------------------------
 pom.xml                                         | 13 ++++
 sdks/java/io/common/pom.xml                     | 38 ++++++++++
 .../sdk/io/common/IOTestPipelineOptions.java    | 79 ++++++++++++++++++++
 sdks/java/io/elasticsearch/pom.xml              |  7 ++
 .../sdk/io/elasticsearch/ElasticsearchIOIT.java |  7 +-
 .../elasticsearch/ElasticsearchTestDataSet.java | 11 +--
 .../elasticsearch/ElasticsearchTestOptions.java | 46 ------------
 sdks/java/io/jdbc/pom.xml                       |  6 ++
 .../org/apache/beam/sdk/io/jdbc/JdbcIOIT.java   | 11 +--
 .../beam/sdk/io/jdbc/JdbcTestDataSet.java       | 12 +--
 .../beam/sdk/io/jdbc/PostgresTestOptions.java   | 60 ---------------
 sdks/java/io/pom.xml                            |  1 +
 12 files changed, 167 insertions(+), 124 deletions(-)
----------------------------------------------------------------------