Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.3 d54ae1909 -> bb67a6534


PHOENIX-4765 Add client and server side config property to enable rollback of 
splittable System Catalog if required


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

Branch: refs/heads/4.x-HBase-1.3
Commit: bb67a653423e804a9f0b4b7e11d114f922427e5e
Parents: d54ae19
Author: Thomas D'Silva <tdsi...@apache.org>
Authored: Tue Nov 20 12:10:05 2018 -0800
Committer: Thomas D'Silva <tdsi...@apache.org>
Committed: Mon Nov 26 16:35:56 2018 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/SystemCatalogIT.java | 40 ++++++++-
 .../coprocessor/MetaDataEndpointImpl.java       | 90 ++++++++++++++------
 .../phoenix/coprocessor/MetaDataProtocol.java   |  5 +-
 .../org/apache/phoenix/query/QueryServices.java | 17 ++++
 .../phoenix/query/QueryServicesOptions.java     |  2 +
 .../apache/phoenix/schema/MetaDataClient.java   | 26 +++++-
 6 files changed, 146 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb67a653/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
index 8a41fad..dd3721f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -31,10 +32,12 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -44,11 +47,12 @@ import com.google.common.collect.Maps;
 @Category(NeedsOwnMiniClusterTest.class)
 public class SystemCatalogIT extends BaseTest {
     private HBaseTestingUtility testUtil = null;
-    
+
        @BeforeClass
        public static void doSetup() throws Exception {
                Map<String, String> serverProps = 
Maps.newHashMapWithExpectedSize(1);
                serverProps.put(QueryServices.SYSTEM_CATALOG_SPLITTABLE, 
"false");
+        
serverProps.put(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK, "true");
                Map<String, String> clientProps = Collections.emptyMap();
                setUpTestDriver(new 
ReadOnlyProps(serverProps.entrySet().iterator()),
                                new 
ReadOnlyProps(clientProps.entrySet().iterator()));
@@ -87,7 +91,8 @@ public class SystemCatalogIT extends BaseTest {
             Statement stmt = conn.createStatement();) {
             stmt.execute("DROP TABLE IF EXISTS " + tableName);
             stmt.execute("CREATE TABLE " + tableName
-                + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 
VARCHAR CONSTRAINT PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT=true");
+                    + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 
VARCHAR CONSTRAINT PK " +
+                    "PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT=true");
             try (Connection tenant1Conn = getTenantConnection("tenant1")) {
                 String view1DDL = "CREATE VIEW " + tableName + "_view AS 
SELECT * FROM " + tableName;
                 tenant1Conn.createStatement().execute(view1DDL);
@@ -97,7 +102,7 @@ public class SystemCatalogIT extends BaseTest {
     }
 
     private String getJdbcUrl() {
-        return "jdbc:phoenix:localhost:" + 
testUtil.getZkCluster().getClientPort() + ":/hbase";
+        return "jdbc:phoenix:localhost:" + 
getUtility().getZkCluster().getClientPort() + ":/hbase";
     }
 
     private Connection getTenantConnection(String tenantId) throws 
SQLException {
@@ -105,4 +110,31 @@ public class SystemCatalogIT extends BaseTest {
         tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
         return DriverManager.getConnection(getJdbcUrl(), tenantProps);
     }
-}
+
+    /**
+     * Ensure that we cannot add a column to a base table if 
QueryServices.BLOCK_METADATA_CHANGES_REQUIRE_PROPAGATION
+     * is true
+     */
+    @Test
+    public void testAddingColumnFails() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getJdbcUrl())) {
+            String fullTableName = 
SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
+            String fullViewName = 
SchemaUtil.getTableName(generateUniqueName(), generateUniqueName());
+            String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT 
NULL, v1 INTEGER " +
+                    "CONSTRAINT pk PRIMARY KEY (k1))";
+            conn.createStatement().execute(ddl);
+
+            ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + 
fullTableName;
+            conn.createStatement().execute(ddl);
+
+            try {
+                ddl = "ALTER TABLE " + fullTableName + " ADD v2 INTEGER";
+                conn.createStatement().execute(ddl);
+                fail();
+            }
+            catch (SQLException e) {
+                
assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), 
e.getErrorCode());
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb67a653/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
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 b3b9185..14caca3 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
@@ -530,6 +530,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
     private int maxIndexesPerTable;
     private boolean isTablesMappingEnabled;
 
+    // this flag denotes that we will continue to write parent table column 
metadata while creating
+    // a child view and also block metadata changes that were previously 
propagated to children
+    // before 4.15, so that we can rollback the upgrade to 4.15 if required
+    private boolean allowSystemCatalogRollback;
 
     /**
      * Stores a reference to the coprocessor environment provided by the
@@ -559,6 +563,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
                     QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
         this.isTablesMappingEnabled = 
SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                 new ReadOnlyProps(config.iterator()));
+        this.allowSystemCatalogRollback = 
config.getBoolean(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK,
+                
QueryServicesOptions.DEFAULT_ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
 
         logger.info("Starting Tracing-Metrics Systems");
         // Start the phoenix trace collection
@@ -1477,6 +1483,7 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
         EncodedCQCounter cqCounter =
                 (!EncodedColumnsUtil.usesEncodedColumnNames(encodingScheme) || 
tableType == PTableType.VIEW) ? PTable.EncodedCQCounter.NULL_COUNTER
                         : new EncodedCQCounter();
+        boolean isRegularView = (tableType == PTableType.VIEW && 
viewType!=ViewType.MAPPED);
         while (true) {
           results.clear();
           scanner.next(results);
@@ -1505,7 +1512,6 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                   addExcludedColumnToTable(columns, colName, famName, 
colKv.getTimestamp());
               }
           } else {
-              boolean isRegularView = (tableType == PTableType.VIEW && 
viewType!=ViewType.MAPPED);
               addColumnToTable(results, colName, famName, colKeyValues, 
columns, saltBucketNum != null, baseColumnCount, isRegularView);
           }
         }
@@ -2150,11 +2156,21 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                 ImmutableBytesPtr parentCacheKey = null;
                 PTable parentTable = null;
                 if (parentTableName != null) {
-                    // we lock the parent table when creating an index on a 
table or a view
-                    if (tableType == PTableType.INDEX) {
+                    // From 4.15 onwards we only need to lock the parent table 
:
+                    // 1) when creating an index on a table or a view
+                    // 2) if allowSystemCatalogRollback is true we try to lock 
the parent table to prevent it
+                    // from changing concurrently while a view is being created
+                    if (tableType == PTableType.INDEX || 
allowSystemCatalogRollback) {
                         result = checkTableKeyInRegion(parentTableKey, region);
                         if (result != null) {
-                            
builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_IN_REGION);
+                            logger.error("Unable to lock parentTableKey 
"+Bytes.toStringBinary(parentTableKey));
+                            // if allowSystemCatalogRollback is true and we 
can't lock the parentTableKey (because
+                            // SYSTEM.CATALOG already split) return 
UNALLOWED_TABLE_MUTATION so that the client
+                            // knows the create statement failed
+                            MetaDataProtos.MutationCode code = tableType == 
PTableType.INDEX ?
+                                    
MetaDataProtos.MutationCode.TABLE_NOT_IN_REGION :
+                                    
MetaDataProtos.MutationCode.UNALLOWED_TABLE_MUTATION;
+                            builder.setReturnCode(code);
                             
builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
                             done.run(builder.build());
                             return;
@@ -2322,7 +2338,7 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                 // 3. Finally write the mutations to create the table
 
                 // From 4.15 the parent->child links are stored in a separate 
table SYSTEM.CHILD_LINK
-                // TODO remove this after PHOENIX-4763 is implemented
+                // TODO remove this after PHOENIX-4810 is implemented
                 List<Mutation> childLinkMutations = 
MetaDataUtil.removeChildLinks(tableMetadata);
                 MetaDataResponse response =
                         processRemoteRegionMutations(
@@ -2333,6 +2349,7 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                     return;
                 }
 
+                // When we drop a view we first drop the view metadata and 
then drop the parent->child linking row
                 List<Mutation> localMutations =
                         
Lists.newArrayListWithExpectedSize(tableMetadata.size());
                 List<Mutation> remoteMutations = 
Lists.newArrayListWithExpectedSize(2);
@@ -2620,7 +2637,7 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                     metaDataCache.invalidate(parentCacheKey);
                 }
 
-                // drop parent->child link when dropping a child view
+                // after the view metadata is dropped drop parent->child link
                 MetaDataResponse response =
                         processRemoteRegionMutations(
                             
PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
@@ -2909,7 +2926,7 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                 Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                         GlobalCache.getInstance(this.env).getMetaDataCache();
 
-                // The mutations to create a table are written in the 
following order:
+                // The mutations to add a column are written in the following 
order:
                 // 1. Update the encoded column qualifier for the parent table 
if its on a
                 // different region server (for tables that use column 
qualifier encoding)
                 // if the next step fails we end up wasting a few col 
qualifiers
@@ -3358,28 +3375,51 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
                         TableViewFinderResult childViewsResult = new 
TableViewFinderResult();
                         findAllChildViews(tenantId, 
table.getSchemaName().getBytes(), table.getTableName().getBytes(), 
childViewsResult);
                         if (childViewsResult.hasLinks()) {
-                            /* 
-                             * Dis-allow if:
-                             * 
-                             * 1) The base column count is 0 which means that 
the metadata hasn't been upgraded yet or
-                             * the upgrade is currently in progress.
-                             * 
-                             * 2) If the request is from a client that is 
older than 4.5 version of phoenix. 
-                             * Starting from 4.5, metadata requests have the 
client version included in them. 
-                             * We don't want to allow clients before 4.5 to 
add a column to the base table if it has views.
-                             * 
-                             * 3) Trying to switch tenancy of a table that has 
views
-                             */
-                            if (table.getBaseColumnCount() == 0 
+                            // Dis-allow if:
+                            //
+                            // 1) The base column count is 0 which means that 
the metadata hasn't been upgraded yet or
+                            // the upgrade is currently in progress.
+                            //
+                            // 2) If the request is from a client that is 
older than 4.5 version of phoenix.
+                            // Starting from 4.5, metadata requests have the 
client version included in them.
+                            // We don't want to allow clients before 4.5 to 
add a column to the base table if it
+                            // has views.
+                            //
+                            // 3) Trying to switch tenancy of a table that has 
views
+                            //
+                            // 4) From 4.15 onwards we allow SYSTEM.CATALOG to 
split and no longer propagate parent
+                            // metadata changes to child views.
+                            // If the client is on a version older than 4.15 
we have to block adding a column to a
+                            // parent able as we no longer lock the parent 
table on the server side while creating a
+                            // child view to prevent conflicting changes. This 
is handled on the client side from
+                            // 4.15 onwards.
+                            // Also if 
QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK is true, we block adding
+                            // a column to a parent table so that we can 
rollback the upgrade if required.
+                            if (table.getBaseColumnCount() == 0
                                     || !request.hasClientVersion()
                                     || switchAttribute(table, 
table.isMultiTenant(), tableMetaData, MULTI_TENANT_BYTES)) {
                                 return new 
MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
                                         
EnvironmentEdgeManager.currentTimeMillis(), null);
-                            } else {
-                                        MetaDataMutationResult mutationResult =
-                                                
validateColumnForAddToBaseTable(table,
-                                                    tableMetaData, 
rowKeyMetaData, childViewsResult,
-                                                    clientTimeStamp, 
request.getClientVersion());
+                            }
+                            else if (request.getClientVersion()< 
MIN_SPLITTABLE_SYSTEM_CATALOG ) {
+                                logger.error(
+                                    "Unable to add a column as the client is 
older than "
+                                            + MIN_SPLITTABLE_SYSTEM_CATALOG);
+                                return new 
MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
+                                        
EnvironmentEdgeManager.currentTimeMillis(), null);
+                            }
+                            else if (allowSystemCatalogRollback) {
+                                logger.error("Unable to add a column as the "
+                                        + 
QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK
+                                        + " config is set to true");
+                                return new 
MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
+                                        
EnvironmentEdgeManager.currentTimeMillis(), null);
+                            }
+                            else {
+                                MetaDataMutationResult mutationResult =
+                                        validateColumnForAddToBaseTable(table,
+                                            tableMetaData, rowKeyMetaData, 
childViewsResult,
+                                            clientTimeStamp, 
request.getClientVersion());
                                 // return if validation was not successful
                                 if (mutationResult!=null)
                                     return mutationResult;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb67a653/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 115d45b..86878c2 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -65,7 +65,7 @@ import org.apache.phoenix.util.MetaDataUtil;
  */
 public abstract class MetaDataProtocol extends MetaDataService {
     public static final int PHOENIX_MAJOR_VERSION = 4;
-    public static final int PHOENIX_MINOR_VERSION = 14;
+    public static final int PHOENIX_MINOR_VERSION = 15;
     public static final int PHOENIX_PATCH_NUMBER = 0;
     public static final int PHOENIX_VERSION =
             VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, 
PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER);
@@ -110,6 +110,8 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
     public static final int ESSENTIAL_FAMILY_VERSION_THRESHOLD = 
VersionUtil.encodeVersion("0", "94", "7");
     /** Version below which we fall back on the generic KeyValueBuilder */
     public static final int CLIENT_KEY_VALUE_BUILDER_THRESHOLD = 
VersionUtil.encodeVersion("0", "94", "14");
+    // Version at which we allow SYSTEM.CATALOG to split
+    public static final int MIN_SPLITTABLE_SYSTEM_CATALOG = 
VersionUtil.encodeVersion("4", "15", "0");
 
     // ALWAYS update this map whenever rolling out a new release (major, minor 
or patch release). 
     // Key is the SYSTEM.CATALOG timestamp for the version and value is the 
version string.
@@ -130,6 +132,7 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0, "4.12.x");
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0, "4.13.x");
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0, "4.14.x");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0, "4.15.x");
     }
     
     public static final String CURRENT_CLIENT_VERSION = PHOENIX_MAJOR_VERSION 
+ "." + PHOENIX_MINOR_VERSION + "." + PHOENIX_PATCH_NUMBER;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb67a653/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 78b72a1..728f3f8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -331,6 +331,23 @@ public interface QueryServices extends SQLCloseable {
     // The initial delay before the first task from table SYSTEM.TASK is 
handled
     public static final String TASK_HANDLING_INITIAL_DELAY_MS_ATTRIB = 
"phoenix.task.handling.initial.delay.ms";
 
+    // Before 4.15 when we created a view we included the parent table column 
metadata in the view
+    // metadata. After PHOENIX-3534 we allow SYSTEM.CATALOG to split and no 
longer store the parent
+    // table column metadata along with the child view metadata. When we 
resolve a child view, we
+    // resolve its ancestors and include their columns.
+    // Also, before 4.15 when we added a column to a base table we would have 
to propagate the
+    // column metadata to all its child views. After PHOENIX-3534 we no longer 
propagate metadata
+    // changes from a parent to its children (we just resolve its ancestors 
and include their columns)
+    // 
+    // The following config is used to continue writing the parent table 
column metadata while
+    // creating a view and also prevent metadata changes to a parent 
table/view that needs to be
+    // propagated to its children. This is done to allow rollback of the 
splittable SYSTEM.CATALOG
+    // feature
+    //
+    // By default this config is false meaning that rolling back the upgrade 
is not possible
+    public static final String ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK =
+            "phoenix.allow.system.catalog.rollback";
+
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb67a653/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 4e507d2..076b7e3 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -345,6 +345,8 @@ public class QueryServicesOptions {
     public static final long DEFAULT_TASK_HANDLING_MAX_INTERVAL_MS = 
30*60*1000; // 30 min
     public static final long DEFAULT_TASK_HANDLING_INITIAL_DELAY_MS = 10*1000; 
// 10 sec
 
+    public static final boolean 
DEFAULT_ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK = false;
+
     @SuppressWarnings("serial")
     public static final Set<String> DEFAULT_QUERY_SERVER_SKIP_WORDS = new 
HashSet<String>() {
       {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb67a653/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 4c8af0b..4b7c9af 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -1993,6 +1993,10 @@ public class MetaDataClient {
             Map<String,Object> commonFamilyProps) throws SQLException {
         final PTableType tableType = statement.getTableType();
         boolean wasAutoCommit = connection.getAutoCommit();
+        boolean allowSystemCatalogRollback =
+                connection.getQueryServices().getProps().getBoolean(
+                    QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK,
+                    
QueryServicesOptions.DEFAULT_ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
         connection.rollback();
         try {
             connection.setAutoCommit(false);
@@ -2023,6 +2027,9 @@ public class MetaDataClient {
             boolean isLocalIndex = indexType == IndexType.LOCAL;
             QualifierEncodingScheme encodingScheme = NON_ENCODED_QUALIFIERS;
             ImmutableStorageScheme immutableStorageScheme = 
ONE_CELL_PER_COLUMN;
+            int baseTableColumnCount =
+                    tableType == PTableType.VIEW ? parent.getColumns().size()
+                            : QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
             if (parent != null && tableType == PTableType.INDEX) {
                 timestamp = TransactionUtil.getTableTimestamp(connection, 
transactionProvider != null, transactionProvider);
                 isImmutableRows = parent.isImmutableRows();
@@ -2738,6 +2745,7 @@ public class MetaDataClient {
             short nextKeySeq = 0;
 
             List<Mutation> columnMetadata = 
Lists.newArrayListWithExpectedSize(columns.size());
+            boolean isRegularView = (tableType == PTableType.VIEW && 
viewType!=ViewType.MAPPED);
             try (PreparedStatement colUpsert = 
connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
                 for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
                     PColumn column = entry.getValue();
@@ -2782,9 +2790,19 @@ public class MetaDataClient {
                         }
                     }
                     Short keySeq = SchemaUtil.isPKColumn(column) ? 
++nextKeySeq : null;
-                    addColumnMutation(schemaName, tableName, column, 
colUpsert, parentTableName, pkName, keySeq, saltBucketNum != null);
-                    
columnMetadata.addAll(connection.getMutationState().toMutations(timestamp).next().getSecond());
-                    connection.rollback();
+                    // Prior to PHOENIX-3534 we were sending the parent table 
column metadata while creating a
+                    // child view, now that we combine columns by resolving 
the parent table hierarchy we
+                    // don't need to include the parent table columns while 
creating a view
+                    // If 
QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK is true we continue
+                    // to store the parent table column metadata along with 
the child view metadata
+                    // so that we can rollback the upgrade if required.
+                    if (allowSystemCatalogRollback || !isRegularView
+                            || columnPosition >= baseTableColumnCount) {
+                        addColumnMutation(schemaName, tableName, column, 
colUpsert, parentTableName,
+                            pkName, keySeq, saltBucketNum != null);
+                        
columnMetadata.addAll(connection.getMutationState().toMutations(timestamp).next().getSecond());
+                        connection.rollback();
+                    }
                 }
             }
             // add the columns in reverse order since we reverse the list later
@@ -2993,7 +3011,7 @@ public class MetaDataClient {
                                 ImmutableStorageScheme.ONE_CELL_PER_COLUMN : 
immutableStorageScheme)
                         .setQualifierEncodingScheme(encodingScheme == null ?
                                 QualifierEncodingScheme.NON_ENCODED_QUALIFIERS 
: encodingScheme)
-                        
.setBaseColumnCount(QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT)
+                        .setBaseColumnCount(baseTableColumnCount)
                         .setEncodedCQCounter(cqCounterToBe)
                         
.setUseStatsForParallelization(useStatsForParallelizationProp)
                         .setExcludedColumns(ImmutableList.<PColumn>of())

Reply via email to