You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by vi...@apache.org on 2019/02/05 01:33:55 UTC

[phoenix] branch 4.x-HBase-1.2 updated: PHOENIX-5080 Index becomes Active during Partial Index Rebuilder if Index Failure happens

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

vincentpoon pushed a commit to branch 4.x-HBase-1.2
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x-HBase-1.2 by this push:
     new 977b2c3  PHOENIX-5080 Index becomes Active during Partial Index Rebuilder if Index Failure happens
977b2c3 is described below

commit 977b2c38c3f559faac4f5fbf579dfb839c7a9ce2
Author: Monani Mihir <mo...@gmail.com>
AuthorDate: Sat Feb 2 11:00:19 2019 +0530

    PHOENIX-5080 Index becomes Active during Partial Index Rebuilder if Index Failure happens
---
 .../end2end/index/PartialIndexRebuilderIT.java     | 66 +++++++++++++++++++++-
 .../coprocessor/BaseScannerRegionObserver.java     |  9 ++-
 .../UngroupedAggregateRegionObserver.java          | 25 +++++++-
 .../org/apache/phoenix/execute/MutationState.java  | 14 ++++-
 .../org/apache/phoenix/hbase/index/Indexer.java    | 10 +---
 .../hbase/index/builder/IndexBuildManager.java     |  8 +++
 .../phoenix/index/PhoenixIndexFailurePolicy.java   | 32 +++++++++--
 .../apache/phoenix/index/PhoenixIndexMetaData.java |  3 +-
 .../java/org/apache/phoenix/query/BaseTest.java    | 23 ++++++++
 9 files changed, 168 insertions(+), 22 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
index 46443e3..cda282b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.phoenix.coprocessor.MetaDataRegionObserver;
 import org.apache.phoenix.coprocessor.MetaDataRegionObserver.BuildIndexScheduleTask;
 import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
-import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.execute.CommitException;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -86,6 +85,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
     private static final long REBUILD_PERIOD = 50000;
     private static final long REBUILD_INTERVAL = 2000;
     private static RegionCoprocessorEnvironment indexRebuildTaskRegionEnvironment;
+    private static Boolean runRebuildOnce = true;
 
     
     @BeforeClass
@@ -125,6 +125,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
         runIndexRebuilderAsync(interval, cancel, Collections.<String>singletonList(table));
     }
     private static void runIndexRebuilderAsync(final int interval, final boolean[] cancel, final List<String> tables) {
+        runRebuildOnce = true;
         Thread thread = new Thread(new Runnable() {
             @Override
             public void run() {
@@ -137,6 +138,8 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
                         throw new RuntimeException(e);
                     } catch (SQLException e) {
                         LOG.error(e.getMessage(),e);
+                    } finally {
+                        runRebuildOnce = false;
                     }
                 }
             }
@@ -554,7 +557,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
         
         @Override
         public long currentTime() {
-            return time;
+            return time++;
         }
     }
     
@@ -1068,6 +1071,65 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
         }
     }
 
+    @Test
+    @Repeat(5)
+    public void testIndexActiveIfRegionMovesWhileRebuilding() throws Throwable {
+        final MyClock clock = new MyClock(1000);
+        EnvironmentEdgeManager.injectEdge(clock);
+        String schemaName = generateUniqueName();
+        String tableName = generateUniqueName();
+        String indexName = generateUniqueName();
+        int nThreads = 5;
+        int nRows = 50;
+        int nIndexValues = 23;
+        int batchSize = 200;
+        final CountDownLatch doneSignal = new CountDownLatch(nThreads);
+        boolean[] cancel = new boolean[1];
+
+        final String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        final String fullIndexName = SchemaUtil.getTableName(schemaName, indexName);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            try {
+                conn.createStatement().execute("CREATE TABLE " + fullTableName
+                    + "(k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 INTEGER, "
+                    + "CONSTRAINT pk PRIMARY KEY (k1,k2)) STORE_NULLS=true, VERSIONS=1");
+                conn.createStatement().execute("CREATE INDEX " + indexName + " ON "
+                    + fullTableName + "(v1)");
+                conn.commit();
+                long disableTS = clock.currentTime();
+                HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices()
+                        .getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+                IndexUtil.updateIndexState(fullIndexName, disableTS,
+                    metaTable, PIndexState.DISABLE);
+                assertTrue(TestUtil.checkIndexState(conn, fullIndexName,
+                    PIndexState.DISABLE, disableTS));
+                mutateRandomly(fullTableName, nThreads, nRows,
+                    nIndexValues, batchSize, doneSignal);
+                assertTrue("Ran out of time", doneSignal.await(120, TimeUnit.SECONDS));
+                runIndexRebuilder(fullTableName);
+                assertTrue(TestUtil.checkIndexState(conn, fullIndexName,
+                    PIndexState.INACTIVE, disableTS));
+                clock.time += WAIT_AFTER_DISABLED;
+                runIndexRebuilderAsync(500,cancel,fullTableName);
+                unassignRegionAsync(fullIndexName);
+                while (runRebuildOnce) {
+                    PIndexState indexState = TestUtil.getIndexState(conn, fullIndexName);
+                    if (indexState != PIndexState.INACTIVE && indexState != PIndexState.ACTIVE) {
+                        cancel[0] = true;
+                        throw new Exception("Index State should not transtion from INACTIVE to "
+                            + indexState);
+                    }
+                }
+                assertTrue(TestUtil.checkIndexState(conn, fullIndexName, PIndexState.ACTIVE, 0L));
+            } finally {
+                cancel[0] = true;
+                EnvironmentEdgeManager.injectEdge(null);
+            }
+            long totalRows = IndexScrutiny.scrutinizeIndex(conn, fullTableName, fullIndexName);
+            assertEquals(nRows, totalRows);
+        }
+    }
+
     public static class WriteFailingRegionObserver extends SimpleRegionObserver {
         @Override
         public void postBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index a426e39..5f82c6d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -110,10 +110,14 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     
     public final static byte[] REPLAY_TABLE_AND_INDEX_WRITES = PUnsignedTinyint.INSTANCE.toBytes(1);
     public final static byte[] REPLAY_ONLY_INDEX_WRITES = PUnsignedTinyint.INSTANCE.toBytes(2);
+    // In case of Index Write failure, we need to determine that Index mutation
+    // is part of normal client write or Index Rebuilder. # PHOENIX-5080
+    public final static byte[] REPLAY_INDEX_REBUILD_WRITES = PUnsignedTinyint.INSTANCE.toBytes(3);
     
     public enum ReplayWrite {
         TABLE_AND_INDEX,
-        INDEX_ONLY;
+        INDEX_ONLY,
+        REBUILD_INDEX_ONLY;
         
         public static ReplayWrite fromBytes(byte[] replayWriteBytes) {
             if (replayWriteBytes == null) {
@@ -125,6 +129,9 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             if (Bytes.compareTo(REPLAY_ONLY_INDEX_WRITES, replayWriteBytes) == 0) {
                 return INDEX_ONLY;
             }
+            if (Bytes.compareTo(REPLAY_INDEX_REBUILD_WRITES, replayWriteBytes) == 0) {
+                return REBUILD_INDEX_ONLY;
+            }
             throw new IllegalArgumentException("Unknown ReplayWrite code of " + Bytes.toStringBinary(replayWriteBytes));
         }
     };
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 26e338f..cab13f1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -99,6 +99,7 @@ import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.index.PhoenixIndexFailurePolicy;
 import org.apache.phoenix.index.PhoenixIndexFailurePolicy.MutateCommand;
+import org.apache.phoenix.index.PhoenixIndexMetaData;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.join.HashJoinInfo;
@@ -248,6 +249,11 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                 public void doMutation() throws IOException {
                     commitBatch(region, localRegionMutations, blockingMemstoreSize);
                 }
+
+                @Override
+                public List<Mutation> getMutationList() {
+                    return localRegionMutations;
+                }
             });
         }
     }
@@ -921,6 +927,11 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                 public void doMutation() throws IOException {
                     commitBatchWithHTable(targetHTable, remoteRegionMutations);
                 }
+
+                @Override
+                public List<Mutation> getMutationList() {
+                    return remoteRegionMutations;
+                }
             });
         }
         localRegionMutations.clear();
@@ -935,7 +946,13 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             // For an index write failure, the data table write succeeded,
             // so when we retry we need to set REPLAY_WRITES
             for (Mutation mutation : localRegionMutations) {
-                mutation.setAttribute(REPLAY_WRITES, REPLAY_ONLY_INDEX_WRITES);
+                if (PhoenixIndexMetaData.isIndexRebuild(mutation.getAttributesMap())) {
+                    mutation.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
+                        BaseScannerRegionObserver.REPLAY_INDEX_REBUILD_WRITES);
+                } else {
+                    mutation.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
+                        BaseScannerRegionObserver.REPLAY_ONLY_INDEX_WRITES);
+                }
                 // use the server timestamp for index write retrys
                 KeyValueUtil.setTimestamp(mutation, serverTimestamp);
             }
@@ -1060,7 +1077,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                     put = new Put(CellUtil.cloneRow(cell));
                                     put.setAttribute(useProto ? PhoenixIndexCodec.INDEX_PROTO_MD : PhoenixIndexCodec.INDEX_MD, indexMetaData);
                                     put.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                                    put.setAttribute(REPLAY_WRITES, REPLAY_ONLY_INDEX_WRITES);
+                                    put.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
+                                        BaseScannerRegionObserver.REPLAY_INDEX_REBUILD_WRITES);
                                     put.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, clientVersionBytes);
                                     mutations.add(put);
                                     // Since we're replaying existing mutations, it makes no sense to write them to the wal
@@ -1072,7 +1090,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                     del = new Delete(CellUtil.cloneRow(cell));
                                     del.setAttribute(useProto ? PhoenixIndexCodec.INDEX_PROTO_MD : PhoenixIndexCodec.INDEX_MD, indexMetaData);
                                     del.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                                    del.setAttribute(REPLAY_WRITES, REPLAY_ONLY_INDEX_WRITES);
+                                    del.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
+                                        BaseScannerRegionObserver.REPLAY_INDEX_REBUILD_WRITES);
                                     del.setAttribute(BaseScannerRegionObserver.CLIENT_VERSION, clientVersionBytes);
                                     mutations.add(del);
                                     // Since we're replaying existing mutations, it makes no sense to write them to the wal
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 e33a005..87b4945 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
@@ -59,6 +59,7 @@ import org.apache.phoenix.index.IndexMetaDataCacheClient;
 import org.apache.phoenix.index.PhoenixIndexBuilder;
 import org.apache.phoenix.index.PhoenixIndexFailurePolicy;
 import org.apache.phoenix.index.PhoenixIndexFailurePolicy.MutateCommand;
+import org.apache.phoenix.index.PhoenixIndexMetaData;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.monitoring.GlobalClientMetrics;
@@ -996,6 +997,11 @@ public class MutationState implements SQLCloseable {
                                             throw new IOException(e);
                                         }
                                     }
+
+                                    @Override
+                                    public List<Mutation> getMutationList() {
+                                        return mutationBatch;
+                                    }
                                 }, iwe, connection, connection.getQueryServices().getProps());
                             } else {
                                 hTable.batch(mutationBatch);
@@ -1051,8 +1057,12 @@ public class MutationState implements SQLCloseable {
                                     // For an index write failure, the data table write succeeded,
                                     // so when we retry we need to set REPLAY_WRITES
                                     for (Mutation m : mutationList) {
-                                        m.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
-                                                BaseScannerRegionObserver.REPLAY_ONLY_INDEX_WRITES);
+                                        if (!PhoenixIndexMetaData.
+                                                isIndexRebuild(m.getAttributesMap())) {
+                                            m.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
+                                                BaseScannerRegionObserver.REPLAY_ONLY_INDEX_WRITES
+                                                );
+                                        }
                                         KeyValueUtil.setTimestamp(m, serverTimestamp);
                                     }
                                     shouldRetry = true;
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
index e4e6991..92f6d0b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
@@ -18,10 +18,6 @@
 package org.apache.phoenix.hbase.index;
 
 import static org.apache.phoenix.hbase.index.util.IndexManagementUtil.rethrowIndexingException;
-import static org.apache.phoenix.hbase.index.write.IndexWriterUtils.DEFAULT_INDEX_WRITER_RPC_PAUSE;
-import static org.apache.phoenix.hbase.index.write.IndexWriterUtils.DEFAULT_INDEX_WRITER_RPC_RETRIES_NUMBER;
-import static org.apache.phoenix.hbase.index.write.IndexWriterUtils.INDEX_WRITER_RPC_PAUSE;
-import static org.apache.phoenix.hbase.index.write.IndexWriterUtils.INDEX_WRITER_RPC_RETRIES_NUMBER;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
@@ -52,8 +48,6 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.ipc.controller.InterRegionServerIndexRpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
@@ -90,7 +84,6 @@ import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.trace.TracingUtils;
 import org.apache.phoenix.trace.util.NullSpan;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
-import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.ServerUtil.ConnectionType;
@@ -460,7 +453,8 @@ public class Indexer extends BaseRegionObserver {
               }
               // No need to write the table mutations when we're rebuilding
               // the index as they're already written and just being replayed.
-              if (replayWrite == ReplayWrite.INDEX_ONLY) {
+              if (replayWrite == ReplayWrite.INDEX_ONLY
+                      || replayWrite == ReplayWrite.REBUILD_INDEX_ONLY) {
                   miniBatchOp.setOperationStatus(i, NOWRITE);
               }
     
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildManager.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildManager.java
index 2550dd1..07a05bc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildManager.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/builder/IndexBuildManager.java
@@ -32,9 +32,11 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver.ReplayWrite;
 import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.hbase.index.covered.IndexMetaData;
+import org.apache.phoenix.index.PhoenixIndexMetaData;
 
 /**
  * Manage the building of index updates from primary table updates.
@@ -88,6 +90,12 @@ public class IndexBuildManager implements Stoppable {
     ArrayList<Pair<Mutation, byte[]>> results = new ArrayList<>(mutations.size());
     for (Mutation m : mutations) {
       Collection<Pair<Mutation, byte[]>> updates = delegate.getIndexUpdate(m, indexMetaData);
+      if (PhoenixIndexMetaData.isIndexRebuild(m.getAttributesMap())) {
+          for (Pair<Mutation, byte[]> update : updates) {
+            update.getFirst().setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
+                BaseScannerRegionObserver.REPLAY_INDEX_REBUILD_WRITES);
+          }
+      }
       results.addAll(updates);
     }
     return results;
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index 26a1fbe..b9da5e9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -153,6 +153,24 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
     public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
         boolean throwing = true;
         long timestamp = HConstants.LATEST_TIMESTAMP;
+        // we should check if failed list of mutation are part of Index Rebuilder or not.
+        // If its part of Index Rebuilder, we throw exception and do retries.
+        // If succeeds, we don't update Index State.
+        // Once those retries are exhausted, we transition Index to DISABLE
+        // It's being handled as part of PhoenixIndexFailurePolicy.doBatchWithRetries
+        Mutation checkMutationForRebuilder = attempted.entries().iterator().next().getValue();
+        boolean isIndexRebuild =
+                PhoenixIndexMetaData.isIndexRebuild(checkMutationForRebuilder.getAttributesMap());
+        if (isIndexRebuild) {
+            SQLException sqlException =
+                    new SQLExceptionInfo.Builder(SQLExceptionCode.INDEX_WRITE_FAILURE)
+                            .setRootCause(cause).setMessage(cause.getLocalizedMessage()).build()
+                            .buildException();
+            IOException ioException = ServerUtil.wrapInDoNotRetryIOException(
+                        "Retrying Index rebuild mutation, we will update Index state to DISABLE "
+                        + "if all retries are exhusated", sqlException, timestamp);
+            throw ioException;
+        }
         try {
             timestamp = handleFailureWithExceptions(attempted, cause);
             throwing = false;
@@ -167,10 +185,8 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
                                 .setRootCause(cause).setMessage(cause.getLocalizedMessage()).build()
                                 .buildException();
                 IOException ioException = ServerUtil.wrapInDoNotRetryIOException(null, sqlException, timestamp);
-            	Mutation m = attempted.entries().iterator().next().getValue();
-            	boolean isIndexRebuild = PhoenixIndexMetaData.isIndexRebuild(m.getAttributesMap());
-            	// Always throw if rebuilding index since the rebuilder needs to know if it was successful
-            	if (throwIndexWriteFailure || isIndexRebuild) {
+                // Here we throw index write failure to client so it can retry index mutation.
+                if (throwIndexWriteFailure) {
             		throw ioException;
             	} else {
                     LOG.warn("Swallowing index write failure", ioException);
@@ -430,6 +446,8 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
 
     public static interface MutateCommand {
         void doMutation() throws IOException;
+
+        List<Mutation> getMutationList();
     }
 
     /**
@@ -464,7 +482,11 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
                 Thread.sleep(ConnectionUtils.getPauseTime(pause, numRetry)); // HBase's exponential backoff
                 mutateCommand.doMutation();
                 // success - change the index state from PENDING_DISABLE back to ACTIVE
-                handleIndexWriteSuccessFromClient(iwe, connection);
+                // If it's not Index Rebuild
+                if (!PhoenixIndexMetaData.isIndexRebuild(
+                    mutateCommand.getMutationList().get(0).getAttributesMap())) {
+                    handleIndexWriteSuccessFromClient(iwe, connection);
+                }
                 return;
             } catch (IOException e) {
                 SQLException inferredE = ServerUtil.parseLocalOrRemoteServerException(e);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
index 46f5b77..75ce9f4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
@@ -37,7 +37,8 @@ public class PhoenixIndexMetaData implements IndexMetaData {
     private final boolean hasLocalIndexes;
     
     public static boolean isIndexRebuild(Map<String,byte[]> attributes) {
-        return attributes.get(BaseScannerRegionObserver.REPLAY_WRITES) != null;
+        return attributes.get(BaseScannerRegionObserver.REPLAY_WRITES)
+                == BaseScannerRegionObserver.REPLAY_INDEX_REBUILD_WRITES;
     }
     
     public static ReplayWrite getReplayWrite(Map<String,byte[]> attributes) {
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 42fa085..a42d682 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -1926,4 +1926,27 @@ public abstract class BaseTest {
             Thread.sleep(100);
         }
     }
+
+    /**
+     * It always unassign first region of table.
+     * @param tableName move region of table.
+     * @throws IOException
+     */
+    protected static void unassignRegionAsync(final String tableName) throws IOException {
+        Thread thread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                    try {
+                        final HBaseAdmin admin = utility.getHBaseAdmin();
+                        final HRegionInfo tableRegion =
+                                admin.getTableRegions(TableName.valueOf(tableName)).get(0);
+                        admin.unassign(tableRegion.getRegionName(), false);
+                    } catch (IOException e) {
+                        e.printStackTrace();
+                    }
+            }
+        });
+        thread.setDaemon(true);
+        thread.start();
+    }
 }