You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ch...@apache.org on 2019/07/24 22:38:13 UTC

[phoenix] branch 4.14-HBase-1.3 updated: PHOENIX-5302: Different isNamespaceMappingEnabled for server / client causes TableNotFoundException

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

chinmayskulkarni pushed a commit to branch 4.14-HBase-1.3
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.14-HBase-1.3 by this push:
     new 3aee060  PHOENIX-5302: Different isNamespaceMappingEnabled for server / client causes TableNotFoundException
3aee060 is described below

commit 3aee060cfed5a3182c13e19962db2f64a96eae3b
Author: Chinmay Kulkarni <ch...@gmail.com>
AuthorDate: Tue Jul 16 16:24:30 2019 -0700

    PHOENIX-5302: Different isNamespaceMappingEnabled for server / client causes TableNotFoundException
---
 .../SystemCatalogCreationOnConnectionIT.java       | 168 ++++++++-------------
 .../phoenix/query/ConnectionQueryServicesImpl.java |  70 +++++++--
 2 files changed, 121 insertions(+), 117 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index 59af533..99f1216 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -26,6 +27,7 @@ import static org.apache.phoenix.query.BaseTest.generateUniqueName;
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -39,6 +41,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -47,7 +50,11 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
-import org.apache.phoenix.query.*;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.ConnectionQueryServicesImpl;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesTestImpl;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.UpgradeUtil;
 import org.junit.After;
@@ -85,7 +92,7 @@ public class SystemCatalogCreationOnConnectionIT {
 
     private static class PhoenixSysCatCreationServices extends ConnectionQueryServicesImpl {
 
-        public PhoenixSysCatCreationServices(QueryServices services, PhoenixEmbeddedDriver.ConnectionInfo connectionInfo, Properties info) {
+        PhoenixSysCatCreationServices(QueryServices services, PhoenixEmbeddedDriver.ConnectionInfo connectionInfo, Properties info) {
             super(services, connectionInfo, info);
         }
 
@@ -119,7 +126,7 @@ public class SystemCatalogCreationOnConnectionIT {
         private ConnectionQueryServices cqs;
         private final ReadOnlyProps overrideProps;
 
-        public PhoenixSysCatCreationTestingDriver(ReadOnlyProps props) {
+        PhoenixSysCatCreationTestingDriver(ReadOnlyProps props) {
             overrideProps = props;
         }
 
@@ -136,7 +143,7 @@ public class SystemCatalogCreationOnConnectionIT {
         // used ConnectionQueryServices instance. This is used only in cases where we need to test server-side
         // changes and don't care about client-side properties set from the init method.
         // Reset the Connection Query Services instance so we can create a new connection to the cluster
-        public void resetCQS() {
+        void resetCQS() {
             cqs = null;
         }
     }
@@ -176,7 +183,7 @@ public class SystemCatalogCreationOnConnectionIT {
         driver.getConnectionQueryServices(getJdbcUrl(), propsDoNotUpgradePropSet);
         hbaseTables = getHBaseTables();
         assertFalse(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG) || hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
-        assertTrue(hbaseTables.size() == 0);
+        assertEquals(0, hbaseTables.size());
         assertEquals(1, countUpgradeAttempts);
     }
 
@@ -184,23 +191,6 @@ public class SystemCatalogCreationOnConnectionIT {
     /********************* Testing SYSTEM.CATALOG/SYSTEM:CATALOG creation/upgrade behavior for subsequent connections *********************/
 
 
-    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create unmapped
-    // SYSTEM tables i.e. SYSTEM\..*, the second connection has client-side namespace mapping enabled and
-    // system table to system namespace mapping enabled
-    // Expected: We will migrate all SYSTEM\..* tables to the SYSTEM namespace
-    @Test
-    public void testMigrateToSystemNamespace() throws Exception {
-        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
-          firstConnectionNSMappingServerEnabledClientEnabledMappingDisabled();
-        driver.resetCQS();
-        // Setting this to true to effect migration of SYSTEM tables to the SYSTEM namespace
-        Properties clientProps = getClientProperties(true, true);
-        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
-        hbaseTables = getHBaseTables();
-        assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
-        assertEquals(1, countUpgradeAttempts);
-    }
-
     // Conditions: server-side namespace mapping is enabled, the first connection to the server will create all namespace
     // mapped SYSTEM tables i.e. SYSTEM:.*, the SYSTEM:CATALOG timestamp at creation is purposefully set to be <
     // MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP. The subsequent connection has client-side namespace mapping enabled
@@ -256,22 +246,6 @@ public class SystemCatalogCreationOnConnectionIT {
         }
     }
 
-    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create only SYSTEM.CATALOG,
-    // the second connection has client-side namespace mapping enabled
-    // Expected: We will migrate SYSTEM.CATALOG to SYSTEM namespace and create all other SYSTEM:.* tables
-    @Test
-    public void testMigrateSysCatCreateOthers() throws Exception {
-        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
-          firstConnectionNSMappingServerEnabledClientDisabled();
-        driver.resetCQS();
-        Properties clientProps = getClientProperties(true, true);
-        driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
-        hbaseTables = getHBaseTables();
-        assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
-        // SYSTEM.CATALOG migration to the SYSTEM namespace is counted as an upgrade
-        assertEquals(1, countUpgradeAttempts);
-    }
-
     // Conditions: server-side namespace mapping is enabled, the first connection to the server will create unmapped SYSTEM
     // tables SYSTEM\..* whose timestamp at creation is purposefully set to be < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP.
     // The second connection has client-side namespace mapping enabled and system table to system namespace mapping enabled
@@ -311,32 +285,35 @@ public class SystemCatalogCreationOnConnectionIT {
         assertEquals(0, countUpgradeAttempts);
     }
 
-    // Conditions: server-side namespace mapping is enabled, the first connection to the server will create only SYSTEM.CATALOG,
-    // the second connection has client-side namespace mapping disabled
-    // Expected: Throw Inconsistent namespace mapping exception when you check client-server compatibility
+    // Conditions: server-side namespace mapping is enabled, the first connection to the server will not create any
+    // SYSTEM tables. The second connection has client-side namespace mapping enabled
+    // Expected: We create SYSTEM:.* tables
     @Test
-    public void testUnmappedSysCatExistsInconsistentNSMappingFails() throws Exception {
+    public void testIncompatibleNSMappingServerEnabledConnectionFails() throws Exception {
         SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
           firstConnectionNSMappingServerEnabledClientDisabled();
         driver.resetCQS();
-        Properties clientProps = getClientProperties(false, false);
-        try {
-            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
-            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
-        } catch (SQLException sqlE) {
-            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
-        }
+        // now try a client with ns mapping enabled
+        Properties clientProps = getClientProperties(true, true);
+        Connection conn = driver.getConnectionQueryServices(getJdbcUrl(), clientProps)
+                .connect(getJdbcUrl(), new Properties());
         hbaseTables = getHBaseTables();
-        assertTrue(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG));
-        assertTrue(hbaseTables.size() == 1);
+        assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
         assertEquals(0, countUpgradeAttempts);
+
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM SYSTEM.CATALOG LIMIT 1");
+        // Tests that SYSTEM:CATALOG contains necessary metadata rows for itself (See PHOENIX-5302)
+        assertTrue(rs.next());
     }
 
     // Conditions: server-side namespace mapping is disabled, the first connection to the server will create all unmapped
     // SYSTEM tables i.e. SYSTEM\..*, the second connection has client-side namespace mapping enabled
     // Expected: Throw Inconsistent namespace mapping exception when you check client-server compatibility
+    //
+    // Then another connection has client-side namespace mapping disabled
+    // Expected: All SYSTEM\..* tables exist and no upgrade is required
     @Test
-    public void testSysTablesExistInconsistentNSMappingFails() throws Exception {
+    public void testSysTablesExistNSMappingDisabled() throws Exception {
         SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
           firstConnectionNSMappingServerDisabledClientDisabled();
         driver.resetCQS();
@@ -350,63 +327,34 @@ public class SystemCatalogCreationOnConnectionIT {
         hbaseTables = getHBaseTables();
         assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
         assertEquals(0, countUpgradeAttempts);
-    }
-
-    // Conditions: server-side namespace mapping is disabled, the first connection to the server will create only SYSTEM:CATALOG
-    // and the second connection has client-side namespace mapping enabled
-    // Expected: Throw Inconsistent namespace mapping exception when you check client-server compatibility
-    @Test
-    public void testMappedSysCatExistsInconsistentNSMappingFails() throws Exception {
-        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
-          firstConnectionNSMappingServerDisabledClientEnabled();
-        driver.resetCQS();
-        Properties clientProps = getClientProperties(true, true);
-        try{
-            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
-            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
-        } catch (SQLException sqlE) {
-            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
-        }
-        hbaseTables = getHBaseTables();
-        assertTrue(hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
-        assertTrue(hbaseTables.size() == 1);
-        assertEquals(0, countUpgradeAttempts);
-    }
 
-    // Conditions: server-side namespace mapping is disabled, the first connection to the server will create all SYSTEM\..*
-    // tables and the second connection has client-side namespace mapping disabled
-    // Expected: All SYSTEM\..* tables exist and no upgrade is required
-    @Test
-    public void testNSMappingDisabledNoUpgradeRequired() throws Exception {
-        SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
-          firstConnectionNSMappingServerDisabledClientDisabled();
         driver.resetCQS();
-        Properties clientProps = getClientProperties(false, false);
+        clientProps = getClientProperties(false, false);
         driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
         hbaseTables = getHBaseTables();
         assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
         assertEquals(0, countUpgradeAttempts);
     }
 
-    // Conditions: server-side namespace mapping is disabled, the first connection to the server will create only SYSTEM:CATALOG
-    // and the second connection has client-side namespace mapping disabled
-    // Expected: The second connection should fail with Inconsistent namespace mapping exception
+    // Conditions: server-side namespace mapping is disabled, the first connection to the server will not create any
+    // SYSTEM tables. The second connection has client-side namespace mapping disabled
+    // Expected: The second connection should create all SYSTEM.* tables
     @Test
-    public void testClientNSMappingDisabledConnectionFails() throws Exception {
+    public void testIncompatibleNSMappingServerDisabledConnectionFails() throws Exception {
         SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver driver =
           firstConnectionNSMappingServerDisabledClientEnabled();
         driver.resetCQS();
+        // now try a client with ns mapping disabled
         Properties clientProps = getClientProperties(false, false);
-        try{
-            driver.getConnectionQueryServices(getJdbcUrl(), clientProps);
-            fail("Client should not be able to connect to cluster with inconsistent client-server namespace mapping properties");
-        } catch (SQLException sqlE) {
-            assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
-        }
+        Connection conn = driver.getConnectionQueryServices(getJdbcUrl(), clientProps)
+                .connect(getJdbcUrl(), new Properties());
         hbaseTables = getHBaseTables();
-        assertTrue(hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
-        assertTrue(hbaseTables.size() == 1);
+        assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
         assertEquals(0, countUpgradeAttempts);
+
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM SYSTEM.CATALOG LIMIT 1");
+        // Tests that SYSTEM.CATALOG contains necessary metadata rows for itself (See PHOENIX-5302)
+        assertTrue(rs.next());
     }
 
     // Conditions: The first connection creates all SYSTEM tables via "EXECUTE UPGRADE" since auto-upgrade is disabled
@@ -445,7 +393,7 @@ public class SystemCatalogCreationOnConnectionIT {
         hbaseTables = getHBaseTables();
         assertFalse(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG)
                 || hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
-        assertTrue(hbaseTables.size() == 0);
+        assertEquals(0, hbaseTables.size());
 
         // Test that we are unable to run any other queries using this connection until we upgrade
         final String tableName = generateUniqueName();
@@ -506,6 +454,16 @@ public class SystemCatalogCreationOnConnectionIT {
         return tables;
     }
 
+    // Check if the SYSTEM namespace has been created
+    private boolean isSystemNamespaceCreated() throws IOException {
+        try {
+            testUtil.getHBaseAdmin().getNamespaceDescriptor(SYSTEM_CATALOG_SCHEMA);
+        } catch (NamespaceNotFoundException ex) {
+            return false;
+        }
+        return true;
+    }
+
     /**
      * Alter the table metadata and return modified value
      * @param driver
@@ -592,7 +550,7 @@ public class SystemCatalogCreationOnConnectionIT {
                 .connect(getJdbcUrl(), new Properties());
         hbaseTables = getHBaseTables();
         assertFalse(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG) || hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
-        assertTrue(hbaseTables.size() == 0);
+        assertEquals(0, hbaseTables.size());
         assertEquals(1, countUpgradeAttempts);
 
         // We use the same connection to run "EXECUTE UPGRADE"
@@ -619,6 +577,7 @@ public class SystemCatalogCreationOnConnectionIT {
         hbaseTables = getHBaseTables();
         assertEquals(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, hbaseTables);
         assertEquals(0, countUpgradeAttempts);
+        assertTrue(isSystemNamespaceCreated());
         return driver;
     }
 
@@ -636,12 +595,13 @@ public class SystemCatalogCreationOnConnectionIT {
         hbaseTables = getHBaseTables();
         assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
         assertEquals(0, countUpgradeAttempts);
+        assertFalse(isSystemNamespaceCreated());
         return driver;
     }
 
     // Conditions: server-side namespace mapping is enabled, client-side namespace mapping is disabled
-    // Expected: Since this is the first connection to the server, we will create SYSTEM.CATALOG but immediately
-    // throw an exception for inconsistent namespace mapping
+    // Expected: Since this is the first connection to the server, we will immediately
+    // throw an exception for inconsistent namespace mapping without creating any SYSTEM tables
     private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionNSMappingServerEnabledClientDisabled()
     throws Exception {
         startMiniClusterWithToggleNamespaceMapping(Boolean.TRUE.toString());
@@ -655,15 +615,14 @@ public class SystemCatalogCreationOnConnectionIT {
             assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
         }
         hbaseTables = getHBaseTables();
-        assertTrue(hbaseTables.contains(PHOENIX_SYSTEM_CATALOG));
-        assertTrue(hbaseTables.size() == 1);
+        assertEquals(0, hbaseTables.size());
         assertEquals(0, countUpgradeAttempts);
         return driver;
     }
 
     // Conditions: server-side namespace mapping is disabled, client-side namespace mapping is enabled
-    // Expected: Since this is the first connection to the server, we will create the SYSTEM namespace and create
-    // SYSTEM:CATALOG and then immediately throw an exception for inconsistent namespace mapping
+    // Expected: Since this is the first connection to the server, we will immediately throw an exception for
+    // inconsistent namespace mapping without creating any SYSTEM tables or SYSTEM namespace
     private SystemCatalogCreationOnConnectionIT.PhoenixSysCatCreationTestingDriver firstConnectionNSMappingServerDisabledClientEnabled()
     throws Exception {
         startMiniClusterWithToggleNamespaceMapping(Boolean.FALSE.toString());
@@ -677,9 +636,9 @@ public class SystemCatalogCreationOnConnectionIT {
             assertEquals(SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode(), sqlE.getErrorCode());
         }
         hbaseTables = getHBaseTables();
-        assertTrue(hbaseTables.contains(PHOENIX_NAMESPACE_MAPPED_SYSTEM_CATALOG));
-        assertTrue(hbaseTables.size() == 1);
+        assertEquals(0, hbaseTables.size());
         assertEquals(0, countUpgradeAttempts);
+        assertFalse(isSystemNamespaceCreated());
         return driver;
     }
 
@@ -696,6 +655,7 @@ public class SystemCatalogCreationOnConnectionIT {
         hbaseTables = getHBaseTables();
         assertEquals(PHOENIX_SYSTEM_TABLES, hbaseTables);
         assertEquals(0, countUpgradeAttempts);
+        assertFalse(isSystemNamespaceCreated());
         return driver;
     }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 003d4d2..9f098d3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1061,25 +1061,33 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 QueryServicesOptions.DEFAULT_ALLOW_ONLINE_TABLE_SCHEMA_UPDATE);
     }
 
-    void ensureNamespaceCreated(String schemaName) throws SQLException {
+    /**
+     * Ensure that the HBase namespace is created/exists already
+     * @param schemaName Phoenix schema name for which we ensure existence of the HBase namespace
+     * @return true if we created the HBase namespace because it didn't already exist
+     * @throws SQLException If there is an exception creating the HBase namespace
+     */
+    boolean ensureNamespaceCreated(String schemaName) throws SQLException {
         SQLException sqlE = null;
+        boolean createdNamespace = false;
         try (HBaseAdmin admin = getAdmin()) {
             NamespaceDescriptor namespaceDescriptor = null;
             try {
                 namespaceDescriptor = admin.getNamespaceDescriptor(schemaName);
-            } catch (org.apache.hadoop.hbase.NamespaceNotFoundException e) {
+            } catch (NamespaceNotFoundException ignored) {
 
             }
             if (namespaceDescriptor == null) {
                 namespaceDescriptor = NamespaceDescriptor.create(schemaName).build();
                 admin.createNamespace(namespaceDescriptor);
+                createdNamespace = true;
             }
-            return;
         } catch (IOException e) {
             sqlE = ServerUtil.parseServerException(e);
         } finally {
             if (sqlE != null) { throw sqlE; }
         }
+        return createdNamespace;
     }
 
     /**
@@ -1105,6 +1113,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
             final String znode = this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+            boolean createdNamespace = false;
             LOGGER.debug("Found quorum: " + quorum + ":" + znode);
 
             if (isMetaTable) {
@@ -1112,7 +1121,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     try {
                         // SYSTEM namespace needs to be created via HBase APIs because "CREATE SCHEMA" statement tries to write
                         // its metadata in SYSTEM:CATALOG table. Without SYSTEM namespace, SYSTEM:CATALOG table cannot be created
-                        ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+                        createdNamespace = ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
                     } catch (PhoenixIOException e) {
                         // We could either:
                         // 1) Not access the NS descriptor. The NS may or may not exist at this point
@@ -1124,14 +1133,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                     if (admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, false))) {
                         // SYSTEM.CATALOG exists, so at this point, we have 3 cases:
-                        // 1) If server-side namespace mapping is disabled, throw Inconsistent namespace mapping exception
-                        // 2) If server-side namespace mapping is enabled and SYSCAT needs to be upgraded, upgrade SYSCAT
-                        //    and also migrate SYSTEM tables to the SYSTEM namespace
-                        // 3. If server-side namespace mapping is enabled and SYSCAT doesn't need to be upgraded, we still
-                        //    need to migrate SYSTEM tables to the SYSTEM namespace using the
+                        // 1) If server-side namespace mapping is disabled, drop the SYSTEM namespace if it was created
+                        //    above and throw Inconsistent namespace mapping exception
+                        // 2) If server-side namespace mapping is enabled and SYSTEM.CATALOG needs to be upgraded,
+                        //    upgrade SYSTEM.CATALOG and also migrate SYSTEM tables to the SYSTEM namespace
+                        // 3. If server-side namespace mapping is enabled and SYSTEM.CATALOG doesn't need to be
+                        //    upgraded, we still need to migrate SYSTEM tables to the SYSTEM namespace using the
                         //    {@link ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method (as part of
                         //    {@link upgradeSystemTables(String, Properties)})
-                        checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
+                        try {
+                            checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
+                        } catch (SQLException possibleCompatException) {
+                            // Handles Case 1: Drop the SYSTEM namespace in case it was created above
+                            if (createdNamespace && possibleCompatException.getErrorCode() ==
+                                    SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode()) {
+                                ensureNamespaceDropped(QueryConstants.SYSTEM_SCHEMA_NAME);
+                            }
+                            // rethrow the SQLException
+                            throw possibleCompatException;
+                        }
                         // Thrown so we can force an upgrade which will just migrate SYSTEM tables to the SYSTEM namespace
                         throw new UpgradeRequiredException(MIN_SYSTEM_TABLE_TIMESTAMP);
                     }
@@ -1187,7 +1207,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return null;
                 }
                 if (isMetaTable && !isUpgradeRequired()) {
-                    checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
+                    try {
+                        checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES,
+                                this.getProps()).getName());
+                    } catch (SQLException possibleCompatException) {
+                        if (possibleCompatException.getErrorCode() ==
+                                SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES.getErrorCode()) {
+                            try {
+                                // In case we wrongly created SYSTEM.CATALOG or SYSTEM:CATALOG, we should drop it
+                                admin.disableTable(physicalTableName);
+                                admin.deleteTable(physicalTableName);
+                            } catch (org.apache.hadoop.hbase.TableNotFoundException ignored) {
+                                // Ignore this since it just means that another client with a similar set of
+                                // incompatible configs and conditions beat us to dropping the SYSCAT HBase table
+                            }
+                            if (createdNamespace &&
+                                    SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, this.getProps())) {
+                                // We should drop the SYSTEM namespace which we just created, since
+                                // server-side namespace mapping is disabled
+                                ensureNamespaceDropped(QueryConstants.SYSTEM_SCHEMA_NAME);
+                            }
+                        }
+                        // rethrow the SQLException
+                        throw possibleCompatException;
+                    }
+
                 }
                 return null;
             } else {
@@ -4646,7 +4690,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             ReadOnlyProps props = this.getProps();
             boolean dropMetadata = props.getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
             if (dropMetadata) {
-                ensureNamespaceDropped(schemaName, result.getMutationTime());
+                ensureNamespaceDropped(schemaName);
             }
             break;
         default:
@@ -4655,7 +4699,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
     }
 
-    private void ensureNamespaceDropped(String schemaName, long mutationTime) throws SQLException {
+    private void ensureNamespaceDropped(String schemaName) throws SQLException {
         SQLException sqlE = null;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);