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/10/29 19:10:49 UTC

[phoenix] branch 4.x-HBase-1.3 updated: PHOENIX-5103: Can't create/drop table using 4.14 client against 4.15 server (Addendum)

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

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


The following commit(s) were added to refs/heads/4.x-HBase-1.3 by this push:
     new dfcbc5e  PHOENIX-5103: Can't create/drop table using 4.14 client against 4.15 server (Addendum)
dfcbc5e is described below

commit dfcbc5ef82eaa179f582e67705ba8a8af4a3c7c7
Author: Chinmay Kulkarni <ch...@gmail.com>
AuthorDate: Thu Oct 24 21:59:40 2019 -0700

    PHOENIX-5103: Can't create/drop table using 4.14 client against 4.15 server (Addendum)
---
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  | 38 +++++++++++++++-------
 .../coprocessor/tasks/DropChildViewsTask.java      |  8 ++++-
 .../phoenix/query/ConnectionQueryServices.java     |  1 +
 .../phoenix/query/ConnectionQueryServicesImpl.java |  7 +++-
 .../query/ConnectionlessQueryServicesImpl.java     |  3 ++
 .../query/DelegateConnectionQueryServices.java     |  5 +++
 .../java/org/apache/phoenix/util/QueryUtil.java    | 10 ++++--
 .../java/org/apache/phoenix/util/ViewUtil.java     | 25 ++++++++------
 8 files changed, 70 insertions(+), 27 deletions(-)

diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 7558b8d..3651927 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -57,6 +57,8 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORAGE_SCHEME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
@@ -1714,9 +1716,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
 
             // check if the table was dropped, but had child views that were have not yet been cleaned up
-            if (!Bytes.toString(schemaName).equals(QueryConstants.SYSTEM_SCHEMA_NAME) &&
-                    clientVersion >= MIN_SPLITTABLE_SYSTEM_CATALOG) {
-                ViewUtil.dropChildViews(env, tenantIdBytes, schemaName, tableName);
+            if (!Bytes.toString(schemaName).equals(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                byte[] sysCatOrSysChildLink = SchemaUtil.getPhysicalTableName(
+                        clientVersion >= MIN_SPLITTABLE_SYSTEM_CATALOG ?
+                        SYSTEM_CHILD_LINK_NAME_BYTES : SYSTEM_CATALOG_NAME_BYTES,
+                        env.getConfiguration()).getName();
+                // TODO: PHOENIX-5544 In the case of old clients, this actually does not do anything since the
+                // parent->child links were already removed when dropping the base table
+                ViewUtil.dropChildViews(env, tenantIdBytes, schemaName, tableName, sysCatOrSysChildLink);
             }
 
             byte[] parentTableKey = null;
@@ -2079,7 +2086,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         TableViewFinderResult result = new TableViewFinderResult();
         try (Table hTable =
                      env.getTable(SchemaUtil.getPhysicalTableName(
-                             PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+                             SYSTEM_CHILD_LINK_NAME_BYTES,
                              env.getConfiguration()))) {
             ViewUtil.findAllRelatives(hTable, tenantId, schemaName, tableName,
                     LinkType.CHILD_TABLE, result);
@@ -2237,7 +2244,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // after the view metadata is dropped, drop parent->child link
                 MetaDataResponse response =
                         processRemoteRegionMutations(
-                                PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+                                SYSTEM_CHILD_LINK_NAME_BYTES,
                                 childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_DELETE_CHILD_LINK);
                 if (response != null) {
                     done.run(response);
@@ -2341,10 +2348,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
             if (tableType == PTableType.TABLE || tableType == PTableType.VIEW || tableType == PTableType.SYSTEM) {
                 // check to see if the table has any child views
-                try (Table hTable =
-                             env.getTable(SchemaUtil.getPhysicalTableName(
-                                     PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                                     env.getConfiguration()))) {
+                try (Table hTable = env.getTable(SchemaUtil.getPhysicalTableName(
+                        clientVersion >= MIN_SPLITTABLE_SYSTEM_CATALOG ?
+                                SYSTEM_CHILD_LINK_NAME_BYTES : SYSTEM_CATALOG_NAME_BYTES,
+                        env.getConfiguration()))) {
                     boolean hasChildViews =
                             ViewUtil.hasChildViews(hTable, tenantId, schemaName, tableName,
                                     clientTimeStamp);
@@ -2355,9 +2362,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     EnvironmentEdgeManager.currentTimeMillis(), null);
                         }
                         try {
-                            PhoenixConnection conn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
-                            Task.addTask(conn, PTable.TaskType.DROP_CHILD_VIEWS, Bytes.toString(tenantId),
-                                    Bytes.toString(schemaName), Bytes.toString(tableName), this.accessCheckEnabled);
+                            if (clientVersion >= MIN_SPLITTABLE_SYSTEM_CATALOG) {
+                                PhoenixConnection conn =
+                                        QueryUtil.getConnectionOnServer(env.getConfiguration())
+                                                .unwrap(PhoenixConnection.class);
+                                Task.addTask(conn, PTable.TaskType.DROP_CHILD_VIEWS,
+                                        Bytes.toString(tenantId), Bytes.toString(schemaName),
+                                        Bytes.toString(tableName), this.accessCheckEnabled);
+                            }
+                            // else: the client version is old, so we cannot add a task to cleanup
+                            // child view metadata since SYSTEM.TASK may not exist
                         } catch (Throwable t) {
                             LOGGER.error("Adding a task to drop child views failed!", t);
                         }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/tasks/DropChildViewsTask.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/tasks/DropChildViewsTask.java
index 62a0f28..4609cf7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/tasks/DropChildViewsTask.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/tasks/DropChildViewsTask.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.coprocessor.tasks;
 
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.TaskRegionObserver;
+import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ViewUtil;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.MetaDataClient;
@@ -32,6 +33,8 @@ import java.sql.SQLException;
 import java.sql.Timestamp;
 import java.util.Properties;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES;
+
 /**
  * Task runs periodically to clean up task of child views whose parent is dropped
  *
@@ -58,7 +61,10 @@ public class DropChildViewsTask extends BaseTask {
                     taskRecord.getSchemaName(), taskRecord.getTableName(), true);
             if (result.getMutationCode() != MetaDataProtocol.MutationCode.TABLE_ALREADY_EXISTS) {
                 ViewUtil.dropChildViews(env, taskRecord.getTenantIdBytes(),
-                        taskRecord.getSchemaNameBytes(), taskRecord.getTableNameBytes());
+                        taskRecord.getSchemaNameBytes(), taskRecord.getTableNameBytes(),
+                        SchemaUtil.getPhysicalTableName(
+                                SYSTEM_CHILD_LINK_NAME_BYTES,
+                                env.getConfiguration()).getName());
                 return new TaskRegionObserver.TaskResult(TaskRegionObserver.TaskResultCode.SUCCESS, "");
             } else if (System.currentTimeMillis() < timeMaxInterval + timestamp.getTime()) {
                 // skip this task as it has not been expired and its parent table has not been dropped yet
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index 18009ad..db63d48 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -162,6 +162,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public MetaDataMutationResult dropSchema(List<Mutation> schemaMetaData, String schemaName) throws SQLException;
 
     boolean isUpgradeRequired();
+    void clearUpgradeRequired();
     void upgradeSystemTables(String url, Properties props) throws SQLException;
     
     public Configuration getConfiguration();
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 a932d59..a1f22bf 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
@@ -3636,7 +3636,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // with SYSTEM Namespace
             createSchemaIfNotExistsSystemNSMappingEnabled(metaConnection);
 
-            ConnectionQueryServicesImpl.this.upgradeRequired.set(false);
+            clearUpgradeRequired();
             success = true;
         } catch (UpgradeInProgressException | UpgradeNotRequiredException e) {
             // don't set it as initializationException because otherwise client won't be able to retry
@@ -5350,6 +5350,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
+    public void clearUpgradeRequired() {
+        upgradeRequired.set(false);
+    }
+
+    @Override
     public Configuration getConfiguration() {
         return config;
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 62b1def..7b52688 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -752,6 +752,9 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
+    public void clearUpgradeRequired() {}
+
+    @Override
     public Configuration getConfiguration() {
         return config;
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index c228270..761b758 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -362,6 +362,11 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
+    public void clearUpgradeRequired() {
+        getDelegate().isUpgradeRequired();
+    }
+
+    @Override
     public Configuration getConfiguration() {
         return getDelegate().getConfiguration();
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index 8fe513e..829c633 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -323,7 +323,8 @@ public final class QueryUtil {
     }
 
     /**
-     * @return {@link PhoenixConnection} with {@value UpgradeUtil#RUN_UPGRADE} set so that we don't initiate server upgrade
+     * @return {@link PhoenixConnection} with {@value UpgradeUtil#DO_NOT_UPGRADE} set so that we
+     * don't initiate metadata upgrade
      */
     public static Connection getConnectionOnServer(Configuration conf) throws ClassNotFoundException,
             SQLException {
@@ -340,13 +341,16 @@ public final class QueryUtil {
     }
 
     /**
-     * @return {@link PhoenixConnection} with {@value UpgradeUtil#DO_NOT_UPGRADE} set so that we don't initiate metadata upgrade.
+     * @return {@link PhoenixConnection} with {@value UpgradeUtil#DO_NOT_UPGRADE} set
+     * and with the upgrade-required flag cleared so that we don't initiate metadata upgrade.
      */
     public static Connection getConnectionOnServer(Properties props, Configuration conf)
             throws ClassNotFoundException,
             SQLException {
         setServerConnection(props);
-        return getConnection(props, conf);
+        Connection conn = getConnection(props, conf);
+        conn.unwrap(PhoenixConnection.class).getQueryServices().clearUpgradeRequired();
+        return conn;
     }
 
     public static Connection getConnectionOnServerWithCustomUrl(Properties props, String principal)
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java
index 6badf1f..100c9ab 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java
@@ -140,10 +140,18 @@ public class ViewUtil {
     }
     
     /**
+     * Check metadata to find all child views for a given table/view
+     * @param sysCatOrsysChildLink For older (pre-4.15.0) clients, we look for child links inside SYSTEM.CATALOG,
+     *                             otherwise we look for them inside SYSTEM.CHILD_LINK
+     * @param tenantId tenantId
+     * @param schemaName table schema name
+     * @param tableName table name
+     * @param timestamp passed client-side timestamp
      * @return true if the given table has at least one child view
-     * @throws IOException 
+     * @throws IOException
      */
-    public static boolean hasChildViews(Table systemCatalog, byte[] tenantId, byte[] schemaName, byte[] tableName, long timestamp) throws IOException {
+    public static boolean hasChildViews(Table sysCatOrsysChildLink, byte[] tenantId, byte[] schemaName,
+                                        byte[] tableName, long timestamp) throws IOException {
         byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
         Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
         SingleColumnValueFilter linkFilter =
@@ -153,28 +161,25 @@ public class ViewUtil {
                     // if we found a row with the CHILD_TABLE link type we are done and can
                     // terminate the scan
                     @Override
-                    public boolean filterAllRemaining() throws IOException {
+                    public boolean filterAllRemaining() {
                         return matchedColumn;
                     }
                 };
         linkFilter.setFilterIfMissing(true);
         scan.setFilter(linkFilter);
         scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
-        try (ResultScanner scanner = systemCatalog.getScanner(scan)) {
+        try (ResultScanner scanner = sysCatOrsysChildLink.getScanner(scan)) {
             Result result = scanner.next();
             return result!=null; 
         }
     }
 
-    public static void dropChildViews(RegionCoprocessorEnvironment env, byte[] tenantIdBytes, byte[] schemaName, byte[] tableName)
+    public static void dropChildViews(RegionCoprocessorEnvironment env, byte[] tenantIdBytes,
+            byte[] schemaName, byte[] tableName, byte[] sysCatOrSysChildLink)
             throws IOException, SQLException, ClassNotFoundException {
         Table hTable = null;
         try {
-            hTable =
-                    ServerUtil.getHTableForCoprocessorScan(env,
-                            SchemaUtil.getPhysicalTableName(
-                                    PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                                    env.getConfiguration()).getName());
+            hTable = ServerUtil.getHTableForCoprocessorScan(env, sysCatOrSysChildLink);
         }
         catch (Exception e){
         }