You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ag...@apache.org on 2018/04/05 17:53:45 UTC

[geode] 06/23: GEODE-4947: Convert distributed tests to use mysql and postgres

This is an automated email from the ASF dual-hosted git repository.

agingade pushed a commit to branch feature/GEODE-4947
in repository https://gitbox.apache.org/repos/asf/geode.git

commit c3ac6b30ca81126394160f22c9f68da3896f0c21
Author: Nick Reich <nr...@pivotal.io>
AuthorDate: Wed Mar 28 14:57:26 2018 -0700

    GEODE-4947: Convert distributed tests to use mysql and postgres
---
 ...JdbcDUnitTest.java => JdbcDistributedTest.java} | 87 ++++++++++------------
 .../connectors/jdbc/MySqlJdbcDistributedTest.java  | 85 +++++++++++++++++++++
 .../jdbc/PostgresJdbcDistributedTest.java          | 85 +++++++++++++++++++++
 .../jdbc/internal/SqlStatementFactoryTest.java     |  8 +-
 .../junit/rules/SqlDatabaseConnectionRule.java     |  2 +-
 5 files changed, 213 insertions(+), 54 deletions(-)

diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/JdbcDUnitTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/JdbcDistributedTest.java
similarity index 91%
rename from geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/JdbcDUnitTest.java
rename to geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/JdbcDistributedTest.java
index 90cfd17..ed748f7 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/JdbcDUnitTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/JdbcDistributedTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.connectors.jdbc;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
+import java.io.IOException;
 import java.io.Serializable;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -50,6 +51,7 @@ import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
 import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.DatabaseConnectionRule;
 import org.apache.geode.test.junit.rules.GfshCommandRule;
 import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
 
@@ -57,15 +59,13 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
  * End-to-end dunits for jdbc connector
  */
 @Category(DistributedTest.class)
-public class JdbcDUnitTest implements Serializable {
+public abstract class JdbcDistributedTest implements Serializable {
 
-  private static final String DB_NAME = "DerbyDB";
+  static final String DB_NAME = "test";
   private static final String TABLE_NAME = "employees";
   private static final String REGION_NAME = "employees";
-  private static final String CONNECTION_URL = "jdbc:derby:memory:" + DB_NAME + ";create=true";
   private static final String CONNECTION_NAME = "TestConnection";
 
-
   @Rule
   public transient GfshCommandRule gfsh = new GfshCommandRule();
 
@@ -81,70 +81,59 @@ public class JdbcDUnitTest implements Serializable {
 
   private MemberVM server;
   private MemberVM locator;
+  private String connectionUrl;
 
   @Before
   public void setup() throws Exception {
     locator = startupRule.startLocatorVM(0);
     gfsh.connectAndVerify(locator);
+    connectionUrl = getConnectionUrl();
   }
 
+  public abstract Connection getConnection() throws SQLException;
+
+  public abstract String getConnectionUrl() throws IOException, InterruptedException;
+
   private void createTable() throws SQLException {
     server = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locator.getPort()));
-    server.invoke(() -> {
-      Connection connection = DriverManager.getConnection(CONNECTION_URL);
+      Connection connection = getConnection();
       Statement statement = connection.createStatement();
       statement.execute("Create Table " + TABLE_NAME
           + " (id varchar(10) primary key not null, name varchar(10), age int)");
-    });
   }
 
   private void createTableForAllSupportedFields() throws SQLException {
     server = startupRule.startServerVM(1,
         x -> x.withConnectionToLocator(locator.getPort()).withPDXReadSerialized());
-    server.invoke(() -> {
-      Connection connection = DriverManager.getConnection(CONNECTION_URL);
+      Connection connection = getConnection();
       Statement statement = connection.createStatement();
-      statement
-          .execute("Create Table \"" + TABLE_NAME + "\" (\"id\" varchar(10) primary key not null, "
-              + "aboolean smallint, " + "abyte smallint, " + "ashort smallint, " + "anint int, "
-              + "\"along\" bigint, " + "\"aFloat\" float, " + "\"ADOUBLE\" double, "
-              + "astring varchar(10), " + "adate timestamp, " + "anobject varchar(20), "
-              + "abytearray blob(100), " + "achar char(1))");
-    });
+      createSupportedFieldsTable(statement, TABLE_NAME);
   }
 
-  private void insertNullDataForAllSupportedFieldsTable(String key) {
-    server.invoke(() -> {
-      Connection connection = DriverManager.getConnection(CONNECTION_URL);
+  protected abstract void createSupportedFieldsTable(Statement statement, String tableName)
+      throws SQLException;
+
+  private void insertNullDataForAllSupportedFieldsTable(String key) throws SQLException {
+      Connection connection = DriverManager.getConnection(connectionUrl);
 
       String insertQuery =
-          "Insert into \"" + TABLE_NAME + "\" values (" + "?,?,?,?,?,?,?,?,?,?,?,?,?)";
+          "Insert into " + TABLE_NAME + " values (" + "?,?,?,?,?,?,?,?,?,?,?,?,?)";
       System.out.println("### Query is :" + insertQuery);
       PreparedStatement statement = connection.prepareStatement(insertQuery);
-      statement.setObject(1, key);
-      statement.setNull(2, Types.SMALLINT);
-      statement.setNull(3, Types.SMALLINT);
-      statement.setNull(4, Types.SMALLINT);
-      statement.setNull(5, Types.INTEGER);
-      statement.setNull(6, Types.BIGINT);
-      statement.setNull(7, Types.FLOAT);
-      statement.setNull(8, Types.DOUBLE);
-      statement.setNull(9, Types.VARCHAR);
-      statement.setNull(10, Types.TIMESTAMP);
-      statement.setNull(11, Types.VARCHAR);
-      statement.setNull(12, Types.BLOB);
-      statement.setNull(13, Types.CHAR);
+    createNullStatement(key, statement);
 
       statement.execute();
-    });
   }
 
-  private void insertDataForAllSupportedFieldsTable(String key, ClassWithSupportedPdxFields data) {
-    server.invoke(() -> {
-      Connection connection = DriverManager.getConnection(CONNECTION_URL);
+  protected abstract void createNullStatement(String key, PreparedStatement statement)
+      throws SQLException;
+
+  private void insertDataForAllSupportedFieldsTable(String key, ClassWithSupportedPdxFields data)
+      throws SQLException {
+      Connection connection = DriverManager.getConnection(connectionUrl);
 
       String insertQuery =
-          "Insert into \"" + TABLE_NAME + "\" values (" + "?,?,?,?,?,?,?,?,?,?,?,?,?)";
+          "Insert into " + TABLE_NAME + " values (" + "?,?,?,?,?,?,?,?,?,?,?,?,?)";
       System.out.println("### Query is :" + insertQuery);
       PreparedStatement statement = connection.prepareStatement(insertQuery);
       statement.setObject(1, key);
@@ -156,24 +145,24 @@ public class JdbcDUnitTest implements Serializable {
       statement.setObject(7, data.getAfloat());
       statement.setObject(8, data.getAdouble());
       statement.setObject(9, data.getAstring());
-      statement.setObject(10, data.getAdate());
+      statement.setObject(10, new java.sql.Timestamp(data.getAdate().getTime()));
       statement.setObject(11, data.getAnobject());
       statement.setObject(12, data.getAbytearray());
       statement.setObject(13, new Character(data.getAchar()).toString());
 
       statement.execute();
-    });
   }
 
   @After
   public void tearDown() throws Exception {
-    server.invoke(() -> {
+    closeDB();
+    /*server.invoke(() -> {
       closeDB();
-    });
+    });*/
   }
 
   private void closeDB() throws SQLException {
-    try (Connection connection = DriverManager.getConnection(CONNECTION_URL)) {
+    try (Connection connection = DriverManager.getConnection(connectionUrl)) {
       try (Statement statement = connection.createStatement()) {
         try {
           statement.execute("Drop table " + TABLE_NAME);
@@ -181,7 +170,7 @@ public class JdbcDUnitTest implements Serializable {
         }
 
         try {
-          statement.execute("Drop table \"" + TABLE_NAME + "\"");
+          statement.execute("Drop table " + TABLE_NAME + "");
         } catch (SQLException ignore) {
         }
       }
@@ -267,7 +256,7 @@ public class JdbcDUnitTest implements Serializable {
   public void verifyDateToDate() throws Exception {
     server = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locator.getPort()));
     server.invoke(() -> {
-      Connection connection = DriverManager.getConnection(CONNECTION_URL);
+      Connection connection = DriverManager.getConnection(connectionUrl);
       Statement statement = connection.createStatement();
       statement.execute(
           "Create Table " + TABLE_NAME + " (id varchar(10) primary key not null, mydate date)");
@@ -302,7 +291,7 @@ public class JdbcDUnitTest implements Serializable {
   public void verifyDateToTime() throws Exception {
     server = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locator.getPort()));
     server.invoke(() -> {
-      Connection connection = DriverManager.getConnection(CONNECTION_URL);
+      Connection connection = DriverManager.getConnection(connectionUrl);
       Statement statement = connection.createStatement();
       statement.execute(
           "Create Table " + TABLE_NAME + " (id varchar(10) primary key not null, mytime time)");
@@ -337,7 +326,7 @@ public class JdbcDUnitTest implements Serializable {
   public void verifyDateToTimestamp() throws Exception {
     server = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locator.getPort()));
     server.invoke(() -> {
-      Connection connection = DriverManager.getConnection(CONNECTION_URL);
+      Connection connection = DriverManager.getConnection(connectionUrl);
       Statement statement = connection.createStatement();
       statement.execute("Create Table " + TABLE_NAME
           + " (id varchar(10) primary key not null, mytimestamp timestamp)");
@@ -614,7 +603,7 @@ public class JdbcDUnitTest implements Serializable {
 
   private void createJdbcConnection() {
     final String commandStr =
-        "create jdbc-connection --name=" + CONNECTION_NAME + " --url=" + CONNECTION_URL;
+        "create jdbc-connection --name=" + CONNECTION_NAME + " --url=" + connectionUrl;
     gfsh.executeAndAssertThat(commandStr).statusIsSuccess();
   }
 
@@ -661,7 +650,7 @@ public class JdbcDUnitTest implements Serializable {
 
   private void assertTableHasEmployeeData(int size, PdxInstance employee, String key)
       throws SQLException {
-    Connection connection = DriverManager.getConnection(CONNECTION_URL);
+    Connection connection = DriverManager.getConnection(connectionUrl);
     Statement statement = connection.createStatement();
     Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> {
       assertThat(getRowCount(statement, TABLE_NAME)).isEqualTo(size);
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/MySqlJdbcDistributedTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/MySqlJdbcDistributedTest.java
new file mode 100644
index 0000000..c86748d
--- /dev/null
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/MySqlJdbcDistributedTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.geode.connectors.jdbc;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Types;
+
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.MySqlConnectionRule;
+import org.apache.geode.test.junit.rules.SqlDatabaseConnectionRule;
+
+/**
+ * End-to-end dunits for jdbc connector
+ */
+@Category(DistributedTest.class)
+public class MySqlJdbcDistributedTest extends JdbcDistributedTest {
+
+  @ClassRule
+  public static transient SqlDatabaseConnectionRule dbRule = createConnectionRule();// = new MySqlConnectionRule.Builder().file("src/test/resources/docker/mysql.yml").serviceName("db").port(3306).database(DB_NAME).build();
+
+  private static SqlDatabaseConnectionRule createConnectionRule() {
+    try {
+      return new MySqlConnectionRule.Builder().file("src/test/resources/docker/mysql.yml")
+          .serviceName("db").port(3306).database(DB_NAME).build();
+    } catch (IllegalStateException e) {
+      return null;
+    }
+  }
+
+  @Override
+  public Connection getConnection() throws SQLException {
+    return dbRule.getConnection();
+  }
+
+  @Override
+  public String getConnectionUrl() {
+    return dbRule.getConnectionUrl();
+  }
+
+  @Override
+  protected void createSupportedFieldsTable(Statement statement, String tableName)
+      throws SQLException {
+    statement
+        .execute("CREATE TABLE " + tableName + " (id varchar(10) primary key not null, "
+            + "aboolean smallint, " + "abyte smallint, " + "ashort smallint, " + "anint int, "
+            + "along bigint, " + "aFloat float, " + "ADOUBLE double, "
+            + "astring varchar(10), " + "adate datetime, " + "anobject varchar(20), "
+            + "abytearray blob(100), " + "achar char(1))");
+  }
+
+  @Override
+  protected void createNullStatement(String key, PreparedStatement statement) throws SQLException {
+    statement.setObject(1, key);
+    statement.setNull(2, Types.BOOLEAN);
+    statement.setNull(3, Types.SMALLINT);
+    statement.setNull(4, Types.SMALLINT);
+    statement.setNull(5, Types.INTEGER);
+    statement.setNull(6, Types.BIGINT);
+    statement.setNull(7, Types.FLOAT);
+    statement.setNull(8, Types.DOUBLE);
+    statement.setNull(9, Types.VARCHAR);
+    statement.setNull(10, Types.TIMESTAMP);
+    statement.setNull(11, Types.VARCHAR);
+    statement.setNull(12, Types.BLOB);
+    statement.setNull(13, Types.CHAR);
+  }
+}
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/PostgresJdbcDistributedTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/PostgresJdbcDistributedTest.java
new file mode 100644
index 0000000..8a91407
--- /dev/null
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/PostgresJdbcDistributedTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.geode.connectors.jdbc;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Types;
+
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.PostgresConnectionRule;
+import org.apache.geode.test.junit.rules.SqlDatabaseConnectionRule;
+
+/**
+ * End-to-end dunits for jdbc connector
+ */
+@Category(DistributedTest.class)
+public class PostgresJdbcDistributedTest extends JdbcDistributedTest {
+
+  @ClassRule
+  public static transient SqlDatabaseConnectionRule dbRule = createConnectionRule();// = new MySqlConnectionRule.Builder().file("src/test/resources/docker/mysql.yml").serviceName("db").port(3306).database(DB_NAME).build();
+
+  private static SqlDatabaseConnectionRule createConnectionRule() {
+    try {
+      return new PostgresConnectionRule.Builder().file("src/test/resources/docker/postgres.yml")
+          .serviceName("db").port(5432).database(DB_NAME).build();
+    } catch (IllegalStateException e) {
+      return null;
+    }
+  }
+
+  @Override
+  public Connection getConnection() throws SQLException {
+    return dbRule.getConnection();
+  }
+
+  @Override
+  public String getConnectionUrl() {
+    return dbRule.getConnectionUrl();
+  }
+
+  @Override
+  protected void createSupportedFieldsTable(Statement statement, String tableName)
+      throws SQLException {
+    statement
+        .execute("CREATE TABLE " + tableName + " (id varchar(10) primary key not null, "
+            + "aboolean boolean, " + "abyte smallint, " + "ashort smallint, " + "anint int, "
+            + "along bigint, " + "aFloat float, " + "ADOUBLE double precision, "
+            + "astring varchar(10), " + "adate timestamp, " + "anobject varchar(20), "
+            + "abytearray bytea, " + "achar char(1))");
+  }
+
+  @Override
+  protected void createNullStatement(String key, PreparedStatement statement) throws SQLException {
+    statement.setObject(1, key);
+    statement.setNull(2, Types.BOOLEAN);
+    statement.setNull(3, Types.SMALLINT);
+    statement.setNull(4, Types.SMALLINT);
+    statement.setNull(5, Types.INTEGER);
+    statement.setNull(6, Types.BIGINT);
+    statement.setNull(7, Types.FLOAT);
+    statement.setNull(8, Types.DOUBLE);
+    statement.setNull(9, Types.VARCHAR);
+    statement.setNull(10, Types.TIMESTAMP);
+    statement.setNull(11, Types.VARCHAR);
+    statement.setNull(12, Types.ARRAY);
+    statement.setNull(13, Types.CHAR);
+  }
+}
diff --git a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/SqlStatementFactoryTest.java b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/SqlStatementFactoryTest.java
index 3bc3ae9..59636f9 100644
--- a/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/SqlStatementFactoryTest.java
+++ b/geode-connectors/src/test/java/org/apache/geode/connectors/jdbc/internal/SqlStatementFactoryTest.java
@@ -44,7 +44,7 @@ public class SqlStatementFactoryTest {
   @Test
   public void getSelectQueryString() throws Exception {
     String expectedStatement =
-        String.format("SELECT * FROM \"%s\" WHERE \"%s\" = ?", TABLE_NAME, KEY_COLUMN_NAME);
+        String.format("SELECT * FROM %s WHERE %s = ?", TABLE_NAME, KEY_COLUMN_NAME);
     List<ColumnValue> keyColumn = new ArrayList<>();
     keyColumn.add(new ColumnValue(true, KEY_COLUMN_NAME, null, 0));
 
@@ -56,7 +56,7 @@ public class SqlStatementFactoryTest {
   @Test
   public void getDestroySqlString() throws Exception {
     String expectedStatement =
-        String.format("DELETE FROM \"%s\" WHERE \"%s\" = ?", TABLE_NAME, KEY_COLUMN_NAME);
+        String.format("DELETE FROM %s WHERE %s = ?", TABLE_NAME, KEY_COLUMN_NAME);
     List<ColumnValue> keyColumn = new ArrayList<>();
     keyColumn.add(new ColumnValue(true, KEY_COLUMN_NAME, null, 0));
 
@@ -68,7 +68,7 @@ public class SqlStatementFactoryTest {
   @Test
   public void getUpdateSqlString() throws Exception {
     String expectedStatement = String.format(
-        "UPDATE \"%s\" SET \"%s\" = ?, \"%s\" = ? WHERE \"%s\" = ?", TABLE_NAME,
+        "UPDATE %s SET %s = ?, %s = ? WHERE %s = ?", TABLE_NAME,
         columnValues.get(0).getColumnName(), columnValues.get(2).getColumnName(), KEY_COLUMN_NAME);
 
     String statement = factory.createUpdateSqlString(TABLE_NAME, columnValues);
@@ -79,7 +79,7 @@ public class SqlStatementFactoryTest {
   @Test
   public void getInsertSqlString() throws Exception {
     String expectedStatement =
-        String.format("INSERT INTO \"%s\" (\"%s\", \"%s\", \"%s\") VALUES (?,?,?)", TABLE_NAME,
+        String.format("INSERT INTO %s (%s, %s, %s) VALUES (?,?,?)", TABLE_NAME,
             columnValues.get(0).getColumnName(), columnValues.get(1).getColumnName(),
             columnValues.get(2).getColumnName());
 
diff --git a/geode-connectors/src/test/java/org/apache/geode/test/junit/rules/SqlDatabaseConnectionRule.java b/geode-connectors/src/test/java/org/apache/geode/test/junit/rules/SqlDatabaseConnectionRule.java
index 96fae8a..628dfb3 100644
--- a/geode-connectors/src/test/java/org/apache/geode/test/junit/rules/SqlDatabaseConnectionRule.java
+++ b/geode-connectors/src/test/java/org/apache/geode/test/junit/rules/SqlDatabaseConnectionRule.java
@@ -79,7 +79,7 @@ public abstract class SqlDatabaseConnectionRule extends ExternalResource
     private int port;
     private String dbName;
 
-    public abstract DatabaseConnectionRule build();
+    public abstract SqlDatabaseConnectionRule build();
 
     public Builder file(String filePath) {
       this.filePath = filePath;

-- 
To stop receiving notification emails like this one, please contact
agingade@apache.org.