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/09/12 20:31:36 UTC

phoenix git commit: PHOENIX-3272 Fix CalciteIT CalciteGlobalIndexIT CalciteLocalIndexIT regressions

Repository: phoenix
Updated Branches:
  refs/heads/calcite fed476fb5 -> 6861a5eb3


PHOENIX-3272 Fix CalciteIT CalciteGlobalIndexIT CalciteLocalIndexIT regressions


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

Branch: refs/heads/calcite
Commit: 6861a5eb31a2d2ce3a22ccb7e5837bb1d5e87165
Parents: fed476f
Author: maryannxue <ma...@gmail.com>
Authored: Mon Sep 12 13:31:20 2016 -0700
Committer: maryannxue <ma...@gmail.com>
Committed: Mon Sep 12 13:31:20 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/calcite/BaseCalciteIT.java   | 16 +++++++-----
 .../query/BaseConnectionlessQueryTest.java      |  2 +-
 .../java/org/apache/phoenix/query/BaseTest.java | 26 +++++++++++---------
 3 files changed, 26 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6861a5eb/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
index adac540..3ea8db7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.calcite;
 
+import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -45,6 +46,7 @@ import org.apache.phoenix.calcite.rel.PhoenixRel;
 import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -95,8 +97,10 @@ public class BaseCalciteIT extends BaseClientManagedTimeIT {
         }
 
         Connection createConnection() throws Exception {
-            return DriverManager.getConnection(getUrl(), 
-                    props);
+        	// FIXME Cannot get correct stats with 'test=true' property
+        	final String testProp = PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
+        	final String url = getUrl().replaceAll(testProp, "");
+            return DriverManager.getConnection(url, props);
         }
         
         String getExplainPlanString() {
@@ -340,14 +344,14 @@ public class BaseCalciteIT extends BaseClientManagedTimeIT {
             + "      type: 'custom',\n"
             + "      factory: 'org.apache.phoenix.calcite.PhoenixSchema$Factory',\n"
             + "      operand: {\n"
-            + "        url: \"" + getUrl() + "\"\n"
+            + "        url: \"" + getOldUrl() + "\"\n"
             + "      }\n"
             + "    }";
     }
 
     protected static Connection connectUsingModel(Properties props) throws Exception {
         final File file = File.createTempFile("model", ".json");
-        final String url = getUrl();
+        final String url = getOldUrl();
         final PrintWriter pw = new PrintWriter(new FileWriter(file));
         pw.print(
             "{\n"
@@ -666,7 +670,7 @@ public class BaseCalciteIT extends BaseClientManagedTimeIT {
             
             conn.close();
             props.setProperty("TenantId", "10");
-            conn = DriverManager.getConnection(getUrl(), props);
+            conn = DriverManager.getConnection(getOldUrl(), props);
             conn.createStatement().execute("CREATE VIEW " + MULTI_TENANT_VIEW1
                     + " AS select * from " + MULTI_TENANT_TABLE);
             conn.commit();
@@ -680,7 +684,7 @@ public class BaseCalciteIT extends BaseClientManagedTimeIT {
             
             conn.close();
             props.setProperty("TenantId", "20");
-            conn = DriverManager.getConnection(getUrl(), props);
+            conn = DriverManager.getConnection(getOldUrl(), props);
             conn.createStatement().execute("CREATE VIEW " + MULTI_TENANT_VIEW2
                     + " AS select * from " + MULTI_TENANT_TABLE + " where col2 > 7");
             conn.commit();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6861a5eb/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
index b9ed592..dccf4a9 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
@@ -106,7 +106,7 @@ public class BaseConnectionlessQueryTest extends BaseTest {
             driver = new PhoenixTestDriver(props);
             DriverManager.registerDriver(driver);
             // Register Calcite-Phoenix Test Driver
-            DriverManager.registerDriver(new PhoenixCalciteTestDriver());
+            Class.forName(org.apache.phoenix.jdbc.PhoenixCalciteTestDriver.class.getName());
         }
         return driver;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6861a5eb/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 30d496ba..d511b36 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -505,6 +505,7 @@ public abstract class BaseTest {
     }
     
     protected static String url;
+    protected static String calciteUrl;
     protected static PhoenixTestDriver driver;
     protected static PhoenixDriver realDriver;
     protected static boolean clusterInitialized = false;
@@ -512,8 +513,10 @@ public abstract class BaseTest {
     protected static final Configuration config = HBaseConfiguration.create(); 
     
     protected static String getUrl() {
-        String url = getOldUrl();
-        return url.replaceFirst(PhoenixRuntime.JDBC_PROTOCOL, PhoenixRuntime.JDBC_PROTOCOL_CALCITE);
+        if (!clusterInitialized) {
+            throw new IllegalStateException("Cluster must be initialized before attempting to get the URL");
+        }
+        return calciteUrl;
     }
     
     protected static String getOldUrl() {
@@ -568,6 +571,7 @@ public abstract class BaseTest {
     protected static String checkClusterInitialized(ReadOnlyProps overrideProps) throws Exception {
         if (!clusterInitialized) {
             url = setUpTestCluster(config, overrideProps);
+            calciteUrl = url.replaceFirst(PhoenixRuntime.JDBC_PROTOCOL, PhoenixRuntime.JDBC_PROTOCOL_CALCITE);
             clusterInitialized = true;
         }
         return url;
@@ -653,6 +657,7 @@ public abstract class BaseTest {
                 utility.startMiniCluster(NUM_SLAVES_BASE);
                 utility.startMiniMapReduceCluster();
                 url = QueryUtil.getConnectionUrl(new Properties(), utility.getConfiguration(), false);
+                calciteUrl = url.replaceFirst(PhoenixRuntime.JDBC_PROTOCOL, PhoenixRuntime.JDBC_PROTOCOL_CALCITE);
             } catch (Throwable t) {
                 throw new RuntimeException(t);
             }
@@ -779,8 +784,7 @@ public abstract class BaseTest {
         PhoenixTestDriver newDriver = new PhoenixTestDriver(props);
         DriverManager.registerDriver(newDriver);
         // Register Calcite-Phoenix test driver at the same time.
-        PhoenixCalciteTestDriver newCalciteDriver = new PhoenixCalciteTestDriver();
-        DriverManager.registerDriver(newCalciteDriver);
+        Class.forName(PhoenixCalciteTestDriver.class.getName());
         Driver oldDriver = DriverManager.getDriver(url); 
         if (oldDriver != newDriver) {
             destroyDriver(oldDriver);
@@ -1250,15 +1254,15 @@ public abstract class BaseTest {
     }
     
     protected static void initATableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
-        initATableValues(tenantId, splits, date, ts, getUrl());
+        initATableValues(tenantId, splits, date, ts, getOldUrl());
     }
     
     protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
-        initEntityHistoryTableValues(tenantId, splits, date, ts, getUrl());
+        initEntityHistoryTableValues(tenantId, splits, date, ts, getOldUrl());
     }
     
     protected static void initSaltedEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
-        initSaltedEntityHistoryTableValues(tenantId, splits, date, ts, getUrl());
+        initSaltedEntityHistoryTableValues(tenantId, splits, date, ts, getOldUrl());
     }
         
     protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, String url) throws Exception {
@@ -1495,7 +1499,7 @@ public abstract class BaseTest {
         if (ts != null) {
             props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
         }
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = DriverManager.getConnection(getOldUrl(), props);
         try {
             //conn.createStatement().execute("CREATE SEQUENCE IF NOT EXISTS my.seq");
             // Insert into customer table
@@ -1890,7 +1894,7 @@ public abstract class BaseTest {
     // Populate the test table with data.
     public static void populateTestTable(String fullTableName) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+        try (Connection conn = DriverManager.getConnection(getOldUrl(), props)) {
         	upsertRows(conn, fullTableName, 3);
             conn.commit();
         }
@@ -1917,7 +1921,7 @@ public abstract class BaseTest {
                 "   CONSTRAINT pk PRIMARY KEY (varchar_pk, char_pk, int_pk, long_pk DESC, decimal_pk)) "
                 + (options!=null? options : "");
             Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            Connection conn = DriverManager.getConnection(getUrl(), props);
+            Connection conn = DriverManager.getConnection(getOldUrl(), props);
             conn.createStatement().execute(ddl);
             conn.close();
     }
@@ -1930,7 +1934,7 @@ public abstract class BaseTest {
     // Populate the test table with data.
     protected static void populateMultiCFTestTable(String tableName, Date date) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = DriverManager.getConnection(getOldUrl(), props);
         try {
             String upsert = "UPSERT INTO " + tableName
                     + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";