You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@phoenix.apache.org by GitBox <gi...@apache.org> on 2021/03/22 17:03:50 UTC

[GitHub] [phoenix] gjacoby126 commented on a change in pull request #1170: PHOENIX-6247 Separating logical and physical table names

gjacoby126 commented on a change in pull request #1170:
URL: https://github.com/apache/phoenix/pull/1170#discussion_r598007293



##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
##########
@@ -0,0 +1,793 @@
+package org.apache.phoenix.end2end;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.end2end.index.SingleCellIndexIT;
+import org.apache.phoenix.hbase.index.IndexRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.PhoenixTestBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.*;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.*;
+
+import static java.util.Arrays.asList;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.printResultSet;
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class LogicalTableNameIT extends ParallelStatsDisabledIT  {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LogicalTableNameIT.class);
+
+    private final boolean createChildAfterTransform;
+    private final boolean immutable;
+    private String dataTableDdl;
+    public static final String NEW_TABLE_PREFIX = "NEW_TBL_";
+    private Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newConcurrentMap();
+        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.TRUE.toString());
+        props.put(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3000));
+        //When we run all tests together we are using global cluster(driver)
+        //so to make drop work we need to re register driver with DROP_METADATA_ATTRIB property
+        destroyDriver();

Review comment:
       Rather than doing this manually, should this be a NeedsOwnCluster test?

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java
##########
@@ -443,7 +445,52 @@ public void testImportOneIndexTable(String tableName, boolean localIndex) throws
             checkIndexTableIsVerified(indexTableName);
         }
     }
-    
+
+    @Test
+    public void testImportWithDifferentPhysicalName() throws Exception {
+        String tableName = generateUniqueName();
+        String indexTableName = String.format("%s_IDX", tableName);
+        Statement stmt = conn.createStatement();
+        stmt.execute("CREATE TABLE " + tableName + "(ID INTEGER NOT NULL PRIMARY KEY, "
+                + "FIRST_NAME VARCHAR, LAST_NAME VARCHAR)");
+        String ddl = "CREATE  INDEX " + indexTableName + " ON " + tableName + "(FIRST_NAME ASC)";
+        stmt.execute(ddl);
+        String newTableName = LogicalTableNameIT.NEW_TABLE_PREFIX + generateUniqueName();
+        try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+            String snapshotName = new StringBuilder(tableName).append("-Snapshot").toString();
+            admin.snapshot(snapshotName, TableName.valueOf(tableName));
+            admin.cloneSnapshot(Bytes.toBytes(snapshotName), Bytes.toBytes(newTableName));
+        }
+        LogicalTableNameIT.renameAndDropPhysicalTable(conn, null, null, tableName, newTableName);
+
+        FileSystem fs = FileSystem.get(getUtility().getConfiguration());
+        FSDataOutputStream outputStream = fs.create(new Path("/tmp/input4.csv"));
+        PrintWriter printWriter = new PrintWriter(outputStream);
+        printWriter.println("1,FirstName 1,LastName 1");
+        printWriter.println("2,FirstName 2,LastName 2");
+        printWriter.close();
+
+        CsvBulkLoadTool csvBulkLoadTool = new CsvBulkLoadTool();
+        csvBulkLoadTool.setConf(getUtility().getConfiguration());
+        int
+                exitCode =
+                csvBulkLoadTool
+                        .run(new String[] { "--input", "/tmp/input4.csv", "--table", tableName,
+                                "--index-table", indexTableName, "--zookeeper", zkQuorum });
+        assertEquals(0, exitCode);
+
+        ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
+        assertFalse(rs.next());

Review comment:
       I don't understand this check -- how can SELECT * FROM FOO return no rows but SELECT * FROM FOO WHERE <some predicate> return a row? 

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
##########
@@ -0,0 +1,793 @@
+package org.apache.phoenix.end2end;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.end2end.index.SingleCellIndexIT;
+import org.apache.phoenix.hbase.index.IndexRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.PhoenixTestBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.*;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.*;
+
+import static java.util.Arrays.asList;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.printResultSet;
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class LogicalTableNameIT extends ParallelStatsDisabledIT  {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LogicalTableNameIT.class);
+
+    private final boolean createChildAfterTransform;
+    private final boolean immutable;
+    private String dataTableDdl;
+    public static final String NEW_TABLE_PREFIX = "NEW_TBL_";
+    private Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newConcurrentMap();
+        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.TRUE.toString());
+        props.put(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3000));
+        //When we run all tests together we are using global cluster(driver)
+        //so to make drop work we need to re register driver with DROP_METADATA_ATTRIB property
+        destroyDriver();
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        //Registering real Phoenix driver to have multiple ConnectionQueryServices created across connections
+        //so that metadata changes doesn't get propagated across connections
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    }
+
+    public LogicalTableNameIT(boolean createChildAfterTransform, boolean immutable)  {
+        this.createChildAfterTransform = createChildAfterTransform;
+        this.immutable = immutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (immutable) {
+            optionBuilder.append(" ,IMMUTABLE_STORAGE_SCHEME=ONE_CELL_PER_COLUMN, IMMUTABLE_ROWS=true");
+        }
+        this.dataTableDdl = optionBuilder.toString();
+    }
+
+    @Parameterized.Parameters(
+            name = "createChildAfterTransform={0}, immutable={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayListWithExpectedSize(2);
+        boolean[] Booleans = new boolean[] { false, true };
+        for (boolean immutable : Booleans) {
+            for (boolean createAfter : Booleans) {
+                list.add(new Object[] { createAfter, immutable });
+            }
+        }
+
+        return list;
+    }
+
+    private Connection getConnection(Properties props) throws Exception {
+        props.setProperty(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        // Force real driver to be used as the test one doesn't handle creating
+        // more than one ConnectionQueryService
+        props.setProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, StringUtil.EMPTY_STRING);
+        // Create new ConnectionQueryServices so that we can set DROP_METADATA_ATTRIB
+        String url = QueryUtil.getConnectionUrl(props, config, "PRINCIPAL");
+        return DriverManager.getConnection(url, props);
+    }
+
+    private  HashMap<String, ArrayList<String>> testBaseTableWithIndex_BaseTableChange(Connection conn, Connection conn2, String schemaName, String tableName, String indexName) throws Exception {
+        conn.setAutoCommit(true);
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        createTable(conn, fullTableName);
+        if (!createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+        HashMap<String, ArrayList<String>> expected = populateTable(conn, fullTableName, 1, 2);
+
+        // Create another hbase table and add 1 more row
+        String newTableName =  NEW_TABLE_PREFIX + tableName;
+        String fullNewTableName = SchemaUtil.getTableName(schemaName, newTableName);
+        try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+            String snapshotName = new StringBuilder(fullTableName).append("-Snapshot").toString();
+            admin.snapshot(snapshotName, TableName.valueOf(fullTableName));
+            admin.cloneSnapshot(Bytes.toBytes(snapshotName), Bytes.toBytes(fullNewTableName));
+
+            try (HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(fullNewTableName))) {
+                Put put = new Put(ByteUtil.concat(Bytes.toBytes("PK3")));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES,
+                        QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V1"), Bytes.toBytes("V13"));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V2"),
+                        PInteger.INSTANCE.toBytes(3));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V3"),
+                        PInteger.INSTANCE.toBytes(4));
+                htable.put(put);
+                expected.put("PK3", Lists.newArrayList("PK3", "V13", "3", "4"));
+            }
+        }
+
+        // Query to cache on the second connection
+        String selectTable1 = "SELECT PK1, V1, V2, V3 FROM " + fullTableName + " ORDER BY PK1 DESC";
+        ResultSet rs1 = conn2.createStatement().executeQuery(selectTable1);
+        assertTrue(rs1.next());
+
+        // Rename table to point to the new hbase table
+        renameAndDropPhysicalTable(conn, "NULL", schemaName, tableName, newTableName);
+
+        if (createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+
+        SingleCellIndexIT.dumpTable(fullNewTableName);
+        return expected;
+    }
+
+    @Test
+    public void testUpdatePhysicalTableNameWithIndex() throws Exception {
+        String schemaName = "S_" + generateUniqueName();
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        String fullIndexName = SchemaUtil.getTableName(schemaName, indexName);
+
+        try (Connection conn = getConnection(props)) {
+            try (Connection conn2 = getConnection(props)) {
+                HashMap<String, ArrayList<String>> expected = testBaseTableWithIndex_BaseTableChange(conn, conn2, schemaName, tableName, indexName);
+
+                // We have to rebuild index for this to work
+                IndexToolIT.runIndexTool(true, false, schemaName, tableName, indexName);
+
+                validateTable(conn, fullTableName);
+                validateTable(conn2, fullTableName);
+                validateIndex(conn, fullIndexName, false, expected);
+                validateIndex(conn2, fullIndexName, false, expected);
+
+                // Add row and check
+                populateTable(conn, fullTableName, 10, 1);
+                ResultSet rs = conn2.createStatement().executeQuery("SELECT * FROM " + fullIndexName + " WHERE \":PK1\"='PK10'");
+                assertEquals(true, rs.next());
+                rs = conn.createStatement().executeQuery("SELECT * FROM " + fullTableName  + " WHERE PK1='PK10'");
+                assertEquals(true, rs.next());
+
+                SingleCellIndexIT.dumpTable(SchemaUtil.getTableName(schemaName, NEW_TABLE_PREFIX+tableName));

Review comment:
       do we need the dumpTable when we're not actively debugging this?

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
##########
@@ -1209,7 +1231,26 @@ private PTable getTable(RegionScanner scanner, long clientTimeStamp, long tableT
                 if (linkType == LinkType.INDEX_TABLE) {
                     addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes, clientVersion);
                 } else if (linkType == LinkType.PHYSICAL_TABLE) {
-                    physicalTables.add(famName);
+                    // famName contains the logical name of the parent table. We need to get the actual physical name of the table
+                    PTable parentTable = getTable(null, schemaName.getBytes(), famName.getBytes(), clientTimeStamp, clientVersion);
+                    if (parentTable == null && indexType != IndexType.LOCAL) {
+                        // parentTable is not in the cache. Since famName is only logical name, we need to find the physical table.
+                        try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)) {
+                            parentTable = PhoenixRuntime.getTableNoCache(connection, famName.getString());
+                        } catch (TableNotFoundException e) {
+

Review comment:
       Please include a comment why it's OK to swallow the exception here. (I assume because of the next if clause?)

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
##########
@@ -0,0 +1,793 @@
+package org.apache.phoenix.end2end;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;

Review comment:
       Good to use the shaded guava (required for 5.x)

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java
##########
@@ -343,18 +343,17 @@ private int getTableTtl() throws SQLException, IOException {
 
     @VisibleForTesting
     public static String getSourceTableName(PTable pSourceTable, boolean isNamespaceEnabled) {
-        String sourcePhysicalName = pSourceTable.getPhysicalName().getString();
+        String sourcePhysicalName = SchemaUtil.getTableNameFromFullName(pSourceTable.getPhysicalName().getString());
         String physicalTable, table, schema;
         if (pSourceTable.getType() == PTableType.VIEW
-                || MetaDataUtil.isViewIndex(sourcePhysicalName)) {
+                || MetaDataUtil.isViewIndex(pSourceTable.getPhysicalName().getString())) {
             // in case of view and view index ptable, getPhysicalName() returns hbase tables
             // i.e. without _IDX_ and with _IDX_ respectively
-            physicalTable = sourcePhysicalName;
+            physicalTable = pSourceTable.getPhysicalName().getString();

Review comment:
       nit: can pull physicalTable up to 346 and refer to it rather than using pSourceTable.getPhysicalName().getString() several times. 

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
##########
@@ -2985,7 +2989,8 @@ private void setSyncedPropertiesForTableIndexes(PTable table,
             tableAndIndexDescriptorMappings.put(origIndexDescriptor, newIndexDescriptor);
         }
         // Also keep properties for the physical view index table in sync
-        String viewIndexName = MetaDataUtil.getViewIndexPhysicalName(table.getPhysicalName().getString());
+        //String viewIndexName = MetaDataUtil.getViewIndexPhysicalName(table.getPhysicalName().getString());

Review comment:
       nit: remove commented line

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
##########
@@ -1583,10 +1622,15 @@ public synchronized boolean getIndexMaintainers(ImmutableBytesWritable ptr, Phoe
         ptr.set(indexMaintainersPtr.get(), indexMaintainersPtr.getOffset(), indexMaintainersPtr.getLength());
         return indexMaintainersPtr.getLength() > 0;
     }
-
+    private static final Logger LOGGER = LoggerFactory.getLogger(PTableImpl.class);
     @Override
     public PName getPhysicalName() {
+        // For views, physicalName is base table name. There might be a case where the Phoenix table is pointing to another physical table.
+        // In that case, pysicalTableName is not null

Review comment:
       spelling: physicalTableName

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
##########
@@ -0,0 +1,793 @@
+package org.apache.phoenix.end2end;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.end2end.index.SingleCellIndexIT;
+import org.apache.phoenix.hbase.index.IndexRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.PhoenixTestBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.*;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.*;
+
+import static java.util.Arrays.asList;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.printResultSet;
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class LogicalTableNameIT extends ParallelStatsDisabledIT  {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LogicalTableNameIT.class);
+
+    private final boolean createChildAfterTransform;
+    private final boolean immutable;
+    private String dataTableDdl;
+    public static final String NEW_TABLE_PREFIX = "NEW_TBL_";
+    private Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newConcurrentMap();
+        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.TRUE.toString());
+        props.put(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3000));
+        //When we run all tests together we are using global cluster(driver)
+        //so to make drop work we need to re register driver with DROP_METADATA_ATTRIB property
+        destroyDriver();
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        //Registering real Phoenix driver to have multiple ConnectionQueryServices created across connections
+        //so that metadata changes doesn't get propagated across connections
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    }
+
+    public LogicalTableNameIT(boolean createChildAfterTransform, boolean immutable)  {
+        this.createChildAfterTransform = createChildAfterTransform;
+        this.immutable = immutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (immutable) {
+            optionBuilder.append(" ,IMMUTABLE_STORAGE_SCHEME=ONE_CELL_PER_COLUMN, IMMUTABLE_ROWS=true");
+        }
+        this.dataTableDdl = optionBuilder.toString();
+    }
+
+    @Parameterized.Parameters(
+            name = "createChildAfterTransform={0}, immutable={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayListWithExpectedSize(2);
+        boolean[] Booleans = new boolean[] { false, true };
+        for (boolean immutable : Booleans) {
+            for (boolean createAfter : Booleans) {
+                list.add(new Object[] { createAfter, immutable });
+            }
+        }
+
+        return list;
+    }
+
+    private Connection getConnection(Properties props) throws Exception {
+        props.setProperty(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        // Force real driver to be used as the test one doesn't handle creating
+        // more than one ConnectionQueryService
+        props.setProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, StringUtil.EMPTY_STRING);
+        // Create new ConnectionQueryServices so that we can set DROP_METADATA_ATTRIB
+        String url = QueryUtil.getConnectionUrl(props, config, "PRINCIPAL");
+        return DriverManager.getConnection(url, props);
+    }
+
+    private  HashMap<String, ArrayList<String>> testBaseTableWithIndex_BaseTableChange(Connection conn, Connection conn2, String schemaName, String tableName, String indexName) throws Exception {
+        conn.setAutoCommit(true);
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        createTable(conn, fullTableName);
+        if (!createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+        HashMap<String, ArrayList<String>> expected = populateTable(conn, fullTableName, 1, 2);
+
+        // Create another hbase table and add 1 more row
+        String newTableName =  NEW_TABLE_PREFIX + tableName;
+        String fullNewTableName = SchemaUtil.getTableName(schemaName, newTableName);
+        try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+            String snapshotName = new StringBuilder(fullTableName).append("-Snapshot").toString();
+            admin.snapshot(snapshotName, TableName.valueOf(fullTableName));
+            admin.cloneSnapshot(Bytes.toBytes(snapshotName), Bytes.toBytes(fullNewTableName));
+
+            try (HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(fullNewTableName))) {
+                Put put = new Put(ByteUtil.concat(Bytes.toBytes("PK3")));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES,
+                        QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V1"), Bytes.toBytes("V13"));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V2"),
+                        PInteger.INSTANCE.toBytes(3));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V3"),
+                        PInteger.INSTANCE.toBytes(4));
+                htable.put(put);
+                expected.put("PK3", Lists.newArrayList("PK3", "V13", "3", "4"));
+            }
+        }
+
+        // Query to cache on the second connection
+        String selectTable1 = "SELECT PK1, V1, V2, V3 FROM " + fullTableName + " ORDER BY PK1 DESC";
+        ResultSet rs1 = conn2.createStatement().executeQuery(selectTable1);
+        assertTrue(rs1.next());
+
+        // Rename table to point to the new hbase table
+        renameAndDropPhysicalTable(conn, "NULL", schemaName, tableName, newTableName);
+
+        if (createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+
+        SingleCellIndexIT.dumpTable(fullNewTableName);
+        return expected;
+    }
+
+    @Test
+    public void testUpdatePhysicalTableNameWithIndex() throws Exception {
+        String schemaName = "S_" + generateUniqueName();
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        String fullIndexName = SchemaUtil.getTableName(schemaName, indexName);
+
+        try (Connection conn = getConnection(props)) {
+            try (Connection conn2 = getConnection(props)) {
+                HashMap<String, ArrayList<String>> expected = testBaseTableWithIndex_BaseTableChange(conn, conn2, schemaName, tableName, indexName);
+
+                // We have to rebuild index for this to work
+                IndexToolIT.runIndexTool(true, false, schemaName, tableName, indexName);
+
+                validateTable(conn, fullTableName);
+                validateTable(conn2, fullTableName);
+                validateIndex(conn, fullIndexName, false, expected);
+                validateIndex(conn2, fullIndexName, false, expected);
+
+                // Add row and check
+                populateTable(conn, fullTableName, 10, 1);
+                ResultSet rs = conn2.createStatement().executeQuery("SELECT * FROM " + fullIndexName + " WHERE \":PK1\"='PK10'");
+                assertEquals(true, rs.next());
+                rs = conn.createStatement().executeQuery("SELECT * FROM " + fullTableName  + " WHERE PK1='PK10'");
+                assertEquals(true, rs.next());

Review comment:
       Ditto

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
##########
@@ -21,18 +21,24 @@
 
 import java.sql.SQLException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 
+import com.google.common.base.Strings;

Review comment:
       nit: use shaded version

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
##########
@@ -44,6 +44,7 @@
 
 import javax.annotation.Nullable;
 
+import org.apache.hadoop.hbase.util.Bytes;

Review comment:
       nit: unnecessary import?

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
##########
@@ -0,0 +1,793 @@
+package org.apache.phoenix.end2end;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.end2end.index.SingleCellIndexIT;
+import org.apache.phoenix.hbase.index.IndexRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.PhoenixTestBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.*;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.*;
+
+import static java.util.Arrays.asList;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.printResultSet;
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class LogicalTableNameIT extends ParallelStatsDisabledIT  {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LogicalTableNameIT.class);
+
+    private final boolean createChildAfterTransform;
+    private final boolean immutable;
+    private String dataTableDdl;
+    public static final String NEW_TABLE_PREFIX = "NEW_TBL_";
+    private Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newConcurrentMap();
+        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.TRUE.toString());
+        props.put(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3000));
+        //When we run all tests together we are using global cluster(driver)
+        //so to make drop work we need to re register driver with DROP_METADATA_ATTRIB property
+        destroyDriver();
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        //Registering real Phoenix driver to have multiple ConnectionQueryServices created across connections
+        //so that metadata changes doesn't get propagated across connections
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    }
+
+    public LogicalTableNameIT(boolean createChildAfterTransform, boolean immutable)  {
+        this.createChildAfterTransform = createChildAfterTransform;
+        this.immutable = immutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (immutable) {
+            optionBuilder.append(" ,IMMUTABLE_STORAGE_SCHEME=ONE_CELL_PER_COLUMN, IMMUTABLE_ROWS=true");
+        }
+        this.dataTableDdl = optionBuilder.toString();
+    }
+
+    @Parameterized.Parameters(
+            name = "createChildAfterTransform={0}, immutable={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayListWithExpectedSize(2);
+        boolean[] Booleans = new boolean[] { false, true };
+        for (boolean immutable : Booleans) {
+            for (boolean createAfter : Booleans) {
+                list.add(new Object[] { createAfter, immutable });
+            }
+        }
+
+        return list;
+    }
+
+    private Connection getConnection(Properties props) throws Exception {
+        props.setProperty(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        // Force real driver to be used as the test one doesn't handle creating
+        // more than one ConnectionQueryService
+        props.setProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, StringUtil.EMPTY_STRING);
+        // Create new ConnectionQueryServices so that we can set DROP_METADATA_ATTRIB
+        String url = QueryUtil.getConnectionUrl(props, config, "PRINCIPAL");
+        return DriverManager.getConnection(url, props);
+    }
+
+    private  HashMap<String, ArrayList<String>> testBaseTableWithIndex_BaseTableChange(Connection conn, Connection conn2, String schemaName, String tableName, String indexName) throws Exception {
+        conn.setAutoCommit(true);
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        createTable(conn, fullTableName);
+        if (!createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+        HashMap<String, ArrayList<String>> expected = populateTable(conn, fullTableName, 1, 2);
+
+        // Create another hbase table and add 1 more row
+        String newTableName =  NEW_TABLE_PREFIX + tableName;
+        String fullNewTableName = SchemaUtil.getTableName(schemaName, newTableName);
+        try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+            String snapshotName = new StringBuilder(fullTableName).append("-Snapshot").toString();
+            admin.snapshot(snapshotName, TableName.valueOf(fullTableName));
+            admin.cloneSnapshot(Bytes.toBytes(snapshotName), Bytes.toBytes(fullNewTableName));
+
+            try (HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(fullNewTableName))) {
+                Put put = new Put(ByteUtil.concat(Bytes.toBytes("PK3")));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES,
+                        QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V1"), Bytes.toBytes("V13"));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V2"),
+                        PInteger.INSTANCE.toBytes(3));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V3"),
+                        PInteger.INSTANCE.toBytes(4));

Review comment:
       if createChildAfterTransform is false, won't doing this through the HBase client API make the index inconsistent?

##########
File path: phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
##########
@@ -2152,10 +2196,10 @@ public void createTable(RpcController controller, CreateTableRequest request,
         }
     }
 
-    private long getViewIndexSequenceValue(PhoenixConnection connection, String tenantIdStr, PTable parentTable, PName physicalName) throws SQLException {
+    private long getViewIndexSequenceValue(PhoenixConnection connection, String tenantIdStr, PTable parentTable) throws SQLException {
         int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
-
-        SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
+        // parentTable is parent of the view index which is the view.
+        SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, SchemaUtil.getTableName(parentTable.getSchemaName(), parentTable.getTableName()) ,   //parentTable.getParentLogicalName(),

Review comment:
       nit: remove commented out code

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
##########
@@ -0,0 +1,793 @@
+package org.apache.phoenix.end2end;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.end2end.index.SingleCellIndexIT;
+import org.apache.phoenix.hbase.index.IndexRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.PhoenixTestBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.*;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.*;
+
+import static java.util.Arrays.asList;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.printResultSet;
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class LogicalTableNameIT extends ParallelStatsDisabledIT  {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LogicalTableNameIT.class);
+
+    private final boolean createChildAfterTransform;
+    private final boolean immutable;
+    private String dataTableDdl;
+    public static final String NEW_TABLE_PREFIX = "NEW_TBL_";
+    private Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newConcurrentMap();
+        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.TRUE.toString());
+        props.put(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3000));
+        //When we run all tests together we are using global cluster(driver)
+        //so to make drop work we need to re register driver with DROP_METADATA_ATTRIB property
+        destroyDriver();
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        //Registering real Phoenix driver to have multiple ConnectionQueryServices created across connections
+        //so that metadata changes doesn't get propagated across connections
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    }
+
+    public LogicalTableNameIT(boolean createChildAfterTransform, boolean immutable)  {
+        this.createChildAfterTransform = createChildAfterTransform;
+        this.immutable = immutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (immutable) {
+            optionBuilder.append(" ,IMMUTABLE_STORAGE_SCHEME=ONE_CELL_PER_COLUMN, IMMUTABLE_ROWS=true");
+        }
+        this.dataTableDdl = optionBuilder.toString();
+    }
+
+    @Parameterized.Parameters(
+            name = "createChildAfterTransform={0}, immutable={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayListWithExpectedSize(2);
+        boolean[] Booleans = new boolean[] { false, true };
+        for (boolean immutable : Booleans) {
+            for (boolean createAfter : Booleans) {
+                list.add(new Object[] { createAfter, immutable });
+            }
+        }
+
+        return list;
+    }
+
+    private Connection getConnection(Properties props) throws Exception {
+        props.setProperty(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        // Force real driver to be used as the test one doesn't handle creating
+        // more than one ConnectionQueryService
+        props.setProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, StringUtil.EMPTY_STRING);
+        // Create new ConnectionQueryServices so that we can set DROP_METADATA_ATTRIB
+        String url = QueryUtil.getConnectionUrl(props, config, "PRINCIPAL");
+        return DriverManager.getConnection(url, props);
+    }
+
+    private  HashMap<String, ArrayList<String>> testBaseTableWithIndex_BaseTableChange(Connection conn, Connection conn2, String schemaName, String tableName, String indexName) throws Exception {
+        conn.setAutoCommit(true);
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        createTable(conn, fullTableName);
+        if (!createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+        HashMap<String, ArrayList<String>> expected = populateTable(conn, fullTableName, 1, 2);
+
+        // Create another hbase table and add 1 more row
+        String newTableName =  NEW_TABLE_PREFIX + tableName;
+        String fullNewTableName = SchemaUtil.getTableName(schemaName, newTableName);
+        try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+            String snapshotName = new StringBuilder(fullTableName).append("-Snapshot").toString();
+            admin.snapshot(snapshotName, TableName.valueOf(fullTableName));
+            admin.cloneSnapshot(Bytes.toBytes(snapshotName), Bytes.toBytes(fullNewTableName));
+
+            try (HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(fullNewTableName))) {
+                Put put = new Put(ByteUtil.concat(Bytes.toBytes("PK3")));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES,
+                        QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V1"), Bytes.toBytes("V13"));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V2"),
+                        PInteger.INSTANCE.toBytes(3));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V3"),
+                        PInteger.INSTANCE.toBytes(4));
+                htable.put(put);
+                expected.put("PK3", Lists.newArrayList("PK3", "V13", "3", "4"));
+            }
+        }
+
+        // Query to cache on the second connection
+        String selectTable1 = "SELECT PK1, V1, V2, V3 FROM " + fullTableName + " ORDER BY PK1 DESC";
+        ResultSet rs1 = conn2.createStatement().executeQuery(selectTable1);
+        assertTrue(rs1.next());
+
+        // Rename table to point to the new hbase table
+        renameAndDropPhysicalTable(conn, "NULL", schemaName, tableName, newTableName);
+
+        if (createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+
+        SingleCellIndexIT.dumpTable(fullNewTableName);

Review comment:
       Consider using TestUtil.dumpTable. Also, do we need to output to stdout in the final version of this? Can probably be cut. 

##########
File path: phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameIT.java
##########
@@ -0,0 +1,793 @@
+package org.apache.phoenix.end2end;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.phoenix.end2end.index.SingleCellIndexIT;
+import org.apache.phoenix.hbase.index.IndexRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.mapreduce.index.IndexScrutinyTool;
+import org.apache.phoenix.query.PhoenixTestBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.*;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.*;
+
+import static java.util.Arrays.asList;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.INVALID_ROW_COUNT;
+import static org.apache.phoenix.mapreduce.index.PhoenixScrutinyJobCounters.VALID_ROW_COUNT;
+import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.printResultSet;
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class LogicalTableNameIT extends ParallelStatsDisabledIT  {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LogicalTableNameIT.class);
+
+    private final boolean createChildAfterTransform;
+    private final boolean immutable;
+    private String dataTableDdl;
+    public static final String NEW_TABLE_PREFIX = "NEW_TBL_";
+    private Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newConcurrentMap();
+        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.TRUE.toString());
+        props.put(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3000));
+        //When we run all tests together we are using global cluster(driver)
+        //so to make drop work we need to re register driver with DROP_METADATA_ATTRIB property
+        destroyDriver();
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        //Registering real Phoenix driver to have multiple ConnectionQueryServices created across connections
+        //so that metadata changes doesn't get propagated across connections
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    }
+
+    public LogicalTableNameIT(boolean createChildAfterTransform, boolean immutable)  {
+        this.createChildAfterTransform = createChildAfterTransform;
+        this.immutable = immutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (immutable) {
+            optionBuilder.append(" ,IMMUTABLE_STORAGE_SCHEME=ONE_CELL_PER_COLUMN, IMMUTABLE_ROWS=true");
+        }
+        this.dataTableDdl = optionBuilder.toString();
+    }
+
+    @Parameterized.Parameters(
+            name = "createChildAfterTransform={0}, immutable={1}")
+    public static synchronized Collection<Object[]> data() {
+        List<Object[]> list = Lists.newArrayListWithExpectedSize(2);
+        boolean[] Booleans = new boolean[] { false, true };
+        for (boolean immutable : Booleans) {
+            for (boolean createAfter : Booleans) {
+                list.add(new Object[] { createAfter, immutable });
+            }
+        }
+
+        return list;
+    }
+
+    private Connection getConnection(Properties props) throws Exception {
+        props.setProperty(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        // Force real driver to be used as the test one doesn't handle creating
+        // more than one ConnectionQueryService
+        props.setProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, StringUtil.EMPTY_STRING);
+        // Create new ConnectionQueryServices so that we can set DROP_METADATA_ATTRIB
+        String url = QueryUtil.getConnectionUrl(props, config, "PRINCIPAL");
+        return DriverManager.getConnection(url, props);
+    }
+
+    private  HashMap<String, ArrayList<String>> testBaseTableWithIndex_BaseTableChange(Connection conn, Connection conn2, String schemaName, String tableName, String indexName) throws Exception {
+        conn.setAutoCommit(true);
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        createTable(conn, fullTableName);
+        if (!createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+        HashMap<String, ArrayList<String>> expected = populateTable(conn, fullTableName, 1, 2);
+
+        // Create another hbase table and add 1 more row
+        String newTableName =  NEW_TABLE_PREFIX + tableName;
+        String fullNewTableName = SchemaUtil.getTableName(schemaName, newTableName);
+        try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+            String snapshotName = new StringBuilder(fullTableName).append("-Snapshot").toString();
+            admin.snapshot(snapshotName, TableName.valueOf(fullTableName));
+            admin.cloneSnapshot(Bytes.toBytes(snapshotName), Bytes.toBytes(fullNewTableName));
+
+            try (HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(fullNewTableName))) {
+                Put put = new Put(ByteUtil.concat(Bytes.toBytes("PK3")));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES,
+                        QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V1"), Bytes.toBytes("V13"));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V2"),
+                        PInteger.INSTANCE.toBytes(3));
+                put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("V3"),
+                        PInteger.INSTANCE.toBytes(4));
+                htable.put(put);
+                expected.put("PK3", Lists.newArrayList("PK3", "V13", "3", "4"));
+            }
+        }
+
+        // Query to cache on the second connection
+        String selectTable1 = "SELECT PK1, V1, V2, V3 FROM " + fullTableName + " ORDER BY PK1 DESC";
+        ResultSet rs1 = conn2.createStatement().executeQuery(selectTable1);
+        assertTrue(rs1.next());
+
+        // Rename table to point to the new hbase table
+        renameAndDropPhysicalTable(conn, "NULL", schemaName, tableName, newTableName);
+
+        if (createChildAfterTransform) {
+            createIndexOnTable(conn, fullTableName, indexName);
+        }
+
+        SingleCellIndexIT.dumpTable(fullNewTableName);
+        return expected;
+    }
+
+    @Test
+    public void testUpdatePhysicalTableNameWithIndex() throws Exception {
+        String schemaName = "S_" + generateUniqueName();
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        String fullIndexName = SchemaUtil.getTableName(schemaName, indexName);
+
+        try (Connection conn = getConnection(props)) {
+            try (Connection conn2 = getConnection(props)) {
+                HashMap<String, ArrayList<String>> expected = testBaseTableWithIndex_BaseTableChange(conn, conn2, schemaName, tableName, indexName);
+
+                // We have to rebuild index for this to work
+                IndexToolIT.runIndexTool(true, false, schemaName, tableName, indexName);
+
+                validateTable(conn, fullTableName);
+                validateTable(conn2, fullTableName);
+                validateIndex(conn, fullIndexName, false, expected);
+                validateIndex(conn2, fullIndexName, false, expected);
+
+                // Add row and check
+                populateTable(conn, fullTableName, 10, 1);
+                ResultSet rs = conn2.createStatement().executeQuery("SELECT * FROM " + fullIndexName + " WHERE \":PK1\"='PK10'");
+                assertEquals(true, rs.next());

Review comment:
       we're just checking if a row exists but not its value?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org