You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2016/02/02 05:00:17 UTC

[43/50] [abbrv] phoenix git commit: PHOENIX-2543 Modify StatsCollectorIT to test transactional test cases

PHOENIX-2543 Modify StatsCollectorIT to test transactional test cases


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/791a27cf
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/791a27cf
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/791a27cf

Branch: refs/heads/calcite
Commit: 791a27cfd141af969bc3e2e25ac9c14bd1b17a90
Parents: ee63702
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Jan 28 13:26:40 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Thu Jan 28 14:14:33 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/StatsCollectorIT.java       |  73 ++++++----
 .../org/apache/phoenix/tx/TransactionIT.java    | 137 +++++++++++++++----
 2 files changed, 152 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/791a27cf/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index 706ae4a..caba259 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -30,6 +30,8 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -43,14 +45,23 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Maps;
 
-
+@RunWith(Parameterized.class)
 public class StatsCollectorIT extends StatsCollectorAbstractIT {
     private static final String STATS_TEST_TABLE_NAME = "S";
+    
+    private final String tableDDLOptions;
+    private final String tableName;
+    private final String fullTableName;
         
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -62,30 +73,31 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-
-    @Test
-    public void testUpdateStatsForNonTxnTable() throws Throwable {
-        helpTestUpdateStats(false);
+    
+    public StatsCollectorIT( boolean transactional) {
+        this.tableDDLOptions= transactional ? " TRANSACTIONAL=true" : "";
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
     }
     
-    @Test
-    public void testUpdateStatsForTxnTable() throws Throwable {
-        helpTestUpdateStats(true);
+    @Parameters(name="transactional = {0}")
+    public static Collection<Boolean> data() {
+        return Arrays.asList(false,true);
     }
 
-	private void helpTestUpdateStats(boolean transactional) throws SQLException, IOException,
+    @Test
+    public void testUpdateStats() throws SQLException, IOException,
 			InterruptedException {
 		Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String tableName = "T" + (transactional ? "_TXN" : "");
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE " + tableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" 
-                		+ (transactional ? " TRANSACTIONAL=true" : ""));
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))"
+                		+ tableDDLOptions );
         String[] s;
         Array array;
         conn = upsertValues(props, tableName);
@@ -119,14 +131,14 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement()
-                .execute("CREATE TABLE x ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k)) \n");
-        conn.createStatement().execute("upsert into x values ('abc',1,3)");
-        conn.createStatement().execute("upsert into x values ('def',2,4)");
+                .execute("CREATE TABLE " + fullTableName +" ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k))" + tableDDLOptions );
+        conn.createStatement().execute("upsert into " + fullTableName +" values ('abc',1,3)");
+        conn.createStatement().execute("upsert into " + fullTableName +" values ('def',2,4)");
         conn.commit();
         // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS X");
+        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM x");
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
         assertTrue(rs.next());
         assertEquals("abc", rs.getString(1));
         assertTrue(rs.next());
@@ -137,6 +149,7 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
 
     @Test
     public void testUpdateStatsWithMultipleTables() throws Throwable {
+        String fullTableName2 = fullTableName+"_2";
         Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
@@ -144,21 +157,21 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE x ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC)) \n");
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
         conn.createStatement().execute(
-                "CREATE TABLE z ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC)) \n");
+                "CREATE TABLE " + fullTableName2 +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
         String[] s;
         Array array;
-        conn = upsertValues(props, "x");
-        conn = upsertValues(props, "z");
+        conn = upsertValues(props, fullTableName);
+        conn = upsertValues(props, fullTableName2);
         // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS X");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName);
         stmt.execute();
-        stmt = conn.prepareStatement("UPDATE STATISTICS Z");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
         stmt.execute();
-        stmt = upsertStmt(conn, "x");
+        stmt = upsertStmt(conn, fullTableName);
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -167,7 +180,7 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         array = conn.createArrayOf("VARCHAR", s);
         stmt.setArray(3, array);
         stmt.execute();
-        stmt = upsertStmt(conn, "z");
+        stmt = upsertStmt(conn, fullTableName2);
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -179,9 +192,9 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         conn.close();
         conn = DriverManager.getConnection(getUrl(), props);
         // This analyze would not work
-        stmt = conn.prepareStatement("UPDATE STATISTICS Z");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM Z");
+        rs = conn.createStatement().executeQuery("SELECT k FROM "+fullTableName2);
         assertTrue(rs.next());
         conn.close();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/791a27cf/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index 2794c47..b65b856 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -20,6 +20,8 @@ package org.apache.phoenix.tx;
 import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.TRANSACTIONAL_DATA_TABLE;
+import static org.apache.phoenix.util.TestUtil.analyzeTable;
+import static org.apache.phoenix.util.TestUtil.getAllSplits;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -36,12 +38,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import co.cask.tephra.TransactionContext;
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.hbase11.TransactionAwareHTable;
-import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
-
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -56,6 +52,7 @@ import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -71,6 +68,12 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import co.cask.tephra.TransactionContext;
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.hbase11.TransactionAwareHTable;
+import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -86,8 +89,9 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
+        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Integer.toString(20));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
         
@@ -581,30 +585,107 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     }
     
     @Test
-    public void testReadOnlyView() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
-        conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>4";
-        conn.createStatement().execute(ddl);
-        for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+    public void testReadOnlyViewWithStats() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
+            conn.createStatement().execute(ddl);
+            for (int i = 0; i < 10; i++) {
+                conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            }
+            conn.commit();
+            
+            // verify rows are visible for stats
+            analyzeTable(conn, "v", true);
+            List<KeyRange> splits = getAllSplits(conn, "v");
+            assertEquals(4, splits.size());
+            
+            int count = 0;
+            ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM t");
+            while (rs.next()) {
+                assertEquals(count++, rs.getInt(1));
+            }
+            assertEquals(10, count);
+            
+            count = 0;
+            rs = conn.createStatement().executeQuery("SELECT k FROM v");
+            while (rs.next()) {
+                assertEquals(6+count++, rs.getInt(1));
+            }
+            assertEquals(4, count);
         }
-        conn.commit();
-        
-        int count = 0;
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM t");
-        while (rs.next()) {
-            assertEquals(count++, rs.getInt(1));
+    }
+    
+    @Test
+    public void testReadOwnWritesWithStats() throws Exception {
+        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+            conn1.createStatement().execute(ddl);
+            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
+            conn1.createStatement().execute(ddl);
+            for (int i = 0; i < 10; i++) {
+                conn1.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            }
+    
+            // verify you can read your own writes
+            int count = 0;
+            ResultSet rs = conn1.createStatement().executeQuery("SELECT k FROM t");
+            while (rs.next()) {
+                assertEquals(count++, rs.getInt(1));
+            }
+            assertEquals(10, count);
+            
+            count = 0;
+            rs = conn1.createStatement().executeQuery("SELECT k FROM v");
+            while (rs.next()) {
+                assertEquals(6+count++, rs.getInt(1));
+            }
+            assertEquals(4, count);
+            
+            // verify stats can see the read own writes rows
+            analyzeTable(conn2, "v", true);
+            List<KeyRange> splits = getAllSplits(conn2, "v");
+            assertEquals(4, splits.size());
         }
-        assertEquals(10, count);
-        
-        count = 0;
-        rs = conn.createStatement().executeQuery("SELECT k FROM v");
-        while (rs.next()) {
-            assertEquals(5+count++, rs.getInt(1));
+    }
+    
+    @Test
+    public void testInvalidRowsWithStats() throws Exception {
+        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+            conn1.createStatement().execute(ddl);
+            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
+            conn1.createStatement().execute(ddl);
+            for (int i = 0; i < 10; i++) {
+                conn1.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            }
+    
+            // verify you can read your own writes
+            int count = 0;
+            ResultSet rs = conn1.createStatement().executeQuery("SELECT k FROM t");
+            while (rs.next()) {
+                assertEquals(count++, rs.getInt(1));
+            }
+            assertEquals(10, count);
+            
+            count = 0;
+            rs = conn1.createStatement().executeQuery("SELECT k FROM v");
+            while (rs.next()) {
+                assertEquals(6+count++, rs.getInt(1));
+            }
+            assertEquals(4, count);
+            
+            Thread.sleep(DEFAULT_TXN_TIMEOUT_SECONDS*1000+20000);
+            assertEquals("There should be one invalid transaction", 1, txManager.getInvalidSize());
+            
+            // verify stats can see the rows from the invalid transaction
+            analyzeTable(conn2, "v", true);
+            List<KeyRange> splits = getAllSplits(conn2, "v");
+            assertEquals(4, splits.size());
         }
-        assertEquals(5, count);
     }
     
     @Test