You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by st...@apache.org on 2022/01/05 10:04:44 UTC

[phoenix] branch master updated: PHOENIX-6611 Fix IndexTool -snap option and set VERIFIED in PhoenixIndexImportDirectReducer

This is an automated email from the ASF dual-hosted git repository.

stoty pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/master by this push:
     new 01e1d2b  PHOENIX-6611 Fix IndexTool -snap option and set VERIFIED in PhoenixIndexImportDirectReducer
01e1d2b is described below

commit 01e1d2b4ef1758afef6b967f0959c8874e094596
Author: Istvan Toth <st...@apache.org>
AuthorDate: Tue Dec 14 18:34:23 2021 +0100

    PHOENIX-6611 Fix IndexTool -snap option and set VERIFIED in PhoenixIndexImportDirectReducer
    
    also
    * Set VERIFIED status for transactional index table records in normal write path
    * Extend test cases for namespacemapped and snapshot cases
    * Remove obsolte directApi option from test code
    * Explicitly check and forbid using the -snapshot option for local index
---
 .../end2end/ConcurrentMutationsExtendedIT.java     |   8 +-
 .../apache/phoenix/end2end/CsvBulkLoadToolIT.java  |  40 +---
 .../phoenix/end2end/IndexBuildTimestampIT.java     |  37 +++-
 .../apache/phoenix/end2end/IndexExtendedIT.java    |  13 +-
 .../end2end/IndexRepairRegionScannerIT.java        |  36 ++--
 .../end2end/IndexToolForNonTxGlobalIndexIT.java    |  99 +++++-----
 .../org/apache/phoenix/end2end/IndexToolIT.java    | 220 +++++++++++----------
 .../phoenix/end2end/IndexToolTimeRangeIT.java      |   2 +-
 .../end2end/IndexVerificationOldDesignIT.java      |  10 +-
 .../end2end/LogicalTableNameExtendedIT.java        |   4 +-
 .../apache/phoenix/end2end/LogicalTableNameIT.java |  16 +-
 .../end2end/index/GlobalIndexCheckerIT.java        |  28 +--
 .../end2end/index/ImmutableIndexExtendedIT.java    |  10 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java    |  46 +----
 .../phoenix/end2end/index/IndexTestUtil.java       |  33 ++++
 .../apache/phoenix/end2end/index/ViewIndexIT.java  |   2 +-
 .../DataTableLocalIndexRegionScanner.java          |   4 +-
 .../coprocessor/GlobalIndexRegionScanner.java      |   6 +-
 .../phoenix/coprocessor/IndexerRegionScanner.java  |   2 +-
 .../UngroupedAggregateRegionScanner.java           |   3 +-
 .../org/apache/phoenix/execute/MutationState.java  |   8 +-
 .../execute/PhoenixTxIndexMutationGenerator.java   |   3 +-
 .../phoenix/hbase/index/IndexRegionObserver.java   |  18 +-
 .../phoenix/hbase/index/covered/IndexCodec.java    |   3 +-
 .../hbase/index/covered/NonTxIndexBuilder.java     |   4 +-
 .../apache/phoenix/index/GlobalIndexChecker.java   |   2 +-
 .../org/apache/phoenix/index/IndexMaintainer.java  |   8 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java    |   6 +-
 .../mapreduce/FormatToBytesWritableMapper.java     |  40 +---
 .../apache/phoenix/mapreduce/index/IndexTool.java  |  17 +-
 .../index/PhoenixIndexImportDirectMapper.java      |  18 +-
 .../org/apache/phoenix/query/QueryConstants.java   |   8 +
 .../schema/transform/TransformMaintainer.java      |   2 +-
 .../java/org/apache/phoenix/util/IndexUtil.java    |  57 +++++-
 .../index/covered/CoveredColumnIndexCodec.java     |   2 +-
 .../index/covered/CoveredIndexCodecForTesting.java |   2 +-
 .../index/covered/TestCoveredColumnIndexCodec.java |   4 +-
 .../org/apache/phoenix/index/IndexToolTest.java    |  36 ++--
 .../index/PrepareIndexMutationsForRebuildTest.java |   3 +-
 .../phoenix/index/VerifySingleIndexRowTest.java    |   8 +-
 40 files changed, 470 insertions(+), 398 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConcurrentMutationsExtendedIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConcurrentMutationsExtendedIT.java
index b123c79..48768ea 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConcurrentMutationsExtendedIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConcurrentMutationsExtendedIT.java
@@ -86,7 +86,7 @@ public class ConcurrentMutationsExtendedIT extends ParallelStatsDisabledIT {
     static long verifyIndexTable(String tableName, String indexName,
             Connection conn) throws Exception {
         // This checks the state of every raw index row without rebuilding any row
-        IndexTool indexTool = IndexToolIT.runIndexTool(true, false, "", tableName,
+        IndexTool indexTool = IndexToolIT.runIndexTool(false, "", tableName,
                 indexName, null, 0, IndexTool.IndexVerifyType.ONLY);
         System.out.println(indexTool.getJob().getCounters());
         assertEquals(0, indexTool.getJob().getCounters().findCounter(REBUILT_INDEX_ROW_COUNT).getValue());
@@ -100,7 +100,7 @@ public class ConcurrentMutationsExtendedIT extends ParallelStatsDisabledIT {
         // This checks the state of an index row after it is repaired
         long actualRowCount = IndexScrutiny.scrutinizeIndex(conn, tableName, indexName);
         // We want to check the index rows again as they may be modified by the read repair
-        indexTool = IndexToolIT.runIndexTool(true, false, "", tableName, indexName,
+        indexTool = IndexToolIT.runIndexTool(false, "", tableName, indexName,
                 null, 0, IndexTool.IndexVerifyType.ONLY);
         System.out.println(indexTool.getJob().getCounters());
 
@@ -115,7 +115,7 @@ public class ConcurrentMutationsExtendedIT extends ParallelStatsDisabledIT {
         assertEquals(0, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_OLD_INDEX_ROW_COUNT).getValue());
         assertEquals(0, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_UNKNOWN_INDEX_ROW_COUNT).getValue());
         // Now we rebuild the entire index table and expect that it is still good after the full rebuild
-        indexTool = IndexToolIT.runIndexTool(true, false, "", tableName, indexName,
+        indexTool = IndexToolIT.runIndexTool(false, "", tableName, indexName,
                 null, 0, IndexTool.IndexVerifyType.AFTER);
         assertEquals(indexTool.getJob().getCounters().findCounter(AFTER_REBUILD_VALID_INDEX_ROW_COUNT).getValue(),
                 indexTool.getJob().getCounters().findCounter(REBUILT_INDEX_ROW_COUNT).getValue());
@@ -131,7 +131,7 @@ public class ConcurrentMutationsExtendedIT extends ParallelStatsDisabledIT {
             admin.disableTable(physicalTableName);
             admin.truncateTable(physicalTableName, true);
         }
-        indexTool = IndexToolIT.runIndexTool(true, false, "", tableName, indexName,
+        indexTool = IndexToolIT.runIndexTool(false, "", tableName, indexName,
                 null, 0, IndexTool.IndexVerifyType.AFTER);
         assertEquals(0, indexTool.getJob().getCounters().findCounter(AFTER_REBUILD_INVALID_INDEX_ROW_COUNT).getValue());
         assertEquals(0, indexTool.getJob().getCounters().findCounter(AFTER_REBUILD_MISSING_INDEX_ROW_COUNT).getValue());
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java
index 4d10a6c..df08d25 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java
@@ -21,16 +21,13 @@ import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_ATTRIB;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.io.IOException;
 import java.io.PrintWriter;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
-import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Map;
 
@@ -38,25 +35,19 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
-import org.apache.phoenix.hbase.index.IndexRegionObserver;
+import org.apache.phoenix.end2end.index.IndexTestUtil;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.mapreduce.CsvBulkLoadTool;
-import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
 import org.apache.phoenix.util.DateUtil;
-import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -431,26 +422,7 @@ public class CsvBulkLoadToolIT extends BaseOwnClusterIT {
         rs.close();
         stmt.close();
 
-        checkIndexTableIsVerified("TABLE3_IDX");
-    }
-
-    private void checkIndexTableIsVerified(String indexTableName) throws SQLException, IOException {
-        ConnectionQueryServices cqs = conn.unwrap(PhoenixConnection.class).getQueryServices();
-        Table hTable = cqs.getTable(Bytes.toBytes(indexTableName));
-        PTable pTable = PhoenixRuntime.getTable(conn, indexTableName);
-
-        byte[] emptyKeyValueCF = SchemaUtil.getEmptyColumnFamily(pTable);
-        byte[] emptyKeyValueQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(pTable).getFirst();
-
-        Scan scan = new Scan();
-        scan.setFilter(new SingleColumnValueFilter(
-                emptyKeyValueCF,
-                emptyKeyValueQualifier,
-                CompareOperator.NOT_EQUAL,
-                new org.apache.hadoop.hbase.filter.BinaryComparator(IndexRegionObserver.VERIFIED_BYTES)));
-        try (ResultScanner scanner = hTable.getScanner(scan)) {
-            assertNull("There are non VERIFIED rows in index", scanner.next());
-        }
+        IndexTestUtil.assertRowsForEmptyColValue(conn, "TABLE3_IDX", QueryConstants.VERIFIED_BYTES);
     }
 
     @Test
@@ -540,7 +512,8 @@ public class CsvBulkLoadToolIT extends BaseOwnClusterIT {
         stmt.close();
 
         if (!localIndex) {
-            checkIndexTableIsVerified(indexTableName);
+            IndexTestUtil.assertRowsForEmptyColValue(conn, indexTableName,
+                QueryConstants.VERIFIED_BYTES);
         }
     }
 
@@ -615,7 +588,8 @@ public class CsvBulkLoadToolIT extends BaseOwnClusterIT {
         rs.close();
         stmt.close();
 
-        checkIndexTableIsVerified(fullIndexTableName);
+        IndexTestUtil.assertRowsForEmptyColValue(conn, fullIndexTableName,
+            QueryConstants.VERIFIED_BYTES);
     }
 
     @Test
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexBuildTimestampIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexBuildTimestampIT.java
index faa6cb0..52d905b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexBuildTimestampIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexBuildTimestampIT.java
@@ -54,6 +54,7 @@ import org.junit.runners.Parameterized.Parameters;
 
 import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
 import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.transaction.TransactionFactory;
 
 @Category(NeedsOwnMiniClusterTest.class)
 @RunWith(Parameterized.class)
@@ -61,17 +62,28 @@ public class IndexBuildTimestampIT extends BaseTest {
     private final boolean localIndex;
     private final boolean async;
     private final boolean view;
+    private final boolean snapshot;
+    private final boolean transactional;
     private final String tableDDLOptions;
+    private String scnPropName;
 
-    public IndexBuildTimestampIT(boolean mutable, boolean localIndex,
-                            boolean async, boolean view) {
+    public IndexBuildTimestampIT(String transactionProvider, boolean mutable, boolean localIndex,
+                            boolean async, boolean view, boolean snapshot) {
         this.localIndex = localIndex;
         this.async = async;
         this.view = view;
+        this.snapshot = snapshot;
+        this.transactional = transactionProvider != null;
         StringBuilder optionBuilder = new StringBuilder();
         if (!mutable) {
             optionBuilder.append(" IMMUTABLE_ROWS=true ");
         }
+        if (transactional) {
+            if (!(optionBuilder.length() == 0)) {
+                optionBuilder.append(",");
+            }
+            optionBuilder.append(" TRANSACTIONAL=true,TRANSACTION_PROVIDER='" + transactionProvider + "'");
+        }
         optionBuilder.append(" SPLIT ON(1,2)");
         this.tableDDLOptions = optionBuilder.toString();
     }
@@ -81,6 +93,7 @@ public class IndexBuildTimestampIT extends BaseTest {
         Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(2);
         serverProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
         serverProps.put(QueryServices.MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS_ATTRIB, Long.toString(5));
+        serverProps.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
         serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB,
             QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
         serverProps.put(QueryServices.INDEX_REBUILD_PAGE_SIZE_IN_ROWS, Long.toString(8));
@@ -94,7 +107,7 @@ public class IndexBuildTimestampIT extends BaseTest {
     }
 
     @Parameters(
-            name = "mutable={0},localIndex={1},async={2},view={3}")
+            name = "transactionProvider={0},mutable={1},localIndex={2},async={3},view={4},snapshot={5}")
     public static synchronized Collection<Object[]> data() {
         List<Object[]> list = Lists.newArrayListWithExpectedSize(16);
         boolean[] Booleans = new boolean[]{false, true};
@@ -102,8 +115,20 @@ public class IndexBuildTimestampIT extends BaseTest {
             for (boolean localIndex : Booleans) {
                 for (boolean async : Booleans) {
                     for (boolean view : Booleans) {
-                        list.add(new Object[]{mutable, localIndex, async, view});
-                    }
+                        for (boolean snapshot : Booleans) {
+                            for (String transactionProvider : new String[]
+                                    {"TEPHRA", "OMID", null}) {
+                                if(snapshot || transactionProvider !=null) {
+                                    //FIXME PHOENIX-5375 TS is set to index creation time
+                                    continue;
+                                }
+                                if ((localIndex || !async) && snapshot) {
+                                    continue;
+                                }
+                                list.add(new Object[]{transactionProvider, mutable, localIndex, async, view, snapshot});
+                            }
+                        }
+                     }
                 }
             }
         }
@@ -218,7 +243,7 @@ public class IndexBuildTimestampIT extends BaseTest {
 
             if (async) {
                 // run the index MR job.
-                IndexToolIT.runIndexTool(true, false, null, (view ? viewName : dataTableName), indexName);
+                IndexToolIT.runIndexTool(snapshot, null, (view ? viewName : dataTableName), indexName);
             }
 
             // Verify the index timestamps via Phoenix
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java
index caa9ada..0f143af 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java
@@ -116,6 +116,9 @@ public class IndexExtendedIT extends BaseTest {
             for (boolean localIndex : Booleans) {
                 for (boolean useViewIndex : Booleans) {
                     for (boolean useSnapshot : Booleans) {
+                        if(localIndex && useSnapshot) {
+                            continue;
+                        }
                         list.add(new Boolean[] { mutable, localIndex, useViewIndex, useSnapshot});
                     }
                 }
@@ -186,7 +189,7 @@ public class IndexExtendedIT extends BaseTest {
             assertFalse(rs.next());
            
             //run the index MR job.
-            IndexToolIT.runIndexTool(true, useSnapshot, schemaName, dataTableName, indexTableName);
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName);
             
             plan = conn.prepareStatement(selectSql)
                 .unwrap(PhoenixPreparedStatement.class).optimizeQuery()
@@ -244,7 +247,7 @@ public class IndexExtendedIT extends BaseTest {
             conn.commit();
 
             //run the index MR job.
-            IndexToolIT.runIndexTool(true, useSnapshot, schemaName, dataTableName, indexTableName);
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName);
 
             // upsert two more rows
             conn.createStatement().execute(
@@ -344,7 +347,7 @@ public class IndexExtendedIT extends BaseTest {
             assertEquals(dataCnt, indexCnt+1);
 
             //run the index MR job.
-            IndexToolIT.runIndexTool(true, useSnapshot, schemaName, baseTableNameOfIndex, indexName);
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, baseTableNameOfIndex, indexName);
 
             dataCnt = getRowCount(conn, baseTableFullNameOfIndex);
             indexCnt = getUtility().countRows(hIndexTable);
@@ -391,7 +394,7 @@ public class IndexExtendedIT extends BaseTest {
 
             if (CompatBaseScannerRegionObserver.isMaxLookbackTimeEnabled(getUtility().getConfiguration())) {
                 // Run the index MR job and verify that the index table rebuild fails
-                IndexToolIT.runIndexTool(true, false, schemaName, dataTableName,
+                IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                         indexTableName, null, -1, IndexTool.IndexVerifyType.AFTER);
                 // job failed, verify that the index table is still not in the ACTIVE state
                 assertFalse(checkIndexState(conn, indexFullName, PIndexState.ACTIVE, 0L));
@@ -399,7 +402,7 @@ public class IndexExtendedIT extends BaseTest {
 
             IndexRebuildRegionScanner.setIgnoreIndexRebuildForTesting(false);
             // Run the index MR job and verify that the index table rebuild succeeds
-            IndexToolIT.runIndexTool(true, false, schemaName, dataTableName,
+            IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                     indexTableName, null, 0, IndexTool.IndexVerifyType.AFTER);
 
             // job passed, verify that the index table is in the ACTIVE state
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexRepairRegionScannerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexRepairRegionScannerIT.java
index 4440ada..5a0c998 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexRepairRegionScannerIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexRepairRegionScannerIT.java
@@ -77,7 +77,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import static org.apache.phoenix.hbase.index.IndexRegionObserver.VERIFIED_BYTES;
 import static org.apache.phoenix.mapreduce.index.IndexVerificationResultRepository.RESULT_TABLE_NAME;
 import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.AFTER_REPAIR_EXTRA_UNVERIFIED_INDEX_ROW_COUNT;
 import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.AFTER_REPAIR_EXTRA_VERIFIED_INDEX_ROW_COUNT;
@@ -87,6 +86,7 @@ import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEF
 import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REPAIR_EXTRA_UNVERIFIED_INDEX_ROW_COUNT;
 import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REPAIR_EXTRA_VERIFIED_INDEX_ROW_COUNT;
 import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.SCANNED_DATA_ROW_COUNT;
+import static org.apache.phoenix.query.QueryConstants.VERIFIED_BYTES;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -263,7 +263,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
         String indexTableName, String indexTableFullName,
         int expectedStatus) throws Exception {
 
-        IndexTool tool = IndexToolIT.runIndexTool(getUtility().getConfiguration(), true, false, schemaName, dataTableName,
+        IndexTool tool = IndexToolIT.runIndexTool(getUtility().getConfiguration(), false, schemaName, dataTableName,
             indexTableName,
             null,
             expectedStatus, verifyType, disableLoggingType, "-fi");
@@ -328,7 +328,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             initTablesAndAddExtraRowsToIndex(conn, schemaName, dataTableName, indexTableName, NROWS);
 
             // do index rebuild without -fi and check with scrutiny that index tool failed to fix the extra rows
-            IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.BEFORE);
 
             boolean failed;
@@ -341,7 +341,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             assertTrue(failed);
 
             // now repair the index with -fi
-            IndexTool indexTool = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.BEFORE, "-fi");
 
             long actualRowCount = IndexScrutiny.scrutinizeIndex(conn, dataTableFullName, indexTableFullName);
@@ -385,7 +385,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             conn.commit();
             IndexRegionObserver.setFailPostIndexUpdatesForTesting(false);
 
-            IndexTool indexTool = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.BEFORE, "-fi");
 
             CounterGroup mrJobCounters = IndexToolIT.getMRJobCounters(indexTool);
@@ -399,7 +399,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             assertEquals(0,
                 mrJobCounters.findCounter(BEFORE_REPAIR_EXTRA_UNVERIFIED_INDEX_ROW_COUNT.name()).getValue());
 
-            indexTool = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.ONLY, "-fi");
             mrJobCounters = IndexToolIT.getMRJobCounters(indexTool);
             assertEquals(0,
@@ -451,7 +451,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             IndexRegionObserver.setFailPostIndexUpdatesForTesting(false);
             TestUtil.doMajorCompaction(conn, dataTableFullName);
 
-            IndexTool indexTool = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.BEFORE, "-fi");
 
             CounterGroup mrJobCounters = IndexToolIT.getMRJobCounters(indexTool);
@@ -465,7 +465,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             assertEquals(1,
                 mrJobCounters.findCounter(BEFORE_REPAIR_EXTRA_UNVERIFIED_INDEX_ROW_COUNT.name()).getValue());
 
-            indexTool = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.ONLY, "-fi");
             mrJobCounters = IndexToolIT.getMRJobCounters(indexTool);
 
@@ -515,7 +515,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             commitWithException(conn);
             IndexRegionObserver.setFailDataTableUpdatesForTesting(true);
 
-            IndexTool indexTool = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.BEFORE, "-fi");
 
             long actualRowCount = IndexScrutiny.scrutinizeIndex(conn, dataTableFullName, indexTableFullName);
@@ -538,7 +538,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             initTablesAndAddExtraRowsToIndex(conn, schemaName, dataTableName, indexTableName, NROWS);
 
-            IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.ONLY, "-fi");
 
             Cell cell = IndexToolIT.getErrorMessageFromIndexToolOutputTable(conn, dataTableFullName, indexTableFullName);
@@ -562,7 +562,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             initTablesAndAddExtraRowsToIndex(conn, schemaName, dataTableName, indexTableName, NROWS);
 
             // Run -v AFTER and check it doesn't fix the extra rows and the job fails
-            IndexTool indexTool = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, -1, IndexVerifyType.AFTER, "-fi");
 
             boolean failed;
@@ -591,7 +591,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             initTablesAndAddExtraRowsToIndex(conn, schemaName, dataTableName, indexTableName, NROWS);
 
-            IndexTool indexTool = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.BOTH, "-fi");
 
             long actualRowCount = IndexScrutiny.scrutinizeIndex(conn, dataTableFullName, indexTableFullName);
@@ -616,7 +616,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
 
             Configuration conf = new Configuration(getUtility().getConfiguration());
             conf.set(QueryServices.INDEX_REBUILD_PAGE_SIZE_IN_ROWS, Long.toString(2));
-            IndexTool indexTool = IndexToolIT.runIndexTool(conf,false, false, schemaName, dataTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(conf, false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.BEFORE, IndexDisableLoggingType.NONE,"-fi");
 
             long actualRowCount = IndexScrutiny.scrutinizeIndex(conn, dataTableFullName, indexTableFullName);
@@ -661,11 +661,11 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             conn.commit();
             setIndexRowStatusesToVerified(conn, viewFullName, indexTableFullName1);
 
-            IndexTool indexTool = IndexToolIT.runIndexTool(false, false, schemaName, viewName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, viewName,
                 indexTableName1, null, 0, IndexVerifyType.BEFORE, "-fi");
             assertExtraCounters(indexTool, 1, 0, true);
 
-            indexTool = IndexToolIT.runIndexTool(false, false, schemaName, viewName,
+            indexTool = IndexToolIT.runIndexTool(false, schemaName, viewName,
                 indexTableName2, null, 0, IndexVerifyType.BEFORE, "-fi");
             assertExtraCounters(indexTool, 1, 0, true);
 
@@ -720,7 +720,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             long t1 = customEdge.currentTime();
 
             IndexTool it;
-            it = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            it = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.ONLY,
                 "-fi", "-st", String.valueOf(t0), "-et", String.valueOf(t1));
 
@@ -749,7 +749,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
             setIndexRowStatusesToVerified(conn, dataTableFullName, indexTableFullName);
             customEdge.incrementValue(delta);
             long t2 = customEdge.currentTime();
-            it = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            it = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.ONLY,
                 "-fi", "-st", String.valueOf(t1), "-et", String.valueOf(t2));
 
@@ -763,7 +763,7 @@ public class IndexRepairRegionScannerIT extends ParallelStatsDisabledIT {
                 mrJobCounters.findCounter(BEFORE_REPAIR_EXTRA_UNVERIFIED_INDEX_ROW_COUNT.name()).getValue());
 
             // now run another verification over the entire window [t0, t2]
-            it = IndexToolIT.runIndexTool(false, false, schemaName, dataTableName,
+            it = IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, 0, IndexVerifyType.ONLY,
                 "-fi", "-st", String.valueOf(t0), "-et", String.valueOf(t2));
 
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForNonTxGlobalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForNonTxGlobalIndexIT.java
index 369da11..98d9a4d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForNonTxGlobalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForNonTxGlobalIndexIT.java
@@ -128,7 +128,6 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
     public static final int MAX_LOOKBACK_AGE = 3600;
     private final String tableDDLOptions;
 
-    private final boolean directApi = true;
     private final boolean useSnapshot = false;
     private final boolean mutable;
     private final String indexDDLOptions;
@@ -232,7 +231,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
                     "CREATE INDEX %s ON %s (VAL1) INCLUDE (VAL2) ASYNC " + this.indexDDLOptions, indexTableName, dataTableFullName));
             // Run the index MR job and verify that the index table is built correctly
             IndexTool
-                    indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, new String[0]);
+                    indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, new String[0]);
             assertEquals(NROWS, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
             assertTrue("Index rebuild failed!", indexTool.getJob().isSuccessful());
             TestUtil.assertIndexState(conn, indexTableFullName, PIndexState.ACTIVE, null);
@@ -249,7 +248,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             assertEquals(NROWS, actualRowCount);
             actualRowCount = IndexScrutiny.scrutinizeIndex(conn, dataTableFullName, indexTableFullName);
             assertEquals(NROWS, actualRowCount);
-            indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null,
+            indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null,
                     0, IndexTool.IndexVerifyType.ONLY, new String[0]);
             assertEquals(NROWS, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
             assertEquals(NROWS, indexTool.getJob().getCounters().findCounter(SCANNED_DATA_ROW_COUNT).getValue());
@@ -281,7 +280,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             conn.createStatement()
                     .execute(String.format("CREATE INDEX %s ON %s (NAME) INCLUDE (CODE) ASYNC " + this.indexDDLOptions,
                             indexTableName, dataTableFullName));
-            IndexTool indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0,
+            IndexTool indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0,
                     IndexTool.IndexVerifyType.ONLY);
             if (CompatBaseScannerRegionObserver.isMaxLookbackTimeEnabled(getUtility().getConfiguration())) {
                 Cell cell =
@@ -300,7 +299,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             }
 
             // Run the index tool to populate the index while verifying rows
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0,
                     IndexTool.IndexVerifyType.AFTER);
 
             // Set ttl of index table ridiculously low so that all data is expired
@@ -324,7 +323,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             admin.disableTable(indexToolOutputTable);
             admin.deleteTable(indexToolOutputTable);
             // Run the index tool using the only-verify option, verify it gives no mismatch
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0,
                     IndexTool.IndexVerifyType.ONLY);
             Scan scan = new Scan();
             Table hIndexToolTable =
@@ -384,7 +383,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             //add the fast fail coproc and make sure it goes first
             admin.modifyTable(newTD);
             // Run the index MR job and it should fail (return -1)
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, -1, new String[0]);
 
             // Verify that the index table should be still in the ACTIVE state
@@ -421,7 +420,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             conn.createStatement().execute(stmtString2);
 
             // Run the index MR job and verify that the index table is built correctly
-            IndexTool indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
+            IndexTool indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
             assertEquals(NROWS, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
             assertEquals(NROWS, indexTool.getJob().getCounters().findCounter(SCANNED_DATA_ROW_COUNT).getValue());
             assertEquals(NROWS, indexTool.getJob().getCounters().findCounter(REBUILT_INDEX_ROW_COUNT).getValue());
@@ -444,7 +443,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
                 IndexToolIT.upsertRow(stmt1, i);
             }
             conn.commit();
-            indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BOTH, new String[0]);
+            indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BOTH, new String[0]);
             assertEquals(2 * NROWS, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
             assertEquals(2 * NROWS, indexTool.getJob().getCounters().findCounter(SCANNED_DATA_ROW_COUNT).getValue());
             assertEquals(0, indexTool.getJob().getCounters().findCounter(REBUILT_INDEX_ROW_COUNT).getValue());
@@ -502,7 +501,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             conn.createStatement().execute(stmtString2);
 
             // Run the index MR job and verify that the index table is built correctly
-            IndexTool indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
+            IndexTool indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
             assertEquals(N_ROWS, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
             assertEquals(N_ROWS, indexTool.getJob().getCounters().findCounter(SCANNED_DATA_ROW_COUNT).getValue());
             assertEquals(N_ROWS, indexTool.getJob().getCounters().findCounter(REBUILT_INDEX_ROW_COUNT).getValue());
@@ -525,13 +524,13 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             conn.createStatement().execute(stmtString3);
             conn.commit();
             // Verify that IndexTool reports that there are old design index rows
-            indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.ONLY, new String[0]);
+            indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.ONLY, new String[0]);
             assertEquals(N_OLD_ROWS, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_OLD_INDEX_ROW_COUNT).getValue());
             // Clean up all old design rows
-            indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
+            indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
             assertEquals(N_OLD_ROWS, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_OLD_INDEX_ROW_COUNT).getValue());
             // Verify that IndexTool does not report them anymore
-            indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
+            indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
             assertEquals(0, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_OLD_INDEX_ROW_COUNT).getValue());
             actualRowCount = IndexScrutiny.scrutinizeIndex(conn, dataTableFullName, indexTableFullName);
             assertEquals(N_ROWS, actualRowCount);
@@ -561,17 +560,17 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
                     "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) ASYNC " + this.indexDDLOptions, indexTableName, viewFullName));
             TestUtil.addCoprocessor(conn, "_IDX_" + dataTableFullName, IndexToolIT.MutationCountingRegionObserver.class);
             // Run the index MR job and verify that the index table rebuild succeeds
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName, indexTableName,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.AFTER);
             assertEquals(1, IndexToolIT.MutationCountingRegionObserver.getMutationCount());
             IndexToolIT.MutationCountingRegionObserver.setMutationCount(0);
             // Since all the rows are in the index table, running the index tool with the "-v BEFORE" option should not
             // write any index rows
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName, indexTableName,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.BEFORE);
             assertEquals(0, IndexToolIT.MutationCountingRegionObserver.getMutationCount());
             // The "-v BOTH" option should not write any index rows either
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName, indexTableName,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.BOTH);
             assertEquals(0, IndexToolIT.MutationCountingRegionObserver.getMutationCount());
         }
@@ -603,7 +602,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
                     "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) ASYNC " + this.indexDDLOptions, indexTableName, viewFullName));
             if (CompatBaseScannerRegionObserver.isMaxLookbackTimeEnabled(getUtility().getConfiguration())) {
                 // Run the index MR job and verify that the index table rebuild fails
-                IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName, indexTableName,
+                IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName, indexTableName,
                         null, -1, IndexTool.IndexVerifyType.AFTER);
                 // The index tool output table should report that there is a missing index row
                 Cell cell = IndexToolIT.getErrorMessageFromIndexToolOutputTable(conn, dataTableFullName, "_IDX_" + dataTableFullName);
@@ -615,7 +614,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
                     Assert.fail("Fail to parsing the error message from IndexToolOutputTable");
                 }
             } else {
-                IndexTool indexTool = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName, indexTableName,
+                IndexTool indexTool = IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName, indexTableName,
                         null, 0, IndexTool.IndexVerifyType.AFTER);
                 assertEquals(1, indexTool.getJob().getCounters().findCounter(AFTER_REBUILD_BEYOND_MAXLOOKBACK_MISSING_INDEX_ROW_COUNT).getValue());
             }
@@ -646,14 +645,14 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             conn.createStatement().execute(String.format(
                     "CREATE INDEX %s ON %s (NAME) INCLUDE (ZIP) ASYNC " + this.indexDDLOptions, indexTableName, dataTableFullName));
             // Run the index MR job and verify that the index table rebuild fails
-            IndexTool it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            IndexTool it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, -1, IndexTool.IndexVerifyType.BEFORE);
             Assert.assertEquals(PIndexState.BUILDING, TestUtil.getIndexState(conn, indexTableFullName));
 
             // Now there is no exception, so the second partial build should retry
             Long scn = it.getJob().getConfiguration().getLong(CURRENT_SCN_VALUE, 1L);
             IndexRebuildRegionScanner.setThrowExceptionForRebuild(false);
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.BEFORE,"-rv", Long.toString(scn));
             Assert.assertEquals(PIndexState.ACTIVE, TestUtil.getIndexState(conn, indexTableFullName));
             ResultSet rs =
@@ -683,7 +682,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             // Run the index MR job to only verify that each data table row has a corresponding index row
             // IndexTool will go through each data table row and record the mismatches in the output table
             // called PHOENIX_INDEX_TOOL
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY);
             if (CompatBaseScannerRegionObserver.isMaxLookbackTimeEnabled(getUtility().getConfiguration())) {
                 Cell cell = IndexToolIT.getErrorMessageFromIndexToolOutputTable(conn, dataTableFullName, indexTableFullName);
@@ -703,9 +702,9 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             deleteAllRows(conn,
                 TableName.valueOf(IndexVerificationOutputRepository.OUTPUT_TABLE_NAME));
             // Run the index tool to populate the index while verifying rows
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.AFTER);
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY);
         }
     }
@@ -730,7 +729,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             conn.createStatement().execute("upsert into " + dataTableFullName + " values (2, 'Phoenix1', 'B')");
             conn.commit();
 
-            IndexTool it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            IndexTool it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.AFTER);
             Long scn = it.getJob().getConfiguration().getLong(CURRENT_SCN_VALUE, 1L);
 
@@ -746,7 +745,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
 
             deleteOneRowFromResultTable(conn, scn, indexTableFullName);
 
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.AFTER, "-rv", Long.toString(scn));
             scn = it.getJob().getConfiguration().getLong(CURRENT_SCN_VALUE, 1L);
 
@@ -762,7 +761,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             expectedStatus.set(1, RUN_STATUS_SKIPPED);
             expectedStatus.set(2, RUN_STATUS_SKIPPED);
 
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.AFTER, "-rv", Long.toString(scn));
             scn = it.getJob().getConfiguration().getLong(CURRENT_SCN_VALUE, 1L);
             verifyRunStatusFromResultTable(conn, scn, indexTableFullName, 8, expectedStatus);
@@ -832,7 +831,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             IndexTool it;
             if(!mutable) {
                 // job with 2 rows
-                it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+                it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                         null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t0),"-et", String.valueOf(t2));
                 verifyCounters(it, 2, 2);
                 //increment time between rebuilds so that PHOENIX_INDEX_TOOL and
@@ -840,65 +839,65 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
                 customEdge.incrementValue(waitForUpsert);
 
                 // only one row
-                it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+                it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                         null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t1),"-et", String.valueOf(t2));
                 verifyCounters(it, 1, 1);
                 customEdge.incrementValue(waitForUpsert);
                 // no rows
-                it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+                it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                         null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t5),"-et", String.valueOf(t6));
                 verifyCounters(it, 0, 0);
                 customEdge.incrementValue(waitForUpsert);
                 //view index
                 // job with 2 rows
-                it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName, viewIndexName,
+                it = IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName, viewIndexName,
                         null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t0),"-et", String.valueOf(t2));
                 verifyCounters(it, 2, 2);
                 customEdge.incrementValue(waitForUpsert);
                 // only one row
-                it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName, viewIndexName,
+                it = IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName, viewIndexName,
                         null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t1),"-et", String.valueOf(t2));
                 verifyCounters(it, 1, 1);
                 customEdge.incrementValue(waitForUpsert);
                 // no rows
-                it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName, viewIndexName,
+                it = IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName, viewIndexName,
                         null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t5),"-et", String.valueOf(t6));
                 verifyCounters(it, 0, 0);
                 customEdge.incrementValue(waitForUpsert);
                 return;
             }
             // regular job without delete row
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t0),"-et", String.valueOf(t4));
             verifyCounters(it, 2, 3);
             customEdge.incrementValue(waitForUpsert);
 
             // job with 2 rows
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t0),"-et", String.valueOf(t2));
             verifyCounters(it, 2, 2);
             customEdge.incrementValue(waitForUpsert);
 
             // job with update on only one row
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t1),"-et", String.valueOf(t3));
             verifyCounters(it, 1, 2);
             customEdge.incrementValue(waitForUpsert);
 
             // job with update on only one row
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t2),"-et", String.valueOf(t4));
             verifyCounters(it, 1, 2);
             customEdge.incrementValue(waitForUpsert);
 
             // job with update on only one row
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t4),"-et", String.valueOf(t5));
             verifyCounters(it, 1, 1);
             customEdge.incrementValue(waitForUpsert);
 
             // job with no new updates on any row
-            it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+            it = IndexToolIT.runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t5),"-et", String.valueOf(t6));
             verifyCounters(it, 0, 0);
             customEdge.incrementValue(waitForUpsert);
@@ -953,28 +952,28 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             IndexTool it;
             // regular job with delete row
             it =
-                    IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName,
+                    IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName,
                             viewIndexName, null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t1),
                             "-et", String.valueOf(t4));
             verifyCounters(it, 2, 2);
 
             // job with 1 row
             it =
-                    IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName,
+                    IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName,
                             viewIndexName, null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t1),
                             "-et", String.valueOf(t2));
             verifyCounters(it, 1, 1);
 
             // job with update on only one row
             it =
-                    IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName,
+                    IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName,
                             viewIndexName, null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t2),
                             "-et", String.valueOf(t3));
             verifyCounters(it, 1, 1);
 
             // job with update on 2 rows
             it =
-                    IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName,
+                    IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName,
                             viewIndexName, null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t1),
                             "-et", String.valueOf(t3));
             verifyCounters(it, 2, 2);
@@ -993,7 +992,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             */
             // job with no new updates on any row
             it =
-                    IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, viewName,
+                    IndexToolIT.runIndexTool(useSnapshot, schemaName, viewName,
                             viewIndexName, null, 0, IndexTool.IndexVerifyType.ONLY, "-st", String.valueOf(t4),
                             "-et", String.valueOf(t5));
             verifyCounters(it, 0, 0);
@@ -1158,7 +1157,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             deleteAllRows(conn, TableName.valueOf(IndexVerificationOutputRepository.OUTPUT_TABLE_NAME));
             getUtility().getConfiguration().
                 set(IndexRebuildRegionScanner.PHOENIX_INDEX_MR_LOG_BEYOND_MAX_LOOKBACK_ERRORS, "true");
-            IndexTool it = IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName,
+            IndexTool it = IndexToolIT.runIndexTool(useSnapshot, schemaName,
                 dataTableName,
                 indexTableName, null, 0, IndexTool.IndexVerifyType.ONLY);
             TestUtil.dumpTable(conn, TableName.valueOf(IndexVerificationOutputRepository.OUTPUT_TABLE_NAME));
@@ -1210,7 +1209,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             // Run the index MR job and verify that the index table is built correctly
             Configuration conf = new Configuration(getUtility().getConfiguration());
             conf.set(QueryServices.INDEX_REBUILD_PAGE_SIZE_IN_ROWS, Long.toString(2));
-            IndexTool indexTool = IndexToolIT.runIndexTool(conf, directApi, useSnapshot, schemaName, dataTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(conf, useSnapshot, schemaName, dataTableName,
                 indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, IndexTool.IndexDisableLoggingType.NONE, new String[0]);
             assertEquals(NROWS, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
             assertEquals(NROWS, indexTool.getJob().getCounters().findCounter(SCANNED_DATA_ROW_COUNT).getValue());
@@ -1248,7 +1247,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             Configuration conf = new Configuration(getUtility().getConfiguration());
             conf.set(QueryServices.INDEX_REBUILD_PAGE_SIZE_IN_ROWS, Long.toString(1));
             IndexTool indexTool =
-                    IndexToolIT.runIndexTool(conf, directApi, useSnapshot, schemaName,
+                    IndexToolIT.runIndexTool(conf, useSnapshot, schemaName,
                         dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE,
                         IndexTool.IndexDisableLoggingType.NONE ,new String[0]);
             assertEquals(3, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
@@ -1302,7 +1301,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             Configuration conf = new Configuration(getUtility().getConfiguration());
             conf.set(QueryServices.INDEX_REBUILD_PAGE_SIZE_IN_ROWS, Long.toString(2));
             IndexTool indexTool =
-                    IndexToolIT.runIndexTool(conf, directApi, useSnapshot, schemaName,
+                    IndexToolIT.runIndexTool(conf, useSnapshot, schemaName,
                         dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.AFTER,
                         IndexTool.IndexDisableLoggingType.NONE, "-st", String.valueOf(minTs), "-et",
                         String.valueOf(EnvironmentEdgeManager.currentTimeMillis()));
@@ -1396,7 +1395,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             conn.createStatement().execute(createViewIndex);
             conn.commit();
             // Rebuild using index tool
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, view1Name, indexTableName);
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, view1Name, indexTableName);
             ResultSet rs =
                     conn.createStatement()
                             .executeQuery("SELECT COUNT(*) FROM " + indexTableFullName);
@@ -1475,7 +1474,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
             conn.createStatement().execute(createViewIndex);
             conn.commit();
             // Rebuild using index tool
-            IndexToolIT.runIndexTool(directApi, useSnapshot, schemaName, view1Name, indexTableName);
+            IndexToolIT.runIndexTool(useSnapshot, schemaName, view1Name, indexTableName);
             ResultSet rs =
                     conn.createStatement()
                             .executeQuery("SELECT COUNT(*) FROM " + indexTableFullName);
@@ -1558,7 +1557,7 @@ public class IndexToolForNonTxGlobalIndexIT extends BaseTest {
                                       String indexTableName, String indexTableFullName,
                                       int expectedStatus) throws Exception {
 
-        IndexTool tool = IndexToolIT.runIndexTool(getUtility().getConfiguration(), true, false, schemaName, dataTableName,
+        IndexTool tool = IndexToolIT.runIndexTool(getUtility().getConfiguration(), false, schemaName, dataTableName,
             indexTableName,
             null,
             expectedStatus, verifyType, disableLoggingType,new String[0]);
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 b121c3f..7ec4400 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
@@ -17,11 +17,39 @@
  */
 package org.apache.phoenix.end2end;
 
-import org.apache.phoenix.compile.ExplainPlan;
-import org.apache.phoenix.compile.ExplainPlanAttributes;
-import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
-import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
-import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import static org.apache.phoenix.mapreduce.PhoenixJobCounters.INPUT_RECORDS;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_BEYOND_MAXLOOKBACK_INVALID_INDEX_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_BEYOND_MAXLOOKBACK_MISSING_INDEX_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_EXPIRED_INDEX_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT_COZ_EXTRA_CELLS;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT_COZ_MISSING_CELLS;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_MISSING_INDEX_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_VALID_INDEX_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.REBUILT_INDEX_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.SCANNED_DATA_ROW_COUNT;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -43,18 +71,25 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.CounterGroup;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.phoenix.mapreduce.index.IndexVerificationOutputRepository;
-import org.apache.phoenix.mapreduce.index.IndexVerificationResultRepository;
+import org.apache.phoenix.compile.ExplainPlan;
+import org.apache.phoenix.compile.ExplainPlanAttributes;
+import org.apache.phoenix.end2end.index.IndexTestUtil;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
 import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.mapreduce.index.IndexVerificationOutputRepository;
+import org.apache.phoenix.mapreduce.index.IndexVerificationResultRepository;
 import org.apache.phoenix.mapreduce.index.PhoenixIndexImportDirectMapper;
 import org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters;
 import org.apache.phoenix.mapreduce.index.PhoenixServerBuildIndexMapper;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
 import org.apache.phoenix.transaction.PhoenixTransactionProvider.Feature;
 import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -72,39 +107,6 @@ import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.phoenix.mapreduce.PhoenixJobCounters.INPUT_RECORDS;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_BEYOND_MAXLOOKBACK_MISSING_INDEX_ROW_COUNT;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT_COZ_EXTRA_CELLS;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT_COZ_MISSING_CELLS;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_EXPIRED_INDEX_ROW_COUNT;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_MISSING_INDEX_ROW_COUNT;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_VALID_INDEX_ROW_COUNT;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_BEYOND_MAXLOOKBACK_INVALID_INDEX_ROW_COUNT;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.REBUILT_INDEX_ROW_COUNT;
-import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.SCANNED_DATA_ROW_COUNT;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 @Category(NeedsOwnMiniClusterTest.class)
 @RunWith(Parameterized.class)
 public class IndexToolIT extends BaseTest {
@@ -115,7 +117,6 @@ public class IndexToolIT extends BaseTest {
     private final boolean localIndex;
     private final boolean mutable;
     private final boolean transactional;
-    private final boolean directApi;
     private final String tableDDLOptions;
     private final String indexDDLOptions;
     private final boolean useSnapshot;
@@ -123,11 +124,10 @@ public class IndexToolIT extends BaseTest {
     private final boolean namespaceMapped;
 
     public IndexToolIT(String transactionProvider, boolean mutable, boolean localIndex,
-            boolean directApi, boolean useSnapshot, boolean useTenantId, boolean namespaceMapped) {
+            boolean useSnapshot, boolean useTenantId, boolean namespaceMapped) {
         this.localIndex = localIndex;
         this.mutable = mutable;
         this.transactional = transactionProvider != null;
-        this.directApi = directApi;
         this.useSnapshot = useSnapshot;
         this.useTenantId = useTenantId;
         this.namespaceMapped = namespaceMapped;
@@ -157,7 +157,7 @@ public class IndexToolIT extends BaseTest {
     @BeforeParam
     public static synchronized void setup(
             String transactionProvider, boolean mutable, boolean localIndex,
-            boolean directApi, boolean useSnapshot, boolean useTenantId, boolean namespaceMapped)
+            boolean useSnapshot, boolean useTenantId, boolean namespaceMapped)
                     throws Exception {
         if (clusterInitialized && Boolean.valueOf(namespaceMapped).equals(utility.getConfiguration()
                 .getBoolean(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, true))) {
@@ -178,6 +178,7 @@ public class IndexToolIT extends BaseTest {
             serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB,
                 QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
             serverProps.put(QueryServices.INDEX_REBUILD_PAGE_SIZE_IN_ROWS, Long.toString(8));
+            serverProps.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
             serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED,
                 Boolean.toString(namespaceMapped));
             Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
@@ -193,34 +194,30 @@ public class IndexToolIT extends BaseTest {
     }
 
     @Parameters(
-            name = "transactionProvider={0},mutable={1},localIndex={2},directApi={3}," +
-                "useSnapshot={4},useTenant={5},namespaceMapped={6}")
+            name = "transactionProvider={0},mutable={1},localIndex={2}," +
+                "useSnapshot={3},useTenant={4},namespaceMapped={5}")
     public static synchronized Collection<Object[]> data() {
         List<Object[]> list = Lists.newArrayListWithExpectedSize(48);
         boolean[] Booleans = new boolean[] { false, true };
         for (boolean namespaceMapped : Booleans) {
-            for (String transactionProvider : new String[] {"TEPHRA", "OMID", null}) {
-                if(transactionProvider !=null &&
-                        !TransactionFactory.Provider.valueOf(transactionProvider).runTests()) {
-                    continue;
-                }
-                for (boolean mutable : Booleans) {
-                    for (boolean localIndex : Booleans) {
-                        if (!localIndex
-                                || transactionProvider == null
-                                || !TransactionFactory.getTransactionProvider(
-                                        TransactionFactory.Provider.valueOf(transactionProvider))
-                                    .isUnsupported(Feature.ALLOW_LOCAL_INDEX)) {
-                            if (localIndex) {
-                                for (boolean directApi : Booleans) {
-                                    list.add(new Object[]{transactionProvider, mutable, localIndex,
-                                            directApi, false, false, namespaceMapped});
-                                }
+            for (boolean useSnapshot : Booleans) {
+                for (String transactionProvider : new String[] {"TEPHRA", "OMID", null}) {
+                    if(transactionProvider !=null &&
+                            !TransactionFactory.Provider.valueOf(transactionProvider).runTests()) {
+                        continue;
+                    }
+                    for (boolean mutable : Booleans) {
+                        for (boolean localIndex : Booleans) {
+                            if(localIndex && useSnapshot) {
+                                continue;
                             }
-                            else {
-                                // Due to PHOENIX-5375 and PHOENIX-5376, the snapshot and bulk load options are ignored for global indexes
+                            if (!localIndex
+                                    || transactionProvider == null
+                                    || !TransactionFactory.getTransactionProvider(
+                                            TransactionFactory.Provider.valueOf(transactionProvider))
+                                        .isUnsupported(Feature.ALLOW_LOCAL_INDEX)) {
                                 list.add(new Object[]{transactionProvider, mutable, localIndex,
-                                        true, false, false, namespaceMapped});
+                                        useSnapshot, false, namespaceMapped});
                             }
                         }
                     }
@@ -228,7 +225,7 @@ public class IndexToolIT extends BaseTest {
             }
         }
         // Add the usetenantId
-        list.add(new Object[] { null, false, false, true, false, true, false});
+        list.add(new Object[] { null, false, true, false, true, false});
         return TestUtil.filterTxParamData(list,0);
     }
 
@@ -253,7 +250,7 @@ public class IndexToolIT extends BaseTest {
         String indexTableName = generateUniqueName();
         String indexTableFullName = SchemaUtil.getTableName(schemaName, indexTableName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        PhoenixConnection conn = (PhoenixConnection) DriverManager.getConnection(getUrl(), props);
         try {
             if(namespaceMapped) {
                 conn.createStatement().execute("CREATE SCHEMA " + schemaName);
@@ -264,11 +261,11 @@ public class IndexToolIT extends BaseTest {
                             + tableDDLOptions;
             conn.createStatement().execute(stmString1);
             String upsertQuery = String.format("UPSERT INTO %s VALUES(?, ?, ?)", dataTableFullName);
-            PreparedStatement stmt1 = conn.prepareStatement(upsertQuery);
+            PreparedStatement upsertStmt = conn.prepareStatement(upsertQuery);
 
             // insert two rows
-            upsertRow(stmt1, 1);
-            upsertRow(stmt1, 2);
+            upsertRow(upsertStmt, 1);
+            upsertRow(upsertStmt, 2);
             conn.commit();
 
             if (transactional) {
@@ -321,20 +318,28 @@ public class IndexToolIT extends BaseTest {
             assertEquals("SERVER FILTER BY (LPAD(UPPER(NAME, 'en_US'), 8, 'x') || '_xyz') = 'xxUNAME2_xyz'",
                 explainPlanAttributes.getServerWhereFilter().replaceAll(":", "."));
 
-            ResultSet rs = stmt1.executeQuery(selectSql);
+            ResultSet rs = conn.createStatement().executeQuery(selectSql);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
             assertFalse(rs.next());
             conn.commit();
 
             // run the index MR job.
-            runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName);
+            runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName);
 
+            if (!localIndex) {
+                IndexTestUtil.assertRowsForEmptyColValue(conn, indexTableFullName,
+                    QueryConstants.VERIFIED_BYTES);
+            }
             // insert two more rows
-            upsertRow(stmt1, 3);
-            upsertRow(stmt1, 4);
+            upsertRow(upsertStmt, 3);
+            upsertRow(upsertStmt, 4);
             conn.commit();
 
+            if (!localIndex) {
+                IndexTestUtil.assertRowsForEmptyColValue(conn, indexTableFullName,
+                    QueryConstants.VERIFIED_BYTES);
+            }
             // assert we are pulling from index table.
             plan = conn.prepareStatement(selectSql)
                 .unwrap(PhoenixPreparedStatement.class).optimizeQuery()
@@ -359,7 +364,7 @@ public class IndexToolIT extends BaseTest {
                 return;
             }
             // Run the index MR job and verify that the global index table is built correctly
-            IndexTool indexTool = runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
+            IndexTool indexTool = runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.BEFORE, new String[0]);
             assertEquals(4, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
             assertEquals(4, indexTool.getJob().getCounters().findCounter(SCANNED_DATA_ROW_COUNT).getValue());
             assertEquals(0, indexTool.getJob().getCounters().findCounter(REBUILT_INDEX_ROW_COUNT).getValue());
@@ -510,7 +515,7 @@ public class IndexToolIT extends BaseTest {
                     .execute(String.format(upsertQueryStr, viewTenantName, tenantId, 1, "x"));
             connTenant.commit();
 
-            runIndexTool(true, false, "", viewTenantName, indexNameTenant,
+            runIndexTool(false, "", viewTenantName, indexNameTenant,
                     tenantId, 0, new String[0]);
 
             String selectSql = String.format("SELECT ID FROM %s WHERE NAME='x'", viewTenantName);
@@ -538,7 +543,7 @@ public class IndexToolIT extends BaseTest {
             admin.disableTable(tableName);
             admin.truncateTable(tableName, false);
 
-            runIndexTool(true, false, "", viewTenantName, indexNameTenant,
+            runIndexTool(false, "", viewTenantName, indexNameTenant,
                     tenantId, 0, new String[0]);
             Table htable= queryServices.getTable(Bytes.toBytes(viewIndexTableName));
             int count = getUtility().countRows(htable);
@@ -558,7 +563,7 @@ public class IndexToolIT extends BaseTest {
 
             // run the index MR job this time with tenant id.
             // We expect it to return -1 because indexTable is not correct for this tenant.
-            runIndexTool(true, false, schemaName, dataTableName, indexNameGlobal,
+            runIndexTool(false, schemaName, dataTableName, indexNameGlobal,
                     tenantId, -1, new String[0]);
 
         } finally {
@@ -600,7 +605,7 @@ public class IndexToolIT extends BaseTest {
                         (localIndex ? "LOCAL" : ""), indexTableName, dataTableFullName);
             conn.createStatement().execute(indexDDL);
 
-            runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName);
+            runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName);
 
             ResultSet rs =
                     conn.createStatement().executeQuery(
@@ -681,7 +686,7 @@ public class IndexToolIT extends BaseTest {
             conn.createStatement().execute(indexDDL);
 
             // run with 50% sampling rate, split if data table more than 3 regions
-            runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, "-sp", "50", "-spa", "3");
+            runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, "-sp", "50", "-spa", "3");
 
             TableName hIndexName = TableName.valueOf(
                 SchemaUtil.getPhysicalHBaseTableName(schemaName, indexTableName, namespaceMapped)
@@ -788,7 +793,7 @@ public class IndexToolIT extends BaseTest {
             conn.commit();
 
             // run the index MR job.
-            runIndexTool(directApi, useSnapshot, sSchemaName, sDataTableName, sIndexTableName);
+            runIndexTool(useSnapshot, sSchemaName, sDataTableName, sIndexTableName);
 
             // insert two more rows
             upsertRow(stmt1, 3);
@@ -836,15 +841,15 @@ public class IndexToolIT extends BaseTest {
         return indexTool.getJob().getCounters().getGroup(PhoenixIndexToolJobCounters.class.getName());
     }
 
-    private static List<String> getArgList (boolean directApi, boolean useSnapshot, String schemaName,
+    private static List<String> getArgList (boolean useSnapshot, String schemaName,
                                             String dataTable, String indxTable, String tenantId,
                                             IndexTool.IndexVerifyType verifyType, Long startTime,
                                             Long endTime, Long incrementalVerify) {
-        return getArgList(directApi, useSnapshot, schemaName, dataTable, indxTable, tenantId,
+        return getArgList(useSnapshot, schemaName, dataTable, indxTable, tenantId,
             verifyType, startTime, endTime, IndexTool.IndexDisableLoggingType.NONE, incrementalVerify, false);
     }
 
-    private static List<String> getArgList (boolean directApi, boolean useSnapshot, String schemaName,
+    private static List<String> getArgList (boolean useSnapshot, String schemaName,
                             String dataTable, String indxTable, String tenantId,
                             IndexTool.IndexVerifyType verifyType, Long startTime, Long endTime,
                                             IndexTool.IndexDisableLoggingType disableLoggingType,
@@ -856,9 +861,6 @@ public class IndexToolIT extends BaseTest {
         // Work around CLI-254. The long-form arg parsing doesn't strip off double-quotes
         args.add("--data-table=" + dataTable);
         args.add("--index-table=" + indxTable);
-        if (directApi) {
-            args.add("-direct");
-        }
         args.add("-v");
         args.add(verifyType.getValue()); // verify index rows inline
 
@@ -900,36 +902,36 @@ public class IndexToolIT extends BaseTest {
         return args;
     }
 
-    public static String[] getArgValues(boolean directApi, boolean useSnapshot, String schemaName,
+    public static String[] getArgValues( boolean useSnapshot, String schemaName,
             String dataTable, String indexTable, String tenantId, IndexTool.IndexVerifyType verifyType) {
-        List<String> args = getArgList(directApi, useSnapshot, schemaName, dataTable, indexTable,
+        List<String> args = getArgList(useSnapshot, schemaName, dataTable, indexTable,
                 tenantId, verifyType, null, null, null);
         return args.toArray(new String[0]);
     }
 
-    public static String[] getArgValues(boolean directApi, boolean useSnapshot, String schemaName,
+    public static String[] getArgValues(boolean useSnapshot, String schemaName,
                                         String dataTable, String indexTable, String tenantId, IndexTool.IndexVerifyType verifyType,
                                         IndexTool.IndexDisableLoggingType disableLoggingType) {
-        List<String> args = getArgList(directApi, useSnapshot, schemaName, dataTable, indexTable,
+        List<String> args = getArgList(useSnapshot, schemaName, dataTable, indexTable,
                 tenantId, verifyType, null, null, disableLoggingType, null, false);
         return args.toArray(new String[0]);
     }
 
-    public static String [] getArgValues(boolean directApi, boolean useSnapshot, String schemaName,
+    public static String [] getArgValues(boolean useSnapshot, String schemaName,
             String dataTable, String indexTable, String tenantId,
             IndexTool.IndexVerifyType verifyType, Long startTime, Long endTime) {
-        List<String> args = getArgList(directApi, useSnapshot, schemaName, dataTable, indexTable,
+        List<String> args = getArgList(useSnapshot, schemaName, dataTable, indexTable,
                 tenantId, verifyType, startTime, endTime, null);
         return args.toArray(new String[0]);
     }
 
-    public static String [] getArgValues(boolean directApi, boolean useSnapshot, String schemaName,
+    public static String [] getArgValues(boolean useSnapshot, String schemaName,
                                          String dataTable, String indexTable, String tenantId,
                                          IndexTool.IndexVerifyType verifyType, Long startTime,
                                          Long endTime,
                                          IndexTool.IndexDisableLoggingType disableLoggingType,
                                          Long incrementalVerify ) {
-        List<String> args = getArgList(directApi, useSnapshot, schemaName, dataTable, indexTable,
+        List<String> args = getArgList(useSnapshot, schemaName, dataTable, indexTable,
             tenantId, verifyType, startTime, endTime, disableLoggingType, incrementalVerify, false);
         return args.toArray(new String[0]);
     }
@@ -941,7 +943,7 @@ public class IndexToolIT extends BaseTest {
         IndexTool.IndexDisableLoggingType disableLoggingType,
         Long incrementalVerify,
         boolean useIndexTableAsSource) {
-        List<String> args = getArgList(directApi, useSnapshot, schemaName, dataTable, indexTable,
+        List<String> args = getArgList(useSnapshot, schemaName, dataTable, indexTable,
             tenantId, verifyType, startTime, endTime, disableLoggingType, incrementalVerify, useIndexTableAsSource);
         return args.toArray(new String[0]);
     }
@@ -954,12 +956,12 @@ public class IndexToolIT extends BaseTest {
         stmt.executeUpdate();
     }
 
-    public static void runIndexTool(boolean directApi, boolean useSnapshot, String schemaName,
+    public static void runIndexTool(boolean useSnapshot, String schemaName,
             String dataTableName, String indexTableName) throws Exception {
-        runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, new String[0]);
+        runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, new String[0]);
     }
 
-    private static void verifyMapper(Job job, boolean directApi, boolean useSnapshot, String schemaName,
+    private static void verifyMapper(Job job, boolean useSnapshot, String schemaName,
                                   String dataTableName, String indexTableName, String tenantId) throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         if (tenantId != null) {
@@ -981,36 +983,36 @@ public class IndexToolIT extends BaseTest {
         }
     }
 
-    public static void runIndexTool(boolean directApi, boolean useSnapshot, String schemaName,
+    public static void runIndexTool(boolean useSnapshot, String schemaName,
             String dataTableName, String indexTableName, String... additionalArgs) throws Exception {
-        runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, null, 0, additionalArgs);
+        runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, null, 0, additionalArgs);
     }
 
-    public static IndexTool runIndexTool(boolean directApi, boolean useSnapshot, String schemaName,
+    public static IndexTool runIndexTool(boolean useSnapshot, String schemaName,
             String dataTableName, String indexTableName, String tenantId, int expectedStatus,
             String... additionalArgs) throws Exception {
-        return runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName, tenantId, expectedStatus,
+        return runIndexTool(useSnapshot, schemaName, dataTableName, indexTableName, tenantId, expectedStatus,
                 IndexTool.IndexVerifyType.NONE, additionalArgs);
     }
 
-    public static IndexTool runIndexTool(boolean directApi, boolean useSnapshot, String schemaName,
+    public static IndexTool runIndexTool( boolean useSnapshot, String schemaName,
                                          String dataTableName, String indexTableName, String tenantId,
                                          int expectedStatus, IndexTool.IndexVerifyType verifyType,
                                          String... additionalArgs) throws Exception {
         Configuration conf = new Configuration(getUtility().getConfiguration());
-        return runIndexTool(conf, directApi, useSnapshot, schemaName, dataTableName, indexTableName,
+        return runIndexTool(conf, useSnapshot, schemaName, dataTableName, indexTableName,
             tenantId, expectedStatus, verifyType, IndexTool.IndexDisableLoggingType.NONE, additionalArgs);
     }
 
 
-    public static IndexTool runIndexTool(Configuration conf, boolean directApi, boolean useSnapshot, String schemaName,
+    public static IndexTool runIndexTool(Configuration conf, boolean useSnapshot, String schemaName,
             String dataTableName, String indexTableName, String tenantId,
             int expectedStatus, IndexTool.IndexVerifyType verifyType, IndexTool.IndexDisableLoggingType disableLoggingType,
             String... additionalArgs) throws Exception {
         IndexTool indexingTool = new IndexTool();
         conf.set(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
         indexingTool.setConf(conf);
-        final String[] cmdArgs = getArgValues(directApi, useSnapshot, schemaName, dataTableName,
+        final String[] cmdArgs = getArgValues(useSnapshot, schemaName, dataTableName,
             indexTableName, tenantId, verifyType, disableLoggingType);
         List<String> cmdArgList = new ArrayList<>(Arrays.asList(cmdArgs));
         cmdArgList.addAll(Arrays.asList(additionalArgs));
@@ -1018,7 +1020,7 @@ public class IndexToolIT extends BaseTest {
         int status = indexingTool.run(cmdArgList.toArray(new String[cmdArgList.size()]));
 
         if (expectedStatus == 0) {
-            verifyMapper(indexingTool.getJob(), directApi, useSnapshot, schemaName, dataTableName, indexTableName, tenantId);
+            verifyMapper(indexingTool.getJob(), useSnapshot, schemaName, dataTableName, indexTableName, tenantId);
         }
         assertEquals(expectedStatus, status);
         return indexingTool;
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolTimeRangeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolTimeRangeIT.java
index c3125d0..a2f9e03 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolTimeRangeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolTimeRangeIT.java
@@ -195,7 +195,7 @@ public class IndexToolTimeRangeIT extends BaseTest {
     }
 
     private void runIndexTool(String [] args, int expectedStatus) throws Exception {
-        IndexToolIT.runIndexTool(true, false, schemaName, dataTableName,
+        IndexToolIT.runIndexTool(false, schemaName, dataTableName,
                 indexTableName, null, expectedStatus,
                 IndexTool.IndexVerifyType.NONE, args);
     }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexVerificationOldDesignIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexVerificationOldDesignIT.java
index eba60bc..86095c5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexVerificationOldDesignIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexVerificationOldDesignIT.java
@@ -89,7 +89,7 @@ public class IndexVerificationOldDesignIT extends BaseTest {
 
             upsertValidRows(conn, dataTableFullName);
 
-            IndexTool indexTool = IndexToolIT.runIndexTool(true, false, schemaName, dataTableName, indexTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY);
 
             assertEquals(0, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT).getValue());
@@ -98,7 +98,7 @@ public class IndexVerificationOldDesignIT extends BaseTest {
 
             conn.createStatement().execute("upsert into " + indexTableFullName + " values ('Phoenix5', 6,'G')");
             conn.commit();
-            indexTool = IndexToolIT.runIndexTool(true, false, schemaName, dataTableName, indexTableName,
+            indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY);
 
             assertEquals(1, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT).getValue());
@@ -109,7 +109,7 @@ public class IndexVerificationOldDesignIT extends BaseTest {
             injectEdge.setValue(EnvironmentEdgeManager.currentTimeMillis() + ttl*1000);
             EnvironmentEdgeManager.injectEdge(injectEdge);
 
-            indexTool = IndexToolIT.runIndexTool(true, false, schemaName, dataTableName, indexTableName,
+            indexTool = IndexToolIT.runIndexTool(false, schemaName, dataTableName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY);
 
             assertEquals(0, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT).getValue());
@@ -141,14 +141,14 @@ public class IndexVerificationOldDesignIT extends BaseTest {
 
             upsertValidRows(conn, fullViewName);
 
-            IndexToolIT.runIndexTool(true, false, schemaName, viewName, indexTableName,
+            IndexToolIT.runIndexTool(false, schemaName, viewName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY);
 
             conn.createStatement().execute("upsert into " + indexTableFullName + " values ('Phoenix5', 6,'G')");
             conn.createStatement().execute("delete from " + indexTableFullName + " where \"0:CODE\" = 'D'");
             conn.commit();
 
-            IndexTool indexTool = IndexToolIT.runIndexTool(true, false, schemaName, viewName, indexTableName,
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, schemaName, viewName, indexTableName,
                     null, 0, IndexTool.IndexVerifyType.ONLY);
             assertEquals(1, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT).getValue());
             assertEquals(4, indexTool.getJob().getCounters().findCounter(BEFORE_REBUILD_VALID_INDEX_ROW_COUNT).getValue());
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameExtendedIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameExtendedIT.java
index e2fd941..73eddfb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameExtendedIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameExtendedIT.java
@@ -91,7 +91,7 @@ public class LogicalTableNameExtendedIT extends LogicalTableNameBaseIT {
         conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName);
         // Create tables and change physical index table
         test_IndexTableChange(conn, conn2, schemaName, tableName, indexName,
-                IndexRegionObserver.UNVERIFIED_BYTES, true);
+                QueryConstants.UNVERIFIED_BYTES, true);
         // Now change physical data table
         createAndPointToNewPhysicalTable(conn, fullTableHName, true);
         try (Admin admin = conn.unwrap(PhoenixConnection.class).getQueryServices()
@@ -149,7 +149,7 @@ public class LogicalTableNameExtendedIT extends LogicalTableNameBaseIT {
                         htable.put(put);
                     }
 
-                    IndexToolIT.runIndexTool(true, false, schemaName, tableName, indexName);
+                    IndexToolIT.runIndexTool(false, schemaName, tableName, indexName);
                     rs = conn.createStatement().executeQuery("SELECT * FROM " +  fullIndexName + " WHERE \":PK1\"='PK30'");
                     assertEquals(true, rs.next());
 
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
index 8e6510f..66e8f16 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.phoenix.end2end.join.HashJoinGlobalIndexIT;
-import org.apache.phoenix.hbase.index.IndexRegionObserver;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
 import org.apache.phoenix.schema.PTable;
@@ -30,6 +29,7 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.query.QueryConstants;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -108,7 +108,7 @@ public class LogicalTableNameIT extends LogicalTableNameBaseIT {
                         indexName, false, createChildAfterRename);
 
                 // We have to rebuild index for this to work
-                IndexToolIT.runIndexTool(true, false, schemaName, tableName, indexName);
+                IndexToolIT.runIndexTool(false, schemaName, tableName, indexName);
 
                 validateTable(conn, fullTableName);
                 validateTable(conn2, fullTableName);
@@ -181,7 +181,7 @@ public class LogicalTableNameIT extends LogicalTableNameBaseIT {
         String fullIndexName = SchemaUtil.getTableName(schemaName, indexName);
         try (Connection conn = getConnection(props)) {
             try (Connection conn2 = getConnection(props)) {
-                HashMap<String, ArrayList<String>> expected = test_IndexTableChange(conn, conn2, schemaName, tableName, indexName, IndexRegionObserver.VERIFIED_BYTES, false);
+                HashMap<String, ArrayList<String>> expected = test_IndexTableChange(conn, conn2, schemaName, tableName, indexName, QueryConstants.VERIFIED_BYTES, false);
 
                 validateIndex(conn, fullIndexName, false, expected);
                 validateIndex(conn2, fullIndexName, false, expected);
@@ -213,7 +213,7 @@ public class LogicalTableNameIT extends LogicalTableNameBaseIT {
         String fullIndexName = SchemaUtil.getTableName(schemaName, indexName);
         try (Connection conn = getConnection(props)) {
             try (Connection conn2 = getConnection(props)) {
-                test_IndexTableChange(conn, conn2, schemaName, tableName, indexName, IndexRegionObserver.VERIFIED_BYTES, false);
+                test_IndexTableChange(conn, conn2, schemaName, tableName, indexName, QueryConstants.VERIFIED_BYTES, false);
                 List<Job>
                         completedJobs =
                         IndexScrutinyToolBaseIT.runScrutinyTool(schemaName, tableName, indexName, 1L,
@@ -231,7 +231,7 @@ public class LogicalTableNameIT extends LogicalTableNameBaseIT {
                 // Try with unverified bytes
                 String tableName2 = "TBL_" + generateUniqueName();
                 String indexName2 = "IDX_" + generateUniqueName();
-                test_IndexTableChange(conn, conn2, schemaName, tableName2, indexName2, IndexRegionObserver.UNVERIFIED_BYTES, false);
+                test_IndexTableChange(conn, conn2, schemaName, tableName2, indexName2, QueryConstants.UNVERIFIED_BYTES, false);
 
                 completedJobs =
                         IndexScrutinyToolBaseIT.runScrutinyTool(schemaName, tableName2, indexName2, 1L,
@@ -271,9 +271,9 @@ public class LogicalTableNameIT extends LogicalTableNameBaseIT {
                         schemaName, tableName, view1Name, view1IndexName1, view1IndexName2, view2Name, view2IndexName1, false, createChildAfterRename);
 
                 // We have to rebuild index for this to work
-                IndexToolIT.runIndexTool(true, false, schemaName, view1Name, view1IndexName1);
-                IndexToolIT.runIndexTool(true, false, schemaName, view1Name, view1IndexName2);
-                IndexToolIT.runIndexTool(true, false, schemaName, view2Name, view2IndexName1);
+                IndexToolIT.runIndexTool(false, schemaName, view1Name, view1IndexName1);
+                IndexToolIT.runIndexTool(false, schemaName, view1Name, view1IndexName2);
+                IndexToolIT.runIndexTool(false, schemaName, view2Name, view2IndexName1);
 
                 validateIndex(conn, fullView1IndexName1, true, expected);
                 validateIndex(conn2, fullView1IndexName2, true, expected);
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexCheckerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexCheckerIT.java
index 80a5618..4a91102 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexCheckerIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexCheckerIT.java
@@ -156,7 +156,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
                     dataTableName + " (val1) include (val2, val3)" + (async ? "ASYNC" : "") + this.indexDDLOptions);
             if (async) {
                 // run the index MR job.
-                IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName);
+                IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName);
             }
             // Count the number of index rows
             String query = "SELECT COUNT(*) from " + indexTableName;
@@ -189,7 +189,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
                     dataTableName + " (val1, PHOENIX_ROW_TIMESTAMP()) " + "include (val2, val3) " + (async ? "ASYNC" : "")+ this.indexDDLOptions);
             if (async) {
                 // Run the index MR job to rebuild the index and verify that index is built correctly
-                IndexToolIT.runIndexTool(true, false, null, dataTableName,
+                IndexToolIT.runIndexTool(false, null, dataTableName,
                         indexTableName, null, 0, IndexTool.IndexVerifyType.AFTER);
             }
 
@@ -274,7 +274,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
                     (async ? "ASYNC" : "")+ this.indexDDLOptions);
             if (async) {
                 // Run the index MR job to rebuild the index and verify that index is built correctly
-                IndexToolIT.runIndexTool(true, false, null, dataTableName,
+                IndexToolIT.runIndexTool(false, null, dataTableName,
                         indexTableName, null, 0, IndexTool.IndexVerifyType.AFTER);
             }
             // Add one more row
@@ -327,7 +327,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
             conn.createStatement().executeUpdate(dml);
             conn.commit();
             // Make sure this delete attempt did not make the index and data table inconsistent
-            IndexToolIT.runIndexTool(true, false, "", dataTableName, indexTableName, null,
+            IndexToolIT.runIndexTool(false, "", dataTableName, indexTableName, null,
                     0, IndexTool.IndexVerifyType.ONLY);
         }
     }
@@ -438,7 +438,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
                     dataTableName + " (val1) include (val2, val3)" + (async ? "ASYNC" : "") + this.indexDDLOptions);
             if (async) {
                 // run the index MR job.
-                IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName);
+                IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName);
             }
             // For the concurrent updates on the same row, the last write phase is ignored.
             // Configure IndexRegionObserver to fail the last write phase (i.e., the post index update phase) where the
@@ -482,7 +482,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
                     dataTableName + " (val1) include (val2, val3)" + (async ? "ASYNC" : "") + this.indexDDLOptions);
             if (async) {
                 // run the index MR job.
-                IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName);
+                IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName);
             }
             String selectSql = "SELECT id from " + dataTableName + " WHERE val1  = 'ab'";
             ResultSet rs = conn.createStatement().executeQuery(selectSql);
@@ -527,7 +527,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
                 dataTableName + " (val1) include (val2, val3)" + (async ? "ASYNC" : "") + this.indexDDLOptions);
         if (async) {
             // run the index MR job.
-            IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName);
+            IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName);
         }
         conn.createStatement().execute("upsert into " + dataTableName + " (id, val2) values ('a', 'abcc')");
         conn.commit();
@@ -620,7 +620,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
                     dataTableName + " (val1) include (val2, val3)" + (async ? "ASYNC" : "") + this.indexDDLOptions);
             if (async) {
                 // run the index MR job.
-                IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName);
+                IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName);
             }
             // Configure IndexRegionObserver to fail the first write phase (i.e., the pre index update phase). This should not
             // lead to any change on index or data table rows
@@ -653,7 +653,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
                     dataTableName + " (val1) include (val2, val3)" + (async ? "ASYNC" : "") + this.indexDDLOptions);
             if (async) {
                 // run the index MR job.
-                IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName);
+                IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName);
             }
             // Configure IndexRegionObserver to fail the last write phase (i.e., the post index update phase) where the verify flag is set
             // to true and/or index rows are deleted and check that this does not impact the correctness
@@ -662,7 +662,7 @@ public class GlobalIndexCheckerIT extends BaseTest {
             conn.commit();
             conn.createStatement().execute("upsert into " + dataTableName + " (id, val1, val2) values ('c', 'cd','cde')");
             conn.commit();
-            IndexTool indexTool = IndexToolIT.runIndexTool(true, false, "", dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.ONLY);
+            IndexTool indexTool = IndexToolIT.runIndexTool(false, "", dataTableName, indexTableName, null, 0, IndexTool.IndexVerifyType.ONLY);
             assertEquals(3, indexTool.getJob().getCounters().findCounter(INPUT_RECORDS).getValue());
             assertEquals(3, indexTool.getJob().getCounters().findCounter(SCANNED_DATA_ROW_COUNT).getValue());
             assertEquals(0, indexTool.getJob().getCounters().findCounter(REBUILT_INDEX_ROW_COUNT).getValue());
@@ -800,8 +800,8 @@ public class GlobalIndexCheckerIT extends BaseTest {
                     dataTableName + " (val2) include (val1, val3)" + (async ? "ASYNC" : "") + this.indexDDLOptions);
             if (async) {
                 // run the index MR job.
-                IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName + "1");
-                IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName + "2");
+                IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName + "1");
+                IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName + "2");
             }
             // Two Phase write. This write is recoverable
             IndexRegionObserver.setFailPostIndexUpdatesForTesting(true);
@@ -920,8 +920,8 @@ public class GlobalIndexCheckerIT extends BaseTest {
         conn.commit();
         if (async) {
             // run the index MR job.
-            IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName + "1");
-            IndexToolIT.runIndexTool(true, false, null, dataTableName, indexTableName + "2");
+            IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName + "1");
+            IndexToolIT.runIndexTool(false, null, dataTableName, indexTableName + "2");
         }
     }
 
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexExtendedIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexExtendedIT.java
index e80056a..5d3b515 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexExtendedIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexExtendedIT.java
@@ -32,11 +32,11 @@ import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.end2end.ParallelStatsDisabledTest;
-import org.apache.phoenix.hbase.index.IndexRegionObserver;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -243,10 +243,10 @@ public class ImmutableIndexExtendedIT extends ParallelStatsDisabledIT {
     private static void verifyRowCountForEmptyCol(Connection conn, String indexTable,
             int expectedVerifiedCount, int expectedUnverifiedCount) throws Exception {
 
-        assertEquals(expectedVerifiedCount,
-                getRowCountForEmptyColValue(conn, indexTable, IndexRegionObserver.VERIFIED_BYTES));
-        assertEquals(expectedUnverifiedCount,
-                getRowCountForEmptyColValue(conn, indexTable, IndexRegionObserver.UNVERIFIED_BYTES));
+        assertEquals(expectedVerifiedCount, getRowCountForEmptyColValue(conn, indexTable,
+                QueryConstants.VERIFIED_BYTES));
+        assertEquals(expectedUnverifiedCount, getRowCountForEmptyColValue(conn, indexTable,
+                QueryConstants.UNVERIFIED_BYTES));
     }
 
     @Test
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index d3cb529..8ff182f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -48,19 +48,13 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
@@ -68,24 +62,23 @@ import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
-import org.apache.phoenix.hbase.index.IndexRegionObserver;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PIndexState;
-import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
 import org.apache.phoenix.transaction.PhoenixTransactionProvider;
 import org.apache.phoenix.transaction.PhoenixTransactionProvider.Feature;
 import org.apache.phoenix.transaction.TransactionFactory;
-import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 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.TestUtil;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -94,9 +87,6 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
-import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
-
 
 @Category(NeedsOwnMiniClusterTest.class)
 @RunWith(Parameterized.class)
@@ -343,8 +333,8 @@ public class ImmutableIndexIT extends BaseTest {
                 rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
                 assertTrue(rs.next());
                 assertEquals(numRows, rs.getInt(1));
-                assertEquals(true, verifyRowsForEmptyColValue(conn, fullIndexName,
-                        IndexRegionObserver.VERIFIED_BYTES));
+                IndexTestUtil.assertRowsForEmptyColValue(conn, fullIndexName,
+                    QueryConstants.VERIFIED_BYTES);
                 rs = conn.createStatement().executeQuery("SELECT * FROM " + fullIndexName);
                 assertTrue(rs.next());
                 assertEquals("1", rs.getString(1));
@@ -394,7 +384,8 @@ public class ImmutableIndexIT extends BaseTest {
             rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
             assertTrue(rs.next());
             assertEquals(numRows - 1, rs.getInt(1));
-            assertEquals(true, verifyRowsForEmptyColValue(conn, fullIndexName, IndexRegionObserver.VERIFIED_BYTES));
+            IndexTestUtil.assertRowsForEmptyColValue(conn, fullIndexName,
+                QueryConstants.VERIFIED_BYTES);
 
             // Force delete to fail (data removed but operation failed) on data table and check index table row remains as unverified
             TestUtil.addCoprocessor(conn, fullTableName, DeleteFailingRegionObserver.class);
@@ -409,7 +400,8 @@ public class ImmutableIndexIT extends BaseTest {
             TestUtil.removeCoprocessor(conn, fullTableName, DeleteFailingRegionObserver.class);
             assertEquals(numRows - 1, getRowCount(conn.unwrap(PhoenixConnection.class).getQueryServices()
                     .getTable(Bytes.toBytes(fullIndexName)), false));
-            assertEquals(true, verifyRowsForEmptyColValue(conn, fullIndexName, IndexRegionObserver.UNVERIFIED_BYTES));
+            IndexTestUtil.assertRowsForEmptyColValue(conn, fullIndexName,
+                QueryConstants.UNVERIFIED_BYTES);
 
             // Now delete via hbase, read from unverified index and see that we don't get any data
             admin.disableTable(TableName.valueOf(fullTableName));
@@ -442,26 +434,6 @@ public class ImmutableIndexIT extends BaseTest {
         }
     }
 
-    public static boolean verifyRowsForEmptyColValue(Connection conn, String tableName, byte[] valueBytes)
-            throws IOException, SQLException {
-        PTable table = PhoenixRuntime.getTable(conn, tableName);
-        byte[] emptyCF = SchemaUtil.getEmptyColumnFamily(table);
-        byte[] emptyCQ = EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst();
-        HTable htable = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(table.getPhysicalName().getBytes());
-        Scan scan = new Scan();
-        scan.addColumn(emptyCF, emptyCQ);
-        ResultScanner resultScanner = htable.getScanner(scan);
-
-        for (Result result = resultScanner.next(); result != null; result = resultScanner.next()) {
-            if (Bytes.compareTo(result.getValue(emptyCF, emptyCQ), 0, valueBytes.length,
-                    valueBytes, 0, valueBytes.length) != 0) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-
     // This test is know to flap. We need PHOENIX-2582 to be fixed before enabling this back.
     @Ignore
     @Test
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
index bfa9ed6..565be05 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
@@ -25,7 +25,9 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TAB
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
+import static org.junit.Assert.assertNull;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -37,12 +39,18 @@ import java.util.Map;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.GenericTestUtils;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.IndexRegionObserver;
@@ -52,10 +60,13 @@ import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.GlobalIndexChecker;
 import org.apache.phoenix.index.PhoenixIndexBuilder;
 import org.apache.phoenix.index.PhoenixIndexCodec;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PRow;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.RowKeySchema;
@@ -64,6 +75,7 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 
 import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
@@ -277,4 +289,25 @@ public class IndexTestUtil {
         Assert.assertFalse("Could find coproc " + expectedCoprocName +
                 " in descriptor " + descriptor,foundCoproc);
     }
+
+    public static void assertRowsForEmptyColValue(Connection conn, String tableName,
+            byte[] emptyValue) throws SQLException, IOException {
+        ConnectionQueryServices cqs = conn.unwrap(PhoenixConnection.class).getQueryServices();
+        PTable pTable = PhoenixRuntime.getTable(conn, tableName);
+        Table hTable = cqs.getTable(pTable.getPhysicalName().getBytes());
+
+        byte[] emptyKeyValueCF = SchemaUtil.getEmptyColumnFamily(pTable);
+        byte[] emptyKeyValueQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(pTable).getFirst();
+
+        Scan scan = new Scan();
+        scan.setFilter(new SingleColumnValueFilter(
+                emptyKeyValueCF,
+                emptyKeyValueQualifier,
+                CompareOperator.NOT_EQUAL,
+                new org.apache.hadoop.hbase.filter.BinaryComparator(emptyValue)));
+        try (ResultScanner scanner = hTable.getScanner(scan)) {
+            assertNull("There are rows with in the table where the empty value is not "
+                    + Bytes.toStringBinary(emptyValue), scanner.next());
+        }
+    }
 }
\ No newline at end of file
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
index c193462..accfa6c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
@@ -361,7 +361,7 @@ public class ViewIndexIT extends SplitSystemCatalogIT {
             }
 
             // run the MR job
-            IndexToolIT.runIndexTool(true, false, SCHEMA2, globalViewName, globalViewIdx);
+            IndexToolIT.runIndexTool(false, SCHEMA2, globalViewName, globalViewIdx);
             try (Connection tenantConn = DriverManager.getConnection(getUrl(), tenantProps)) {
                 // Verify that query uses the global view index works while querying the tenant view
                 PreparedStatement stmt = tenantConn.prepareStatement("SELECT KV1 FROM  " + tenantView + " WHERE PK3 = ? AND KV3 = ?");
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/DataTableLocalIndexRegionScanner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/DataTableLocalIndexRegionScanner.java
index 859b9ba..ec84cbc 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/DataTableLocalIndexRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/DataTableLocalIndexRegionScanner.java
@@ -107,8 +107,8 @@ public class DataTableLocalIndexRegionScanner extends DelegateRegionScanner {
                 ValueGetter valueGetter = maintainer
                         .createGetterFromKeyValues(ImmutableBytesPtr.copyBytesIfNecessary(ptr), dataTableResults);
                 List<Cell> list = maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr,
-                        dataTableResults.get(0).getTimestamp(), startKey, endKey).getFamilyCellMap()
-                        .get(localIndexFamily);
+                        dataTableResults.get(0).getTimestamp(), startKey, endKey, false)
+                        .getFamilyCellMap().get(localIndexFamily);
                 Put put = null;
                 Delete del = null;
                 for (Cell cell : list) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
index bdc4008..b05854b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java
@@ -88,8 +88,6 @@ import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
-import static org.apache.phoenix.hbase.index.IndexRegionObserver.UNVERIFIED_BYTES;
-import static org.apache.phoenix.hbase.index.IndexRegionObserver.VERIFIED_BYTES;
 import static org.apache.phoenix.hbase.index.IndexRegionObserver.removeEmptyColumn;
 import static org.apache.phoenix.hbase.index.write.AbstractParallelWriterIndexCommitter.INDEX_WRITER_KEEP_ALIVE_TIME_CONF_KEY;
 import static org.apache.phoenix.mapreduce.index.IndexVerificationOutputRepository.IndexVerificationErrorType.BEYOND_MAX_LOOKBACK_INVALID;
@@ -98,6 +96,8 @@ import static org.apache.phoenix.mapreduce.index.IndexVerificationOutputReposito
 import static org.apache.phoenix.mapreduce.index.IndexVerificationOutputRepository.IndexVerificationErrorType.EXTRA_ROW;
 import static org.apache.phoenix.mapreduce.index.IndexVerificationOutputRepository.IndexVerificationErrorType.INVALID_ROW;
 import static org.apache.phoenix.mapreduce.index.IndexVerificationOutputRepository.IndexVerificationErrorType.MISSING_ROW;
+import static org.apache.phoenix.query.QueryConstants.UNVERIFIED_BYTES;
+import static org.apache.phoenix.query.QueryConstants.VERIFIED_BYTES;
 import static org.apache.phoenix.query.QueryServices.INDEX_REBUILD_PAGE_SIZE_IN_ROWS;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB;
@@ -1224,7 +1224,7 @@ public abstract class GlobalIndexRegionScanner extends BaseRegionScanner {
                                                 ValueGetter mergedRowVG, long ts)
             throws IOException {
         Put indexPut = indexMaintainer.buildUpdateMutation(GenericKeyValueBuilder.INSTANCE,
-                mergedRowVG, rowKeyPtr, ts, null, null);
+                mergedRowVG, rowKeyPtr, ts, null, null, false);
         if (indexPut == null) {
             // No covered column. Just prepare an index row with the empty column
             byte[] indexRowKey = indexMaintainer.buildRowKey(mergedRowVG, rowKeyPtr,
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java
index 6333a8c..2920468 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java
@@ -142,7 +142,7 @@ public class IndexerRegionScanner extends GlobalIndexRegionScanner {
         ValueGetter valueGetter = new SimpleValueGetter(dataRow);
         long ts = getMaxTimestamp(dataRow);
         Put indexPut = indexMaintainer.buildUpdateMutation(GenericKeyValueBuilder.INSTANCE,
-                valueGetter, new ImmutableBytesWritable(dataRow.getRow()), ts, null, null);
+                valueGetter, new ImmutableBytesWritable(dataRow.getRow()), ts, null, null, false);
 
         if (indexPut == null) {
             // This means the data row does not have any covered column values
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionScanner.java
index 017aab5..95f0136 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionScanner.java
@@ -439,7 +439,8 @@ public class UngroupedAggregateRegionScanner extends BaseRegionScanner {
                 Put put = maintainer.buildUpdateMutation(GenericKeyValueBuilder.INSTANCE,
                         valueGetter, ptr, results.get(0).getTimestamp(),
                         env.getRegion().getRegionInfo().getStartKey(),
-                        env.getRegion().getRegionInfo().getEndKey());
+                        env.getRegion().getRegionInfo().getEndKey(),
+                        false);
 
                 if (txnProvider != null) {
                     put = txnProvider.markPutAsCommitted(put, ts, ts);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 184ac59..0a6b92a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -1697,7 +1697,7 @@ public class MutationState implements SQLCloseable {
                     if (m instanceof Delete) {
                         Put put = new Put(m.getRow());
                         put.addColumn(emptyCF, emptyCQ, IndexRegionObserver.getMaxTimestamp(m),
-                                IndexRegionObserver.UNVERIFIED_BYTES);
+                                QueryConstants.UNVERIFIED_BYTES);
                         // The Delete gets marked as unverified in Phase 1 and gets deleted on Phase 3.
                         addToMap(unverifiedIndexMutations, tableInfo, put);
                         addToMap(verifiedOrDeletedIndexMutations, tableInfo, m);
@@ -1708,12 +1708,14 @@ public class MutationState implements SQLCloseable {
                         // Send entire mutation with the unverified status
                         // Remove the empty column prepared by Index codec as we need to change its value
                         IndexRegionObserver.removeEmptyColumn(m, emptyCF, emptyCQ);
-                        ((Put) m).addColumn(emptyCF, emptyCQ, timestamp, IndexRegionObserver.UNVERIFIED_BYTES);
+                        ((Put) m).addColumn(emptyCF, emptyCQ, timestamp,
+                                QueryConstants.UNVERIFIED_BYTES);
                         addToMap(unverifiedIndexMutations, tableInfo, m);
 
                         // Phase 3 mutations are verified
                         Put verifiedPut = new Put(m.getRow());
-                        verifiedPut.addColumn(emptyCF, emptyCQ, timestamp, IndexRegionObserver.VERIFIED_BYTES);
+                        verifiedPut.addColumn(emptyCF, emptyCQ, timestamp,
+                                 QueryConstants.VERIFIED_BYTES);
                         addToMap(verifiedOrDeletedIndexMutations, tableInfo, verifiedPut);
                     } else {
                         addToMap(unverifiedIndexMutations, tableInfo, m);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
index a489edc..6cfcd9c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/PhoenixTxIndexMutationGenerator.java
@@ -364,7 +364,8 @@ public class PhoenixTxIndexMutationGenerator {
                 throw new IOException(e);
             }
         }
-        Iterable<IndexUpdate> puts = codec.getIndexUpserts(state, indexMetaData, regionStartKey, regionEndKey);
+        Iterable<IndexUpdate> puts =
+                codec.getIndexUpserts(state, indexMetaData, regionStartKey, regionEndKey, true);
         boolean validPut = false;
         for (IndexUpdate put : puts) {
             if (put.isValid()) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
index 345c393..2c40c98 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexRegionObserver.java
@@ -107,6 +107,7 @@ import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.PhoenixIndexBuilder;
 import org.apache.phoenix.index.PhoenixIndexMetaData;
 import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PRow;
@@ -150,10 +151,6 @@ public class IndexRegionObserver extends CompatIndexRegionObserver implements Re
     private static final Logger LOG = LoggerFactory.getLogger(IndexRegionObserver.class);
     private static final OperationStatus IGNORE = new OperationStatus(OperationStatusCode.SUCCESS);
     private static final OperationStatus NOWRITE = new OperationStatus(OperationStatusCode.SUCCESS);
-    protected static final byte VERIFIED_BYTE = 1;
-    protected static final byte UNVERIFIED_BYTE = 2;
-    public static final byte[] UNVERIFIED_BYTES = new byte[] { UNVERIFIED_BYTE };
-    public static final byte[] VERIFIED_BYTES = new byte[] { VERIFIED_BYTE };
     public static final String PHOENIX_APPEND_METADATA_TO_WAL = "phoenix.append.metadata.to.wal";
     public static final boolean DEFAULT_PHOENIX_APPEND_METADATA_TO_WAL = false;
 
@@ -893,7 +890,7 @@ public class IndexRegionObserver extends CompatIndexRegionObserver implements Re
                 if (nextDataRowState != null) {
                     ValueGetter nextDataRowVG = new GlobalIndexRegionScanner.SimpleValueGetter(nextDataRowState);
                     Put indexPut = indexMaintainer.buildUpdateMutation(GenericKeyValueBuilder.INSTANCE,
-                            nextDataRowVG, rowKeyPtr, ts, null, null);
+                            nextDataRowVG, rowKeyPtr, ts, null, null, false);
                     if (indexPut == null) {
                         // No covered column. Just prepare an index row with the empty column
                         byte[] indexRowKey = indexMaintainer.buildRowKey(nextDataRowVG, rowKeyPtr,
@@ -903,8 +900,10 @@ public class IndexRegionObserver extends CompatIndexRegionObserver implements Re
                         removeEmptyColumn(indexPut, indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary(),
                                 indexMaintainer.getEmptyKeyValueQualifier());
                     }
-                    indexPut.addColumn(indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary(),
-                            indexMaintainer.getEmptyKeyValueQualifier(), ts, UNVERIFIED_BYTES);
+                    indexPut.addColumn(
+                            indexMaintainer.getEmptyKeyValueFamily().copyBytesIfNecessary(),
+                            indexMaintainer.getEmptyKeyValueQualifier(), ts,
+                            QueryConstants.UNVERIFIED_BYTES);
                     context.indexUpdates.put(hTableInterfaceReference,
                             new Pair<Mutation, byte[]>(indexPut, rowKeyPtr.get()));
                     // Delete the current index row if the new index key is different than the current one
@@ -971,7 +970,8 @@ public class IndexRegionObserver extends CompatIndexRegionObserver implements Re
                     } else {
                         // Set the status of the index row to "unverified"
                         Put unverifiedPut = new Put(m.getRow());
-                        unverifiedPut.addColumn(emptyCF, emptyCQ, now, UNVERIFIED_BYTES);
+                        unverifiedPut.addColumn(
+                            emptyCF, emptyCQ, now, QueryConstants.UNVERIFIED_BYTES);
                         // This will be done before the data table row is updated (i.e., in the first write phase)
                         context.preIndexUpdates.put(hTableInterfaceReference, unverifiedPut);
                     }
@@ -1010,7 +1010,7 @@ public class IndexRegionObserver extends CompatIndexRegionObserver implements Re
                 if (m instanceof Put) {
                     Put verifiedPut = new Put(m.getRow());
                     // Set the status of the index row to "verified"
-                    verifiedPut.addColumn(emptyCF, emptyCQ, now, VERIFIED_BYTES);
+                    verifiedPut.addColumn(emptyCF, emptyCQ, now, QueryConstants.VERIFIED_BYTES);
                     context.postIndexUpdates.put(hTableInterfaceReference, verifiedPut);
                 } else {
                     context.postIndexUpdates.put(hTableInterfaceReference, m);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexCodec.java
index abcb347..2972022 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexCodec.java
@@ -66,7 +66,8 @@ public interface IndexCodec {
      * @return the pairs of (updates,index table name) that should be applied.
      * @throws IOException
      */
-    public Iterable<IndexUpdate> getIndexUpserts(TableState state, IndexMetaData context, byte[] regionStartKey, byte[] regionEndKey) throws IOException;
+    public Iterable<IndexUpdate> getIndexUpserts(TableState state, IndexMetaData context,
+            byte[] regionStartKey, byte[] regionEndKey, boolean verified) throws IOException;
 
     /**
      * This allows the codec to dynamically change whether or not indexing should take place for a table. If it doesn't
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java
index 63d2751..41d0d6c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java
@@ -179,7 +179,9 @@ public class NonTxIndexBuilder extends BaseIndexBuilder {
             throws IOException {
 
         // get the index updates for this current batch
-        Iterable<IndexUpdate> upserts = codec.getIndexUpserts(state, indexMetaData, env.getRegionInfo().getStartKey(), env.getRegionInfo().getEndKey());
+        Iterable<IndexUpdate> upserts = codec.getIndexUpserts(
+            state, indexMetaData,
+            env.getRegionInfo().getStartKey(), env.getRegionInfo().getEndKey(), false);
         state.resetTrackedColumns();
 
         /*
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/GlobalIndexChecker.java b/phoenix-core/src/main/java/org/apache/phoenix/index/GlobalIndexChecker.java
index 75775fa..d9185cd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/GlobalIndexChecker.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/GlobalIndexChecker.java
@@ -17,8 +17,8 @@
  */
 package org.apache.phoenix.index;
 
-import static org.apache.phoenix.hbase.index.IndexRegionObserver.VERIFIED_BYTES;
 import static org.apache.phoenix.index.IndexMaintainer.getIndexMaintainer;
+import static org.apache.phoenix.query.QueryConstants.VERIFIED_BYTES;
 import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
 import static org.apache.phoenix.util.ScanUtil.getDummyResult;
 import static org.apache.phoenix.util.ScanUtil.getPageSizeMsForRegionScanner;
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 0ead138..fbb5182 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -1054,11 +1054,11 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         return indexRowKeySchema;
     }
 
-    public Put buildUpdateMutation(KeyValueBuilder kvBuilder, ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts, byte[] regionStartKey, byte[] regionEndKey) throws IOException {
+    public Put buildUpdateMutation(KeyValueBuilder kvBuilder, ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts, byte[] regionStartKey, byte[] regionEndKey, boolean verified) throws IOException {
         byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey, ts);
         return buildUpdateMutation(kvBuilder, valueGetter, dataRowKeyPtr, ts, regionStartKey, regionEndKey,
                 indexRowKey, this.getEmptyKeyValueFamily(), coveredColumnsMap,
-                indexEmptyKeyValueRef, indexWALDisabled, dataImmutableStorageScheme, immutableStorageScheme, encodingScheme);
+                indexEmptyKeyValueRef, indexWALDisabled, dataImmutableStorageScheme, immutableStorageScheme, encodingScheme, verified);
     }
 
     public static Put buildUpdateMutation(KeyValueBuilder kvBuilder, ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts,
@@ -1066,7 +1066,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                                           Map<ColumnReference, ColumnReference> coveredColumnsMap,
                                           ColumnReference destEmptyKeyValueRef, boolean destWALDisabled,
                                           ImmutableStorageScheme srcImmutableStroageScheme, ImmutableStorageScheme destImmutableStorageScheme,
-                                          QualifierEncodingScheme destEncodingScheme) throws IOException {
+                                          QualifierEncodingScheme destEncodingScheme, boolean verified) throws IOException {
         Set<ColumnReference> coveredColumns = coveredColumnsMap.keySet();
         Put put = null;
         // New row being inserted: add the empty key value
@@ -1084,7 +1084,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             // add the keyvalue for the empty row
             put.add(kvBuilder.buildPut(new ImmutableBytesPtr(destRowKey),
                     emptyKeyValueCFPtr, destEmptyKeyValueRef.getQualifierWritable(), ts,
-                    QueryConstants.EMPTY_COLUMN_VALUE_BYTES_PTR));
+                    verified ? QueryConstants.VERIFIED_BYTES_PTR : QueryConstants.EMPTY_COLUMN_VALUE_BYTES_PTR));
             put.setDurability(!destWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
         }
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
index 65ada53..5ac8fe6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
@@ -79,7 +79,9 @@ public class PhoenixIndexCodec extends BaseIndexCodec {
     }
 
     @Override
-    public Iterable<IndexUpdate> getIndexUpserts(TableState state, IndexMetaData context, byte[] regionStartKey, byte[] regionEndKey) throws IOException {
+    public Iterable<IndexUpdate> getIndexUpserts(
+            TableState state, IndexMetaData context, byte[] regionStartKey, byte[] regionEndKey,
+            boolean verified) throws IOException {
         PhoenixIndexMetaData metaData = (PhoenixIndexMetaData)context;
         List<IndexMaintainer> indexMaintainers = metaData.getIndexMaintainers();
         if (indexMaintainers.get(0).isRowDeleted(state.getPendingUpdate())) {
@@ -94,7 +96,7 @@ public class PhoenixIndexCodec extends BaseIndexCodec {
             IndexUpdate indexUpdate = statePair.getSecond();
             indexUpdate.setTable(maintainer.isLocalIndex() ? tableName : maintainer.getIndexTableName());
             Put put = maintainer.buildUpdateMutation(KV_BUILDER, valueGetter, ptr, state.getCurrentTimestamp(),
-                    regionStartKey, regionEndKey);
+                    regionStartKey, regionEndKey, verified);
             indexUpdate.setUpdate(put);
             indexUpdates.add(indexUpdate);
         }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java
index 31ab60b..5a61eff 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java
@@ -33,7 +33,6 @@ import javax.annotation.Nullable;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -52,11 +51,11 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.util.ColumnInfo;
 import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.IndexUtil.IndexStatusUpdater;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.UpsertExecutor;
-import org.apache.phoenix.hbase.index.IndexRegionObserver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -425,41 +424,4 @@ public abstract class FormatToBytesWritableMapper<RECORD> extends Mapper<LongWri
         }
     }
 
-    /**
-     * Updates the EMPTY cell value to VERIFIED for global index table rows.
-     */
-    private static class IndexStatusUpdater {
-
-        private final byte[] emptyKeyValueCF;
-        private final int emptyKeyValueCFLength;
-        private final byte[] emptyKeyValueQualifier;
-        private final int emptyKeyValueQualifierLength;
-
-        public IndexStatusUpdater(final byte[] emptyKeyValueCF, final byte[] emptyKeyValueQualifier) {
-            this.emptyKeyValueCF = emptyKeyValueCF;
-            this.emptyKeyValueQualifier = emptyKeyValueQualifier;
-            emptyKeyValueCFLength = emptyKeyValueCF.length;
-            emptyKeyValueQualifierLength = emptyKeyValueQualifier.length;
-        }
-
-        /**
-         * Update the Empty cell values to VERIFIED in the passed keyValues list
-         * 
-         * @param keyValues will be modified
-         */
-        public void setVerified(List<Cell> keyValues) {
-            for (int i = 0; i < keyValues.size(); i++) {
-                Cell kv = keyValues.get(i);
-                if (CellUtil.compareFamilies(kv, emptyKeyValueCF, 0, emptyKeyValueCFLength) == 0
-                        && CellUtil.compareQualifiers(kv, emptyKeyValueQualifier, 0, emptyKeyValueQualifierLength) == 0) {
-                    if (kv.getValueLength() != 1) {
-                        //This should never happen. Fail fast if it does.
-                       throw new IllegalArgumentException("Empty cell value length is not 1");
-                    }
-                    //We are directly overwriting the value for performance
-                    kv.getValueArray()[kv.getValueOffset()] = IndexRegionObserver.VERIFIED_BYTES[0];
-                }
-            }
-        }
-    }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
index b5d9808..9d6d0b3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
@@ -493,6 +493,10 @@ public class IndexTool extends Configured implements Tool {
                 }
                 if (useSnapshot || (!isLocalIndexBuild && pDataTable.isTransactional())) {
                     PhoenixConfigurationUtil.setCurrentScnValue(configuration, maxTimeRange);
+                    if (indexVerifyType != IndexVerifyType.NONE) {
+                        LOGGER.warn("Verification is not supported for snapshots and transactional"
+                                + "table index rebuilds, verification parameter ignored");
+                    }
                     return configureJobForAsyncIndex();
                 } else {
                     // Local and non-transactional global indexes to be built on the server side
@@ -636,6 +640,7 @@ public class IndexTool extends Configured implements Tool {
 
             configuration.set(PhoenixConfigurationUtil.UPSERT_STATEMENT, upsertQuery);
             PhoenixConfigurationUtil.setPhysicalTableName(configuration, physicalIndexTable);
+            PhoenixConfigurationUtil.setIndexToolIndexTableName(configuration, qIndexTable);
             PhoenixConfigurationUtil.setDisableIndexes(configuration, indexTable);
 
             PhoenixConfigurationUtil.setUpsertColumnNames(configuration,
@@ -667,7 +672,12 @@ public class IndexTool extends Configured implements Tool {
                 try {
                     admin = pConnection.getQueryServices().getAdmin();
                     TableName hDdataTableName = TableName.valueOf(pDataTable.getPhysicalName().getBytes());
-                    snapshotName = new StringBuilder(hDdataTableName.toString()).append("-Snapshot").toString();
+                    snapshotName = new StringBuilder("INDEXTOOL-")
+                            .append(pDataTable.getName().getString())
+                            .append("-Snapshot-")
+                            .append(System.currentTimeMillis())
+                            .toString();
+                    //FIXME Drop this snapshot after we're done ?
                     admin.snapshot(snapshotName, hDdataTableName);
                 } finally {
                     if (admin != null) {
@@ -972,6 +982,11 @@ public class IndexTool extends Configured implements Tool {
         qIndexTable = SchemaUtil.getQualifiedTableName(schemaName, indexTable);
         if (IndexType.LOCAL.equals(indexType)) {
             isLocalIndexBuild = true;
+            if (useSnapshot) {
+                throw new IllegalArgumentException(String.format(
+                    "%s is a local index. snapshots are not supported for local indexes.",
+                    qIndexTable));
+            }
             try (org.apache.hadoop.hbase.client.Connection hConn
                     = getTemporaryHConnection(connection.unwrap(PhoenixConnection.class))) {
                 RegionLocator regionLocator = hConn
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
index d4c9798..2e1743a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
@@ -42,8 +42,12 @@ import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.ColumnInfo;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.IndexUtil.IndexStatusUpdater;
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,6 +75,7 @@ public class PhoenixIndexImportDirectMapper extends
     private MutationState mutationState;
     private int currentBatchCount = 0;
 
+    private IndexStatusUpdater indexStatusUpdater;
 
     @Override
     protected void setup(final Context context) throws IOException, InterruptedException {
@@ -86,7 +91,7 @@ public class PhoenixIndexImportDirectMapper extends
             final Properties overrideProps = new Properties();
             String scn = configuration.get(PhoenixConfigurationUtil.CURRENT_SCN_VALUE);
             String txScnValue = configuration.get(PhoenixConfigurationUtil.TX_SCN_VALUE);
-            if(txScnValue==null) {
+            if (txScnValue == null && scn != null) {
                 overrideProps.put(PhoenixRuntime.BUILD_INDEX_AT_ATTRIB, scn);
             }
             connection = ConnectionUtil.getOutputConnection(configuration, overrideProps);
@@ -106,6 +111,14 @@ public class PhoenixIndexImportDirectMapper extends
             final String upsertQuery = PhoenixConfigurationUtil.getUpsertStatement(configuration);
             this.pStatement = connection.prepareStatement(upsertQuery);
 
+            String indexTableName =
+                    PhoenixConfigurationUtil.getIndexToolIndexTableName(configuration);
+            PTable pIndexTable = PhoenixRuntime.getTable(connection, indexTableName);
+
+            indexStatusUpdater = new IndexStatusUpdater(
+                SchemaUtil.getEmptyColumnFamily(pIndexTable),
+                EncodedColumnsUtil.getEmptyKeyValueInfo(pIndexTable).getFirst());
+
         } catch (Exception e) {
             tryClosingResources();
             throw new RuntimeException(e);
@@ -155,6 +168,9 @@ public class PhoenixIndexImportDirectMapper extends
             List<List<Mutation>> batchOfBatchMutations =
                 MutationState.getMutationBatchList(batchSize, batchSizeBytes, batchMutations);
             for (List<Mutation> mutationList : batchOfBatchMutations) {
+                for (Mutation mutation: mutationList) {
+                    indexStatusUpdater.setVerified(mutation.cellScanner());
+                }
                 writer.write(mutationList);
             }
             context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(
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 bc537ae..8f11d41 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
@@ -265,6 +265,14 @@ public interface QueryConstants {
     byte[] DEFAULT_COLUMN_FAMILY_BYTES = Bytes.toBytes(DEFAULT_COLUMN_FAMILY);
     ImmutableBytesPtr DEFAULT_COLUMN_FAMILY_BYTES_PTR = new ImmutableBytesPtr(
             DEFAULT_COLUMN_FAMILY_BYTES);
+
+    byte VERIFIED_BYTE = 1;
+    byte UNVERIFIED_BYTE = 2;
+    byte[] VERIFIED_BYTES = new byte[] { VERIFIED_BYTE };
+    byte[] UNVERIFIED_BYTES = new byte[] { UNVERIFIED_BYTE };
+    ImmutableBytesPtr VERIFIED_BYTES_PTR = new ImmutableBytesPtr(VERIFIED_BYTES);
+    ImmutableBytesPtr UNVERIFIED_BYTES_PTR = new ImmutableBytesPtr(UNVERIFIED_BYTES);
+
     // column qualifier of the single key value used to store all columns for the
     // COLUMNS_STORED_IN_SINGLE_CELL storage scheme
     String SINGLE_KEYVALUE_COLUMN_QUALIFIER = "1";
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/transform/TransformMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/transform/TransformMaintainer.java
index 750750e..79e80d3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/transform/TransformMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/transform/TransformMaintainer.java
@@ -440,7 +440,7 @@ public class TransformMaintainer extends IndexMaintainer {
         byte[] newRowKey = this.buildRowKey(valueGetter, oldRowKeyPtr, regionStartKey, regionEndKey, ts);
         return buildUpdateMutation(kvBuilder, valueGetter, oldRowKeyPtr, ts, regionStartKey, regionEndKey,
                 newRowKey, this.getEmptyKeyValueFamily(), coveredColumnsMap,
-                newTableEmptyKeyValueRef, newTableWALDisabled, oldTableImmutableStorageScheme, newTableImmutableStorageScheme, newTableEncodingScheme);
+                newTableEmptyKeyValueRef, newTableWALDisabled, oldTableImmutableStorageScheme, newTableImmutableStorageScheme, newTableEncodingScheme, false);
     }
 
     public Delete buildRowDeleteMutation(byte[] rowKey, DeleteType deleteType, long ts) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 4fc8430..ae28ac1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -42,6 +42,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.PhoenixTagType;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.RawCell;
@@ -390,7 +391,7 @@ public class IndexUtil {
                         regionStartKey = tableRegionLocation.getRegion().getStartKey();
                         regionEndkey = tableRegionLocation.getRegion().getEndKey();
                     }
-                    indexMutations.add(maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr, ts, regionStartKey, regionEndkey));
+                    indexMutations.add(maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr, ts, regionStartKey, regionEndkey, false));
                 }
             }
             return indexMutations;
@@ -875,4 +876,58 @@ public class IndexUtil {
             }
         }
     }
+
+    /**
+     * Updates the EMPTY cell value to VERIFIED for global index table rows.
+     */
+    public static class IndexStatusUpdater {
+
+        private final byte[] emptyKeyValueCF;
+        private final int emptyKeyValueCFLength;
+        private final byte[] emptyKeyValueQualifier;
+        private final int emptyKeyValueQualifierLength;
+
+        public IndexStatusUpdater(final byte[] emptyKeyValueCF, final byte[] emptyKeyValueQualifier) {
+            this.emptyKeyValueCF = emptyKeyValueCF;
+            this.emptyKeyValueQualifier = emptyKeyValueQualifier;
+            this.emptyKeyValueCFLength = emptyKeyValueCF.length;
+            this.emptyKeyValueQualifierLength = emptyKeyValueQualifier.length;
+        }
+
+        /**
+         * Update the Empty cell values to VERIFIED in the passed keyValues list
+         *
+         * @param keyValues will be modified
+         */
+        public void setVerified(List<Cell> keyValues) {
+            for (int i = 0; i < keyValues.size(); i++) {
+                updateVerified(keyValues.get(i));
+            }
+        }
+
+        /**
+         * Update the Empty cell values to VERIFIED in the passed keyValues list
+         *
+         * @param cellScanner contents will be modified
+         * @throws IOException
+         */
+        public void setVerified(CellScanner cellScanner) throws IOException {
+            while (cellScanner.advance()) {
+                updateVerified(cellScanner.current());
+            }
+        }
+
+        private void updateVerified(Cell cell) {
+            if (CellUtil.compareFamilies(cell, emptyKeyValueCF, 0, emptyKeyValueCFLength) == 0
+                    && CellUtil.compareQualifiers(cell, emptyKeyValueQualifier,
+                        0, emptyKeyValueQualifierLength) == 0) {
+                if (cell.getValueLength() != 1) {
+                    //This should never happen. Fail fast if it does.
+                   throw new IllegalArgumentException("Empty cell value length is not 1");
+                }
+                //We are directly overwriting the value for performance
+                cell.getValueArray()[cell.getValueOffset()] = QueryConstants.VERIFIED_BYTE;
+            }
+        }
+    }
 }
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredColumnIndexCodec.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredColumnIndexCodec.java
index 2c7b0b9..4e3be7e 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredColumnIndexCodec.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredColumnIndexCodec.java
@@ -58,7 +58,7 @@ public class CoveredColumnIndexCodec extends BaseIndexCodec {
     }
 
     @Override
-    public Iterable<IndexUpdate> getIndexUpserts(TableState state, IndexMetaData indexMetaData, byte[] regionStartKey, byte[] regionEndKey) {
+    public Iterable<IndexUpdate> getIndexUpserts(TableState state, IndexMetaData indexMetaData, byte[] regionStartKey, byte[] regionEndKey, boolean verified) {
         List<IndexUpdate> updates = new ArrayList<IndexUpdate>(groups.size());
         for (ColumnGroup group : groups) {
             IndexUpdate update = getIndexUpdateForGroup(group, state, indexMetaData);
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredIndexCodecForTesting.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredIndexCodecForTesting.java
index 665d2bf..39030f7 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredIndexCodecForTesting.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/CoveredIndexCodecForTesting.java
@@ -53,7 +53,7 @@ public class CoveredIndexCodecForTesting extends BaseIndexCodec {
     }
 
     @Override
-    public Iterable<IndexUpdate> getIndexUpserts(TableState state, IndexMetaData context, byte[] regionStartKey, byte[] regionEndKey) {
+    public Iterable<IndexUpdate> getIndexUpserts(TableState state, IndexMetaData context, byte[] regionStartKey, byte[] regionEndKey, boolean verified) {
         return this.updates;
     }
 
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestCoveredColumnIndexCodec.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestCoveredColumnIndexCodec.java
index 64ba5e3..a2ed4e2 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestCoveredColumnIndexCodec.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestCoveredColumnIndexCodec.java
@@ -184,7 +184,7 @@ public class TestCoveredColumnIndexCodec {
     // get the updates with the pending update
     state.setCurrentTimestamp(1);
     state.addPendingUpdates(kvs);
-    updates = codec.getIndexUpserts(state, IndexMetaData.NULL_INDEX_META_DATA, null, null);
+    updates = codec.getIndexUpserts(state, IndexMetaData.NULL_INDEX_META_DATA, null, null, false);
     assertTrue("Didn't find index updates for pending primary table update!", updates.iterator()
         .hasNext());
     for (IndexUpdate update : updates) {
@@ -237,7 +237,7 @@ public class TestCoveredColumnIndexCodec {
     state.setCurrentTimestamp(d.getTimeStamp());
     // now we shouldn't see anything when getting the index update
     state.addPendingUpdates(d.getFamilyCellMap().get(FAMILY));
-    Iterable<IndexUpdate> updates = codec.getIndexUpserts(state, IndexMetaData.NULL_INDEX_META_DATA, null, null);
+    Iterable<IndexUpdate> updates = codec.getIndexUpserts(state, IndexMetaData.NULL_INDEX_META_DATA, null, null, false);
     for (IndexUpdate update : updates) {
       assertFalse("Had some index updates, though it should have been covered by the delete",
         update.isValid());
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexToolTest.java b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexToolTest.java
index 248a9a1..1ce12a6 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexToolTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexToolTest.java
@@ -73,7 +73,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = 10L;
         Long endTime = 15L;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         startTime , endTime);
         CommandLine cmdLine = it.parseOptions(args);
@@ -85,7 +85,7 @@ public class IndexToolTest extends BaseTest {
     @Test
     public void testParseOptions_timeRange_null() throws Exception {
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE);
         CommandLine cmdLine = it.parseOptions(args);
         it.populateIndexToolAttributes(cmdLine);
@@ -98,7 +98,7 @@ public class IndexToolTest extends BaseTest {
         Assume.assumeTrue(HbaseCompatCapabilities.isRawFilterSupported());
         Long startTime = 10L;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         startTime , null);
         CommandLine cmdLine = it.parseOptions(args);
@@ -111,7 +111,7 @@ public class IndexToolTest extends BaseTest {
     public void testParseOptions_timeRange_endTimeNotNull() throws Exception {
         Long endTime = 15L;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         null , endTime);
         CommandLine cmdLine = it.parseOptions(args);
@@ -124,7 +124,7 @@ public class IndexToolTest extends BaseTest {
     public void testParseOptions_timeRange_startTimeNullEndTimeInFuture() throws Exception {
         Long endTime = EnvironmentEdgeManager.currentTimeMillis() + 100000;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         null , endTime);
         CommandLine cmdLine = it.parseOptions(args);
@@ -138,7 +138,7 @@ public class IndexToolTest extends BaseTest {
         Assume.assumeTrue(HbaseCompatCapabilities.isRawFilterSupported());
         Long startTime = EnvironmentEdgeManager.currentTimeMillis() + 100000;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         startTime , null);
         CommandLine cmdLine = it.parseOptions(args);
@@ -153,7 +153,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = EnvironmentEdgeManager.currentTimeMillis() + 100000;
         Long endTime = EnvironmentEdgeManager.currentTimeMillis() + 200000;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         startTime , endTime);
         CommandLine cmdLine = it.parseOptions(args);
@@ -168,7 +168,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = EnvironmentEdgeManager.currentTimeMillis();
         Long endTime = EnvironmentEdgeManager.currentTimeMillis() + 100000;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         startTime , endTime);
         CommandLine cmdLine = it.parseOptions(args);
@@ -183,7 +183,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = 10L;
         Long endTime = 10L;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         startTime , endTime);
         CommandLine cmdLine = it.parseOptions(args);
@@ -198,7 +198,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = 10L;
         Long endTime = 1L;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         startTime , endTime);
         CommandLine cmdLine = it.parseOptions(args);
@@ -222,7 +222,7 @@ public class IndexToolTest extends BaseTest {
         when(mockTool.getLastVerifyTime()).thenCallRealMethod();
         Long lastVerifyTime = 10L;
         String [] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                         dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                         lastVerifyTime, null, IndexTool.IndexDisableLoggingType.NONE, lastVerifyTime);
         when(mockTool.parseOptions(args)).thenCallRealMethod();
@@ -248,7 +248,7 @@ public class IndexToolTest extends BaseTest {
         when(mockTool.getLastVerifyTime()).thenCallRealMethod();
         Long lastVerifyTime = 10L;
         String [] args =
-            IndexToolIT.getArgValues(true, true, schema,
+            IndexToolIT.getArgValues(true, schema,
                 dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.AFTER,
                 lastVerifyTime, null, IndexTool.IndexDisableLoggingType.NONE,
                 lastVerifyTime);
@@ -274,7 +274,7 @@ public class IndexToolTest extends BaseTest {
             IndexTool it = new IndexTool();
             Long lastVerifyTime = 10L;
             String[] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                     dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.AFTER,
                     lastVerifyTime, null, IndexTool.IndexDisableLoggingType.NONE,
                     lastVerifyTime);
@@ -288,7 +288,7 @@ public class IndexToolTest extends BaseTest {
             IndexTool it = new IndexTool();
             Long lastVerifyTime = 10L;
             String[] args =
-                IndexToolIT.getArgValues(true, true, schema,
+                IndexToolIT.getArgValues(true, schema,
                     dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.AFTER,
                     null, null, IndexTool.IndexDisableLoggingType.NONE,
                     lastVerifyTime);
@@ -303,7 +303,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = null;
         Long endTime = 10L;
         String [] args =
-            IndexToolIT.getArgValues(true, true, schema,
+            IndexToolIT.getArgValues(true, schema,
                 dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                 startTime , endTime);
         CommandLine cmdLine = it.parseOptions(args);
@@ -361,7 +361,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = null;
         Long endTime = 10L;
         String[] args =
-            IndexToolIT.getArgValues(true, true, schema,
+            IndexToolIT.getArgValues(true, schema,
                 dataTable, indexTable, tenantId, verifyType,
                 startTime, endTime, disableType, null);
         CommandLine cmdLine = it.parseOptions(args);
@@ -374,7 +374,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = null;
         Long endTime = 10L;
         String[] args =
-            IndexToolIT.getArgValues(true, true, schema,
+            IndexToolIT.getArgValues(true, schema,
                 dataTable, indexTable, tenantId, verifyType,
                 startTime, endTime, disableType, null);
         exceptionRule.expect(IllegalStateException.class);
@@ -387,7 +387,7 @@ public class IndexToolTest extends BaseTest {
         Long startTime = 1L;
         Long endTime = 10L;
         String [] args =
-            IndexToolIT.getArgValues(true, true, schema,
+            IndexToolIT.getArgValues(true, schema,
                 dataTable, indexTable, tenantId, IndexTool.IndexVerifyType.NONE,
                 startTime , endTime);
         CommandLine cmdLine = it.parseOptions(args);
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/index/PrepareIndexMutationsForRebuildTest.java b/phoenix-core/src/test/java/org/apache/phoenix/index/PrepareIndexMutationsForRebuildTest.java
index d461552..98619b2 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/index/PrepareIndexMutationsForRebuildTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/index/PrepareIndexMutationsForRebuildTest.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.IndexRebuildRegionScanner;
-import org.apache.phoenix.hbase.index.IndexRegionObserver;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.QueryConstants;
@@ -789,7 +788,7 @@ public class PrepareIndexMutationsForRebuildTest extends BaseConnectionlessQuery
                 im.getEmptyKeyValueFamily().copyBytesIfNecessary(),
                 QueryConstants.EMPTY_COLUMN_BYTES,
                 ts,
-                IndexRegionObserver.VERIFIED_BYTES);
+                QueryConstants.VERIFIED_BYTES);
     }
 
     /**
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/index/VerifySingleIndexRowTest.java b/phoenix-core/src/test/java/org/apache/phoenix/index/VerifySingleIndexRowTest.java
index cd146c5..e3c3e9b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/index/VerifySingleIndexRowTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/index/VerifySingleIndexRowTest.java
@@ -61,8 +61,8 @@ import java.sql.SQLException;
 import java.util.*;
 
 import static org.apache.phoenix.coprocessor.GlobalIndexRegionScanner.MUTATION_TS_DESC_COMPARATOR;
-import static org.apache.phoenix.hbase.index.IndexRegionObserver.UNVERIFIED_BYTES;
-import static org.apache.phoenix.hbase.index.IndexRegionObserver.VERIFIED_BYTES;
+import static org.apache.phoenix.query.QueryConstants.UNVERIFIED_BYTES;
+import static org.apache.phoenix.query.QueryConstants.VERIFIED_BYTES;
 import static org.apache.phoenix.query.QueryConstants.EMPTY_COLUMN_BYTES;
 import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
@@ -490,7 +490,7 @@ public class VerifySingleIndexRowTest extends BaseConnectionlessQueryTest {
                 QueryConstants.EMPTY_COLUMN_BYTES,
                 EnvironmentEdgeManager.currentTimeMillis(),
                 Cell.Type.Put,
-                IndexRegionObserver.VERIFIED_BYTES);
+                QueryConstants.VERIFIED_BYTES);
         put.add(cell);
         // This mutation is beyond maxLookBack, so add it to expectedMutations only.
         expectedMutations.add(put);
@@ -504,7 +504,7 @@ public class VerifySingleIndexRowTest extends BaseConnectionlessQueryTest {
                 QueryConstants.EMPTY_COLUMN_BYTES,
                 EnvironmentEdgeManager.currentTimeMillis(),
                 Cell.Type.Put,
-                IndexRegionObserver.VERIFIED_BYTES);
+                QueryConstants.VERIFIED_BYTES);
         put.add(cell);
         // This mutation is in both expectedMutations and actualMutations, as it is within the maxLookBack, so it will not get chance to be compacted away
         expectedMutations.add(put);