From commits-return-20764-archive-asf-public=cust-asf.ponee.io@phoenix.apache.org Thu Apr 26 19:35:27 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 2015A18077B for ; Thu, 26 Apr 2018 19:35:25 +0200 (CEST) Received: (qmail 2695 invoked by uid 500); 26 Apr 2018 17:35:25 -0000 Mailing-List: contact commits-help@phoenix.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.apache.org Delivered-To: mailing list commits@phoenix.apache.org Received: (qmail 2533 invoked by uid 99); 26 Apr 2018 17:35:25 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 26 Apr 2018 17:35:25 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 400A5F32B1; Thu, 26 Apr 2018 17:35:24 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jamestaylor@apache.org To: commits@phoenix.apache.org Date: Thu, 26 Apr 2018 17:35:25 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/6] phoenix git commit: PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/761a60e2 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/761a60e2 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/761a60e2 Branch: refs/heads/4.x-HBase-0.98 Commit: 761a60e2349d3db240ae07492bedd47e391eb78e Parents: e18363c Author: James Taylor Authored: Thu Apr 26 09:40:35 2018 -0700 Committer: James Taylor Committed: Thu Apr 26 10:35:06 2018 -0700 ---------------------------------------------------------------------- .../phoenix/end2end/AlterTableWithViewsIT.java | 22 ++ .../coprocessor/MetaDataEndpointImpl.java | 115 +++++--- .../apache/phoenix/schema/TableProperty.java | 270 ++++++++++--------- 3 files changed, 248 insertions(+), 159 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/761a60e2/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java index 237a8d2..e1b1372 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java @@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.sql.Connection; +import java.sql.DatabaseMetaData; import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -39,12 +40,14 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.coprocessor.TephraTransactionalProcessor; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.PNameFactory; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableKey; import org.apache.phoenix.schema.PTableType; +import org.apache.phoenix.util.StringUtil; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -166,6 +169,25 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT { assertTrue(viewTable2.isImmutableRows()); // update cache frequency is not propagated to the view since it was altered on the view assertEquals(1, viewTable2.getUpdateCacheFrequency()); + + long gpw = 1000000; + conn.createStatement().execute("ALTER TABLE " + tableName + " SET GUIDE_POSTS_WIDTH=" + gpw); + + ResultSet rs; + DatabaseMetaData md = conn.getMetaData(); + rs = md.getTables("", "", StringUtil.escapeLike(tableName), null); + assertTrue(rs.next()); + assertEquals(gpw, rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH)); + + rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable1), null); + assertTrue(rs.next()); + rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH); + assertTrue(rs.wasNull()); + + rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable2), null); + assertTrue(rs.next()); + rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH); + assertTrue(rs.wasNull()); } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/761a60e2/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 3511b0b..1d9c786 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 @@ -90,6 +90,7 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.Iterator; @@ -3022,7 +3023,56 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso byte[] tableNameBytes = index.getTableName().getBytes(); return ByteUtil.concat(tenantIdBytes, SEPARATOR_BYTE_ARRAY, schemaNameBytes, SEPARATOR_BYTE_ARRAY, tableNameBytes); } + + /** + * Determines whether or not we have a change that needs to be propagated from a base table + * to it's views. For example, a change to GUIDE_POSTS_WIDTH does not need to be propogated + * since it's only set on the physical table. + * @param table the table being altered + * @param rowKeyMetaData the filled in values for schemaName and tableName + * @param tableMetaData the metadata passed over from the client + * @return true if changes need to be propagated to the views and false otherwise. + */ + private static boolean hasChangesToPropagate(PTable table, byte[][] rowKeyMetaData, List tableMetaData) { + boolean hasChangesToPropagate = true; + byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX]; + byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX]; + for (Mutation m : tableMetaData) { + byte[] key = m.getRow(); + int pkCount = getVarChars(key, rowKeyMetaData); + if (pkCount >= COLUMN_NAME_INDEX + && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0 + && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0) { + return true; + } else { + Collection>cellLists = m.getFamilyCellMap().values(); + for (List cells : cellLists) { + if (cells != null) { + for (Cell cell : cells) { + byte[] qualifier = CellUtil.cloneQualifier(cell); + String columnName = Bytes.toString(qualifier); + try { + // Often Phoenix table properties aren't valid to be set on a view so thus + // do not need to be propogated. Here we check if the column name corresponds + // to a table property and whether that property is valid to set on a view. + TableProperty tableProp = TableProperty.valueOf(columnName); + if (tableProp.propagateToViews()) { + return true; + } else { + hasChangesToPropagate = false; + } + } catch (IllegalArgumentException e) { + } + } + } + } + } + } + return hasChangesToPropagate; + } + + @Override public void addColumn(RpcController controller, final AddColumnRequest request, RpcCallback done) { @@ -3042,37 +3092,40 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso // Size for worst case - all new columns are PK column List mutationsForAddingColumnsToViews = Lists.newArrayListWithExpectedSize(tableMetaData.size() * ( 1 + table.getIndexes().size())); if (type == PTableType.TABLE || type == PTableType.SYSTEM) { - TableViewFinder childViewsResult = new TableViewFinder(); - findAllChildViews(region, tenantId, table, childViewsResult, clientTimeStamp, request.getClientVersion()); - if (childViewsResult.hasViews()) { - /* - * Dis-allow if: - * 1) The meta-data for child view/s spans over - * more than one region (since the changes cannot be made in a transactional fashion) - * - * 2) The base column count is 0 which means that the metadata hasn't been upgraded yet or - * the upgrade is currently in progress. - * - * 3) 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. - * - * 4) Trying to switch a table that has views from multi-tenant to global. - */ - if (!childViewsResult.allViewsInSingleRegion() - || table.getBaseColumnCount() == 0 - || !request.hasClientVersion() - || switchAttribute(table, table.isMultiTenant(), tableMetaData, MULTI_TENANT_BYTES)) { - return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, - EnvironmentEdgeManager.currentTimeMillis(), null); - } else { - mutationsForAddingColumnsToViews = new ArrayList<>(childViewsResult.getViewInfoList().size() * tableMetaData.size()); - MetaDataMutationResult mutationResult = addColumnsAndTablePropertiesToChildViews(table, tableMetaData, mutationsForAddingColumnsToViews, schemaName, tableName, invalidateList, clientTimeStamp, - childViewsResult, region, locks, request.getClientVersion()); - // return if we were not able to add the column successfully - if (mutationResult!=null) - return mutationResult; - } + // If change doesn't need to be propagated, don't bother finding children + if (hasChangesToPropagate(table, rowKeyMetaData, tableMetaData)) { + TableViewFinder childViewsResult = new TableViewFinder(); + findAllChildViews(region, tenantId, table, childViewsResult, clientTimeStamp, request.getClientVersion()); + if (childViewsResult.hasViews()) { + /* + * Dis-allow if: + * 1) The meta-data for child view/s spans over + * more than one region (since the changes cannot be made in a transactional fashion) + * + * 2) The base column count is 0 which means that the metadata hasn't been upgraded yet or + * the upgrade is currently in progress. + * + * 3) 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. + * + * 4) Trying to swtich tenancy of a table that has views + */ + if (!childViewsResult.allViewsInSingleRegion() + || table.getBaseColumnCount() == 0 + || !request.hasClientVersion() + || switchAttribute(table, table.isMultiTenant(), tableMetaData, MULTI_TENANT_BYTES)) { + return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, + EnvironmentEdgeManager.currentTimeMillis(), null); + } else { + mutationsForAddingColumnsToViews = new ArrayList<>(childViewsResult.getViewInfoList().size() * tableMetaData.size()); + MetaDataMutationResult mutationResult = addColumnsAndTablePropertiesToChildViews(table, tableMetaData, mutationsForAddingColumnsToViews, schemaName, tableName, invalidateList, clientTimeStamp, + childViewsResult, region, locks, request.getClientVersion()); + // return if we were not able to add the column successfully + if (mutationResult!=null) + return mutationResult; + } + } } } else if (type == PTableType.VIEW && EncodedColumnsUtil.usesEncodedColumnNames(table)) { http://git-wip-us.apache.org/repos/asf/phoenix/blob/761a60e2/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java index 78b9beb..3d473c4 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java @@ -37,44 +37,44 @@ import org.apache.phoenix.transaction.TransactionFactory; import org.apache.phoenix.util.SchemaUtil; public enum TableProperty { - + @Deprecated // use the IMMUTABLE keyword while creating the table - IMMUTABLE_ROWS(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, true, true, false) { + IMMUTABLE_ROWS(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, true, true, false) { @Override public Object getPTableValue(PTable table) { return table.isImmutableRows(); } }, - MULTI_TENANT(PhoenixDatabaseMetaData.MULTI_TENANT, true, false, false) { + MULTI_TENANT(PhoenixDatabaseMetaData.MULTI_TENANT, true, false, false) { @Override public Object getPTableValue(PTable table) { return table.isMultiTenant(); } }, - DISABLE_WAL(PhoenixDatabaseMetaData.DISABLE_WAL, true, false, false) { + DISABLE_WAL(PhoenixDatabaseMetaData.DISABLE_WAL, true, false, false) { @Override public Object getPTableValue(PTable table) { return table.isWALDisabled(); } }, - SALT_BUCKETS(PhoenixDatabaseMetaData.SALT_BUCKETS, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, SALT_ONLY_ON_CREATE_TABLE, false, false) { + SALT_BUCKETS(PhoenixDatabaseMetaData.SALT_BUCKETS, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, SALT_ONLY_ON_CREATE_TABLE, false, false) { @Override public Object getPTableValue(PTable table) { return table.getBucketNum(); } }, - DEFAULT_COLUMN_FAMILY(DEFAULT_COLUMN_FAMILY_NAME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE, false, false) { + DEFAULT_COLUMN_FAMILY(DEFAULT_COLUMN_FAMILY_NAME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE, false, false) { @Override public Object getPTableValue(PTable table) { return table.getDefaultFamilyName(); } }, - TTL(HColumnDescriptor.TTL, COLUMN_FAMILY_NOT_ALLOWED_FOR_TTL, true, CANNOT_ALTER_PROPERTY, false, false) { + TTL(HColumnDescriptor.TTL, COLUMN_FAMILY_NOT_ALLOWED_FOR_TTL, true, CANNOT_ALTER_PROPERTY, false, false) { @Override public Object getPTableValue(PTable table) { return null; @@ -87,14 +87,14 @@ public enum TableProperty { return table.getStoreNulls(); } }, - + TRANSACTIONAL(PhoenixDatabaseMetaData.TRANSACTIONAL, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false) { @Override public Object getPTableValue(PTable table) { return table.isTransactional(); } }, - + TRANSACTION_PROVIDER(PhoenixDatabaseMetaData.TRANSACTION_PROVIDER, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false) { @Override public Object getPTableValue(PTable table) { @@ -113,28 +113,28 @@ public enum TableProperty { }, UPDATE_CACHE_FREQUENCY(PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY, true, true, true) { - @Override + @Override public Object getValue(Object value) { - if (value instanceof String) { - String strValue = (String) value; - if ("ALWAYS".equalsIgnoreCase(strValue)) { - return 0L; - } else if ("NEVER".equalsIgnoreCase(strValue)) { - return Long.MAX_VALUE; - } - } else { - return value == null ? null : ((Number) value).longValue(); - } - return value; - } + if (value instanceof String) { + String strValue = (String) value; + if ("ALWAYS".equalsIgnoreCase(strValue)) { + return 0L; + } else if ("NEVER".equalsIgnoreCase(strValue)) { + return Long.MAX_VALUE; + } + } else { + return value == null ? null : ((Number) value).longValue(); + } + return value; + } @Override public Object getPTableValue(PTable table) { return table.getUpdateCacheFrequency(); - } - }, - - AUTO_PARTITION_SEQ(PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, false, false) { + } + }, + + AUTO_PARTITION_SEQ(PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, false, false) { @Override public Object getValue(Object value) { return value == null ? null : SchemaUtil.normalizeIdentifier(value.toString()); @@ -144,15 +144,15 @@ public enum TableProperty { public Object getPTableValue(PTable table) { return table.getAutoPartitionSeqName(); } - }, - - APPEND_ONLY_SCHEMA(PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, true, false) { + }, + + APPEND_ONLY_SCHEMA(PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, true, false) { @Override public Object getPTableValue(PTable table) { return table.isAppendOnlySchema(); } }, - GUIDE_POSTS_WIDTH(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH, true, false, false) { + GUIDE_POSTS_WIDTH(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH, true, false, false, false) { @Override public Object getValue(Object value) { return value == null ? null : ((Number) value).longValue(); @@ -162,30 +162,30 @@ public enum TableProperty { public Object getPTableValue(PTable table) { return null; } - - }, - - COLUMN_ENCODED_BYTES(PhoenixDatabaseMetaData.ENCODING_SCHEME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, false, false) { - @Override + + }, + + COLUMN_ENCODED_BYTES(PhoenixDatabaseMetaData.ENCODING_SCHEME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, false, false) { + @Override public Object getValue(Object value) { - if (value instanceof String) { - String strValue = (String) value; - if ("NONE".equalsIgnoreCase(strValue)) { - return (byte)0; - } - } else { - return value == null ? null : ((Number) value).byteValue(); - } - return value; - } - - @Override - public Object getPTableValue(PTable table) { - return table.getEncodingScheme(); - } - - }, - + if (value instanceof String) { + String strValue = (String) value; + if ("NONE".equalsIgnoreCase(strValue)) { + return (byte)0; + } + } else { + return value == null ? null : ((Number) value).byteValue(); + } + return value; + } + + @Override + public Object getPTableValue(PTable table) { + return table.getEncodingScheme(); + } + + }, + IMMUTABLE_STORAGE_SCHEME(PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false) { @Override public ImmutableStorageScheme getValue(Object value) { @@ -203,7 +203,7 @@ public enum TableProperty { public Object getPTableValue(PTable table) { return table.getImmutableStorageScheme(); } - + }, USE_STATS_FOR_PARALLELIZATION(PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION, true, true, true) { @@ -224,81 +224,91 @@ public enum TableProperty { } } ; - - private final String propertyName; - private final SQLExceptionCode colFamSpecifiedException; - private final boolean isMutable; // whether or not a property can be changed through statements like ALTER TABLE. - private final SQLExceptionCode mutatingImmutablePropException; - private final boolean isValidOnView; - private final boolean isMutableOnView; - - private TableProperty(String propertyName, boolean isMutable, boolean isValidOnView, boolean isMutableOnView) { - this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView); - } - - private TableProperty(String propertyName, SQLExceptionCode colFamilySpecifiedException, boolean isMutable, boolean isValidOnView, boolean isMutableOnView) { - this(propertyName, colFamilySpecifiedException, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView); - } - - private TableProperty(String propertyName, boolean isMutable, boolean isValidOnView, boolean isMutableOnView, SQLExceptionCode isMutatingException) { - this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, isMutatingException, isValidOnView, isMutableOnView); - } - - private TableProperty(String propertyName, SQLExceptionCode colFamSpecifiedException, boolean isMutable, SQLExceptionCode mutatingException, boolean isValidOnView, boolean isMutableOnView) { - this.propertyName = propertyName; - this.colFamSpecifiedException = colFamSpecifiedException; - this.isMutable = isMutable; - this.mutatingImmutablePropException = mutatingException; - this.isValidOnView = isValidOnView; - this.isMutableOnView = isMutableOnView; - } - - public static boolean isPhoenixTableProperty(String property) { - try { - TableProperty.valueOf(property); - } catch (IllegalArgumentException e) { - return false; - } - return true; - } - - public Object getValue(Object value) { - return value; - } - + + private final String propertyName; + private final SQLExceptionCode colFamSpecifiedException; + private final boolean isMutable; // whether or not a property can be changed through statements like ALTER TABLE. + private final SQLExceptionCode mutatingImmutablePropException; + private final boolean isValidOnView; + private final boolean isMutableOnView; + private final boolean propagateToViews; + + private TableProperty(String propertyName, boolean isMutable, boolean isValidOnView, boolean isMutableOnView) { + this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, true); + } + + private TableProperty(String propertyName, boolean isMutable, boolean isValidOnView, boolean isMutableOnView, boolean propagateToViews) { + this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, propagateToViews); + } + + private TableProperty(String propertyName, SQLExceptionCode colFamilySpecifiedException, boolean isMutable, boolean isValidOnView, boolean isMutableOnView) { + this(propertyName, colFamilySpecifiedException, isMutable, CANNOT_ALTER_PROPERTY, isValidOnView, isMutableOnView, true); + } + + private TableProperty(String propertyName, boolean isMutable, boolean isValidOnView, boolean isMutableOnView, SQLExceptionCode isMutatingException) { + this(propertyName, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, isMutable, isMutatingException, isValidOnView, isMutableOnView, true); + } + + private TableProperty(String propertyName, SQLExceptionCode colFamSpecifiedException, boolean isMutable, SQLExceptionCode mutatingException, boolean isValidOnView, boolean isMutableOnView) { + this(propertyName, colFamSpecifiedException, isMutable, mutatingException, isValidOnView, isMutableOnView, true); + } + + private TableProperty(String propertyName, SQLExceptionCode colFamSpecifiedException, boolean isMutable, SQLExceptionCode mutatingException, boolean isValidOnView, boolean isMutableOnView, boolean propagateToViews) { + this.propertyName = propertyName; + this.colFamSpecifiedException = colFamSpecifiedException; + this.isMutable = isMutable; + this.mutatingImmutablePropException = mutatingException; + this.isValidOnView = isValidOnView; + this.isMutableOnView = isMutableOnView; + this.propagateToViews = propagateToViews; + } + + public static boolean isPhoenixTableProperty(String property) { + try { + TableProperty.valueOf(property); + } catch (IllegalArgumentException e) { + return false; + } + return true; + } + + public Object getValue(Object value) { + return value; + } + public Object getValue(Map props) { return getValue(props.get(this.toString())); } - - // isQualified is true if column family name is specified in property name - public void validate(boolean isMutating, boolean isQualified, PTableType tableType) throws SQLException { - checkForColumnFamily(isQualified); - checkIfApplicableForView(tableType); - checkForMutability(isMutating,tableType); - } - - private void checkForColumnFamily(boolean isQualified) throws SQLException { - if (isQualified) { - throw new SQLExceptionInfo.Builder(colFamSpecifiedException).setMessage(". Property: " + propertyName).build().buildException(); - } - } - - private void checkForMutability(boolean isMutating, PTableType tableType) throws SQLException { - if (isMutating && !isMutable) { - throw new SQLExceptionInfo.Builder(mutatingImmutablePropException).setMessage(". Property: " + propertyName).build().buildException(); - } - if (isMutating && tableType == PTableType.VIEW && !isMutableOnView) { - throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW).setMessage(". Property: " + propertyName).build().buildException(); - } - } - - private void checkIfApplicableForView(PTableType tableType) - throws SQLException { - if (tableType == PTableType.VIEW && !isValidOnView) { - throw new SQLExceptionInfo.Builder( - VIEW_WITH_PROPERTIES).setMessage("Property: " + propertyName).build().buildException(); - } - } + + // isQualified is true if column family name is specified in property name + public void validate(boolean isMutating, boolean isQualified, PTableType tableType) throws SQLException { + checkForColumnFamily(isQualified); + checkIfApplicableForView(tableType); + checkForMutability(isMutating,tableType); + } + + private void checkForColumnFamily(boolean isQualified) throws SQLException { + if (isQualified) { + throw new SQLExceptionInfo.Builder(colFamSpecifiedException).setMessage(". Property: " + propertyName).build().buildException(); + } + } + + private void checkForMutability(boolean isMutating, PTableType tableType) throws SQLException { + if (isMutating && !isMutable) { + throw new SQLExceptionInfo.Builder(mutatingImmutablePropException).setMessage(". Property: " + propertyName).build().buildException(); + } + if (isMutating && tableType == PTableType.VIEW && !isMutableOnView) { + throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW).setMessage(". Property: " + propertyName).build().buildException(); + } + } + + private void checkIfApplicableForView(PTableType tableType) + throws SQLException { + if (tableType == PTableType.VIEW && !isValidOnView) { + throw new SQLExceptionInfo.Builder( + VIEW_WITH_PROPERTIES).setMessage("Property: " + propertyName).build().buildException(); + } + } public String getPropertyName() { return propertyName; @@ -315,7 +325,11 @@ public enum TableProperty { public boolean isMutableOnView() { return isMutableOnView; } - + + public boolean propagateToViews() { + return propagateToViews; + } + abstract public Object getPTableValue(PTable table); - + }