You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2016/07/28 06:35:50 UTC

phoenix git commit: Automatic build of async index will happen even if mapreduce.framework.name is not set in any configuration

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 6de5b613a -> 5e2537a65


Automatic build of async index will happen even if mapreduce.framework.name is not set in any configuration


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 5e2537a655f3dc291d2f0dd6d84fa887f41f00cf
Parents: 6de5b61
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Wed Jul 27 22:39:07 2016 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Wed Jul 27 23:25:38 2016 -0700

----------------------------------------------------------------------
 .../end2end/index/AsyncImmutableIndexIT.java    | 27 +++++++++--------
 .../end2end/index/AsyncIndexAutoBuildIT.java    | 19 +++++++-----
 .../end2end/index/AsyncIndexRegularBuildIT.java | 13 ++++----
 .../end2end/index/AsyncIndexTestUtil.java       | 31 +++++++++-----------
 .../coprocessor/MetaDataRegionObserver.java     |  6 ++--
 5 files changed, 49 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5e2537a6/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java
index 74ac2fd..6b65f9a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java
@@ -28,7 +28,7 @@ import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -36,14 +36,16 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.junit.Test;
 
-public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeIT {
+public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
     private static final long MAX_WAIT_FOR_INDEX_BUILD_TIME_MS = 45000;
 
     @Test
     public void testDeleteFromImmutable() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            conn.createStatement().execute("CREATE TABLE TEST_TABLE (\n" + 
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" + 
                     "        pk1 VARCHAR NOT NULL,\n" + 
                     "        pk2 VARCHAR NOT NULL,\n" + 
                     "        pk3 VARCHAR\n" + 
@@ -54,17 +56,17 @@ public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeIT {
                     "        pk3\n" + 
                     "        )\n" + 
                     "        ) IMMUTABLE_ROWS=true");
-            conn.createStatement().execute("upsert into TEST_TABLE (pk1, pk2, pk3) values ('a', '1', '1')");
-            conn.createStatement().execute("upsert into TEST_TABLE (pk1, pk2, pk3) values ('b', '2', '2')");
+            conn.createStatement().execute("upsert into " + tableName + " (pk1, pk2, pk3) values ('a', '1', '1')");
+            conn.createStatement().execute("upsert into " + tableName + " (pk1, pk2, pk3) values ('b', '2', '2')");
             conn.commit();
-            conn.createStatement().execute("CREATE INDEX TEST_INDEX ON TEST_TABLE (pk3, pk2) ASYNC");
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + " (pk3, pk2) ASYNC");
             
             // this delete will be issued at a timestamp later than the above timestamp of the index table
-            conn.createStatement().execute("delete from TEST_TABLE where pk1 = 'a'");
+            conn.createStatement().execute("delete from " + tableName + " where pk1 = 'a'");
             conn.commit();
 
             DatabaseMetaData dbmd = conn.getMetaData();
-            String escapedTableName = StringUtil.escapeLike("TEST_INDEX");
+            String escapedTableName = StringUtil.escapeLike(indexName);
             String[] tableType = new String[] {PTableType.INDEX.toString()};
             long startTime = System.currentTimeMillis();
             boolean isIndexActive = false;
@@ -77,22 +79,21 @@ public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeIT {
                 }
                 Thread.sleep(3000);
             } while (System.currentTimeMillis() - startTime < MAX_WAIT_FOR_INDEX_BUILD_TIME_MS);
-            
             assertTrue(isIndexActive);
 
             // upsert two more rows
             conn.createStatement().execute(
-                "upsert into TEST_TABLE (pk1, pk2, pk3) values ('a', '3', '3')");
+                "upsert into " + tableName + " (pk1, pk2, pk3) values ('a', '3', '3')");
             conn.createStatement().execute(
-                "upsert into TEST_TABLE (pk1, pk2, pk3) values ('b', '4', '4')");
+                "upsert into " + tableName + " (pk1, pk2, pk3) values ('b', '4', '4')");
             conn.commit();
 
             // validate that delete markers were issued correctly and only ('a', '1', 'value1') was
             // deleted
-            String query = "SELECT pk3 from TEST_TABLE ORDER BY pk3";
+            String query = "SELECT pk3 from " + tableName + " ORDER BY pk3";
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             String expectedPlan =
-                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER TEST_INDEX\n" + 
+                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + indexName + "\n" + 
                     "    SERVER FILTER BY FIRST KEY ONLY";
             assertEquals("Wrong plan ", expectedPlan, QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(query);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5e2537a6/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
index 96dccea..9e7862b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
@@ -25,24 +25,27 @@ import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.Statement;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.junit.Test;
 
-public class AsyncIndexAutoBuildIT extends BaseHBaseManagedTimeIT {
+public class AsyncIndexAutoBuildIT extends BaseHBaseManagedTimeTableReuseIT {
     
     @Test
     public void testAsyncIndexAutoBuild() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         Statement stmt = conn.createStatement();
-        AsyncIndexTestUtil.createTableAndLoadData(stmt);
-        AsyncIndexTestUtil.createAsyncIndex(stmt);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        AsyncIndexTestUtil.createTableAndLoadData(stmt, tableName);
+        AsyncIndexTestUtil.createAsyncIndex(stmt, indexName, tableName);
 
-        ResultSet rs = stmt.executeQuery(AsyncIndexTestUtil.PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        String personTableAsyncIndexInfoQuery = AsyncIndexTestUtil.getPersonTableAsyncIndexInfoQuery(tableName);
+        ResultSet rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
         assertTrue(rs.next());
+        
+        AsyncIndexTestUtil.retryWithSleep(tableName, 4, 5, stmt);
 
-        AsyncIndexTestUtil.retryWithSleep(4, 5, stmt);
-
-        rs = stmt.executeQuery(AsyncIndexTestUtil.PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
         assertFalse(rs.next());
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5e2537a6/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
index cafb6f1..5a53333 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
@@ -45,15 +45,18 @@ public class AsyncIndexRegularBuildIT extends BaseOwnClusterHBaseManagedTimeIT {
     public void testAsyncIndexRegularBuild() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         Statement stmt = conn.createStatement();
-        AsyncIndexTestUtil.createTableAndLoadData(stmt);
-        AsyncIndexTestUtil.createAsyncIndex(stmt);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        AsyncIndexTestUtil.createTableAndLoadData(stmt, tableName);
+        AsyncIndexTestUtil.createAsyncIndex(stmt, indexName, tableName);
 
-        ResultSet rs = stmt.executeQuery(AsyncIndexTestUtil.PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        String personTableAsyncIndexInfoQuery = AsyncIndexTestUtil.getPersonTableAsyncIndexInfoQuery(tableName);
+        ResultSet rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
         assertTrue(rs.next());
 
-        AsyncIndexTestUtil.retryWithSleep(4, 5, stmt);
+        AsyncIndexTestUtil.retryWithSleep(tableName, 4, 5, stmt);
 
-        rs = stmt.executeQuery(AsyncIndexTestUtil.PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
         assertTrue(rs.next());
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5e2537a6/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
index 3a85b54..d025961 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
@@ -24,38 +24,35 @@ import java.sql.SQLException;
 import java.sql.Statement;
 
 public class AsyncIndexTestUtil {
-    private static final String PERSON_TABLE_NAME = "PERSON";
-    private static final String PERSON_TABLE_NAME_WITH_SCHEMA = "TEST.PERSON";
-    private static final String TEST_SCHEMA = "TEST";
-
-    public static final String PERSON_TABLE_ASYNC_INDEX_INFO_QUERY = 
-            ASYNC_INDEX_INFO_QUERY + " and DATA_TABLE_NAME='" + PERSON_TABLE_NAME 
-            + "' and TABLE_SCHEM='" + TEST_SCHEMA + "'";
-
-    public static void createTableAndLoadData(Statement stmt) throws SQLException {
-        String ddl = "CREATE TABLE " + PERSON_TABLE_NAME_WITH_SCHEMA + " (ID INTEGER NOT NULL PRIMARY KEY, " +
+    public static void createTableAndLoadData(Statement stmt, String tableName) throws SQLException {
+        String ddl = "CREATE TABLE " + tableName + " (ID INTEGER NOT NULL PRIMARY KEY, " +
                      "FNAME VARCHAR, LNAME VARCHAR)";
         
         stmt.execute(ddl);
-        stmt.execute("UPSERT INTO " + PERSON_TABLE_NAME_WITH_SCHEMA + " values(1, 'FIRST', 'F')");
-        stmt.execute("UPSERT INTO " + PERSON_TABLE_NAME_WITH_SCHEMA + " values(2, 'SECOND', 'S')");
+        stmt.execute("UPSERT INTO " + tableName + " values(1, 'FIRST', 'F')");
+        stmt.execute("UPSERT INTO " + tableName + " values(2, 'SECOND', 'S')");
     }
 
-    public static void createAsyncIndex(Statement stmt) throws SQLException {
-        stmt.execute("CREATE INDEX FNAME_INDEX ON " + PERSON_TABLE_NAME_WITH_SCHEMA + "(FNAME) ASYNC");
+    public static void createAsyncIndex(Statement stmt, String indexName, String tableName) throws SQLException {
+        stmt.execute("CREATE INDEX " + indexName + " ON " + tableName + "(FNAME) ASYNC");
     }
 
-    public static void retryWithSleep(int maxRetries, int sleepInSecs, Statement stmt) throws Exception {
-        ResultSet rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+    public static void retryWithSleep(String tableName, int maxRetries, int sleepInSecs, Statement stmt) throws Exception {
+        String personTableAsyncIndexInfoQuery = getPersonTableAsyncIndexInfoQuery(tableName);
+        ResultSet rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
         // Wait for max of 5 retries with each retry of 5 sec sleep
         int retries = 0;
         while(retries <= maxRetries) {
             Thread.sleep(sleepInSecs * 1000);
-            rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+            rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
             if (!rs.next()) {
                 break;
             }
             retries++;
         }
     }
+    
+    public static String getPersonTableAsyncIndexInfoQuery(String tableName) {
+        return ASYNC_INDEX_INFO_QUERY + " and DATA_TABLE_NAME='" + tableName + "'";
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5e2537a6/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index a58b551..e8a35dd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -204,7 +204,6 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
      */
     public static class AsyncIndexRebuilderTask extends TimerTask {
         RegionCoprocessorEnvironment env;
-        PhoenixConnection conn = null;
 
         public AsyncIndexRebuilderTask(RegionCoprocessorEnvironment env) {
             this.env = env;
@@ -212,10 +211,9 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
 
         @Override
         public void run() {
+            PhoenixConnection conn = null;
             try {
-                if (conn == null) {
-                   conn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
-                }
+                conn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
                 Statement s = conn.createStatement();
                 ResultSet rs = s.executeQuery(ASYNC_INDEX_INFO_QUERY);