You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2016/08/05 21:03:24 UTC

[25/50] [abbrv] phoenix git commit: PHOENIX-3113 Automatic build of async index will happen even if mapreduce.framework.name is not set in any configuration

PHOENIX-3113 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/cc7c6151
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/cc7c6151
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/cc7c6151

Branch: refs/heads/encodecolumns
Commit: cc7c6151a4dda87d960691e55550e3e0e0628b60
Parents: badae2a
Author: Loknath Priyatham Teja Singamsetty <si...@gmail.com>
Authored: Tue Jul 26 23:47:53 2016 +0530
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Tue Jul 26 23:11:47 2016 -0700

----------------------------------------------------------------------
 .../end2end/index/AsyncImmutableIndexIT.java    |  15 +-
 .../end2end/index/AsyncIndexAutoBuildIT.java    |  48 ++++++
 .../phoenix/end2end/index/AsyncIndexIT.java     | 158 -------------------
 .../end2end/index/AsyncIndexRegularBuildIT.java |  59 +++++++
 .../end2end/index/AsyncIndexTestUtil.java       |  61 +++++++
 .../coprocessor/MetaDataRegionObserver.java     |  18 +--
 .../org/apache/phoenix/query/QueryServices.java |   3 +
 .../phoenix/query/QueryServicesOptions.java     |  11 +-
 .../phoenix/query/QueryServicesTestImpl.java    |   4 +-
 9 files changed, 194 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/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 df047d6..74ac2fd 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
@@ -26,28 +26,18 @@ import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
-import java.util.Collections;
-import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.StringUtil;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class AsyncImmutableIndexIT extends BaseOwnClusterHBaseManagedTimeIT {
+public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeIT {
     private static final long MAX_WAIT_FOR_INDEX_BUILD_TIME_MS = 45000;
-    
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        Map<String, String> props = Collections.emptyMap();
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
 
     @Test
     public void testDeleteFromImmutable() throws Exception {
@@ -117,3 +107,4 @@ public class AsyncImmutableIndexIT extends BaseOwnClusterHBaseManagedTimeIT {
     }
 
 }
+

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/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
new file mode 100644
index 0000000..96dccea
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
@@ -0,0 +1,48 @@
+/*
+ * 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.index;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.junit.Test;
+
+public class AsyncIndexAutoBuildIT extends BaseHBaseManagedTimeIT {
+    
+    @Test
+    public void testAsyncIndexAutoBuild() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        Statement stmt = conn.createStatement();
+        AsyncIndexTestUtil.createTableAndLoadData(stmt);
+        AsyncIndexTestUtil.createAsyncIndex(stmt);
+
+        ResultSet rs = stmt.executeQuery(AsyncIndexTestUtil.PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        assertTrue(rs.next());
+
+        AsyncIndexTestUtil.retryWithSleep(4, 5, stmt);
+
+        rs = stmt.executeQuery(AsyncIndexTestUtil.PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        assertFalse(rs.next());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexIT.java
deleted file mode 100644
index 19b460d..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexIT.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * 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.index;
-
-import static org.apache.phoenix.query.QueryConstants.ASYNC_INDEX_INFO_QUERY;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
-import org.apache.phoenix.query.BaseTest;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.After;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(NeedsOwnMiniClusterTest.class)
-public class AsyncIndexIT extends BaseTest {
-
-    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";
-
-    private 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 + "'";
-
-    private void dropTable(Statement stmt) throws SQLException, IOException {
-        stmt.execute("DROP TABLE IF EXISTS " + PERSON_TABLE_NAME_WITH_SCHEMA);
-    }
-
-    private void createTableAndLoadData(Statement stmt) throws SQLException {
-        String ddl = "CREATE TABLE " + PERSON_TABLE_NAME_WITH_SCHEMA + " (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')");
-    }
-
-    private void createAsyncIndex(Statement stmt) throws SQLException {
-        stmt.execute("CREATE INDEX FNAME_INDEX ON " + PERSON_TABLE_NAME_WITH_SCHEMA + "(FNAME) ASYNC");
-    }
-    
-    private void dropAsyncIndex(Statement stmt) throws SQLException {
-        stmt.execute("DROP INDEX IF EXISTS FNAME_INDEX ON " + PERSON_TABLE_NAME_WITH_SCHEMA);
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        tearDownMiniCluster();
-    }
-
-    private void retryWithSleep(int maxRetries, int sleepInSecs, Statement stmt) throws Exception {
-        ResultSet rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
-        // 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);
-            if (!rs.next()) {
-                break;
-            }
-            retries++;
-        }
-    }
-    
-    @Test
-    public void testAsyncIndexBuilderNonDistributed() throws Exception {
-        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS);
-        Connection conn = DriverManager.getConnection(getUrl());
-        Statement stmt = conn.createStatement();
-        createTableAndLoadData(stmt);
-        createAsyncIndex(stmt);
-
-        ResultSet rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
-        assertTrue(rs.next());
-
-        retryWithSleep(5, 5, stmt);
-
-        rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
-        assertFalse(rs.next());
-
-        dropAsyncIndex(stmt);
-        dropTable(stmt);
-    }
-    
-    @Test
-    public void testAsyncIndexBuilderNonDistributedMapreduceYarn() throws Exception {
-        Map<String,String> props = new HashMap<>();
-        props.put(QueryServices.MAPRED_FRAMEWORK_NAME, "yarn");
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-        
-        Connection conn = DriverManager.getConnection(getUrl());
-        Statement stmt = conn.createStatement();
-        createTableAndLoadData(stmt);
-        createAsyncIndex(stmt);
-
-        ResultSet rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
-        assertTrue(rs.next());
-
-        retryWithSleep(5, 5, stmt);
-
-        rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
-        assertFalse(rs.next());
-
-        dropAsyncIndex(stmt);
-        dropTable(stmt);
-    }
-
-    @Test
-    public void testAsyncIndexBuilderDistributed() throws Exception {
-        Map<String,String> props = new HashMap<>();
-        props.put(QueryServices.HBASE_CLUSTER_DISTRIBUTED_ATTRIB, "true");
-        props.put(QueryServices.MAPRED_FRAMEWORK_NAME, "yarn");
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-        
-        Connection conn = DriverManager.getConnection(getUrl());
-        Statement stmt = conn.createStatement();
-        createTableAndLoadData(stmt);
-        createAsyncIndex(stmt);
-
-        ResultSet rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
-        assertTrue(rs.next());
-
-        retryWithSleep(5, 5, stmt);
-
-        rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
-        assertTrue(rs.next());
-
-        dropAsyncIndex(stmt);
-        dropTable(stmt);
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/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
new file mode 100644
index 0000000..cafb6f1
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
@@ -0,0 +1,59 @@
+/*
+ * 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.index;
+
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Map;
+
+import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class AsyncIndexRegularBuildIT extends BaseOwnClusterHBaseManagedTimeIT {
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+        serverProps.put("phoenix.async.index.automatic.build", Boolean.toString(false));
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()));
+    }
+    
+    @Test
+    public void testAsyncIndexRegularBuild() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        Statement stmt = conn.createStatement();
+        AsyncIndexTestUtil.createTableAndLoadData(stmt);
+        AsyncIndexTestUtil.createAsyncIndex(stmt);
+
+        ResultSet rs = stmt.executeQuery(AsyncIndexTestUtil.PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        assertTrue(rs.next());
+
+        AsyncIndexTestUtil.retryWithSleep(4, 5, stmt);
+
+        rs = stmt.executeQuery(AsyncIndexTestUtil.PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        assertTrue(rs.next());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/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
new file mode 100644
index 0000000..3a85b54
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
@@ -0,0 +1,61 @@
+/*
+ * 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.index;
+
+import static org.apache.phoenix.query.QueryConstants.ASYNC_INDEX_INFO_QUERY;
+
+import java.sql.ResultSet;
+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, " +
+                     "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')");
+    }
+
+    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 retryWithSleep(int maxRetries, int sleepInSecs, Statement stmt) throws Exception {
+        ResultSet rs = stmt.executeQuery(PERSON_TABLE_ASYNC_INDEX_INFO_QUERY);
+        // 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);
+            if (!rs.next()) {
+                break;
+            }
+            retries++;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/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 bf396a9..a58b551 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
@@ -34,7 +34,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -95,9 +94,8 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
     protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(2);
     private boolean enableRebuildIndex = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD;
     private long rebuildIndexTimeInterval = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL;
+    private boolean autoAsyncIndexBuild = QueryServicesOptions.DEFAULT_ASYNC_INDEX_AUTO_BUILD; 
     private boolean blockWriteRebuildIndex = false;
-    private final String HBASE_CLUSTER_DISTRIBUTED_CONFIG = "true";
-    private final String MAPRED_FRAMEWORK_YARN_CONFIG = "yarn";
 
     @Override
     public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
@@ -124,6 +122,8 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
             QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD);
         rebuildIndexTimeInterval = env.getConfiguration().getLong(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB, 
             QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL);
+        autoAsyncIndexBuild = env.getConfiguration().getBoolean(QueryServices.ASYNC_INDEX_AUTO_BUILD_ATTRIB, 
+                QueryServicesOptions.DEFAULT_ASYNC_INDEX_AUTO_BUILD);
         blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE,
         	QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
     }
@@ -177,13 +177,8 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
             LOG.error("Phoenix Driver class is not found. Fix the classpath.", ex);
         }
          
-        Configuration conf = env.getConfiguration();
-        String hbaseClusterDistributedMode = conf.get(QueryServices.HBASE_CLUSTER_DISTRIBUTED_ATTRIB);
-        String mapredFrameworkName = conf.get(QueryServices.MAPRED_FRAMEWORK_NAME); 
-
-        // In case of non-distributed mode of hbase service or local mode of map reduce service, add timer task to rebuild the async indexes  
-        if ((hbaseClusterDistributedMode != null && !hbaseClusterDistributedMode.equals(HBASE_CLUSTER_DISTRIBUTED_CONFIG)) || 
-            (mapredFrameworkName != null && !mapredFrameworkName.equals(MAPRED_FRAMEWORK_YARN_CONFIG)))
+        // Enable async index rebuilder when autoAsyncIndexBuild is set to true 
+        if (autoAsyncIndexBuild)
         {
             LOG.info("Enabling Async Index rebuilder");
             AsyncIndexRebuilderTask asyncIndexRebuilderTask = new AsyncIndexRebuilderTask(e.getEnvironment());
@@ -237,7 +232,8 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
 
                     try {
                         final Properties props = new Properties();
-                        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(maxTimeRange));
+                        if (!pindexTable.isTransactional())
+                            props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(maxTimeRange));
                         alterIndexConnection = QueryUtil.getConnectionOnServer(props, env.getConfiguration()).unwrap(PhoenixConnection.class);
 
                         // Alter index query for rebuilding async indexes

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 3f3518b..980611a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -220,6 +220,9 @@ public interface QueryServices extends SQLCloseable {
     public static final String LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB = "phoenix.client.localIndexUpgrade";
     public static final String LIMITED_QUERY_SERIAL_THRESHOLD = "phoenix.limited.query.serial.threshold";
 
+    // A master switch to enable auto rebuild an async secondary index 
+    public static final String ASYNC_INDEX_AUTO_BUILD_ATTRIB = "phoenix.async.index.automatic.build";
+
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 6c49fbb..7abd7ee 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.query;
 import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD;
 import static org.apache.phoenix.query.QueryServices.ALLOW_ONLINE_TABLE_SCHEMA_UPDATE;
 import static org.apache.phoenix.query.QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE;
+import static org.apache.phoenix.query.QueryServices.ASYNC_INDEX_AUTO_BUILD_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_PRODUCER_ATTRIB_NAME;
 import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_ROUND_ROBIN_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.COLLECT_REQUEST_LEVEL_METRICS;
@@ -137,6 +138,7 @@ public class QueryServicesOptions {
     public static final long DEFAULT_SCAN_RESULT_CHUNK_SIZE = 2999;
     public static final boolean DEFAULT_IS_NAMESPACE_MAPPING_ENABLED = false;
     public static final boolean DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE = true;
+    public static final boolean DEFAULT_ASYNC_INDEX_AUTO_BUILD = false;
 
     //
     // Spillable GroupBy - SPGBY prefix
@@ -323,7 +325,9 @@ public class QueryServicesOptions {
             .setIfUnset(RENEW_LEASE_THREAD_POOL_SIZE, DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE)
             .setIfUnset(IS_NAMESPACE_MAPPING_ENABLED, DEFAULT_IS_NAMESPACE_MAPPING_ENABLED)
             .setIfUnset(IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE)
-            .setIfUnset(LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE);
+            .setIfUnset(LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)
+            .setIfUnset(ASYNC_INDEX_AUTO_BUILD_ATTRIB, DEFAULT_ASYNC_INDEX_AUTO_BUILD);
+
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
         // it to 1, so we'll change it.
@@ -583,6 +587,11 @@ public class QueryServicesOptions {
         return this;
     }
 
+    public QueryServicesOptions setAsyncIndexAutoBuild(boolean autoAsyncIndexBuild) {
+        config.setBoolean(ASYNC_INDEX_AUTO_BUILD_ATTRIB, autoAsyncIndexBuild);
+        return this;
+    }
+
     public QueryServicesOptions setExplainRowCount(boolean showRowCount) {
         config.setBoolean(EXPLAIN_ROW_COUNT_ATTRIB, showRowCount);
         return this;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cc7c6151/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index 6ae655c..4ec42bd 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -58,6 +58,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     public static final String DEFAULT_EXTRA_JDBC_ARGUMENTS = PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
     private static final boolean DEFAULT_RUN_UPDATE_STATS_ASYNC = false;
     private static final boolean DEFAULT_COMMIT_STATS_ASYNC = false;
+    private static final boolean DEFAULT_ASYNC_INDEX_AUTO_BUILD = true;
     public static final int DEFAULT_INDEX_HANDLER_COUNT = 5;
     public static final int DEFAULT_METADATA_HANDLER_COUNT = 5;
     public static final int DEFAULT_HCONNECTION_POOL_CORE_SIZE = 10;
@@ -111,7 +112,8 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .setHConnectionPoolCoreSize(DEFAULT_HCONNECTION_POOL_CORE_SIZE)
                 .setHConnectionPoolMaxSize(DEFAULT_HCONNECTION_POOL_MAX_SIZE)
                 .setMaxThreadsPerHTable(DEFAULT_HTABLE_MAX_THREADS)
-                .setDefaultIndexPopulationWaitTime(DEFAULT_INDEX_POPULATION_WAIT_TIME);
+                .setDefaultIndexPopulationWaitTime(DEFAULT_INDEX_POPULATION_WAIT_TIME)
+                .setAsyncIndexAutoBuild(DEFAULT_ASYNC_INDEX_AUTO_BUILD);
     }
     
     public QueryServicesTestImpl(ReadOnlyProps defaultProps, ReadOnlyProps overrideProps) {