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/14 21:43:03 UTC

[05/50] [abbrv] phoenix git commit: PHOENIX-3130 Ignore ASYNC and build indexes synchronously based on config property

PHOENIX-3130 Ignore ASYNC and build indexes synchronously based on config property


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

Branch: refs/heads/calcite
Commit: 243e5e22a1338aa60f22b23c64f28d7c364bb53f
Parents: 92c7faf
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Mon Aug 1 12:00:27 2016 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Tue Aug 2 20:30:08 2016 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/IndexToolIT.java |   4 +-
 .../phoenix/end2end/MutableIndexToolIT.java     |   3 +-
 .../end2end/index/AsyncImmutableIndexIT.java    |  73 +++++++------
 .../end2end/index/AsyncIndexAutoBuildIT.java    |  51 ---------
 .../end2end/index/AsyncIndexDisabledIT.java     |  78 ++++++++++++++
 .../end2end/index/AsyncIndexRegularBuildIT.java |  62 -----------
 .../end2end/index/AsyncIndexTestUtil.java       |  58 ----------
 .../apache/phoenix/end2end/index/IndexIT.java   | 105 ++----------------
 .../coprocessor/MetaDataRegionObserver.java     | 107 ++-----------------
 .../phoenix/mapreduce/PhoenixInputFormat.java   |   2 -
 .../phoenix/mapreduce/index/IndexToolUtil.java  |   4 -
 .../index/automation/PhoenixMRJobSubmitter.java |  22 +++-
 .../apache/phoenix/query/QueryConstants.java    |  21 +---
 .../org/apache/phoenix/query/QueryServices.java |   6 +-
 .../phoenix/query/QueryServicesOptions.java     |  13 +--
 .../apache/phoenix/schema/MetaDataClient.java   |  16 +--
 .../phoenix/query/QueryServicesTestImpl.java    |   4 +-
 17 files changed, 173 insertions(+), 456 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
index 5621634..c66fea3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
@@ -37,6 +37,7 @@ import java.util.UUID;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -86,9 +87,8 @@ public class IndexToolIT extends BaseOwnClusterHBaseManagedTimeIT {
     
     @BeforeClass
     public static void doSetup() throws Exception {
-        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(2);
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
         serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
-        serverProps.put(QueryServices.ASYNC_INDEX_AUTO_BUILD_ATTRIB, Boolean.toString(false));
         Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
         clientProps.put(QueryServices.TRANSACTIONS_ENABLED, "true");
         setUpRealDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java
index cb41d2b..c335ff8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java
@@ -45,9 +45,8 @@ public class MutableIndexToolIT extends BaseOwnClusterHBaseManagedTimeIT {
     
     @BeforeClass
     public static void doSetup() throws Exception {
-        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(2);
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
         serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
-        serverProps.put(QueryServices.ASYNC_INDEX_AUTO_BUILD_ATTRIB, Boolean.toString(false));
         setUpRealDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), ReadOnlyProps.EMPTY_PROPS);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/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 6b65f9a..8c90b6e 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
@@ -23,29 +23,41 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
-import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
+import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
-import org.apache.phoenix.schema.PIndexState;
-import org.apache.phoenix.schema.PTableType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.IndexToolIT;
+import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.StringUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
-    private static final long MAX_WAIT_FOR_INDEX_BUILD_TIME_MS = 45000;
+import com.google.common.collect.Maps;
 
+public class AsyncImmutableIndexIT extends BaseOwnClusterHBaseManagedTimeIT {
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+        serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB,
+            QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        setUpRealDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),
+            ReadOnlyProps.EMPTY_PROPS);
+    }
+    
     @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 " + tableName + " (\n" + 
+            conn.createStatement().execute("CREATE TABLE TEST_TABLE (\n" + 
                     "        pk1 VARCHAR NOT NULL,\n" + 
                     "        pk2 VARCHAR NOT NULL,\n" + 
                     "        pk3 VARCHAR\n" + 
@@ -56,45 +68,37 @@ public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
                     "        pk3\n" + 
                     "        )\n" + 
                     "        ) IMMUTABLE_ROWS=true");
-            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.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.commit();
-            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + " (pk3, pk2) ASYNC");
+            conn.createStatement().execute("CREATE INDEX TEST_INDEX ON TEST_TABLE (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 " + tableName + " where pk1 = 'a'");
+            conn.createStatement().execute("delete from TEST_TABLE where pk1 = 'a'");
             conn.commit();
 
-            DatabaseMetaData dbmd = conn.getMetaData();
-            String escapedTableName = StringUtil.escapeLike(indexName);
-            String[] tableType = new String[] {PTableType.INDEX.toString()};
-            long startTime = System.currentTimeMillis();
-            boolean isIndexActive = false;
-            do {
-                ResultSet rs = dbmd.getTables("", "", escapedTableName, tableType);
-                assertTrue(rs.next());
-                if (PIndexState.valueOf(rs.getString("INDEX_STATE")) == PIndexState.ACTIVE) {
-                    isIndexActive = true;
-                    break;
-                }
-                Thread.sleep(3000);
-            } while (System.currentTimeMillis() - startTime < MAX_WAIT_FOR_INDEX_BUILD_TIME_MS);
-            assertTrue(isIndexActive);
+            // run the index MR job
+            final IndexTool indexingTool = new IndexTool();
+            indexingTool.setConf(new Configuration(getUtility().getConfiguration()));
+            final String[] cmdArgs =
+                    IndexToolIT.getArgValues(null, "TEST_TABLE", "TEST_INDEX", true);
+            int status = indexingTool.run(cmdArgs);
+            assertEquals(0, status);
 
             // upsert two more rows
             conn.createStatement().execute(
-                "upsert into " + tableName + " (pk1, pk2, pk3) values ('a', '3', '3')");
+                "upsert into TEST_TABLE (pk1, pk2, pk3) values ('a', '3', '3')");
             conn.createStatement().execute(
-                "upsert into " + tableName + " (pk1, pk2, pk3) values ('b', '4', '4')");
+                "upsert into TEST_TABLE (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 " + tableName + " ORDER BY pk3";
+            String query = "SELECT pk3 from TEST_TABLE ORDER BY pk3";
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             String expectedPlan =
-                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + indexName + "\n" + 
-                    "    SERVER FILTER BY FIRST KEY ONLY";
+                    "CLIENT 1-CHUNK PARALLEL 1-WAY FULL SCAN OVER TEST_INDEX\n"
+                            + "    SERVER FILTER BY FIRST KEY ONLY";
             assertEquals("Wrong plan ", expectedPlan, QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
@@ -108,4 +112,3 @@ public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
     }
 
 }
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/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
deleted file mode 100644
index 9e7862b..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
+++ /dev/null
@@ -1,51 +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.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.BaseHBaseManagedTimeTableReuseIT;
-import org.junit.Test;
-
-public class AsyncIndexAutoBuildIT extends BaseHBaseManagedTimeTableReuseIT {
-    
-    @Test
-    public void testAsyncIndexAutoBuild() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
-        Statement stmt = conn.createStatement();
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        AsyncIndexTestUtil.createTableAndLoadData(stmt, tableName);
-        AsyncIndexTestUtil.createAsyncIndex(stmt, indexName, tableName);
-
-        String personTableAsyncIndexInfoQuery = AsyncIndexTestUtil.getPersonTableAsyncIndexInfoQuery(tableName);
-        ResultSet rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        assertTrue(rs.next());
-        
-        AsyncIndexTestUtil.retryWithSleep(tableName, 4, 5, stmt);
-
-        rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        assertFalse(rs.next());
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexDisabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexDisabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexDisabledIT.java
new file mode 100644
index 0000000..1e4cbcc
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexDisabledIT.java
@@ -0,0 +1,78 @@
+/*
+ * 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.assertEquals;
+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 java.util.Map;
+
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class AsyncIndexDisabledIT extends BaseHBaseManagedTimeTableReuseIT {
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        clientProps.put(QueryServices.INDEX_ASYNC_BUILD_ENABLED, Boolean.toString(false));
+        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS, new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+    
+    @Test
+    public void testAsyncIndexRegularBuild() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.setAutoCommit(true);
+            Statement stmt = conn.createStatement();
+            String tableName = "TBL_" + generateRandomString();
+            String indexName = "IND_" + generateRandomString();
+            
+            String ddl = "CREATE TABLE " + tableName + " (pk INTEGER NOT NULL PRIMARY KEY, val VARCHAR)";
+            stmt.execute(ddl);
+            stmt.execute("UPSERT INTO " + tableName + " values(1, 'y')");
+            // create the async index
+            stmt.execute("CREATE INDEX " + indexName + " ON " + tableName + "(val) ASYNC");
+    
+            // it should be built as a regular index
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable table = phxConn.getTable(new PTableKey(null, tableName));
+            assertEquals("Index not built", 1, table.getIndexes().size());
+            assertEquals("Wrong index created", indexName, table.getIndexes().get(0).getName().getString());
+            
+            ResultSet rs = stmt.executeQuery("select /*+ INDEX(" + indexName + ")*/ pk, val from " + tableName);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("y", rs.getString(2));
+            assertFalse(rs.next());
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/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
deleted file mode 100644
index 5a53333..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
+++ /dev/null
@@ -1,62 +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.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();
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        AsyncIndexTestUtil.createTableAndLoadData(stmt, tableName);
-        AsyncIndexTestUtil.createAsyncIndex(stmt, indexName, tableName);
-
-        String personTableAsyncIndexInfoQuery = AsyncIndexTestUtil.getPersonTableAsyncIndexInfoQuery(tableName);
-        ResultSet rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        assertTrue(rs.next());
-
-        AsyncIndexTestUtil.retryWithSleep(tableName, 4, 5, stmt);
-
-        rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        assertTrue(rs.next());
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/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
deleted file mode 100644
index d025961..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
+++ /dev/null
@@ -1,58 +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 java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-public class AsyncIndexTestUtil {
-    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 " + tableName + " values(1, 'FIRST', 'F')");
-        stmt.execute("UPSERT INTO " + tableName + " values(2, 'SECOND', 'S')");
-    }
-
-    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(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(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/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index aba7dac..b7537a6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -51,18 +51,15 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
-import org.apache.phoenix.coprocessor.generated.PTableProtos.PTableType;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 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.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.util.DateUtil;
@@ -70,7 +67,6 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
-import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.apache.phoenix.util.TransactionUtil;
 import org.junit.BeforeClass;
@@ -83,12 +79,15 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseHBaseManagedTimeIT {
-    private static final long MAX_WAIT_FOR_ASYNC_INDEX_BUILD = 30000;
-    	
+	
 	private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
 	private final String tableDDLOptions;
+	private final String tableName;
+    private final String indexName;
+    private final String fullTableName;
+    private final String fullIndexName;
 	
 	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
 		this.localIndex = localIndex;
@@ -103,6 +102,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 			optionBuilder.append(" TRANSACTIONAL=true ");
 		}
 		this.tableDDLOptions = optionBuilder.toString();
+		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+        this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
+        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 	}
 	
 	@BeforeClass
@@ -123,10 +126,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
 	@Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
-	    String tableName = "TBL_" + generateRandomString();
-	    String indexName = "IND_" + generateRandomString();
-	    String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-	    String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -183,10 +182,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -242,10 +237,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         testCreateIndexAfterUpsertStarted(false, fullTableName + "1", fullIndexName + "1");
         if (transactional) {
             testCreateIndexAfterUpsertStarted(true, fullTableName + "2", fullIndexName + "2");
@@ -342,10 +333,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
@@ -396,10 +383,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testGroupByCount() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -418,10 +401,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -445,10 +424,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -471,10 +446,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -514,10 +485,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -573,10 +540,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -652,10 +615,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSelectCF() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -713,10 +672,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -777,10 +732,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
@@ -869,10 +820,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -984,10 +931,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -1005,20 +948,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     }
 
     @Test
-    public void testSyncIndexWithDecimalCol() throws Exception {
-        testIndexWithDecimalCol(false);
-    }
-    
-    @Test
-    public void testAsyncIndexWithDecimalCol() throws Exception {
-        testIndexWithDecimalCol(true);
-    }
-    
-    private void testIndexWithDecimalCol(boolean async) throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+    public void testIndexWithDecimalCol() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -1029,23 +959,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             createMultiCFTestTable(fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
-            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)" + (async ? " ASYNC" : "");
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
-            if (async) {
-                boolean wasBuilt = false;
-                long startTime = System.currentTimeMillis();
-                do {
-                    rs = conn.getMetaData().getTables("", TestUtil.DEFAULT_SCHEMA_NAME, StringUtil.escapeLike(indexName), new String[] {PTableType.INDEX.toString()});
-                    if (rs.next() && PIndexState.ACTIVE.toString().equalsIgnoreCase(rs.getString(PhoenixDatabaseMetaData.INDEX_STATE))) {
-                        wasBuilt = true;
-                        break;
-                    }
-                } while (System.currentTimeMillis() - startTime < MAX_WAIT_FOR_ASYNC_INDEX_BUILD);
-                assertTrue("Asyncronous index failed to build", wasBuilt);
-            }
-            
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/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 f13217a..5243154 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
@@ -17,12 +17,8 @@
  */
 package org.apache.phoenix.coprocessor;
 
-import static org.apache.phoenix.query.QueryConstants.ASYNC_INDEX_INFO_QUERY;
-
 import java.io.IOException;
-import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -72,7 +68,6 @@ import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.ByteUtil;
@@ -92,10 +87,9 @@ import com.google.common.collect.Lists;
  */
 public class MetaDataRegionObserver extends BaseRegionObserver {
     public static final Log LOG = LogFactory.getLog(MetaDataRegionObserver.class);
-    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(2);
+    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
     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;
 
     @Override
@@ -123,8 +117,6 @@ 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);
     }
@@ -166,102 +158,23 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
         };
         (new Thread(r)).start();
 
+        if (!enableRebuildIndex && !blockWriteRebuildIndex) {
+            LOG.info("Failure Index Rebuild is skipped by configuration.");
+            return;
+        }
         // turn off verbose deprecation logging
         Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
         if (deprecationLogger != null) {
             deprecationLogger.setLevel(Level.WARN);
         }
-
         try {
             Class.forName(PhoenixDriver.class.getName());
+            // starts index rebuild schedule work
+            BuildIndexScheduleTask task = new BuildIndexScheduleTask(e.getEnvironment());
+            // run scheduled task every 10 secs
+            executor.scheduleAtFixedRate(task, 10000, rebuildIndexTimeInterval, TimeUnit.MILLISECONDS);
         } catch (ClassNotFoundException ex) {
-            LOG.error("Phoenix Driver class is not found. Fix the classpath.", ex);
-        }
-         
-        // Enable async index rebuilder when autoAsyncIndexBuild is set to true 
-        if (autoAsyncIndexBuild)
-        {
-            LOG.info("Enabling Async Index rebuilder");
-            AsyncIndexRebuilderTask asyncIndexRebuilderTask = new AsyncIndexRebuilderTask(e.getEnvironment());
-            // run async index rebuilder task every 10 secs to rebuild any newly created async indexes
-            executor.scheduleAtFixedRate(asyncIndexRebuilderTask, 10000, rebuildIndexTimeInterval, TimeUnit.MILLISECONDS);
-        }
-
-        if (!enableRebuildIndex && !blockWriteRebuildIndex) {
-            LOG.info("Failure Index Rebuild is skipped by configuration.");
-            return;
-        }
-
-        // starts index rebuild schedule work
-        BuildIndexScheduleTask task = new BuildIndexScheduleTask(e.getEnvironment());
-        // run scheduled task every 10 secs
-        executor.scheduleAtFixedRate(task, 10000, rebuildIndexTimeInterval, TimeUnit.MILLISECONDS);
-    }
-    
-    /**
-     * Task runs periodically to re-build async indexes when hbase is running in non-distributed mode or 
-     * when mapreduce is running in local mode
-     *
-     */
-    public static class AsyncIndexRebuilderTask extends TimerTask {
-        RegionCoprocessorEnvironment env;
-
-        public AsyncIndexRebuilderTask(RegionCoprocessorEnvironment env) {
-            this.env = env;
-        }
-
-        @Override
-        public void run() {
-            PhoenixConnection conn = null;
-            try {
-                conn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
-                Statement s = conn.createStatement();
-                ResultSet rs = s.executeQuery(ASYNC_INDEX_INFO_QUERY);
-                PhoenixConnection metaDataClientConn = conn;
-                while (rs.next()) {
-                    String tableName = rs.getString(PhoenixDatabaseMetaData.DATA_TABLE_NAME);
-                    String tableSchema = rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM);
-                    String indexName = rs.getString(PhoenixDatabaseMetaData.TABLE_NAME);
-                    
-                    final PTable indexTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(tableSchema, indexName));
-                    final PTable dataTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(tableSchema, tableName));
-                    // this is set to ensure index tables remains consistent post population.
-                    long maxTimeRange = indexTable.getTimeStamp()+1;
-
-                    try {
-                        final Properties props = new Properties();
-                        Long txnScn = null;
-                        if (!indexTable.isTransactional()) {
-                            props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(maxTimeRange));
-                            metaDataClientConn = QueryUtil.getConnectionOnServer(props, env.getConfiguration()).unwrap(PhoenixConnection.class);
-                            txnScn = maxTimeRange;
-                        }
-                        MetaDataClient client = new MetaDataClient(conn);
-                        LOG.info("Building Index " + SchemaUtil.getTableName(tableSchema, indexName));
-                        client.buildIndex(indexTable, new TableRef(dataTable), txnScn);
-                    } catch (Throwable t) {
-                        LOG.error("AsyncIndexRebuilderTask failed while building index!", t);
-                    } finally {
-                        if (metaDataClientConn != null) {
-                            try {
-                                metaDataClientConn.close();
-                            } catch (SQLException ignored) {
-                                LOG.debug("AsyncIndexRebuilderTask can't close metaDataClientConn", ignored);
-                            }
-                        }
-                    }
-                }
-            } catch (Throwable t) {
-                LOG.error("AsyncIndexRebuilderTask failed!", t);
-            } finally {
-                if (conn != null) {
-                    try {
-                        conn.close();
-                    } catch (SQLException ignored) {
-                        LOG.debug("AsyncIndexRebuilderTask can't close connection", ignored);
-                    }
-                }
-            }
+            LOG.error("BuildIndexScheduleTask cannot start!", ex);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java
index 5882c14..df96c7b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java
@@ -43,7 +43,6 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.mapreduce.util.ConnectionUtil;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
 
 import com.google.common.base.Preconditions;
@@ -112,7 +111,6 @@ public class PhoenixInputFormat<T extends DBWritable> extends InputFormat<NullWr
             if(txnScnValue==null && currentScnValue!=null) {
                 overridingProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, currentScnValue);
             }
-            overridingProps.put(QueryServices.TRANSACTIONS_ENABLED, "true");
             final Connection connection = ConnectionUtil.getInputConnection(configuration, overridingProps);
             final String selectStatement = PhoenixConfigurationUtil.getSelectStatement(configuration);
             Preconditions.checkNotNull(selectStatement);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
index f955e6b..1058670 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
@@ -24,10 +24,7 @@ import java.util.Properties;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.mapreduce.util.ConnectionUtil;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PIndexState;
-import org.apache.phoenix.util.PhoenixRuntime;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,7 +50,6 @@ public class IndexToolUtil {
 		final String masterTable = PhoenixConfigurationUtil.getInputTableName(configuration);
 		final String indexTable = PhoenixConfigurationUtil.getOutputTableName(configuration);
 		final Properties overrideProps = new Properties();
-		overrideProps.setProperty(QueryServices.TRANSACTIONS_ENABLED, configuration.get(QueryServices.TRANSACTIONS_ENABLED));
 		final Connection connection = ConnectionUtil.getOutputConnection(configuration, overrideProps);
 		try {
 			updateIndexState(connection, masterTable, indexTable , state);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java
index 99f84ec..f6f2482 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java
@@ -17,8 +17,6 @@
  */
 package org.apache.phoenix.mapreduce.index.automation;
 
-import static org.apache.phoenix.query.QueryConstants.ASYNC_INDEX_INFO_QUERY;
-
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -52,6 +50,7 @@ import org.apache.phoenix.mapreduce.index.IndexTool;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.util.PhoenixMRJobUtil;
 import org.apache.phoenix.util.PhoenixMRJobUtil.MR_SCHEDULER_TYPE;
 import org.apache.phoenix.util.ZKBasedMasterElectionUtil;
@@ -62,6 +61,7 @@ import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import com.google.gson.reflect.TypeToken;
 
+
 public class PhoenixMRJobSubmitter {
 
     // Lock to elect a master node that submits the Phoenix Secondary Index MR Jobs
@@ -69,6 +69,22 @@ public class PhoenixMRJobSubmitter {
             "/phoenix/automated-mr-index-build-leader-election";
     private static final String AUTO_INDEX_BUILD_LOCK_NAME = "ActiveStandbyElectorLock";
 
+    public static final String CANDIDATE_INDEX_INFO_QUERY = "SELECT "
+            + PhoenixDatabaseMetaData.INDEX_TYPE + ","
+            + PhoenixDatabaseMetaData.DATA_TABLE_NAME + ", "
+            + PhoenixDatabaseMetaData.TABLE_SCHEM + ", "
+            + PhoenixDatabaseMetaData.TABLE_NAME + ", "
+            + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE 
+            + " FROM "
+            + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA + ".\"" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "\""
+            + " (" + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() + ") "
+            + " WHERE "
+            + PhoenixDatabaseMetaData.COLUMN_NAME + " IS NULL and "
+            + PhoenixDatabaseMetaData.COLUMN_FAMILY + " IS NULL  and "
+            + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " IS NOT NULL and "
+            + PhoenixDatabaseMetaData.TABLE_TYPE + " = '" + PTableType.INDEX.getSerializedValue() + "' and "
+            + PhoenixDatabaseMetaData.INDEX_STATE + " = '" + PIndexState.BUILDING.getSerializedValue() + "'";
+    
     // TODO - Move this to a property?
     private static final int JOB_SUBMIT_POOL_TIMEOUT = 5;
     private Configuration conf;
@@ -144,7 +160,7 @@ public class PhoenixMRJobSubmitter {
     public Map<String, PhoenixAsyncIndex> getCandidateJobs() throws SQLException {
         Connection con = DriverManager.getConnection("jdbc:phoenix:" + zkQuorum);
         Statement s = con.createStatement();
-        ResultSet rs = s.executeQuery(ASYNC_INDEX_INFO_QUERY);
+        ResultSet rs = s.executeQuery(CANDIDATE_INDEX_INFO_QUERY);
         Map<String, PhoenixAsyncIndex> candidateIndexes = new HashMap<String, PhoenixAsyncIndex>();
         while (rs.next()) {
             PhoenixAsyncIndex indexInfo = new PhoenixAsyncIndex();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 28393aa..9f8f58c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -21,7 +21,6 @@ package org.apache.phoenix.query;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_CREATED_DATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
@@ -116,12 +115,9 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
-import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
-import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.util.ByteUtil;
 
 
@@ -171,22 +167,7 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
-    public static final String ASYNC_INDEX_INFO_QUERY = "SELECT "
-            + INDEX_TYPE + ","
-            + DATA_TABLE_NAME + ", "
-            + TABLE_SCHEM + ", "
-            + TABLE_NAME + ", "
-            + ASYNC_CREATED_DATE 
-            + " FROM "
-            + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\""
-            + " (" + ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() + ") "
-            + " WHERE "
-            + COLUMN_NAME + " IS NULL and "
-            + COLUMN_FAMILY + " IS NULL  and "
-            + ASYNC_CREATED_DATE + " IS NOT NULL and "
-            + TABLE_TYPE + " = '" + PTableType.INDEX.getSerializedValue() + "' and "
-            + PhoenixDatabaseMetaData.INDEX_STATE + " = '" + PIndexState.BUILDING.getSerializedValue() + "'";
-    
+
     /**
      * Separator used between variable length keys for a composite key.
      * Variable length data types may not use this byte value.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/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 04e703d..2ad2d43 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
@@ -117,8 +117,6 @@ public interface QueryServices extends SQLCloseable {
     public static final String MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS_ATTRIB = "phoenix.coprocessor.maxMetaDataCacheTimeToLiveMs";
     public static final String MAX_SERVER_METADATA_CACHE_SIZE_ATTRIB = "phoenix.coprocessor.maxMetaDataCacheSize";
     public static final String MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB = "phoenix.client.maxMetaDataCacheSize";
-    public static final String HBASE_CLUSTER_DISTRIBUTED_ATTRIB = "hbase.cluster.distributed";
-    public static final String MAPRED_FRAMEWORK_NAME = "mapreduce.framework.name";
 
     public static final String AUTO_UPGRADE_WHITELIST_ATTRIB = "phoenix.client.autoUpgradeWhiteList";
     // Mainly for testing to force spilling
@@ -221,9 +219,7 @@ 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";
-
+    public static final String INDEX_ASYNC_BUILD_ENABLED = "phoenix.index.async.build.enabled";
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/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 cc23e08..b7ea82e 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,7 +20,6 @@ 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;
@@ -139,7 +138,6 @@ 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
@@ -251,6 +249,8 @@ public class QueryServicesOptions {
     public static final int DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE = 10;
     public static final boolean DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE = true;
     public static final float DEFAULT_LIMITED_QUERY_SERIAL_THRESHOLD = 0.2f;
+    
+    public static final boolean DEFAULT_INDEX_ASYNC_BUILD_ENABLED = true;
 
     @SuppressWarnings("serial")
     public static final Set<String> DEFAULT_QUERY_SERVER_SKIP_WORDS = new HashSet<String>() {
@@ -328,9 +328,7 @@ 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(ASYNC_INDEX_AUTO_BUILD_ATTRIB, DEFAULT_ASYNC_INDEX_AUTO_BUILD);
-
+            .setIfUnset(LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE);
         // 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.
@@ -591,11 +589,6 @@ 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/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 729f5f9..1264e32 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -1095,7 +1095,7 @@ public class MetaDataClient {
         boolean success = false;
         SQLException sqlException = null;
         try {
-            MutationState state = newClientAtNextTimeStamp.buildIndex(index, tableRef, null);
+            MutationState state = newClientAtNextTimeStamp.buildIndex(index, tableRef);
             success = true;
             return state;
         } catch (SQLException e) {
@@ -1121,7 +1121,7 @@ public class MetaDataClient {
         throw new IllegalStateException(); // impossible
     }
 
-    public MutationState buildIndex(PTable index, TableRef dataTableRef, Long txnScn) throws SQLException {
+    private MutationState buildIndex(PTable index, TableRef dataTableRef) throws SQLException {
         AlterIndexStatement indexStatement = null;
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -1138,9 +1138,6 @@ public class MetaDataClient {
             Scan scan = mutationPlan.getContext().getScan();
             Long scn = connection.getSCN();
             try {
-                if (txnScn!=null) {
-                    scan.setAttribute(BaseScannerRegionObserver.TX_SCN, Bytes.toBytes(Long.valueOf(txnScn)));
-                }
                 if (ScanUtil.isDefaultTimeRange(scan.getTimeRange())) {
                     if (scn == null) {
                         scn = mutationPlan.getContext().getCurrentTime();
@@ -1449,8 +1446,11 @@ public class MetaDataClient {
         }
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
+        boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
+            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
-        if(statement.isAsync()) {
+        if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
         
@@ -1459,7 +1459,7 @@ public class MetaDataClient {
         if (connection.getSCN() != null) {
             return buildIndexAtTimeStamp(table, statement.getTable());
         }
-        return buildIndex(table, tableRef, null);
+        return buildIndex(table, tableRef);
     }
 
     public MutationState dropSequence(DropSequenceStatement statement) throws SQLException {
@@ -3550,7 +3550,7 @@ public class MetaDataClient {
                     return buildIndexAtTimeStamp(index, dataTableNode);
                 }
                 TableRef dataTableRef = FromCompiler.getResolver(dataTableNode, connection).getTables().get(0);
-                return buildIndex(index, dataTableRef, null);
+                return buildIndex(index, dataTableRef);
             }
             return new MutationState(1, connection);
         } catch (TableNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/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 4ec42bd..6ae655c 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,7 +58,6 @@ 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;
@@ -112,8 +111,7 @@ 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)
-                .setAsyncIndexAutoBuild(DEFAULT_ASYNC_INDEX_AUTO_BUILD);
+                .setDefaultIndexPopulationWaitTime(DEFAULT_INDEX_POPULATION_WAIT_TIME);
     }
     
     public QueryServicesTestImpl(ReadOnlyProps defaultProps, ReadOnlyProps overrideProps) {