You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ac...@apache.org on 2020/05/19 23:42:02 UTC

[phoenix] branch 4.x updated: PHOENIX-5863 Upsert into view against a table with index throws exception when 4.14.3 client connects to 4.16 server

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

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


The following commit(s) were added to refs/heads/4.x by this push:
     new 422ade0  PHOENIX-5863 Upsert into view against a table with index throws exception when 4.14.3 client connects to 4.16 server
422ade0 is described below

commit 422ade0883b94da758c219d6dc9633b2d901759a
Author: Sandeep Guggilam <sg...@sandeepg-ltm.internal.salesforce.com>
AuthorDate: Tue Apr 28 15:08:58 2020 -0700

    PHOENIX-5863 Upsert into view against a table with index throws exception when 4.14.3 client connects to 4.16 server
    
    Signed-off-by: Abhishek Singh Chouhan <ac...@apache.org>
---
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  |   2 +-
 .../java/org/apache/phoenix/util/ViewUtil.java     | 103 ++++++++++++---------
 2 files changed, 59 insertions(+), 46 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 7c9dcc5..5735711 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
@@ -617,7 +617,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     && table.getViewType() != ViewType.MAPPED) {
                 try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)) {
                     PTable pTable = PhoenixRuntime.getTableNoCache(connection, table.getParentName().getString());
-                    table = ViewUtil.addDerivedColumnsAndIndexesFromParent(connection, table, pTable);
+                    table = ViewUtil.addDerivedColumnsFromParent(connection, table, pTable);
                 }
             }
             builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
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 7e622b5..e5e9b8f 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
@@ -15,10 +15,24 @@
  */
 package org.apache.phoenix.util;
 
-import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import static org.apache.phoenix.coprocessor.MetaDataProtocol.MIN_SPLITTABLE_SYSTEM_CATALOG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_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_TYPE_BYTES;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
+import static org.apache.phoenix.util.SchemaUtil.getVarChars;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
@@ -63,23 +77,10 @@ import org.apache.phoenix.schema.types.PLong;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.sql.SQLException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-import static org.apache.phoenix.coprocessor.MetaDataProtocol.MIN_SPLITTABLE_SYSTEM_CATALOG;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_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_TYPE_BYTES;
-import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
-import static org.apache.phoenix.util.SchemaUtil.getVarChars;
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 public class ViewUtil {
 
@@ -344,13 +345,46 @@ public class ViewUtil {
             }
         }
     }
+    
+    /**
+     * Inherit all indexes and columns from the parent 
+     * @return table with inherited columns and indexes
+     */
+    public static PTable addDerivedColumnsAndIndexesFromParent(PhoenixConnection connection,
+            PTable table, PTable parentTable) throws SQLException {
+        PTable pTable = addDerivedColumnsFromParent(connection, table, parentTable);
+        boolean hasIndexId = table.getViewIndexId() != null;
+        // For views :
+        if (!hasIndexId) {
+            // 1. need to resolve the views's own indexes so that any columns added by ancestors are included
+            List<PTable> allIndexes = Lists.newArrayList();
+            if (!pTable.getIndexes().isEmpty()) {
+                for (PTable viewIndex : pTable.getIndexes()) {
+                    PTable resolvedViewIndex =
+                            ViewUtil.addDerivedColumnsAndIndexesFromParent(connection, viewIndex, pTable);
+                    if (resolvedViewIndex!=null)
+                        allIndexes.add(resolvedViewIndex);
+                }
+            }
+            
+            // 2. include any indexes from ancestors that can be used by this view
+            List<PTable> inheritedIndexes = Lists.newArrayList();
+            addIndexesFromParent(connection, pTable, parentTable, inheritedIndexes);
+            allIndexes.addAll(inheritedIndexes);
+            if (!allIndexes.isEmpty()) {
+                pTable = PTableImpl.builderWithColumns(pTable, getColumnsToClone(pTable))
+                        .setIndexes(allIndexes).build();
+            }
+        }
+        return pTable;
+    }
 
     /**
      * Inherit all columns from the parent unless its an excluded column if the same columns is present in the parent
      * and child (for table metadata created before PHOENIX-3534) we chose the child column over the parent column
-     * @return table with inherited columns and indexes
+     * @return table with inherited columns
      */
-    public static PTable addDerivedColumnsAndIndexesFromParent(PhoenixConnection connection,
+    public static PTable addDerivedColumnsFromParent(PhoenixConnection connection,
                                                                PTable table, PTable parentTable) throws SQLException {
         // combine columns for view and view indexes
         boolean hasIndexId = table.getViewIndexId() != null;
@@ -558,28 +592,7 @@ public class ViewUtil {
                 .setUseStatsForParallelization(useStatsForParallelization)
                 .build();
         pTable = WhereConstantParser.addViewInfoToPColumnsIfNeeded(pTable);
-
-        // For views :
-        if (!hasIndexId) {
-            // 1. need to resolve the views's own indexes so that any columns added by ancestors are included
-            List<PTable> allIndexes = Lists.newArrayList();
-            if (!pTable.getIndexes().isEmpty()) {
-                for (PTable viewIndex : pTable.getIndexes()) {
-                    PTable resolvedViewIndex =
-                            ViewUtil.addDerivedColumnsAndIndexesFromParent(connection, viewIndex, pTable);
-                    if (resolvedViewIndex!=null)
-                        allIndexes.add(resolvedViewIndex);
-                }
-            }
-            // 2. include any indexes from ancestors that can be used by this view
-            List<PTable> inheritedIndexes = Lists.newArrayList();
-            addIndexesFromParent(connection, pTable, parentTable, inheritedIndexes);
-            allIndexes.addAll(inheritedIndexes);
-            if (!allIndexes.isEmpty()) {
-                pTable = PTableImpl.builderWithColumns(pTable, getColumnsToClone(pTable))
-                        .setIndexes(allIndexes).build();
-            }
-        }
+        
         return pTable;
     }