You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2016/07/29 20:54:25 UTC

phoenix git commit: PHOENIX-3120 AsyncIndexRebuilderTask fails for transactional tables

Repository: phoenix
Updated Branches:
  refs/heads/master cb21c8175 -> 3251ac58a


PHOENIX-3120 AsyncIndexRebuilderTask fails for transactional tables


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

Branch: refs/heads/master
Commit: 3251ac58a6a9de890285ae82ba86d76618fa0a1c
Parents: cb21c81
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Jul 28 18:21:45 2016 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Fri Jul 29 13:53:59 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/IndexIT.java   | 108 ++++++++++++++++---
 .../coprocessor/MetaDataRegionObserver.java     |  34 +++---
 .../apache/phoenix/schema/MetaDataClient.java   |  11 +-
 3 files changed, 120 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3251ac58/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index ff26208..35a0aad 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -42,21 +42,26 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.coprocessor.generated.PTableProtos.PTableType;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -67,16 +72,13 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class IndexIT extends BaseHBaseManagedTimeIT {
-	
+public class IndexIT extends BaseHBaseManagedTimeTableReuseIT {
+    private static final long MAX_WAIT_FOR_ASYNC_INDEX_BUILD = 30000;
+    	
 	private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
 	private final String tableDDLOptions;
-	private final String tableName;
-    private final String indexName;
-    private final String fullTableName;
-    private final String fullIndexName;
 	
 	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
 		this.localIndex = localIndex;
@@ -91,10 +93,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 			optionBuilder.append(" TRANSACTIONAL=true ");
 		}
 		this.tableDDLOptions = optionBuilder.toString();
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
-        this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
-        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 	}
 	
 	@BeforeClass
@@ -115,6 +113,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
 	@Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
+	    String tableName = "TBL_" + generateRandomString();
+	    String indexName = "IND_" + generateRandomString();
+	    String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+	    String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -171,6 +173,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -226,6 +232,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         testCreateIndexAfterUpsertStarted(false, fullTableName + "1", fullIndexName + "1");
         if (transactional) {
             testCreateIndexAfterUpsertStarted(true, fullTableName + "2", fullIndexName + "2");
@@ -322,6 +332,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
@@ -372,6 +386,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testGroupByCount() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -390,6 +408,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -413,6 +435,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -435,6 +461,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -474,6 +504,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -529,6 +563,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -604,6 +642,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSelectCF() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -661,6 +703,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -721,6 +767,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
@@ -809,6 +859,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -892,6 +946,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -909,7 +967,20 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     }
 
     @Test
-    public void testIndexWithDecimalCol() throws Exception {
+    public void testSyncIndexWithDecimalCol() throws Exception {
+        testIndexWithDecimalCol(false);
+    }
+    
+    @Test
+    public void testAsyncIndexWithDecimalCol() throws Exception {
+        testIndexWithDecimalCol(true);
+    }
+    
+    private void testIndexWithDecimalCol(boolean async) throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -920,10 +991,23 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             createMultiCFTestTable(fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
-            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)" + (async ? " ASYNC" : "");
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
+            if (async) {
+                boolean wasBuilt = false;
+                long startTime = System.currentTimeMillis();
+                do {
+                    rs = conn.getMetaData().getTables("", TestUtil.DEFAULT_SCHEMA_NAME, StringUtil.escapeLike(indexName), new String[] {PTableType.INDEX.toString()});
+                    if (rs.next() && PIndexState.ACTIVE.toString().equalsIgnoreCase(rs.getString(PhoenixDatabaseMetaData.INDEX_STATE))) {
+                        wasBuilt = true;
+                        break;
+                    }
+                } while (System.currentTimeMillis() - startTime < MAX_WAIT_FOR_ASYNC_INDEX_BUILD);
+                assertTrue("Asyncronous index failed to build", wasBuilt);
+            }
+            
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3251ac58/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index fd3226c..f13217a 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
@@ -72,6 +72,7 @@ import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.ByteUtil;
@@ -216,37 +217,36 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                 conn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
                 Statement s = conn.createStatement();
                 ResultSet rs = s.executeQuery(ASYNC_INDEX_INFO_QUERY);
-
-                PhoenixConnection alterIndexConnection = null;
+                PhoenixConnection metaDataClientConn = conn;
                 while (rs.next()) {
                     String tableName = rs.getString(PhoenixDatabaseMetaData.DATA_TABLE_NAME);
                     String tableSchema = rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM);
                     String indexName = rs.getString(PhoenixDatabaseMetaData.TABLE_NAME);
-                    String tableNameWithSchema = SchemaUtil.getTableName(tableSchema, tableName);
                     
-                    final PTable pindexTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(tableSchema, indexName));
+                    final PTable indexTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(tableSchema, indexName));
+                    final PTable dataTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(tableSchema, tableName));
                     // this is set to ensure index tables remains consistent post population.
-                    long maxTimeRange = pindexTable.getTimeStamp()+1;
+                    long maxTimeRange = indexTable.getTimeStamp()+1;
 
                     try {
                         final Properties props = new Properties();
-                        if (!pindexTable.isTransactional())
+                        Long txnScn = null;
+                        if (!indexTable.isTransactional()) {
                             props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(maxTimeRange));
-                        alterIndexConnection = QueryUtil.getConnectionOnServer(props, env.getConfiguration()).unwrap(PhoenixConnection.class);
-
-                        // Alter index query for rebuilding async indexes
-                        String alterIndexQuery = String.format("ALTER INDEX IF EXISTS %s ON %s REBUILD", indexName, tableNameWithSchema);
-    
-                        LOG.info("Executing Rebuild Index Query:" + alterIndexQuery);
-                        alterIndexConnection.createStatement().execute(alterIndexQuery);
+                            metaDataClientConn = QueryUtil.getConnectionOnServer(props, env.getConfiguration()).unwrap(PhoenixConnection.class);
+                            txnScn = maxTimeRange;
+                        }
+                        MetaDataClient client = new MetaDataClient(conn);
+                        LOG.info("Building Index " + SchemaUtil.getTableName(tableSchema, indexName));
+                        client.buildIndex(indexTable, new TableRef(dataTable), txnScn);
                     } catch (Throwable t) {
-                        LOG.error("AsyncIndexRebuilderTask failed during rebuilding index!", t);
+                        LOG.error("AsyncIndexRebuilderTask failed while building index!", t);
                     } finally {
-                        if (alterIndexConnection != null) {
+                        if (metaDataClientConn != null) {
                             try {
-                                alterIndexConnection.close();
+                                metaDataClientConn.close();
                             } catch (SQLException ignored) {
-                                LOG.debug("AsyncIndexRebuilderTask can't close alterIndexConnection", ignored);
+                                LOG.debug("AsyncIndexRebuilderTask can't close metaDataClientConn", ignored);
                             }
                         }
                     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3251ac58/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index d0e749f..729f5f9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -1095,7 +1095,7 @@ public class MetaDataClient {
         boolean success = false;
         SQLException sqlException = null;
         try {
-            MutationState state = newClientAtNextTimeStamp.buildIndex(index, tableRef);
+            MutationState state = newClientAtNextTimeStamp.buildIndex(index, tableRef, null);
             success = true;
             return state;
         } catch (SQLException e) {
@@ -1121,7 +1121,7 @@ public class MetaDataClient {
         throw new IllegalStateException(); // impossible
     }
 
-    private MutationState buildIndex(PTable index, TableRef dataTableRef) throws SQLException {
+    public MutationState buildIndex(PTable index, TableRef dataTableRef, Long txnScn) throws SQLException {
         AlterIndexStatement indexStatement = null;
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -1138,6 +1138,9 @@ public class MetaDataClient {
             Scan scan = mutationPlan.getContext().getScan();
             Long scn = connection.getSCN();
             try {
+                if (txnScn!=null) {
+                    scan.setAttribute(BaseScannerRegionObserver.TX_SCN, Bytes.toBytes(Long.valueOf(txnScn)));
+                }
                 if (ScanUtil.isDefaultTimeRange(scan.getTimeRange())) {
                     if (scn == null) {
                         scn = mutationPlan.getContext().getCurrentTime();
@@ -1456,7 +1459,7 @@ public class MetaDataClient {
         if (connection.getSCN() != null) {
             return buildIndexAtTimeStamp(table, statement.getTable());
         }
-        return buildIndex(table, tableRef);
+        return buildIndex(table, tableRef, null);
     }
 
     public MutationState dropSequence(DropSequenceStatement statement) throws SQLException {
@@ -3547,7 +3550,7 @@ public class MetaDataClient {
                     return buildIndexAtTimeStamp(index, dataTableNode);
                 }
                 TableRef dataTableRef = FromCompiler.getResolver(dataTableNode, connection).getTables().get(0);
-                return buildIndex(index, dataTableRef);
+                return buildIndex(index, dataTableRef, null);
             }
             return new MutationState(1, connection);
         } catch (TableNotFoundException e) {