Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 0CF96200B50 for ; Fri, 29 Jul 2016 22:55:01 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 0BBB5160A79; Fri, 29 Jul 2016 20:55:01 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id D0102160A6E for ; Fri, 29 Jul 2016 22:54:59 +0200 (CEST) Received: (qmail 12301 invoked by uid 500); 29 Jul 2016 20:54:59 -0000 Mailing-List: contact commits-help@phoenix.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.apache.org Delivered-To: mailing list commits@phoenix.apache.org Received: (qmail 12291 invoked by uid 99); 29 Jul 2016 20:54:59 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 29 Jul 2016 20:54:59 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id E3335E78B5; Fri, 29 Jul 2016 20:54:58 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: tdsilva@apache.org To: commits@phoenix.apache.org Message-Id: <3f36ba1a2ebb475586d7afd815b67a38@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: phoenix git commit: PHOENIX-3120 AsyncIndexRebuilderTask fails for transactional tables Date: Fri, 29 Jul 2016 20:54:58 +0000 (UTC) archived-at: Fri, 29 Jul 2016 20:55:01 -0000 Repository: phoenix Updated Branches: refs/heads/4.x-HBase-1.0 23ac6a487 -> 47da2cf2f 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/47da2cf2 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/47da2cf2 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/47da2cf2 Branch: refs/heads/4.x-HBase-1.0 Commit: 47da2cf2f4716b6455614f62b552931f626de2d7 Parents: 23ac6a4 Author: Thomas D'Silva Authored: Thu Jul 28 18:21:45 2016 -0700 Committer: Thomas D'Silva Committed: Fri Jul 29 13:52:29 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/47da2cf2/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/47da2cf2/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 e8a35dd..522c6c7 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/47da2cf2/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) {