You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2015/07/09 00:36:24 UTC

phoenix git commit: PHOENIX-2058 Check for existence and compatibility of columns being added in view

Repository: phoenix
Updated Branches:
  refs/heads/master 61f1900dd -> 8c9a6b8cb


PHOENIX-2058 Check for existence and compatibility of columns being added in view


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

Branch: refs/heads/master
Commit: 8c9a6b8cb94c0380c8e99aeee18d3b7d8f6e37ef
Parents: 61f1900
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Jul 2 21:19:11 2015 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Wed Jul 8 15:36:00 2015 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    | 221 +++++++++++++++++--
 .../coprocessor/MetaDataEndpointImpl.java       | 108 ++++++++-
 .../apache/phoenix/schema/MetaDataClient.java   |   4 +-
 3 files changed, 302 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8c9a6b8c/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index 56bba9b..fbaded0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -64,6 +64,7 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.base.Objects;
@@ -2009,7 +2010,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
     }
     
     @Test
-    public void testAddColumnToTableWithViews() throws Exception {
+    public void testAddNewColumnToBaseTableWithViews() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         try {       
             conn.createStatement().execute("CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
@@ -2020,12 +2021,176 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
                     + " )");
             assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, -1, "ID", "COL1", "COL2");
             
-            conn.createStatement().execute("CREATE VIEW VIEWOFTABLE ( VIEW_COL1 SMALLINT ) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 4, 3, "ID", "COL1", "COL2", "VIEW_COL1");
+            conn.createStatement().execute("CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM TABLEWITHVIEW");
+            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD COL3 char(10)");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 5, 4, "ID", "COL1", "COL2", "COL3", "VIEW_COL1");
+            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 6, 4, "ID", "COL1", "COL2", "COL3", "VIEW_COL1", "VIEW_COL2");
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testAddExistingViewColumnToBaseTableWithViews() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {       
+            conn.createStatement().execute("CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
+                    + " ID char(10) NOT NULL,"
+                    + " COL1 integer NOT NULL,"
+                    + " COL2 bigint NOT NULL,"
+                    + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
+                    + " )");
+            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, -1, "ID", "COL1", "COL2");
+            
+            conn.createStatement().execute("CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256), VIEW_COL3 VARCHAR, VIEW_COL4 DECIMAL ) AS SELECT * FROM TABLEWITHVIEW");
+            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 7, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4");
+            
+            // upsert single row into view
+            String dml = "UPSERT INTO VIEWOFTABLE VALUES(?,?,?,?,?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "view1");
+            stmt.setInt(2, 12);
+            stmt.setInt(3, 13);
+            stmt.setInt(4, 14);
+            stmt.setString(5, "view5");
+            stmt.setString(6, "view6");
+            stmt.setInt(7, 17);
+            stmt.execute();
+            conn.commit();
+            
+            try {
+            	// should fail because there is already a view column with same name of different type
+            	conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 char(10)");
+            	fail();
+            }
+            catch (SQLException e) {
+            	assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            }           
+            
+            try {
+            	// should fail because there is already a view column with same name with a higher scale
+            	conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,1)");
+            	fail();
+            }
+            catch (SQLException e) {
+            	assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            } 
+            
+            try {
+            	// should fail because there is already a view column with same name with a greater length
+            	conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 VARCHAR(256)");
+            	fail();
+            }
+            catch (SQLException e) {
+            	assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            } 
+            
+            try {
+            	// should fail because there is already a view column with null length
+            	conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL2 DECIMAL(9,2)");
+            	fail();
+            }
+            catch (SQLException e) {
+            	assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            }
+            
+            // validate that there were no columns added to the table or view
+            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, -1, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 7, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4");
+            
+            // should succeed because even though there already is a view column with same name,
+            // the column being added has a scale and max length that is equal to or greater
+            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL2 VARCHAR, VIEW_COL4 DECIMAL");
+            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 1, 4, -1, "ID", "COL1", "COL2", "VIEW_COL2", "VIEW_COL4");
+            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 7, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4");
+            
+            // query table
+            ResultSet rs = stmt.executeQuery("SELECT * FROM TABLEWITHVIEW");
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), "view1");
+            assertEquals(rs.getInt(2), 12);
+            assertEquals(rs.getInt(3), 13);
+            assertEquals(rs.getString(4), "view5");
+            assertEquals(rs.getInt(5), 17);
+            assertFalse(rs.next());
+
+            // query view
+            rs = stmt.executeQuery("SELECT * FROM VIEWOFTABLE");
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), "view1");
+            assertEquals(rs.getInt(2), 12);
+            assertEquals(rs.getInt(3), 13);
+            assertEquals(rs.getInt(4), 14);
+            assertEquals(rs.getString(5),"view5");
+            assertEquals(rs.getString(6),"view6");
+            assertEquals(rs.getInt(7), 17);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    @Ignore
+    // enable this test after https://issues.apache.org/jira/browse/PHOENIX-978 is fixed 
+    public void testAddExistingViewPkColumnToBaseTableWithViews() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        try {       
+            conn.createStatement().execute("CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
+                    + " ID char(10) NOT NULL,"
+                    + " COL1 integer NOT NULL,"
+                    + " COL2 bigint NOT NULL,"
+                    + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
+                    + " )");
+            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, -1, "ID", "COL1", "COL2");
+            
+            conn.createStatement().execute("CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM TABLEWITHVIEW");
+            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
+            // upsert single row into view
+            String dml = "UPSERT INTO VIEWOFTABLE VALUES(?,?,?,?,?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "view1");
+            stmt.setInt(2, 12);
+            stmt.setInt(3, 13);
+            stmt.setInt(4, 14);
+            stmt.setString(5, "view5");
+            stmt.execute();
+            conn.commit();
+            
+            try {
+            	// should fail because there is already a pk view column with same name is a different slot 
+            	conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL2 VARCHAR PRIMARY KEY");
+            	fail();
+            }
+            catch (SQLException e) {
+            	assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            }
+            
+            // add the pk column of the view to the base table
+            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL PRIMARY KEY");
+            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 1, 4, -1, "ID", "COL1", "COL2", "VIEW_COL1");
+            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 5, 4, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            
+            // query table
+            ResultSet rs = stmt.executeQuery("SELECT * FROM TABLEWITHVIEW");
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), "view1");
+            assertEquals(rs.getInt(2), 12);
+            assertEquals(rs.getInt(3), 13);
+            assertEquals(rs.getInt(4), 14);
+            assertFalse(rs.next());
+
+            // query view
+            rs = stmt.executeQuery("SELECT * FROM VIEWOFTABLE");
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), "view1");
+            assertEquals(rs.getInt(2), 12);
+            assertEquals(rs.getInt(3), 13);
+            assertEquals(rs.getInt(4), 14);
+            assertEquals(rs.getString(5),"view5");
+            assertFalse(rs.next());
         } finally {
             conn.close();
         }
@@ -2045,34 +2210,46 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         ResultSet parentTableColumnsRs = null; 
         if (parentTableName != null) {
             parentTableColumnsRs = conn.getMetaData().getColumns(null, null, parentTableName, null);
+            parentTableColumnsRs.next();
         }
         
-        rs = conn.getMetaData().getColumns(null, null, tableName, null);
+        ResultSet viewColumnsRs = conn.getMetaData().getColumns(null, null, tableName, null);
         for (int i = 0; i < columnName.length; i++) {
             if (columnName[i] != null) {
-                assertTrue(rs.next());
-                assertEquals(getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in columnName: i=" + i), columnName[i], rs.getString("COLUMN_NAME"));
-                assertEquals(getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in ordinalPosition: i=" + i), i+1, rs.getInt("ORDINAL_POSITION"));
-                if (i < baseColumnCount && parentTableColumnsRs != null) {
-                    assertTrue(parentTableColumnsRs.next());
-                    ResultSetMetaData md = parentTableColumnsRs.getMetaData();
-                    assertEquals(md.getColumnCount(), rs.getMetaData().getColumnCount());
-                    for (int columnIndex = 1; columnIndex < md.getColumnCount(); columnIndex++) {
-                        String viewColumnValue = rs.getString(columnIndex);
+                assertTrue(viewColumnsRs.next());
+                assertEquals(getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in columnName: i=" + i), columnName[i], viewColumnsRs.getString(PhoenixDatabaseMetaData.COLUMN_NAME));
+                assertEquals(getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in ordinalPosition: i=" + i), i+1, viewColumnsRs.getInt(PhoenixDatabaseMetaData.ORDINAL_POSITION));
+                // validate that all the columns in the base table are present in the view   
+                if (parentTableColumnsRs != null && !parentTableColumnsRs.isAfterLast()) {
+                    ResultSetMetaData parentTableColumnsMetadata = parentTableColumnsRs.getMetaData();
+                    assertEquals(parentTableColumnsMetadata.getColumnCount(), viewColumnsRs.getMetaData().getColumnCount());
+                    
+                    // if you add a non-pk column that already exists in the view
+                    if (!viewColumnsRs.getString(PhoenixDatabaseMetaData.COLUMN_NAME).equals(parentTableColumnsRs.getString(PhoenixDatabaseMetaData.COLUMN_NAME))) {
+                    	continue;
+                    }
+                    
+                    for (int columnIndex = 1; columnIndex < parentTableColumnsMetadata.getColumnCount(); columnIndex++) {
+                        String viewColumnValue = viewColumnsRs.getString(columnIndex);
                         String parentTableColumnValue = parentTableColumnsRs.getString(columnIndex);
                         if (!Objects.equal(viewColumnValue, parentTableColumnValue)) {
-                            if (md.getColumnName(columnIndex).equals("TABLE_NAME")) {
+                            if (parentTableColumnsMetadata.getColumnName(columnIndex).equals(PhoenixDatabaseMetaData.TABLE_NAME)) {
                                 assertEquals(parentTableName, parentTableColumnValue);
                                 assertEquals(tableName, viewColumnValue);
-                            } else {
-                                fail(md.getColumnName(columnIndex) + "=" + parentTableColumnValue);
+                            } 
+                            else if (parentTableColumnsMetadata.getColumnName(columnIndex).equals(PhoenixDatabaseMetaData.ORDINAL_POSITION) && parentTableColumnsRs.getString("COLUMN_FAMILY")!=null) {
+                            	// its ok if the ordinal positions don't match for non-pk columns 
+                            }
+                            else {
+                                fail(parentTableColumnsMetadata.getColumnName(columnIndex) + " of base table " + parentTableColumnValue + " does not match view "+viewColumnValue) ;
                             }
                         }
                     }
+                    parentTableColumnsRs.next();
                 }
             }
         }
-        assertFalse(getSystemCatalogEntriesForTable(conn, tableName, ""), rs.next());
+        assertFalse(getSystemCatalogEntriesForTable(conn, tableName, ""), viewColumnsRs.next());
     }
     
     private String getSystemCatalogEntriesForTable(Connection conn, String tableName, String message) throws Exception {
@@ -2207,7 +2384,13 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
             conn.createStatement().execute(dropColumn);
             
             String alterBaseTable = "ALTER TABLE " + baseTable + " ADD V3 VARCHAR";
-            conn.createStatement().execute(alterBaseTable);
+            try {
+            	conn.createStatement().execute(alterBaseTable);
+	            fail();
+	        }
+	        catch (SQLException e) {
+	        	assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+	        }
             
             // Column V3 shouldn't have propagated to the divorced view.
             String sql = "SELECT V3 FROM " + viewName;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8c9a6b8c/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 1fc1d7f..b27beb4 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
@@ -39,6 +39,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_VIEW_REFERENCED_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.JAR_PATH_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_VALUE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MIN_VALUE_BYTES;
@@ -1580,8 +1581,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             return null; // impossible
         }
     }
+    
+    private boolean isValidLengthOrScale(Integer viewColVal, Cell cell) {
+    	Integer baseColValue = cell.getValueArray()==null ? null :
+			PInteger.INSTANCE.getCodec().decodeInt(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+    	if ( (baseColValue!=null && viewColVal!=null && baseColValue<viewColVal) || // if the base column has a scale/length less then the view column
+    			(viewColVal==null && baseColValue!=null)) {  // or if the view column has no scale/length, but the base column has a scale/length
+    		return false;
+    	}
+    	return true;
+    }
 
-    private void addRowsToChildViews(List<Mutation> tableMetadata, List<Mutation> mutationsForAddingColumnsToViews, byte[] schemaName, byte[] tableName,
+    private MetaDataMutationResult addRowsToChildViews(List<Mutation> tableMetadata, List<Mutation> mutationsForAddingColumnsToViews, byte[] schemaName, byte[] tableName,
             List<ImmutableBytesPtr> invalidateList, long clientTimeStamp, TableViewFinderResult childViewsResult,
             Region region, List<RowLock> locks) throws IOException, SQLException {
         for (Result viewResult : childViewsResult.getResults()) {
@@ -1597,18 +1608,28 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             if (view.getBaseColumnCount() == QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT) {
                 // if a view has divorced itself from the base table, we don't allow schema changes
                 // to be propagated to it.
-                return;
+            	return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
             }
-            int deltaNumberOfColumns = 0;
+            int numColsAddedToBaseTable = 0;
+            int numColsAddedToView = 0;
             short deltaNumPkColsSoFar = 0;
+            PColumn existingViewColumn = null;
             for (Mutation m : tableMetadata) {
                 byte[][] rkmd = new byte[5][];
                 int pkCount = getVarChars(m.getRow(), rkmd);
                 if (m instanceof Put && pkCount > COLUMN_NAME_INDEX
                         && Bytes.compareTo(schemaName, rkmd[SCHEMA_NAME_INDEX]) == 0
                         && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) == 0) {
+                	try {
+                        // Maybe deserving of a new SchemaUtil.getColumnByName(byte[] familyName, String columnName) function 
+                        String columnName = Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
+                        existingViewColumn = rkmd[FAMILY_NAME_INDEX] == null 
+                            ? view.getPKColumn(columnName) 
+                            : view.getColumnFamily(rkmd[FAMILY_NAME_INDEX]).getColumn(columnName);
+                    } catch (ColumnNotFoundException e) {
+                    } // Ignore - means column family or column name don't exist
+                	
                     Put p = (Put)m;
-
                     byte[] columnKey = ByteUtil.concat(viewKey, QueryConstants.SEPARATOR_BYTE_ARRAY, rkmd[COLUMN_NAME_INDEX]);
                     if (rkmd[FAMILY_NAME_INDEX] != null) {
                         columnKey = ByteUtil.concat(columnKey, QueryConstants.SEPARATOR_BYTE_ARRAY, rkmd[FAMILY_NAME_INDEX]);
@@ -1619,7 +1640,70 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                 CellUtil.cloneQualifier(cell), cell.getTimestamp(), cell.getTypeByte(),
                                 CellUtil.cloneValue(cell)));
                     }
-                    deltaNumberOfColumns++;
+                    
+                    // if there is already a view column with the same name as the base table column we are trying to add
+                	if (existingViewColumn!=null) {
+                		List<Cell> dataTypes = viewColumnDefinitionPut
+                                .get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                        PhoenixDatabaseMetaData.DATA_TYPE_BYTES);
+                		if (dataTypes != null && dataTypes.size() > 0) {
+                            Cell cell = dataTypes.get(0);
+                            int typeId = PInteger.INSTANCE.getCodec().decodeInt(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+                    		PDataType dataType = PDataType.fromTypeId(typeId);
+                    		if (!existingViewColumn.getDataType().equals(dataType)) {
+                    			 return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
+                    		}
+                		}
+                		if (rkmd[FAMILY_NAME_INDEX] == null && rkmd[COLUMN_NAME_INDEX] != null) {
+	                        List<Cell> keySeqCells = viewColumnDefinitionPut.get(
+	                                PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+	                                PhoenixDatabaseMetaData.KEY_SEQ_BYTES);
+	                        if (keySeqCells != null && keySeqCells.size() > 0) {
+	                        	Cell cell = keySeqCells.get(0);
+	                        	int keySeq = PSmallint.INSTANCE.getCodec().decodeInt(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+	                     		int pkPosition = SchemaUtil.getPKPosition(view, existingViewColumn)+1;
+	                     		if (pkPosition!=keySeq) {
+	                     			return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
+	                     		}
+	                        }
+                        }
+                		// if there is an existing view column that matches the column being added to the base table and if the column being added has a null
+                    	// scale or maxLength, we need to explicity do a put to set the scale or maxLength to null (in case the view column has the scale or 
+                    	// max length set)
+                		List<Cell> columnSizes = viewColumnDefinitionPut.get(
+                                PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES);
+                        if (columnSizes != null && columnSizes.size() > 0) {
+                            Cell cell = columnSizes.get(0);
+                            if (!isValidLengthOrScale(existingViewColumn.getMaxLength(),cell)) {
+                            	return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null); 
+                            }
+                        }
+                        else if (existingViewColumn.getMaxLength()!=null) {
+                        	viewColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                    PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES, null);
+                        }
+                        List<Cell> decimalDigits = viewColumnDefinitionPut.get(
+                                PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES);
+                        if (decimalDigits != null && decimalDigits.size() > 0) {
+                            Cell cell = decimalDigits.get(0);
+                            if (!isValidLengthOrScale(existingViewColumn.getScale(),cell)) {
+                            	return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null); 
+                            }
+                        }
+                        else if (existingViewColumn.getScale()!=null) {
+                        	viewColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                    PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES, null);
+                        }
+                	}
+                	else {
+                    	// if we are adding a column that already exists in the view, no need to updates the base table or view table column count
+                    	numColsAddedToView++;
+                    } 
+                    	
+                	
+                    numColsAddedToBaseTable++;
                     mutationsForAddingColumnsToViews.add(viewColumnDefinitionPut);
                     if (rkmd[FAMILY_NAME_INDEX] == null && rkmd[COLUMN_NAME_INDEX] != null) {
                         /*
@@ -1743,9 +1827,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
             Put viewHeaderRowPut = new Put(viewKey, clientTimeStamp);
             byte[] baseColumnCountPtr = new byte[PInteger.INSTANCE.getByteSize()];
-            PInteger.INSTANCE.getCodec().encodeInt(oldBaseColumnCount + deltaNumberOfColumns, baseColumnCountPtr, 0);
+            PInteger.INSTANCE.getCodec().encodeInt(oldBaseColumnCount + numColsAddedToBaseTable, baseColumnCountPtr, 0);
             byte[] columnCountPtr = new byte[PInteger.INSTANCE.getByteSize()];
-            PInteger.INSTANCE.getCodec().encodeInt(view.getColumns().size() + deltaNumberOfColumns, columnCountPtr, 0);
+            PInteger.INSTANCE.getCodec().encodeInt(view.getColumns().size() + numColsAddedToView, columnCountPtr, 0);
             byte[] viewSequencePtr = new byte[PLong.INSTANCE.getByteSize()];
             PLong.INSTANCE.getCodec().encodeLong(view.getSequenceNumber() + 1, viewSequencePtr, 0);
             viewHeaderRowPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
@@ -1759,7 +1843,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             // Update positions of view columns
             for (PColumn column : view.getColumns()) {
                 if (column.getPosition() >= oldBaseColumnCount) {
-                    int newPosition = column.getPosition() + deltaNumberOfColumns + 1;
+                    int newPosition = column.getPosition() + numColsAddedToView + 1;
 
                     byte[] k = ByteUtil.concat(viewKey, QueryConstants.SEPARATOR_BYTE_ARRAY, column.getName()
                             .getBytes());
@@ -1777,6 +1861,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             invalidateList.add(new ImmutableBytesPtr(viewKey));
         }
+        return null;
     }
     
     private byte[] getViewIndexHeaderRowKey(PTable index) {
@@ -1820,9 +1905,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                         EnvironmentEdgeManager.currentTimeMillis(), null);
                             } else {
                                 mutationsForAddingColumnsToViews = new ArrayList<>(childViewsResult.getResults().size() * tableMetaData.size());
-                                addRowsToChildViews(tableMetaData, mutationsForAddingColumnsToViews, schemaName, tableName, invalidateList, clientTimeStamp,
+                                MetaDataMutationResult mutationResult = addRowsToChildViews(tableMetaData, mutationsForAddingColumnsToViews, schemaName, tableName, invalidateList, clientTimeStamp,
                                         childViewsResult, region, locks);
-                            }
+                                // return if we were not able to add the column successfully
+                                if (mutationResult!=null)
+                                	return mutationResult;
+                            } 
                         }
                     }
                     for (Mutation m : tableMetaData) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8c9a6b8c/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 f24da44..8a1c14b 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
@@ -2204,10 +2204,10 @@ public class MetaDataClient {
             String familyName = null;
             String msg = null;
             // TODO: better to return error code
-            if (result.getColumnName() != null) {
+            if (result.getColumnName() != null && result.getColumnName() != null) {
                 familyName = result.getFamilyName() == null ? null : Bytes.toString(result.getFamilyName());
                 columnName = Bytes.toString(result.getColumnName());
-                msg = "Cannot drop column referenced by VIEW";
+                msg = "Cannot add/drop column referenced by VIEW";
             }
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
                 .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();