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

stoty pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/master by this push:
     new 2817dee  PHOENIX-3710 Cannot use lowername data table name with 
indextool
2817dee is described below

commit 2817dee00ebb282a8d6985dd742b8f335718c6a9
Author: Istvan Toth <st...@apache.org>
AuthorDate: Wed Jan 6 13:24:19 2021 +0100

    PHOENIX-3710 Cannot use lowername data table name with indextool
    
    Co-authored-by: Ankit Singhal <ankitsingha...@gmail.com>
    Co-authored-by: Josh Elser <els...@apache.org>
---
 .../org/apache/phoenix/end2end/IndexToolIT.java    | 145 +++++++++++++++++++--
 .../apache/phoenix/mapreduce/index/IndexTool.java  |  58 +++++----
 .../phoenix/mapreduce/index/IndexToolUtil.java     |   2 +-
 .../index/PhoenixIndexImportDirectReducer.java     |   2 +-
 .../java/org/apache/phoenix/util/SchemaUtil.java   |  27 +++-
 5 files changed, 190 insertions(+), 44 deletions(-)

diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
index 8a1baea..bfee87f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
@@ -51,11 +51,13 @@ import 
org.apache.phoenix.mapreduce.index.PhoenixServerBuildIndexMapper;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.transaction.PhoenixTransactionProvider.Feature;
 import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.PhoenixRuntime;
 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.TestUtil;
@@ -64,6 +66,8 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -97,6 +101,8 @@ import static org.junit.Assert.fail;
 
 @RunWith(Parameterized.class)
 public class IndexToolIT extends BaseUniqueNamesOwnClusterIT {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(IndexToolIT.class);
+
     private final boolean localIndex;
     private final boolean mutable;
     private final boolean transactional;
@@ -647,13 +653,129 @@ public class IndexToolIT extends 
BaseUniqueNamesOwnClusterIT {
         }
     }
 
+    @Test
+    public void testCaseSensitiveNames() throws Exception {
+        // We have three references to the "same" table name. e.g. for 
`my_table`
+        // The original name, unquoted: `my_table`
+        // The quoted name to prevent Phoenix from auto-capitalizing it (s): 
`"my_table"`
+        // The qualified named with schema (q): `"my_schema":"my_table"`
+        String schemaName = generateUniqueName().toLowerCase();
+        String dataTableName = generateUniqueName().toLowerCase();
+        String sSchemaName = 
SchemaUtil.getCaseSensitiveColumnDisplayName(null, schemaName);
+        String sDataTableName = 
SchemaUtil.getCaseSensitiveColumnDisplayName(null, dataTableName);
+        String qDataTableName = 
SchemaUtil.getCaseSensitiveColumnDisplayName(schemaName, dataTableName);
+        // The explain plan doesn't re-quote the qualified table name: 
`my_schema.my_table`
+        String dataTableNameForExplain = SchemaUtil.getTableName(schemaName, 
dataTableName);
+
+        String indexTableName = generateUniqueName().toLowerCase();
+        String sIndexTableName = 
SchemaUtil.getCaseSensitiveColumnDisplayName(null, indexTableName);
+        String indexTableNameForExplain = SchemaUtil.getTableName(schemaName, 
indexTableName);
+
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            String stmString1 =
+                    "CREATE TABLE " + qDataTableName
+                         + " (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, 
ZIP INTEGER) "
+                         + tableDDLOptions;
+            conn.createStatement().execute(stmString1);
+            String upsertQuery = String.format("UPSERT INTO %s VALUES(?, ?, 
?)", qDataTableName);
+            PreparedStatement stmt1 = conn.prepareStatement(upsertQuery);
+
+            // insert two rows
+            upsertRow(stmt1, 1);
+            upsertRow(stmt1, 2);
+            conn.commit();
+
+            if (transactional) {
+                // insert two rows in another connection without committing so 
that they are not
+                // visible to other transactions
+                try (Connection conn2 = DriverManager.getConnection(getUrl(), 
props)) {
+                    conn2.setAutoCommit(false);
+                    PreparedStatement stmt2 = 
conn2.prepareStatement(upsertQuery);
+                    upsertRow(stmt2, 5);
+                    upsertRow(stmt2, 6);
+                    ResultSet rs =
+                            conn.createStatement()
+                                    .executeQuery("SELECT count(*) from " + 
qDataTableName);
+                    assertTrue(rs.next());
+                    assertEquals("Unexpected row count ", 2, rs.getInt(1));
+                    assertFalse(rs.next());
+                    rs =
+                            conn2.createStatement()
+                                    .executeQuery("SELECT count(*) from " + 
qDataTableName);
+                    assertTrue(rs.next());
+                    assertEquals("Unexpected row count ", 4, rs.getInt(1));
+                    assertFalse(rs.next());
+                }
+            }
+
+            String stmtString2 =
+                    String.format(
+                        "CREATE %s INDEX %s ON %s  (LPAD(UPPER(NAME, 
'en_US'),8,'x')||'_xyz') ASYNC ",
+                        (localIndex ? "LOCAL" : ""), sIndexTableName, 
qDataTableName);
+            conn.createStatement().execute(stmtString2);
+
+            // verify rows are fetched from data table.
+            String selectSql =
+                    String.format(
+                        "SELECT ID FROM %s WHERE LPAD(UPPER(NAME, 
'en_US'),8,'x')||'_xyz' = 'xxUNAME2_xyz'",
+                        qDataTableName);
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + 
selectSql);
+            String actualExplainPlan = QueryUtil.getExplainPlan(rs);
+
+            // assert we are pulling from data table.
+            assertEquals(String.format(
+                "CLIENT PARALLEL 1-WAY FULL SCAN OVER %s\n"
+                     + "    SERVER FILTER BY (LPAD(UPPER(NAME, 'en_US'), 8, 
'x') || '_xyz') = 'xxUNAME2_xyz'",
+                        dataTableNameForExplain), actualExplainPlan);
+
+            rs = stmt1.executeQuery(selectSql);
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            assertFalse(rs.next());
+            conn.commit();
+
+            // run the index MR job.
+            runIndexTool(directApi, useSnapshot, sSchemaName, sDataTableName, 
sIndexTableName);
+
+            // insert two more rows
+            upsertRow(stmt1, 3);
+            upsertRow(stmt1, 4);
+            conn.commit();
+
+            // assert we are pulling from index table.
+            rs = conn.createStatement().executeQuery("EXPLAIN " + selectSql);
+            actualExplainPlan = QueryUtil.getExplainPlan(rs);
+            // Because the explain plan doesn't include double-quotes around 
case-sensitive table names,
+            // we need to tell assertExplainPlan to not normalize our table 
names.
+            assertExplainPlan(localIndex, actualExplainPlan, 
dataTableNameForExplain, indexTableNameForExplain, false);
+
+            rs = conn.createStatement().executeQuery(selectSql);
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
     public static void assertExplainPlan(boolean localIndex, String 
actualExplainPlan,
             String dataTableFullName, String indexTableFullName) {
+        assertExplainPlan(localIndex, actualExplainPlan, dataTableFullName, 
indexTableFullName, true);
+    }
+
+    public static void assertExplainPlan(boolean localIndex, String 
actualExplainPlan,
+            String dataTableFullName, String indexTableFullName, boolean 
normalizeTableNames) {
         String expectedExplainPlan;
         if (localIndex) {
-            expectedExplainPlan = String.format(" RANGE SCAN OVER %s [1,", 
dataTableFullName);
+            expectedExplainPlan = String.format(" RANGE SCAN OVER %s [1,",
+                normalizeTableNames ? 
SchemaUtil.normalizeIdentifier(dataTableFullName)
+                        : dataTableFullName);
         } else {
-            expectedExplainPlan = String.format(" RANGE SCAN OVER %s", 
indexTableFullName);
+            expectedExplainPlan = String.format(" RANGE SCAN OVER %s",
+                normalizeTableNames ? 
SchemaUtil.normalizeIdentifier(indexTableFullName)
+                        : indexTableFullName);
         }
         assertTrue(actualExplainPlan + "\n expected to contain \n" + 
expectedExplainPlan,
             actualExplainPlan.contains(expectedExplainPlan));
@@ -674,13 +796,11 @@ public class IndexToolIT extends 
BaseUniqueNamesOwnClusterIT {
                                             Long incrementalVerify) {
         List<String> args = Lists.newArrayList();
         if (schemaName != null) {
-            args.add("-s");
-            args.add(schemaName);
+            args.add("--schema=" + schemaName);
         }
-        args.add("-dt");
-        args.add(dataTable);
-        args.add("-it");
-        args.add(indxTable);
+        // Work around CLI-254. The long-form arg parsing doesn't strip off 
double-quotes
+        args.add("--data-table=" + dataTable);
+        args.add("--index-table=" + indxTable);
         if (directApi) {
             args.add("-direct");
         }
@@ -773,13 +893,14 @@ public class IndexToolIT extends 
BaseUniqueNamesOwnClusterIT {
         if (tenantId != null) {
             props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
         }
+
         try (Connection conn =
                      DriverManager.getConnection(getUrl(), props)) {
-            PTable dataTable = PhoenixRuntime.getTable(conn, 
SchemaUtil.getTableName(schemaName, dataTableName));
-            PTable indexTable = PhoenixRuntime.getTable(conn, 
SchemaUtil.getTableName(schemaName, indexTableName));
+            PTable indexTable = PhoenixRuntime.getTableNoCache(conn,
+                   
SchemaUtil.normalizeFullTableName(SchemaUtil.getTableName(schemaName, 
indexTableName)));
+            PTable dataTable = PhoenixRuntime.getTableNoCache(conn, 
SchemaUtil.normalizeFullTableName(SchemaUtil.getTableName(schemaName, 
dataTableName)));
             boolean transactional = dataTable.isTransactional();
             boolean localIndex = 
PTable.IndexType.LOCAL.equals(indexTable.getIndexType());
-
             if ((localIndex || !transactional) && !useSnapshot) {
                 assertEquals(job.getMapperClass(), 
PhoenixServerBuildIndexMapper.class);
             } else {
@@ -821,6 +942,8 @@ public class IndexToolIT extends 
BaseUniqueNamesOwnClusterIT {
                 indexTableName, tenantId, verifyType, disableLoggingType);
         List<String> cmdArgList = new ArrayList<>(Arrays.asList(cmdArgs));
         cmdArgList.addAll(Arrays.asList(additionalArgs));
+        LOGGER.info("Running IndexTool with {}", 
Arrays.toString(cmdArgList.toArray()),
+            new Exception("Informational exception to show the caller stack 
trace"));
         int status = indexingTool.run(cmdArgList.toArray(new 
String[cmdArgList.size()]));
 
         if (expectedStatus == 0) {
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java 
b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
index d9fc747..e6fd2a7 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
@@ -194,14 +194,21 @@ public class IndexTool extends Configured implements Tool 
{
 
     private static final Logger LOGGER = 
LoggerFactory.getLogger(IndexTool.class);
 
+    //The raw identifiers as passed in, with the escaping used in SQL
+    //(double quotes for case sensitivity)
     private String schemaName;
     private String dataTable;
     private String indexTable;
+    private String dataTableWithSchema;
+    private String indexTableWithSchema;
+
     private boolean isPartialBuild, isForeground;
     private IndexVerifyType indexVerifyType = IndexVerifyType.NONE;
     private IndexDisableLoggingType disableLoggingType = 
IndexDisableLoggingType.NONE;
-    private String qDataTable;
-    private String qIndexTable;
+    //The qualified normalized table names (no double quotes, case same as 
HBase table)
+    private String qDataTable; //normalized with schema
+    private String qIndexTable; //normalized with schema
+    private String qSchemaName;
     private boolean useSnapshot;
     private boolean isLocalIndexBuild = false;
     private boolean shouldDeleteBeforeRebuild;
@@ -560,7 +567,7 @@ public class IndexTool extends Configured implements Tool {
             String physicalTableName=pDataTable.getPhysicalName().getString();
             final String jobName = String.format("Phoenix Indexes build for " 
+ pDataTable.getName().toString());
             
-            PhoenixConfigurationUtil.setInputTableName(configuration, 
qDataTable);
+            PhoenixConfigurationUtil.setInputTableName(configuration, 
dataTableWithSchema);
             PhoenixConfigurationUtil.setPhysicalTableName(configuration, 
physicalTableName);
             
             //TODO: update disable indexes
@@ -614,7 +621,7 @@ public class IndexTool extends Configured implements Tool {
             final List<String> indexColumns = 
ddlCompiler.getIndexColumnNames();
             final String selectQuery = ddlCompiler.getSelectQuery();
             final String upsertQuery =
-                    QueryUtil.constructUpsertStatement(qIndexTable, 
indexColumns, Hint.NO_INDEX);
+                    QueryUtil.constructUpsertStatement(indexTableWithSchema, 
indexColumns, Hint.NO_INDEX);
 
             configuration.set(PhoenixConfigurationUtil.UPSERT_STATEMENT, 
upsertQuery);
             PhoenixConfigurationUtil.setPhysicalTableName(configuration, 
physicalIndexTable);
@@ -626,7 +633,7 @@ public class IndexTool extends Configured implements Tool {
                 PhoenixConfigurationUtil.setTenantId(configuration, tenantId);
             }
             final List<ColumnInfo> columnMetadataList =
-                    PhoenixRuntime.generateColumnInfo(connection, qIndexTable, 
indexColumns);
+                    PhoenixRuntime.generateColumnInfo(connection, 
indexTableWithSchema, indexColumns);
             ColumnInfoToStringEncoderDecoder.encode(configuration, 
columnMetadataList);
 
             if (outputPath != null) {
@@ -642,7 +649,7 @@ public class IndexTool extends Configured implements Tool {
             }
 
             if (!useSnapshot) {
-                PhoenixMapReduceUtil.setInput(job, 
PhoenixIndexDBWritable.class, qDataTable, selectQuery);
+                PhoenixMapReduceUtil.setInput(job, 
PhoenixIndexDBWritable.class, dataTableWithSchema, selectQuery);
             } else {
                 Admin admin = null;
                 String snapshotName;
@@ -663,7 +670,7 @@ public class IndexTool extends Configured implements Tool {
 
                 // set input for map reduce job using hbase snapshots
                 PhoenixMapReduceUtil
-                            .setInput(job, PhoenixIndexDBWritable.class, 
snapshotName, qDataTable, restoreDir, selectQuery);
+                            .setInput(job, PhoenixIndexDBWritable.class, 
snapshotName, dataTableWithSchema, restoreDir, selectQuery);
             }
             TableMapReduceUtil.initCredentials(job);
             
@@ -695,7 +702,7 @@ public class IndexTool extends Configured implements Tool {
                     Long.toString(indexRebuildRpcRetriesCounter));
             configuration.set("mapreduce.task.timeout", 
Long.toString(indexRebuildQueryTimeoutMs));
 
-            PhoenixConfigurationUtil.setIndexToolDataTableName(configuration, 
qDataTable);
+            PhoenixConfigurationUtil.setIndexToolDataTableName(configuration, 
dataTableWithSchema);
             PhoenixConfigurationUtil.setIndexToolIndexTableName(configuration, 
qIndexTable);
             if (startTime != null) {
                 PhoenixConfigurationUtil.setIndexToolStartTime(configuration, 
startTime);
@@ -723,7 +730,7 @@ public class IndexTool extends Configured implements Tool {
             }
 
             PhoenixMapReduceUtil.setInput(job, 
PhoenixServerBuildIndexDBWritable.class, 
PhoenixServerBuildIndexInputFormat.class,
-                            qDataTable, "");
+                            dataTableWithSchema, "");
 
             TableMapReduceUtil.initCredentials(job);
             job.setMapperClass(PhoenixServerBuildIndexMapper.class);
@@ -872,6 +879,8 @@ public class IndexTool extends Configured implements Tool {
         dataTable = cmdLine.getOptionValue(DATA_TABLE_OPTION.getOpt());
         indexTable = cmdLine.getOptionValue(INDEX_TABLE_OPTION.getOpt());
         isPartialBuild = cmdLine.hasOption(PARTIAL_REBUILD_OPTION.getOpt());
+        dataTableWithSchema = 
SchemaUtil.getQualifiedPhoenixTableName(schemaName, dataTable);
+        indexTableWithSchema = 
SchemaUtil.getQualifiedPhoenixTableName(schemaName, indexTable);
         qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
         basePath = cmdLine.getOptionValue(OUTPUT_PATH_OPTION.getOpt());
         isForeground = cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt());
@@ -898,7 +907,7 @@ public class IndexTool extends Configured implements Tool {
             boolean isNamespaceMapped = 
SchemaUtil.isNamespaceMappingEnabled(null, cqs.getProps());
             s.setRowPrefixFilter(Bytes.toBytes(String.format("%s%s%s", 
lastVerifyTime,
                 ROW_KEY_SEPARATOR,
-                SchemaUtil.getPhysicalHBaseTableName(schemaName, indexTable,
+                SchemaUtil.getPhysicalHBaseTableName(qSchemaName, 
SchemaUtil.normalizeIdentifier(indexTable),
                     isNamespaceMapped))));
             try (ResultScanner rs = hIndexToolTable.getScanner(s)) {
                 return rs.next() != null;
@@ -926,14 +935,11 @@ public class IndexTool extends Configured implements Tool 
{
                     String.format(" %s is not an index table for %s for this 
connection",
                             indexTable, qDataTable));
         }
-        pIndexTable = PhoenixRuntime.getTable(connection, schemaName != null 
&& !schemaName.isEmpty()
-                ? SchemaUtil.getQualifiedTableName(schemaName, indexTable) : 
indexTable);
+        qSchemaName = SchemaUtil.normalizeIdentifier(schemaName);
+        pIndexTable = PhoenixRuntime.getTable(
+            connection, SchemaUtil.getQualifiedTableName(schemaName, 
indexTable));
         indexType = pIndexTable.getIndexType();
-        if (schemaName != null && !schemaName.isEmpty()) {
-            qIndexTable = SchemaUtil.getQualifiedTableName(schemaName, 
indexTable);
-        } else {
-            qIndexTable = indexTable;
-        }
+        qIndexTable = SchemaUtil.getQualifiedTableName(schemaName, indexTable);
         if (IndexType.LOCAL.equals(indexType)) {
             isLocalIndexBuild = true;
             try (org.apache.hadoop.hbase.client.Connection hConn
@@ -989,12 +995,12 @@ public class IndexTool extends Configured implements Tool 
{
         if 
(MetaDataUtil.isViewIndex(pIndexTable.getPhysicalName().getString())) {
             throw new IllegalArgumentException(String.format(
                 "%s is a view index. delete-all-and-rebuild is not supported 
for view indexes",
-                indexTable));
+                qIndexTable));
         }
 
         if (isLocalIndexBuild) {
             throw new IllegalArgumentException(String.format(
-                    "%s is a local index.  delete-all-and-rebuild is not 
supported for local indexes", indexTable));
+                    "%s is a local index.  delete-all-and-rebuild is not 
supported for local indexes", qIndexTable));
         } else {
             ConnectionQueryServices queryServices = 
conn.unwrap(PhoenixConnection.class).getQueryServices();
             try (Admin admin = queryServices.getAdmin()){
@@ -1114,7 +1120,7 @@ public class IndexTool extends Configured implements Tool 
{
             final String indexTable, final String tenantId) throws 
SQLException {
         final DatabaseMetaData dbMetaData = connection.getMetaData();
         final String schemaName = 
SchemaUtil.getSchemaNameFromFullName(masterTable);
-        final String tableName = 
SchemaUtil.normalizeIdentifier(SchemaUtil.getTableNameFromFullName(masterTable));
+        final String tableName = 
SchemaUtil.getTableNameFromFullName(masterTable);
 
         ResultSet rs = null;
         try {
@@ -1125,7 +1131,7 @@ public class IndexTool extends Configured implements Tool 
{
             rs = dbMetaData.getIndexInfo(catalog, schemaName, tableName, 
false, false);
             while (rs.next()) {
                 final String indexName = rs.getString(6);
-                if (indexTable.equalsIgnoreCase(indexName)) {
+                if 
(SchemaUtil.normalizeIdentifier(indexTable).equalsIgnoreCase(indexName)) {
                     return true;
                 }
             }
@@ -1141,13 +1147,11 @@ public class IndexTool extends Configured implements 
Tool {
              boolean useSnapshot, String tenantId, boolean disableBefore, 
boolean shouldDeleteBeforeRebuild, boolean runForeground) throws Exception {
         final List<String> args = Lists.newArrayList();
         if (schemaName != null) {
-            args.add("-s");
-            args.add(schemaName);
+            args.add("--schema=" + schemaName);
         }
-        args.add("-dt");
-        args.add(dataTable);
-        args.add("-it");
-        args.add(indexTable);
+        // Work around CLI-254. The long-form arg parsing doesn't strip off 
double-quotes
+        args.add("--data-table=" + dataTable);
+        args.add("--index-table=" + indexTable);
 
         if (runForeground) {
             args.add("-runfg");
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
index 74a19b9..294e58c 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
@@ -74,7 +74,7 @@ public class IndexToolUtil {
         Preconditions.checkNotNull(connection);
         final String alterQuery = 
String.format(ALTER_INDEX_QUERY_TEMPLATE,indexTable,masterTable,state.name());
         connection.createStatement().execute(alterQuery);
-        LOGGER.info(" Updated the status of the index {} to {} " , indexTable 
, state.name());
+        LOGGER.info(" Updated the status of the index {} on {} to {} " , new 
String[] {indexTable , masterTable, state.name()});
     }
        
 }
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectReducer.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectReducer.java
index b9a0cc6..22b1c99 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectReducer.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectReducer.java
@@ -147,7 +147,7 @@ public class PhoenixIndexImportDirectReducer extends
             try {
                 IndexToolUtil.updateIndexState(context.getConfiguration(), 
PIndexState.ACTIVE);
             } catch (SQLException e) {
-                LOGGER.error(" Failed to update the status to Active");
+                LOGGER.error(" Failed to update the status to Active", e);
                 throw new RuntimeException(e.getMessage());
             }
         }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index cc506f9..fd35bea 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -199,7 +199,8 @@ public class SchemaUtil {
 
     /**
      * Normalize an identifier. If name is surrounded by double quotes,
-     * it is used as-is, otherwise the name is upper caased.
+     * the double quotes are stripped and the rest is used as-is,
+     * otherwise the name is upper caased.
      * @param name the parsed identifier
      * @return the normalized identifier
      */
@@ -702,9 +703,12 @@ public class SchemaUtil {
     }
 
     public static String getSchemaNameFromFullName(String tableName) {
-        if (isExistingTableMappedToPhoenixName(tableName)) { return 
StringUtil.EMPTY_STRING; }
-        if (tableName.contains(QueryConstants.NAMESPACE_SEPARATOR)) { return 
getSchemaNameFromFullName(tableName,
-                QueryConstants.NAMESPACE_SEPARATOR); }
+        if (isExistingTableMappedToPhoenixName(tableName)) {
+            return StringUtil.EMPTY_STRING;
+        }
+        if (tableName.contains(QueryConstants.NAMESPACE_SEPARATOR)) {
+            return getSchemaNameFromFullName(tableName, 
QueryConstants.NAMESPACE_SEPARATOR);
+        }
         return getSchemaNameFromFullName(tableName, 
QueryConstants.NAME_SEPARATOR);
     }
 
@@ -1163,6 +1167,21 @@ public class SchemaUtil {
     }
 
     /**
+     * Calculate the Phoenix Table name without normalization
+     *
+     * @param schemaName import schema name, can be null
+     * @param tableName import table name
+     * @return the qualified Phoenix table name, from the non normalized 
schema and table
+     */
+    public static String getQualifiedPhoenixTableName(String schemaName, 
String tableName) {
+        if (schemaName != null && !schemaName.isEmpty()) {
+            return String.format("%s.%s", schemaName, tableName);
+        } else {
+            return tableName;
+        }
+    }
+
+    /**
      * Pads the data in ptr by the required amount for fixed width data types
      */
     public static void padData(String tableName, PColumn column, 
ImmutableBytesWritable ptr) {

Reply via email to