You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by pb...@apache.org on 2018/02/01 00:06:59 UTC

[27/35] phoenix git commit: PHOENIX-4523 phoenix.schema.isNamespaceMappingEnabled problem (Karan Mehta)

PHOENIX-4523 phoenix.schema.isNamespaceMappingEnabled problem (Karan Mehta)


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

Branch: refs/heads/4.x-cdh5.11.2
Commit: 319ff01175f3f65acf85314d5d137496c8f1a043
Parents: ffee8c0
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Jan 12 00:22:09 2018 +0000
Committer: Pedro Boado <pb...@apache.org>
Committed: Wed Jan 31 22:24:48 2018 +0000

----------------------------------------------------------------------
 .../query/ConnectionQueryServicesImpl.java      | 35 ++++++++++----------
 .../org/apache/phoenix/util/UpgradeUtil.java    |  2 ++
 .../query/ConnectionQueryServicesImplTest.java  |  6 ++--
 3 files changed, 22 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/319ff011/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
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 38be6af..5b7735e 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
@@ -2543,16 +2543,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
     }
 
-    void createSysMutexTable(HBaseAdmin admin, ReadOnlyProps props) throws IOException, SQLException {
+    void createSysMutexTableIfNotExists(HBaseAdmin admin, ReadOnlyProps props) throws IOException, SQLException {
         try {
-            final TableName mutexTableName = SchemaUtil.getPhysicalTableName(
-                PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME, props);
-            List<TableName> systemTables = getSystemTableNames(admin);
-            if (systemTables.contains(mutexTableName) || admin.tableExists( TableName.valueOf(
-                PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME))) {
+            if(admin.tableExists(PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME) || admin.tableExists(TableName.valueOf(
+                    PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME))) {
                 logger.debug("System mutex table already appears to exist, not creating it");
                 return;
             }
+            final TableName mutexTableName = SchemaUtil.getPhysicalTableName(
+                PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME, props);
             HTableDescriptor tableDesc = new HTableDescriptor(mutexTableName);
             HColumnDescriptor columnDesc = new HColumnDescriptor(
                     PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES);
@@ -2566,8 +2565,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 put.add(PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES, UPGRADE_MUTEX, UPGRADE_MUTEX_UNLOCKED);
                 sysMutexTable.put(put);
             }
-        } catch (TableExistsException | AccessDeniedException e) {
-            // Ignore
+        } catch (IOException e) {
+            if(!Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), AccessDeniedException.class)) ||
+                    !Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), org.apache.hadoop.hbase.TableNotFoundException.class))) {
+                // Ignore
+            } else {
+                throw e;
+            }
         }catch(PhoenixIOException e){
             if(e.getCause()!=null && e.getCause() instanceof AccessDeniedException)
             {
@@ -2578,7 +2582,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
     }
 
-    List<TableName> getSystemTableNames(HBaseAdmin admin) throws IOException {
+    List<TableName> getSystemTableNamesInDefaultNamespace(HBaseAdmin admin) throws IOException {
         return Lists.newArrayList(admin.listTableNames(QueryConstants.SYSTEM_SCHEMA_NAME + "\\..*"));
     }
 
@@ -2597,7 +2601,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
         // Catch the IOException to log the error message and then bubble it up for the client to retry.
         try {
-            createSysMutexTable(hbaseAdmin, ConnectionQueryServicesImpl.this.getProps());
+            createSysMutexTableIfNotExists(hbaseAdmin, ConnectionQueryServicesImpl.this.getProps());
         } catch (IOException exception) {
             logger.error("Failed to created SYSMUTEX table. Upgrade or migration is not possible without it. Please retry.");
             throw exception;
@@ -2649,7 +2653,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             !SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
                                     ConnectionQueryServicesImpl.this.getProps())) {
                         try (HBaseAdmin admin = getAdmin()) {
-                            createSysMutexTable(admin, this.getProps());
+                            createSysMutexTableIfNotExists(admin, this.getProps());
                         }
                     }
                     if (acquiredMutexLock = acquireUpgradeMutex(currentServerSideTableTimeStamp, mutexRowKey)) {
@@ -3192,7 +3196,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // below. If the NS does exist and is mapped, the below check will exit gracefully.
             }
 
-            List<TableName> tableNames = getSystemTableNames(admin);
+            List<TableName> tableNames = getSystemTableNamesInDefaultNamespace(admin);
             // No tables exist matching "SYSTEM\..*", they are all already in "SYSTEM:.*"
             if (tableNames.size() == 0) { return; }
             // Try to move any remaining tables matching "SYSTEM\..*" into "SYSTEM:"
@@ -3204,12 +3208,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // If we cannot acquire lock, it means some old client is either migrating SYSCAT or trying to upgrade the
             // schema of SYSCAT table and hence it should not be interrupted
             // Create mutex if not already created
-            if (!tableNames.contains(PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME)) {
-                TableName mutexName = SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME, props);
-                if (PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME.equals(mutexName) || !tableNames.contains(mutexName)) {
-                    createSysMutexTable(admin, props);
-                }
-            }
+            createSysMutexTableIfNotExists(admin, props);
             acquiredMutexLock = acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP, mutexRowKey);
             if(acquiredMutexLock) {
                 logger.debug("Acquired lock in SYSMUTEX table for migrating SYSTEM tables to SYSTEM namespace");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/319ff011/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index 548e306..f09590d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -1779,6 +1779,8 @@ public class UpgradeUtil {
                 admin.deleteTable(srcTableName);
                 logger.info(String.format("deleting snapshot %s..", snapshotName));
                 admin.deleteSnapshot(snapshotName);
+            } else {
+                logger.info(String.format("Destination Table %s already exists. No migration needed.", destTableName));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/319ff011/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
index 4708ffb..b5c3e4a 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
@@ -49,10 +49,10 @@ public class ConnectionQueryServicesImplTest {
         when(cqs.createSchema(any(List.class), anyString())).thenCallRealMethod();
         doCallRealMethod().when(cqs).ensureSystemTablesMigratedToSystemNamespace(any(ReadOnlyProps.class));
         // Do nothing for this method, just check that it was invoked later
-        doNothing().when(cqs).createSysMutexTable(any(HBaseAdmin.class), any(ReadOnlyProps.class));
+        doNothing().when(cqs).createSysMutexTableIfNotExists(any(HBaseAdmin.class), any(ReadOnlyProps.class));
 
         // Spoof out this call so that ensureSystemTablesUpgrade() will return-fast.
-        when(cqs.getSystemTableNames(any(HBaseAdmin.class))).thenReturn(Collections.<TableName> emptyList());
+        when(cqs.getSystemTableNamesInDefaultNamespace(any(HBaseAdmin.class))).thenReturn(Collections.<TableName> emptyList());
 
         // Throw a special exception to check on later
         doThrow(PHOENIX_IO_EXCEPTION).when(cqs).ensureNamespaceCreated(anyString());
@@ -64,7 +64,7 @@ public class ConnectionQueryServicesImplTest {
 
         // Should be called after upgradeSystemTables()
         // Proves that execution proceeded
-        verify(cqs).getSystemTableNames(any(HBaseAdmin.class));
+        verify(cqs).getSystemTableNamesInDefaultNamespace(any(HBaseAdmin.class));
 
         try {
             // Verifies that the exception is propagated back to the caller