You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ya...@apache.org on 2020/12/07 22:33:33 UTC

[phoenix] branch master updated: PHOENIX-6246 : Remove flakes from PointInTimeQueryIT

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

yanxinyi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/master by this push:
     new 3b04c81  PHOENIX-6246 : Remove flakes from PointInTimeQueryIT
3b04c81 is described below

commit 3b04c8163a0ebbbec152bc8dd74544d7332d7639
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Mon Dec 7 18:02:17 2020 +0530

    PHOENIX-6246 : Remove flakes from PointInTimeQueryIT
    
    Signed-off-by: Xinyi Yan <ya...@apache.org>
---
 .../org/apache/phoenix/end2end/BaseQueryIT.java    |   5 +-
 .../apache/phoenix/end2end/PointInTimeQueryIT.java | 130 +--------------
 .../phoenix/end2end/PointInTimeScanQueryIT.java    | 179 +++++++++++++++++++++
 3 files changed, 184 insertions(+), 130 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
index 6e6435b..51c9696 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
@@ -70,7 +70,7 @@ public abstract class BaseQueryIT extends ParallelStatsDisabledIT {
                             + "    A_DATE)",
                     "CREATE LOCAL INDEX %s ON %s (a_integer) INCLUDE (" + "    A_STRING, "
                             + "    B_STRING, " + "    A_DATE)" };
-    protected static String[] INDEX_DDLS;
+    private static final String[] INDEX_DDLS;
     static {
         INDEX_DDLS = new String[GLOBAL_INDEX_DDLS.length + LOCAL_INDEX_DDLS.length];
         int i = 0;
@@ -88,7 +88,8 @@ public abstract class BaseQueryIT extends ParallelStatsDisabledIT {
     private static final Logger LOGGER = LoggerFactory.getLogger(BaseQueryIT.class);
 
     @BeforeParam
-    public static final void initTables(String idxDdl, boolean columnEncoded, boolean keepDeletedCells) throws Exception {
+    public static void initTables(String idxDdl, boolean columnEncoded,
+            boolean keepDeletedCells) throws Exception {
         StringBuilder optionBuilder = new StringBuilder();
         if (!columnEncoded) {
             optionBuilder.append("COLUMN_ENCODED_BYTES=0");
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
index 1702949..ba7fa15 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
@@ -21,8 +21,6 @@ import static org.apache.phoenix.util.TestUtil.A_VALUE;
 import static org.apache.phoenix.util.TestUtil.B_VALUE;
 import static org.apache.phoenix.util.TestUtil.C_VALUE;
 import static org.apache.phoenix.util.TestUtil.E_VALUE;
-import static org.apache.phoenix.util.TestUtil.ROW1;
-import static org.apache.phoenix.util.TestUtil.ROW4;
 import static org.apache.phoenix.util.TestUtil.ROW5;
 import static org.apache.phoenix.util.TestUtil.ROW6;
 import static org.apache.phoenix.util.TestUtil.ROW7;
@@ -36,18 +34,17 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.Statement;
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.List;
 import java.util.Properties;
 
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.runners.Parameterized.Parameters;
 
-import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
 
+@Category(NeedsOwnMiniClusterTest.class)
 public class PointInTimeQueryIT extends BaseQueryIT {
 
     @Parameters(name="PointInTimeQueryIT_{index},columnEncoded={1}")
@@ -122,120 +119,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             assertFalse(rs.next());
         }
     }
-    
-    @Test
-    public void testPointInTimeScan() throws Exception {
-        // Override value that was set at creation time
-        String url = getUrl();
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String upsertStmt =
-                "upsert into " + tableName +
-                " (" +
-                "    ORGANIZATION_ID, " +
-                "    ENTITY_ID, " +
-                "    A_INTEGER) " +
-                "VALUES (?, ?, ?)";
-        
-        try (Connection upsertConn = DriverManager.getConnection(url, props)) {
-
-            upsertConn.setAutoCommit(true); // Test auto commit
-            PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW4);
-            stmt.setInt(3, 5);
-            stmt.execute(); // should commit too
-        }
-        long upsert1Time = System.currentTimeMillis();
-        long timeDelta = 100;
-        Thread.sleep(timeDelta);
-        
-        try(Connection upsertConn = DriverManager.getConnection(url, props)) {
-            upsertConn.setAutoCommit(true); // Test auto commit
-            PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW4);
-            stmt.setInt(3, 9);
-            stmt.execute(); // should commit too
-            upsertConn.close();
-        }
-        
-        long queryTime = upsert1Time + timeDelta / 2;
-        String query = "SELECT organization_id, a_string AS a FROM " + tableName + " WHERE organization_id=? and a_integer = 5";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(queryTime));
-        try(Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(tenantId, rs.getString(1));
-            assertEquals(A_VALUE, rs.getString("a"));
-            assertTrue(rs.next());
-            assertEquals(tenantId, rs.getString(1));
-            assertEquals(B_VALUE, rs.getString(2));
-            assertFalse(rs.next());
-            conn.close();
-        }
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testPointInTimeLimitedScan() throws Exception {
-        // Override value that was set at creation time
-        String url = getUrl();
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String upsertStmt =
-                "upsert into " + tableName +
-                " (" +
-                "    ORGANIZATION_ID, " +
-                "    ENTITY_ID, " +
-                "    A_INTEGER) " +
-                "VALUES (?, ?, ?)";
-        try(Connection upsertConn = DriverManager.getConnection(url, props)) {
-
-            upsertConn.setAutoCommit(true); // Test auto commit
-            // Insert all rows at ts
-            PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW1);
-            stmt.setInt(3, 6);
-            stmt.execute(); // should commit too
-            upsertConn.close();
-        }
-        long upsert1Time = System.currentTimeMillis();
-        long timeDelta = 100;
-        Thread.sleep(timeDelta);
-
-        url = getUrl();
-        try(Connection upsertConn = DriverManager.getConnection(url, props)) {
-            upsertConn.setAutoCommit(true); // Test auto commit
-            // Insert all rows at ts
-            PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW1);
-            stmt.setInt(3, 0);
-            stmt.execute(); // should commit too
-            upsertConn.close();
-        }
-        
-        long queryTime = upsert1Time + timeDelta  / 2;
-        String query = "SELECT a_integer,b_string FROM " + tableName + " WHERE organization_id=? and a_integer <= 5 limit 2";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(queryTime));
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            List<List<Object>> expectedResultsA = Lists.newArrayList(
-                    Arrays.<Object>asList(2, C_VALUE),
-                    Arrays.<Object>asList( 3, E_VALUE));
-            List<List<Object>> expectedResultsB = Lists.newArrayList(
-                    Arrays.<Object>asList( 5, C_VALUE),
-                    Arrays.<Object>asList(4, B_VALUE));
-            // Since we're not ordering and we may be using a descending index, we don't
-            // know which rows we'll get back.
-            assertOneOfValuesEqualsResultSet(rs, expectedResultsA,expectedResultsB);
-            conn.close();
-        }
-    }
 
     @Test
     public void TestPointInTimeGroupedAggregation() throws Exception {
@@ -248,7 +131,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             upsertConn.setAutoCommit(true); // Test auto commit
             Statement stmt = upsertConn.createStatement();
             stmt.execute(updateStmt); // should commit too
-            upsertConn.close();
         }
         
         long upsert1Time = System.currentTimeMillis();
@@ -264,7 +146,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             pstmt.setString(2, ROW5);
             pstmt.setString(3, E_VALUE);
             pstmt.execute(); // should commit too
-            upsertConn.close();
         }
         
         long queryTime = upsert1Time + timeDelta / 2;
@@ -285,7 +166,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             assertEquals(C_VALUE, rs.getString(1));
             assertEquals(2, rs.getInt(2));
             assertFalse(rs.next());
-            conn.close();
         }
     }
     
@@ -310,7 +190,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             stmt.setString(3, E_VALUE);
             stmt.execute();
             upsertConn.commit();
-            upsertConn.close();
         }
         long upsert1Time = System.currentTimeMillis();
         long timeDelta = 100;
@@ -323,7 +202,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             stmt.setString(2, ROW6);
             stmt.setString(3, E_VALUE);
             stmt.execute();
-            upsertConn.close();
         }
         
         long queryTime = upsert1Time + timeDelta / 2;
@@ -339,7 +217,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             assertTrue(rs.next());
             assertEquals(2, rs.getLong(1));
             assertFalse(rs.next());
-            conn.close();
         }
     }
 
@@ -363,7 +240,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             stmt.execute();
             stmt.setString(3, B_VALUE);
             stmt.execute();
-            upsertConn.close();
         }
         long upsert1Time = System.currentTimeMillis();
         long timeDelta = 100;
@@ -376,7 +252,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             stmt.setString(2, ROW6);
             stmt.setString(3, E_VALUE);
             stmt.execute();
-            upsertConn.close();
         }
         long queryTime = upsert1Time + timeDelta / 2;
         String query =
@@ -392,7 +267,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
             assertTrue(rs.next());
             assertEquals(4, rs.getLong(1)); // LIMIT applied at end, so all rows would be counted
             assertFalse(rs.next());
-            conn.close();
         }
     }
 }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeScanQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeScanQueryIT.java
new file mode 100644
index 0000000..0cdf21d
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeScanQueryIT.java
@@ -0,0 +1,179 @@
+/*
+ * 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.phoenix.end2end;
+
+import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.phoenix.util.TestUtil.A_VALUE;
+import static org.apache.phoenix.util.TestUtil.B_VALUE;
+import static org.apache.phoenix.util.TestUtil.C_VALUE;
+import static org.apache.phoenix.util.TestUtil.E_VALUE;
+import static org.apache.phoenix.util.TestUtil.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW4;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+
+@Category(NeedsOwnMiniClusterTest.class)
+public class PointInTimeScanQueryIT extends BaseQueryIT {
+
+    @Parameters(name="PointInTimeScanQueryIT_{index},columnEncoded={1}")
+    public static synchronized Collection<Object> data() {
+        return BaseQueryIT.allIndexesWithEncodedAndKeepDeleted();
+    }
+
+    public PointInTimeScanQueryIT(String idxDdl, boolean columnEncoded,
+            boolean keepDeletedCells) throws Exception {
+        super(idxDdl, columnEncoded, keepDeletedCells);
+
+        // For this class we specifically want to run each test method with
+        // fresh tables, it is expected to be slow
+        initTables(idxDdl, columnEncoded, keepDeletedCells);
+    }
+
+    @Test
+    public void testPointInTimeScan() throws Exception {
+        // Override value that was set at creation time
+        String url = getUrl();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String upsertStmt =
+            "upsert into " + tableName +
+                " (" +
+                "    ORGANIZATION_ID, " +
+                "    ENTITY_ID, " +
+                "    A_INTEGER) " +
+                "VALUES (?, ?, ?)";
+
+        try (Connection upsertConn = DriverManager.getConnection(url, props)) {
+            upsertConn.setAutoCommit(true); // Test auto commit
+            PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW4);
+            stmt.setInt(3, 5);
+            stmt.execute(); // should commit too
+        }
+        long upsert1Time = System.currentTimeMillis();
+        long timeDelta = 100;
+        Thread.sleep(timeDelta);
+
+        try(Connection upsertConn = DriverManager.getConnection(url, props)) {
+            upsertConn.setAutoCommit(true); // Test auto commit
+            PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW4);
+            stmt.setInt(3, 9);
+            stmt.execute(); // should commit too
+        }
+
+        long queryTime = upsert1Time + timeDelta / 2;
+        String query = "SELECT organization_id, a_string AS a FROM "
+            + tableName + " WHERE organization_id=? and a_integer = 5";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
+            Long.toString(queryTime));
+        try(Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(tenantId, rs.getString(1));
+            assertEquals(A_VALUE, rs.getString("a"));
+            assertTrue(rs.next());
+            assertEquals(tenantId, rs.getString(1));
+            assertEquals(B_VALUE, rs.getString(2));
+            assertFalse(rs.next());
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testPointInTimeLimitedScan() throws Exception {
+        // Override value that was set at creation time
+        String url = getUrl();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String upsertStmt =
+            "upsert into " + tableName +
+                " (" +
+                "    ORGANIZATION_ID, " +
+                "    ENTITY_ID, " +
+                "    A_INTEGER) " +
+                "VALUES (?, ?, ?)";
+        try(Connection upsertConn = DriverManager.getConnection(url, props)) {
+
+            upsertConn.setAutoCommit(true); // Test auto commit
+            // Insert all rows at ts
+            PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW1);
+            stmt.setInt(3, 6);
+            stmt.execute(); // should commit too
+        }
+        long upsert1Time = System.currentTimeMillis();
+        long timeDelta = 100;
+        Thread.sleep(timeDelta);
+
+        url = getUrl();
+        try(Connection upsertConn = DriverManager.getConnection(url, props)) {
+            upsertConn.setAutoCommit(true); // Test auto commit
+            // Insert all rows at ts
+            PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW1);
+            stmt.setInt(3, 0);
+            stmt.execute(); // should commit too
+        }
+
+        long queryTime = upsert1Time + timeDelta  / 2;
+        String query = "SELECT a_integer,b_string FROM " + tableName
+            + " WHERE organization_id=? and a_integer <= 5 limit 2";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
+            Long.toString(queryTime));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            List<List<Object>> expectedResultsA = Lists.newArrayList(
+                Arrays.<Object>asList(2, C_VALUE),
+                Arrays.<Object>asList( 3, E_VALUE));
+            List<List<Object>> expectedResultsB = Lists.newArrayList(
+                Arrays.<Object>asList( 5, C_VALUE),
+                Arrays.<Object>asList(4, B_VALUE));
+            // Since we're not ordering and we may be using a descending index, we don't
+            // know which rows we'll get back.
+            assertOneOfValuesEqualsResultSet(rs, expectedResultsA,
+                expectedResultsB);
+        }
+    }
+
+}