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/f8836f7a Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/f8836f7a Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/f8836f7a Branch: refs/heads/4.x-cdh5.15 Commit: f8836f7a2d12273a1bfdad96a79844d1d7db08e6 Parents: b7e6f2d Author: Thomas D'Silva <tdsi...@apache.org> Authored: Tue Nov 20 20:10:05 2018 +0000 Committer: Pedro Boado <pbo...@apache.org> Committed: Tue Nov 27 15:21:03 2018 +0000 ---------------------------------------------------------------------- .../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/f8836f7a/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 ae09bac..1203f3c 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/f8836f7a/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/f8836f7a/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/f8836f7a/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/f8836f7a/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/f8836f7a/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())