You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by pb...@apache.org on 2018/04/26 21:43:16 UTC

[6/9] 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/33ed01d5
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/33ed01d5
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/33ed01d5

Branch: refs/heads/4.x-cdh5.12
Commit: 33ed01d54da92de4c7eadaa65bd6beebcc4d1b01
Parents: d776ce9
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Apr 26 09:40:35 2018 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Apr 26 10:00:17 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/33ed01d5/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/33ed01d5/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 29eee7e..b77f113 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
@@ -91,6 +91,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.HashSet;
@@ -3163,7 +3164,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<Mutation> 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<List<Cell>>cellLists = m.getFamilyCellMap().values();
+                for (List<Cell> 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<MetaDataResponse> done) {
@@ -3188,37 +3238,40 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     // Size for worst case - all new columns are PK column
                     List<Mutation> 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 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;
-                            } 
+                        // 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/33ed01d5/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<String, Object> 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);
-    
+
 }