You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by go...@apache.org on 2022/04/12 22:09:31 UTC

[phoenix] branch master updated: PHOENIX-6681 Optionally disable indexes during creation

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

gokcen 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 3f12a40780 PHOENIX-6681 Optionally disable indexes during creation
3f12a40780 is described below

commit 3f12a40780cca531ae9f43f7c7fa4a8033ea3285
Author: Gokcen Iskender <go...@gmail.com>
AuthorDate: Mon Apr 11 10:12:05 2022 -0700

    PHOENIX-6681 Optionally disable indexes during creation
    
    Signed-off-by: Gokcen Iskender <go...@gmail.com>
---
 .../end2end/index/IndexTwoPhaseCreateIT.java       | 245 +++++++++++++++++++++
 .../org/apache/phoenix/compile/DeleteCompiler.java |   2 +-
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  |   2 +-
 .../coprocessor/MetaDataRegionObserver.java        |   2 +-
 .../org/apache/phoenix/index/IndexMaintainer.java  |  26 ++-
 .../phoenix/index/IndexMetaDataCacheClient.java    |   6 +-
 .../apache/phoenix/mapreduce/index/IndexTool.java  |   2 +-
 .../phoenix/mapreduce/transform/TransformTool.java |   3 +
 .../org/apache/phoenix/query/QueryServices.java    |   2 +
 .../apache/phoenix/query/QueryServicesOptions.java |   6 +
 .../org/apache/phoenix/schema/DelegateTable.java   |   5 +
 .../org/apache/phoenix/schema/MetaDataClient.java  |  27 ++-
 .../org/apache/phoenix/schema/PIndexState.java     |  11 +-
 .../java/org/apache/phoenix/schema/PTable.java     |   5 +-
 .../java/org/apache/phoenix/schema/PTableImpl.java |  17 +-
 .../apache/phoenix/schema/transform/Transform.java |  42 +++-
 .../schema/transform/TransformMaintainer.java      |   6 -
 17 files changed, 375 insertions(+), 34 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTwoPhaseCreateIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTwoPhaseCreateIT.java
new file mode 100644
index 0000000000..4b9f246637
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTwoPhaseCreateIT.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end.index;
+
+import org.apache.phoenix.compat.hbase.coprocessor.CompatBaseScannerRegionObserver;
+import org.apache.phoenix.end2end.IndexToolIT;
+import org.apache.phoenix.end2end.transform.TransformToolIT;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.transform.SystemTransformRecord;
+import org.apache.phoenix.schema.transform.Transform;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.getRowCount;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class IndexTwoPhaseCreateIT extends BaseTest {
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(CompatBaseScannerRegionObserver.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY, Integer.toString(60 * 60)); // An hour
+        props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, Boolean.toString(false));
+        props.put(QueryServices.INDEX_CREATE_DEFAULT_STATE, PIndexState.CREATE_DISABLE.toString());
+        props.put(QueryServices.DEFAULT_IMMUTABLE_STORAGE_SCHEME_ATTRIB, "ONE_CELL_PER_COLUMN");
+        props.put(QueryServices.DEFAULT_COLUMN_ENCODED_BYTES_ATRRIB, "0");
+
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @AfterClass
+    public static synchronized void freeResources() throws Exception {
+        BaseTest.freeResourcesIfBeyondThreshold();
+    }
+
+    @Test
+    public void testIndexCreateWithNonDefaultSettings() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String dataTable = generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String tableDDL = "CREATE TABLE " + dataTable + TestUtil.TEST_TABLE_SCHEMA;
+            conn.createStatement().execute(tableDDL);
+            BaseTest.upsertRows(conn, dataTable, 2);
+            String ddl = "CREATE INDEX " + indexName + " ON " + dataTable
+                    + " (varchar_col1 ASC, varchar_col2 ASC, int_pk DESC)"
+                    + " INCLUDE (int_col1, int_col2) ";
+            conn.createStatement().execute(ddl);
+
+            assertIndexOrTableState(conn, null, indexName, PTableType.INDEX, PIndexState.CREATE_DISABLE);
+
+            assertEquals(0, getRowCount(conn, indexName));
+
+            ddl = "ALTER INDEX " + indexName + " ON " + dataTable + " REBUILD ASYNC";
+            conn.createStatement().execute(ddl);
+            assertIndexOrTableState(conn, null, indexName, PTableType.INDEX, PIndexState.BUILDING);
+        }
+    }
+
+
+    @Test
+    public void testIndexCreateDisabledBuildAfter() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String dataTable = generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(true);
+            String tableDDL = "CREATE TABLE " + dataTable + TestUtil.TEST_TABLE_SCHEMA;
+            conn.createStatement().execute(tableDDL);
+            BaseTest.upsertRows(conn, dataTable, 1);
+            String ddl = "CREATE INDEX " + indexName + " ON " + dataTable
+                    + " (varchar_col1 ASC, varchar_col2 ASC, int_pk DESC)"
+                    + " INCLUDE (int_col1, int_col2) ";
+            conn.createStatement().execute(ddl);
+
+            assertIndexOrTableState(conn, null, indexName, PTableType.INDEX, PIndexState.CREATE_DISABLE);
+
+            BaseTest.upsertRows(conn, dataTable, 3);
+            long rows = getRowCount(conn, indexName);
+            // Disabled table, rows don't go in
+            assertEquals(0, rows);
+
+            IndexToolIT.runIndexTool(false, null, dataTable, indexName);
+            rows = getRowCount(conn, indexName);
+            assertEquals(3, rows);
+            assertIndexOrTableState(conn, null, indexName, PTableType.INDEX, PIndexState.ACTIVE);
+        }
+    }
+
+    @Test
+    public void testTransformingTableAndIndex() throws Exception {
+        Properties testProps = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), testProps)) {
+            conn.setAutoCommit(true);
+            String tableName = "TBL_" + generateUniqueName();
+            String idxName = "IND_" + generateUniqueName();
+
+            String createTableSql = "CREATE TABLE " + tableName + " (PK1 VARCHAR NOT NULL, INT_PK INTEGER NOT NULL, " +
+                    "V1 VARCHAR, V2 INTEGER CONSTRAINT NAME_PK PRIMARY KEY(PK1, INT_PK)) ";
+            conn.createStatement().execute(createTableSql);
+
+            String upsertStmt = "UPSERT INTO " + tableName + " (PK1, INT_PK, V1, V2) VALUES ('%s', %d, '%s', %d)";
+            conn.createStatement().execute(String.format(upsertStmt, "a", 1, "val1", 1));
+
+            // Note that index will not be built, since we create it with ASYNC
+            String createIndexSql = "CREATE INDEX " + idxName + " ON " + tableName + " (PK1, INT_PK) include (V1)";
+            conn.createStatement().execute(createIndexSql);
+            assertMetadata(conn, ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, tableName);
+            assertMetadata(conn, ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, idxName);
+
+            assertEquals(0, getRowCount(conn, idxName));
+            IndexToolIT.runIndexTool(false, null, tableName, idxName);
+            assertEquals(1, getRowCount(conn, idxName));
+
+            // Index transform
+            conn.createStatement().execute("ALTER INDEX " + idxName + " ON " + tableName +
+                    " ACTIVE IMMUTABLE_STORAGE_SCHEME=SINGLE_CELL_ARRAY_WITH_OFFSETS, COLUMN_ENCODED_BYTES=2");
+
+            SystemTransformRecord record = Transform.getTransformRecord(null, idxName, tableName, null, conn.unwrap(PhoenixConnection.class));
+            assertNotNull(record);
+            assertMetadata(conn, PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS, PTable.QualifierEncodingScheme.TWO_BYTE_QUALIFIERS, record.getNewPhysicalTableName());
+            String newIndexTable = record.getNewPhysicalTableName();
+            assertEquals(0, getRowCount(conn, newIndexTable));
+
+            // Now do a table transform
+            conn.createStatement().execute("ALTER TABLE " + tableName + " SET "
+                    + " IMMUTABLE_STORAGE_SCHEME=SINGLE_CELL_ARRAY_WITH_OFFSETS, COLUMN_ENCODED_BYTES=2");
+            record = Transform.getTransformRecord(null, tableName, null, null, conn.unwrap(PhoenixConnection.class));
+            assertNotNull(record);
+            assertMetadata(conn, PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS, PTable.QualifierEncodingScheme.TWO_BYTE_QUALIFIERS, record.getNewPhysicalTableName());
+            String newDataTable = record.getNewPhysicalTableName();
+            assertEquals(0, getRowCount(conn, newDataTable));
+
+            assertIndexOrTableState(conn, null, newDataTable, PTableType.TABLE, PIndexState.CREATE_DISABLE);
+
+            // Now these live mutations should not go into the index table and the transforming table
+            conn.createStatement().execute(String.format(upsertStmt, "b", 2, "val2", 2));
+            conn.commit();
+            assertEquals(0, getRowCount(conn, newDataTable));
+            assertEquals(0, getRowCount(conn, newIndexTable));
+
+            // Activate index and see that transforming table doesn't have records
+            IndexToolIT.runIndexTool(false, null, tableName, newIndexTable);
+            assertEquals(2, getRowCount(conn, newIndexTable));
+            assertIndexOrTableState(conn, null, newIndexTable, PTableType.INDEX, PIndexState.ACTIVE);
+            assertEquals(0, getRowCount(conn, newDataTable));
+
+            // Now activate transforming table
+            List<String> args = TransformToolIT.getArgList(null, tableName, null,
+                    null, null, null, false, false, true, false, false);
+
+            TransformToolIT.runTransformTool(args.toArray(new String[0]), 0);
+            record = Transform.getTransformRecord(null, tableName, null, null, conn.unwrap(PhoenixConnection.class));
+            TransformToolIT.assertTransformStatusOrPartial(PTable.TransformStatus.PENDING_CUTOVER, record);
+            assertEquals(2, getRowCount(conn, newDataTable));
+        }
+    }
+
+    @Test
+    public void testWithViewIndex() throws Exception {
+        Properties testProps = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), testProps)) {
+            conn.setAutoCommit(true);
+            String tableName = "TBL_" + generateUniqueName();
+            String viewName = "VW_" + generateUniqueName();
+            String viewIdxName = "VWIDX_" + generateUniqueName();
+
+            String createTableSql = "CREATE TABLE " + tableName + " (PK1 VARCHAR NOT NULL, INT_PK INTEGER NOT NULL, " +
+                    "V1 VARCHAR, V2 INTEGER CONSTRAINT NAME_PK PRIMARY KEY(PK1, INT_PK)) ";
+            conn.createStatement().execute(createTableSql);
+
+            String createViewSql = "CREATE VIEW " + viewName + " ( VIEW_COL1 INTEGER, VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName;
+            conn.createStatement().execute(createViewSql);
+
+            String createViewIdxSql = "CREATE INDEX " + viewIdxName + " ON " + viewName + " (VIEW_COL1) include (VIEW_COL2) ";
+            conn.createStatement().execute(createViewIdxSql);
+
+            assertIndexOrTableState(conn, null, viewIdxName, PTableType.INDEX, PIndexState.ACTIVE);
+        }
+    }
+
+    @Test
+    public void testWithLocalIndex() throws Exception {
+        Properties testProps = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String dataTableName = "TBL_" + generateUniqueName();
+        String indexName = "LCLIDX_" + generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl(), testProps)) {
+            String createTableSql = "CREATE TABLE " + dataTableName
+                    + " (k INTEGER PRIMARY KEY, a bigint, b bigint, c bigint) ";
+            conn.createStatement().execute(createTableSql);
+            conn.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + dataTableName
+                    + " (b) INCLUDE (c) ");
+            assertIndexOrTableState(conn, null, indexName, PTableType.INDEX, PIndexState.ACTIVE);
+        }
+    }
+
+    private void assertIndexOrTableState(Connection conn, String schema, String tblName, PTableType type,
+                                         PIndexState state) throws SQLException {
+        ResultSet rs = conn.getMetaData().getTables("", schema, tblName
+                , new String[]{type.toString()});
+        assertTrue(rs.next());
+        assertEquals(tblName, rs.getString(3));
+        assertEquals(state.toString(), rs.getString("INDEX_STATE"));
+        assertFalse(rs.next());
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 8e6880d33c..06eda59c85 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -344,7 +344,7 @@ public class DeleteCompiler {
         if (!table.getIndexes().isEmpty()) {
             List<PTable> nonDisabledIndexes = Lists.newArrayListWithExpectedSize(table.getIndexes().size());
             for (PTable index : table.getIndexes()) {
-                if (index.getIndexState() != PIndexState.DISABLE && isMaintainedOnClient(index)) {
+                if (!index.getIndexState().isDisabled() && isMaintainedOnClient(index)) {
                     nonDisabledIndexes.add(index);
                 }
             }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index bc790a019e..ba57e47de6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -4079,7 +4079,7 @@ TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
                         metaDataCache.invalidate(new ImmutableBytesPtr(dataTableKey));
                     }
                     if (setRowKeyOrderOptimizableCell || disableTimeStampKVIndex != -1
-                            || currentState == PIndexState.DISABLE || newState == PIndexState.BUILDING) {
+                            || currentState.isDisabled() || newState == PIndexState.BUILDING) {
                         returnTable = doGetTable(tenantId, schemaName, tableName,
                                 HConstants.LATEST_TIMESTAMP, rowLock, request.getClientVersion());
                     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index 7691aceb50..223c69ef68 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -387,7 +387,7 @@ public class MetaDataRegionObserver implements RegionObserver,RegionCoprocessor
                     // Only perform relatively expensive check for all regions online when index
                     // is disabled or pending active since that's the state it's placed into when
                     // an index write fails.
-                    if ((indexState == PIndexState.DISABLE || indexState == PIndexState.PENDING_ACTIVE)
+                    if ((indexState.isDisabled() || indexState == PIndexState.PENDING_ACTIVE)
                             && !MetaDataUtil.tableRegionsOnline(this.env.getConfiguration(), indexPTable)) {
                         LOGGER.debug("Index rebuild has been skipped because not all regions of" +
                                 " index table=" + indexPTable.getName() + " are online.");
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 a465d98266..b76d0e7231 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
@@ -151,7 +151,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     
     private static boolean sendIndexMaintainer(PTable index) {
         PIndexState indexState = index.getIndexState();
-        return ! ( PIndexState.DISABLE == indexState || PIndexState.PENDING_ACTIVE == indexState );
+        return ! ( indexState.isDisabled() || PIndexState.PENDING_ACTIVE == indexState );
     }
 
     public static Iterator<PTable> maintainedIndexes(Iterator<PTable> indexes) {
@@ -232,7 +232,15 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         }
         int nIndexes = indexes.size();
         if (dataTable.getTransformingNewTable() != null) {
-            nIndexes++;
+            // If the transforming new table is in CREATE_DISABLE state, the mutations don't go into the table.
+            boolean disabled = dataTable.getTransformingNewTable().isIndexStateDisabled();
+            if (disabled && nIndexes == 0) {
+                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                return;
+            }
+            if (!disabled) {
+                nIndexes++;
+            }
         }
         ByteArrayOutputStream stream = new ByteArrayOutputStream();
         DataOutputStream output = new DataOutputStream(stream);
@@ -248,11 +256,15 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                     output.write(protoBytes);
             }
             if (dataTable.getTransformingNewTable() != null) {
-                ServerCachingProtos.TransformMaintainer proto = TransformMaintainer.toProto(
-                        dataTable.getTransformingNewTable().getTransformMaintainer(dataTable, connection));
-                byte[] protoBytes = proto.toByteArray();
-                WritableUtils.writeVInt(output, protoBytes.length);
-                output.write(protoBytes);
+                // We're not serializing the TransformMaintainer if the new transformed table is disabled
+                boolean disabled = dataTable.getTransformingNewTable().isIndexStateDisabled();
+                if (!disabled) {
+                    ServerCachingProtos.TransformMaintainer proto = TransformMaintainer.toProto(
+                            dataTable.getTransformingNewTable().getTransformMaintainer(dataTable, connection));
+                    byte[] protoBytes = proto.toByteArray();
+                    WritableUtils.writeVInt(output, protoBytes.length);
+                    output.write(protoBytes);
+                }
             }
         } catch (IOException e) {
             throw new RuntimeException(e); // Impossible
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java
index be5d8f9493..e5796eb989 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java
@@ -34,6 +34,7 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.join.MaxServerCacheSizeExceededException;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -136,7 +137,10 @@ public class IndexMetaDataCacheClient {
             }
             mutation.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
             if (table.getTransformingNewTable() != null) {
-                mutation.setAttribute(BaseScannerRegionObserver.DO_TRANSFORMING, TRUE_BYTES);
+                boolean disabled = table.getTransformingNewTable().isIndexStateDisabled();
+                if (!disabled) {
+                    mutation.setAttribute(BaseScannerRegionObserver.DO_TRANSFORMING, TRUE_BYTES);
+                }
             }
             if (attribValue != null) {
                 mutation.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
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 9d6d0b3a49..5d83801a33 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
@@ -1011,7 +1011,7 @@ public class IndexTool extends Configured implements Tool {
     }
 
     private void changeDisabledIndexStateToBuiding(Connection connection) throws SQLException {
-        if (pIndexTable != null && pIndexTable.getIndexState() == PIndexState.DISABLE) {
+        if (pIndexTable != null && pIndexTable.getIndexState().isDisabled()) {
             IndexUtil.updateIndexState(connection.unwrap(PhoenixConnection.class),
                     pIndexTable.getName().getString(), PIndexState.BUILDING, null);
         }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/transform/TransformTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/transform/TransformTool.java
index ff83e1b908..f37aee0ab9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/transform/TransformTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/transform/TransformTool.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.mapreduce.transform;
 
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.phoenix.mapreduce.PhoenixTTLTool;
+import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.phoenix.thirdparty.com.google.common.base.Strings;
 import org.apache.phoenix.thirdparty.com.google.common.collect.Lists;
@@ -827,6 +828,8 @@ public class TransformTool extends Configured implements Tool {
             throw new IllegalStateException("Only a paused transform can be resumed");
         }
 
+        Transform.updateNewTableState(connection.unwrap(PhoenixConnection.class), transformRecord, PIndexState.ACTIVE);
+
         runTransform(args, cmdLine);
 
         // Check if we already have a TransformMonitor task. If we do, remove those and start a new monitor
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index a1ee87687b..7be7b01e61 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -167,6 +167,8 @@ public interface QueryServices extends SQLCloseable {
     public static final String INDEX_FAILURE_THROW_EXCEPTION_ATTRIB = "phoenix.index.failure.throw.exception";
     public static final String INDEX_FAILURE_KILL_SERVER = "phoenix.index.failure.unhandled.killserver";
 
+    public static final String INDEX_CREATE_DEFAULT_STATE = "phoenix.index.create.default.state";
+
     // Index will be partially re-built from index disable time stamp - following overlap time
     @Deprecated
     public static final String INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB =
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index b0c5887b59..e59282d1e5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -43,6 +43,7 @@ import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILLABLE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILL_FILES_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.HBASE_CLIENT_SCANNER_TIMEOUT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.IMMUTABLE_ROWS_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.INDEX_CREATE_DEFAULT_STATE;
 import static org.apache.phoenix.query.QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.INDEX_POPULATION_SLEEP_TIME;
 import static org.apache.phoenix.query.QueryServices.INDEX_REBUILD_TASK_INITIAL_DELAY;
@@ -106,6 +107,7 @@ import static org.apache.phoenix.query.QueryServices.PHOENIX_TTL_SERVER_SIDE_MAS
 import java.util.Map.Entry;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.phoenix.schema.PIndexState;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.client.Consistency;
@@ -260,6 +262,9 @@ public class QueryServicesOptions {
 
     public static final boolean DEFAULT_USE_REVERSE_SCAN = true;
 
+    public static final String DEFAULT_CREATE_INDEX_STATE= PIndexState.BUILDING.toString();
+    public static final boolean DEFAULT_DISABLE_ON_DROP = false;
+
     /**
      * Use only first time SYSTEM.SEQUENCE table is created.
      */
@@ -468,6 +473,7 @@ public class QueryServicesOptions {
             .setIfUnset(CLIENT_METRICS_TAG, DEFAULT_CLIENT_METRICS_TAG)
             .setIfUnset(CLIENT_INDEX_ASYNC_THRESHOLD, DEFAULT_CLIENT_INDEX_ASYNC_THRESHOLD)
             .setIfUnset(PHOENIX_TTL_SERVER_SIDE_MASKING_ENABLED, DEFAULT_SERVER_SIDE_MASKING_ENABLED)
+            .setIfUnset(INDEX_CREATE_DEFAULT_STATE, DEFAULT_CREATE_INDEX_STATE)
             ;
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
index 38bd4e2b65..1c9cefd9de 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
@@ -39,6 +39,11 @@ public class DelegateTable implements PTable {
         return delegate.getIndexDisableTimestamp();
     }
 
+    @Override
+    public boolean isIndexStateDisabled() {
+        return delegate.isIndexStateDisabled();
+    }
+
     @Override
     public long getSequenceNumber() {
         return delegate.getSequenceNumber();
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 9663653af1..dc57c70517 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -17,15 +17,16 @@
  */
 package org.apache.phoenix.schema;
 
-import static org.apache.phoenix.exception.SQLExceptionCode.*;
+import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_TRANSFORM_TRANSACTIONAL_TABLE;
+import static org.apache.phoenix.exception.SQLExceptionCode.ERROR_WRITING_TO_SCHEMA_REGISTRY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_TASK_TABLE;
 import static org.apache.phoenix.query.QueryConstants.SYSTEM_SCHEMA_NAME;
+import static org.apache.phoenix.query.QueryServices.INDEX_CREATE_DEFAULT_STATE;
 import static org.apache.phoenix.thirdparty.com.google.common.collect.Sets.newLinkedHashSet;
 import static org.apache.phoenix.thirdparty.com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.RUN_UPDATE_STATS_ASYNC_ATTRIB;
 import static org.apache.phoenix.coprocessor.tasks.IndexRebuildTask.INDEX_NAME;
 import static org.apache.phoenix.coprocessor.tasks.IndexRebuildTask.REBUILD_ALL;
-import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_TRANSFORM_LOCAL_OR_VIEW_INDEX;
 import static org.apache.phoenix.exception.SQLExceptionCode.INSUFFICIENT_MULTI_TENANT_COLUMNS;
 import static org.apache.phoenix.exception.SQLExceptionCode.PARENT_TABLE_NOT_FOUND;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
@@ -435,7 +436,7 @@ public class MetaDataClient {
                     ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() +
                     ") VALUES (?, ?, ?, ?)";
     
-    private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
+    public static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
                     TABLE_SCHEM + "," +
@@ -1677,15 +1678,17 @@ public class MetaDataClient {
             return new MutationState(0, 0, connection);
         }
 
+        // If we create index in create_disabled state, we will build them later
+        if (table.getIndexState() == PIndexState.CREATE_DISABLE) {
+            return new MutationState(0, 0, connection);
+        }
+
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
         if (connection.getSCN() != null) {
             return buildIndexAtTimeStamp(table, statement.getTable());
         }
 
-        String dataTableFullName = SchemaUtil.getTableName(
-                tableRef.getTable().getSchemaName().getString(),
-                tableRef.getTable().getTableName().getString());
         return buildIndex(table, tableRef);
     }
 
@@ -2874,7 +2877,17 @@ public class MetaDataClient {
             Collections.reverse(columnMetadata);
             tableMetaData.addAll(columnMetadata);
             String dataTableName = parent == null || tableType == PTableType.VIEW ? null : parent.getTableName().getString();
-            PIndexState indexState = parent == null || tableType == PTableType.VIEW  ? null : PIndexState.BUILDING;
+            PIndexState defaultCreateState = PIndexState.valueOf(connection.getQueryServices().getConfiguration().
+                            get(INDEX_CREATE_DEFAULT_STATE, QueryServicesOptions.DEFAULT_CREATE_INDEX_STATE));
+            if (defaultCreateState == PIndexState.CREATE_DISABLE) {
+                if  (indexType == IndexType.LOCAL || sharedTable) {
+                    defaultCreateState = PIndexState.BUILDING;
+                }
+            }
+            PIndexState indexState = parent == null || tableType == PTableType.VIEW  ? null : defaultCreateState;
+            if (indexState == null && tableProps.containsKey(INDEX_STATE)) {
+                indexState = PIndexState.fromSerializedValue(tableProps.get(INDEX_STATE).toString());
+            }
             PreparedStatement tableUpsert = connection.prepareStatement(CREATE_TABLE);
             tableUpsert.setString(1, tenantIdStr);
             tableUpsert.setString(2, schemaName);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PIndexState.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PIndexState.java
index 2b6ac4aac1..6d4dd7d682 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PIndexState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PIndexState.java
@@ -32,7 +32,12 @@ public enum PIndexState {
     // When an index write fails, it is put in this state, and we let the client retry the mutation
     // After retries are exhausted, the client should mark the index as disabled, but if that
     // doesn't happen, then the index is considered disabled if it's been in this state too long
-    PENDING_DISABLE("w");
+    PENDING_DISABLE("w"),
+    //When we create/drop some indexes in one cluster with a replication peer, the peer doesn't immediately have this
+    //index and Hbase throws a replication error when we try to write into indexes that don't have a matching table.
+    //To remediate this issue, we can optionally create indexes in CREATE_DISABLED state and enable them after all
+    //the replication peers have the table. Similar for drop.
+    CREATE_DISABLE("c");
 
     private final String serializedValue;
     private final byte[] serializedBytes;
@@ -56,6 +61,10 @@ public enum PIndexState {
         return nameBytesValue;
     }
 
+    public boolean isDisabled() {
+        return (this == DISABLE || this == CREATE_DISABLE);
+    }
+
     private static final PIndexState[] FROM_VALUE;
     private static final int FROM_VALUE_OFFSET;
     static {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index e995e502fc..eb38078c6b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -663,6 +663,9 @@ public interface PTable extends PMetaDataEntity {
     long getTimeStamp();
     long getSequenceNumber();
     long getIndexDisableTimestamp();
+
+    boolean isIndexStateDisabled();
+
     /**
      * @return table name
      */
@@ -892,7 +895,7 @@ public interface PTable extends PMetaDataEntity {
     int getRowTimestampColPos();
     long getUpdateCacheFrequency();
     boolean isNamespaceMapped();
-    
+
     /**
      * @return The sequence name used to get the unique identifier for views
      * that are automatically partitioned.
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 4cfcdf4fb0..af642ba09b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -28,6 +28,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHYSICAL_TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
@@ -310,6 +311,9 @@ public class PTableImpl implements PTable {
         }
 
         public Builder setState(PIndexState state) {
+            if (state != null) {
+                propertyValues.put(INDEX_STATE, state.getSerializedValue());
+            }
             this.state = state;
             return this;
         }
@@ -1586,7 +1590,12 @@ public class PTableImpl implements PTable {
     public long getIndexDisableTimestamp() {
         return indexDisableTimestamp;
     }
-    
+
+    @Override
+    public boolean isIndexStateDisabled() {
+        return getIndexState()!= null && getIndexState().isDisabled();
+    }
+
     @Override
     public PColumn getPKColumn(String name) throws ColumnNotFoundException {
         List<PColumn> columns = columnsByName.get(name);
@@ -2026,10 +2035,10 @@ public class PTableImpl implements PTable {
             builder.setPhysicalTableNameBytes(ByteStringer.wrap(table.getPhysicalName(true).getBytes()));
         }
         builder.setTableType(ProtobufUtil.toPTableTypeProto(table.getType()));
+        if (table.getIndexState() != null) {
+            builder.setIndexState(table.getIndexState().getSerializedValue());
+        }
         if (table.getType() == PTableType.INDEX) {
-            if (table.getIndexState() != null) {
-                builder.setIndexState(table.getIndexState().getSerializedValue());
-            }
             if (table.getViewIndexId() != null) {
                 builder.setViewIndexId(table.getViewIndexId());
                 builder.setViewIndexIdType(table.getviewIndexIdType().getSqlType());
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/transform/Transform.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/transform/Transform.java
index ab4aac4b58..88112ab8ff 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/transform/Transform.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/transform/Transform.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.schema.transform;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.TableInfo;
+import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.thirdparty.com.google.common.base.Strings;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -74,9 +75,11 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSFORM_STATUS;
 import static org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.MAPREDUCE_TENANT_ID;
 import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+import static org.apache.phoenix.query.QueryServices.INDEX_CREATE_DEFAULT_STATE;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static org.apache.phoenix.schema.ColumnMetaDataOps.addColumnMutation;
 import static org.apache.phoenix.schema.MetaDataClient.CREATE_LINK;
+import static org.apache.phoenix.schema.MetaDataClient.UPDATE_INDEX_STATE_TO_ACTIVE;
 import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
 import static org.apache.phoenix.schema.PTableType.INDEX;
 import static org.apache.phoenix.schema.PTableType.VIEW;
@@ -126,6 +129,12 @@ public class Transform {
             // TODO: calculate old and new metadata
             transformBuilder.setNewMetadata(newMetadata);
             transformBuilder.setOldMetadata(oldMetadata);
+            PIndexState defaultCreateState = PIndexState.valueOf(connection.getQueryServices().getConfiguration().
+                    get(INDEX_CREATE_DEFAULT_STATE, QueryServicesOptions.DEFAULT_CREATE_INDEX_STATE));
+            if (defaultCreateState == PIndexState.CREATE_DISABLE) {
+                // Create a paused transform. This can be enabled later by calling TransformTool resume
+                transformBuilder.setTransformStatus(PTable.TransformStatus.PAUSED.name());
+            }
             if (Strings.isNullOrEmpty(newPhysicalTableName)) {
                 newPhysicalTableName = generateNewTableName(schema, logicalTableName, sequenceNum);
             }
@@ -149,11 +158,14 @@ public class Transform {
             long sequenceNum, PhoenixConnection connection) throws Exception {
         PName newTableName = PNameFactory.newName(systemTransformParams.getNewPhysicalTableName());
         PName newTableNameWithoutSchema = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(systemTransformParams.getNewPhysicalTableName()));
+        PIndexState defaultCreateState = PIndexState.valueOf(connection.getQueryServices().getConfiguration().
+                get(INDEX_CREATE_DEFAULT_STATE, QueryServicesOptions.DEFAULT_CREATE_INDEX_STATE));
         PTable newTable = new PTableImpl.Builder()
                 .setTableName(newTableNameWithoutSchema)
                 .setParentTableName(table.getParentTableName())
                 .setBaseTableLogicalName(table.getBaseTableLogicalName())
                 .setPhysicalTableName(newTableNameWithoutSchema)
+                .setState(defaultCreateState)
                 .setAllColumns(table.getColumns())
                 .setAppendOnlySchema(table.isAppendOnlySchema())
                 .setAutoPartitionSeqName(table.getAutoPartitionSeqName())
@@ -213,10 +225,14 @@ public class Transform {
                     (view.getSchemaName()==null? null: Bytes.toString(view.getSchemaName())), Bytes.toString(view.getTableName())
                     , newTableName, sequenceNum);
         }
-        // add a monitoring task
-        TransformMonitorTask.addTransformMonitorTask(connection, connection.getQueryServices().getConfiguration(), systemTransformParams,
-                PTable.TaskStatus.CREATED, new Timestamp(EnvironmentEdgeManager.currentTimeMillis()), null);
 
+        if (defaultCreateState != PIndexState.CREATE_DISABLE) {
+            // add a monitoring task
+            TransformMonitorTask.addTransformMonitorTask(connection, connection.getQueryServices().getConfiguration(), systemTransformParams,
+                    PTable.TaskStatus.CREATED, new Timestamp(EnvironmentEdgeManager.currentTimeMillis()), null);
+        } else {
+            LOGGER.info("Transform will not be monitored until it is resumed again.");
+        }
         return newTable;
     }
 
@@ -333,6 +349,26 @@ public class Transform {
         return false;
     }
 
+    public static void updateNewTableState(PhoenixConnection connection, SystemTransformRecord systemTransformRecord,
+                                           PIndexState state)
+            throws SQLException {
+        String schema = SchemaUtil.getSchemaNameFromFullName(systemTransformRecord.getNewPhysicalTableName());
+        String tableName = SchemaUtil.getTableNameFromFullName(systemTransformRecord.getNewPhysicalTableName());
+        try (PreparedStatement tableUpsert = connection.prepareStatement(UPDATE_INDEX_STATE_TO_ACTIVE)){
+            tableUpsert.setString(1, systemTransformRecord.getTenantId() == null ? null :
+                    systemTransformRecord.getTenantId());
+            tableUpsert.setString(2, schema);
+            tableUpsert.setString(3, tableName);
+            tableUpsert.setString(4, state.getSerializedValue());
+            tableUpsert.setLong(5, 0);
+            tableUpsert.setLong(6, 0);
+            tableUpsert.execute();
+        }
+        // Update cache
+        UpgradeUtil.clearCache(connection, connection.getTenantId(), schema, tableName,
+                systemTransformRecord.getLogicalParentName(), MIN_TABLE_TIMESTAMP);
+    }
+
     public static void removeTransformRecord(
             SystemTransformRecord transformRecord, PhoenixConnection connection) throws SQLException {
         connection.prepareStatement("DELETE FROM  "
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 c084a27699..508e2a1e90 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
@@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.ByteStringer;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos;
@@ -48,10 +47,6 @@ import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
 
-import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.ValueSchema;
-import org.apache.phoenix.schema.tuple.BaseTuple;
-import org.apache.phoenix.schema.tuple.ValueGetterTuple;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -73,7 +68,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
 
 public class TransformMaintainer extends IndexMaintainer {
     private boolean isMultiTenant;