You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/07/01 03:22:08 UTC

[1/2] phoenix git commit: PHOENIX-2050 Avoid checking for child views unless operating on table

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 100313b20 -> 05911ac51


PHOENIX-2050 Avoid checking for child views unless operating
on table


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 84a17ce9f5513c04c8c75a50575896a8a0a74a56
Parents: 100313b
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Jun 29 21:36:19 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Jun 30 18:11:28 2015 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    |  18 +--
 .../coprocessor/MetaDataEndpointImpl.java       | 141 ++++++++++---------
 2 files changed, 81 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/84a17ce9/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 946aaab..cd46927 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
@@ -2149,23 +2149,13 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
                     "CREATE VIEW " + grandChildView + " AS SELECT * FROM " + childView;
             conn.createStatement().execute(grandChildViewDDL);
 
-            // dropping base table column from child view should fail
+            // dropping base table column from child view should succeed
             String dropColumnFromChildView = "ALTER VIEW " + childView + " DROP COLUMN V2";
-            try {
-                conn.createStatement().execute(dropColumnFromChildView);
-                fail("Dropping columns from a view that has child views on it is not allowed");
-            } catch (SQLException e) {
-                assertEquals(CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-            }
+            conn.createStatement().execute(dropColumnFromChildView);
 
-            // dropping view specific column from child view should fail
+            // dropping view specific column from child view should succeed
             dropColumnFromChildView = "ALTER VIEW " + childView + " DROP COLUMN CHILD_VIEW_COL";
-            try {
-                conn.createStatement().execute(dropColumnFromChildView);
-                fail("Dropping columns from a view that has child views on it is not allowed");
-            } catch (SQLException e) {
-                assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-            }
+            conn.createStatement().execute(dropColumnFromChildView);
             
             // Adding column to view that has child views is allowed
             String addColumnToChildView = "ALTER VIEW " + childView + " ADD V5 VARCHAR";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/84a17ce9/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 e13c635..5876290 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
@@ -1361,69 +1361,76 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         List<byte[]> indexNames = Lists.newArrayList();
         List<Cell> results = Lists.newArrayList();
         try (RegionScanner scanner = region.getScanner(scan);) {
-          scanner.next(results);
-          if (results.isEmpty()) { // Should not be possible
-            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
-          }
+            scanner.next(results);
+            if (results.isEmpty()) { // Should not be possible
+                return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
+                        EnvironmentEdgeManager.currentTimeMillis(), null);
+            }
 
-          // Handle any child views that exist
-          TableViewFinderResult tableViewFinderResult = findChildViews(region, tenantId, table, PHYSICAL_TABLE_BYTES);
-          if (tableViewFinderResult.hasViews()) {
-            if (isCascade) {
-              if (tableViewFinderResult.allViewsInMultipleRegions()) {
-                // We don't yet support deleting a table with views where SYSTEM.CATALOG has split and the
-                // view metadata spans multiple regions
-                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
-              } else if (tableViewFinderResult.allViewsInSingleRegion()) {
-                // Recursively delete views - safe as all the views as all in the same region
-                for (Result viewResult : tableViewFinderResult.getResults()) {
-                  byte[][] rowKeyMetaData = new byte[3][];
-                  getVarChars(viewResult.getRow(), 3, rowKeyMetaData);
-                  byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
-                  byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
-                  byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
-                  byte[] viewKey = SchemaUtil.getTableKey(viewTenantId, viewSchemaName, viewName);
-                  Delete delete = new Delete(viewKey, clientTimeStamp);
-                  rowsToDelete.add(delete);
-                  acquireLock(region, viewKey, locks);
-                  MetaDataMutationResult result =
-                      doDropTable(viewKey, viewTenantId, viewSchemaName, viewName, null, PTableType.VIEW,
-                        rowsToDelete, invalidateList, locks, tableNamesToDelete, false);
-                  if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
-                    return result;
-                  }
+            // Only tables may have views, so prevent the running of this potentially
+            // expensive full table scan over the SYSTEM.CATALOG table unless it's needed.
+            if (tableType == PTableType.TABLE || tableType == PTableType.SYSTEM) {
+                // Handle any child views that exist
+                TableViewFinderResult tableViewFinderResult = findChildViews(region, tenantId, table,
+                        PHYSICAL_TABLE_BYTES);
+                if (tableViewFinderResult.hasViews()) {
+                    if (isCascade) {
+                        if (tableViewFinderResult.allViewsInMultipleRegions()) {
+                            // We don't yet support deleting a table with views where SYSTEM.CATALOG has split and the
+                            // view metadata spans multiple regions
+                            return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
+                                    EnvironmentEdgeManager.currentTimeMillis(), null);
+                        } else if (tableViewFinderResult.allViewsInSingleRegion()) {
+                            // Recursively delete views - safe as all the views as all in the same region
+                            for (Result viewResult : tableViewFinderResult.getResults()) {
+                                byte[][] rowKeyMetaData = new byte[3][];
+                                getVarChars(viewResult.getRow(), 3, rowKeyMetaData);
+                                byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+                                byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+                                byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+                                byte[] viewKey = SchemaUtil.getTableKey(viewTenantId, viewSchemaName, viewName);
+                                Delete delete = new Delete(viewKey, clientTimeStamp);
+                                rowsToDelete.add(delete);
+                                acquireLock(region, viewKey, locks);
+                                MetaDataMutationResult result = doDropTable(viewKey, viewTenantId, viewSchemaName,
+                                        viewName, null, PTableType.VIEW, rowsToDelete, invalidateList, locks,
+                                        tableNamesToDelete, false);
+                                if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) { return result; }
+                            }
+                        }
+                    } else {
+                        // DROP without CASCADE on tables with child views is not permitted
+                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
+                                EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
                 }
-              }
-            } else {
-              // DROP without CASCADE on tables with child views is not permitted
-              return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
             }
-          }
 
-          if (tableType != PTableType.VIEW) { // Add to list of HTables to delete, unless it's a view
-            tableNamesToDelete.add(table.getName().getBytes());
-          }
-          invalidateList.add(cacheKey);
-          byte[][] rowKeyMetaData = new byte[5][];
-          do {
-            Cell kv = results.get(LINK_TYPE_INDEX);
-            int nColumns = getVarChars(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), 0, rowKeyMetaData);
-            if (nColumns == 5
-                && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length == 0
-                && rowKeyMetaData[PhoenixDatabaseMetaData.INDEX_NAME_INDEX].length > 0
-                && Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(), LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length) == 0
-                && LinkType.fromSerializedValue(kv.getValueArray()[kv.getValueOffset()]) == LinkType.INDEX_TABLE) {
-              indexNames.add(rowKeyMetaData[PhoenixDatabaseMetaData.INDEX_NAME_INDEX]);
+            if (tableType != PTableType.VIEW) { // Add to list of HTables to delete, unless it's a view
+                tableNamesToDelete.add(table.getName().getBytes());
             }
-            // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
-            // FIXME: the version of the Delete constructor without the lock args was introduced
-            // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
-            // of the client.
-            Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
-            rowsToDelete.add(delete);
-            results.clear();
-            scanner.next(results);
-          } while (!results.isEmpty());
+            invalidateList.add(cacheKey);
+            byte[][] rowKeyMetaData = new byte[5][];
+            do {
+                Cell kv = results.get(LINK_TYPE_INDEX);
+                int nColumns = getVarChars(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), 0, rowKeyMetaData);
+                if (nColumns == 5
+                        && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length == 0
+                        && rowKeyMetaData[PhoenixDatabaseMetaData.INDEX_NAME_INDEX].length > 0
+                        && Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
+                                LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length) == 0
+                        && LinkType.fromSerializedValue(kv.getValueArray()[kv.getValueOffset()]) == LinkType.INDEX_TABLE) {
+                    indexNames.add(rowKeyMetaData[PhoenixDatabaseMetaData.INDEX_NAME_INDEX]);
+                }
+                // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
+                // FIXME: the version of the Delete constructor without the lock args was introduced
+                // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
+                // of the client.
+                Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
+                rowsToDelete.add(delete);
+                results.clear();
+                scanner.next(results);
+            } while (!results.isEmpty());
         }
 
         // Recursively delete indexes
@@ -1800,7 +1807,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                      * and https://issues.apache.org/jira/browse/PHOENIX-2054 for enabling meta-data changes to a view
                      * to be propagated to its view hierarchy.
                      */
-                    if (type == PTableType.TABLE) {
+                    if (type == PTableType.TABLE || type == PTableType.SYSTEM) {
                         TableViewFinderResult childViewsResult = findChildViews(region, tenantId, table, PHYSICAL_TABLE_BYTES);
                         if (childViewsResult.hasViews()) {
                             // Adding a column is not allowed if the meta-data for child view/s spans over
@@ -2013,11 +2020,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     List<Mutation> additionalTableMetaData = Lists.newArrayList();
                     
                     PTableType type = table.getType();
-                    TableViewFinderResult childViewsResult = findChildViews(region, tenantId, table,
-                            (type == PTableType.VIEW ? PARENT_TABLE_BYTES : PHYSICAL_TABLE_BYTES));
-                    if (childViewsResult.hasViews()) {
-                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager
-                                .currentTimeMillis(), null);
+                    // Only tables may have views, so prevent the running of this potentially
+                    // expensive full table scan over the SYSTEM.CATALOG table unless it's needed.
+                    // In the case of a view, we allow a column to be dropped without checking for
+                    // child views, but in the future we'll allow it and propagate it as necessary.
+                    if (type == PTableType.TABLE || type == PTableType.SYSTEM) {
+                        TableViewFinderResult childViewsResult = 
+                                findChildViews(region, tenantId, table, PHYSICAL_TABLE_BYTES);
+                        if (childViewsResult.hasViews()) {
+                            return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager
+                                    .currentTimeMillis(), null);
+                        }
                     }
                     
                     for (Mutation m : tableMetaData) {


[2/2] phoenix git commit: PHOENIX-2087 Ensure predictable column position during alter table

Posted by ja...@apache.org.
PHOENIX-2087 Ensure predictable column position during alter table

Conflicts:
	phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 05911ac511b7277aaa3e6599181385fcac4b6096
Parents: 84a17ce
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Jun 30 08:44:37 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Jun 30 18:21:49 2015 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    | 51 +++++++++++++++-----
 .../coprocessor/MetaDataEndpointImpl.java       |  3 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  9 +++-
 3 files changed, 46 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/05911ac5/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 cd46927..56bba9b 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
@@ -448,7 +448,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         conn.commit();
 
         assertIndexExists(conn,true);
-        conn.createStatement().execute("ALTER TABLE " + DATA_TABLE_FULL_NAME + " ADD v3 VARCHAR, k2 DECIMAL PRIMARY KEY");
+        conn.createStatement().execute("ALTER TABLE " + DATA_TABLE_FULL_NAME + " ADD v3 VARCHAR, k2 DECIMAL PRIMARY KEY, k3 DECIMAL PRIMARY KEY");
         rs = conn.getMetaData().getPrimaryKeys("", SCHEMA_NAME, DATA_TABLE_NAME);
         assertTrue(rs.next());
         assertEquals("K",rs.getString("COLUMN_NAME"));
@@ -456,6 +456,10 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         assertTrue(rs.next());
         assertEquals("K2",rs.getString("COLUMN_NAME"));
         assertEquals(2, rs.getShort("KEY_SEQ"));
+        assertTrue(rs.next());
+        assertEquals("K3",rs.getString("COLUMN_NAME"));
+        assertEquals(3, rs.getShort("KEY_SEQ"));
+        assertFalse(rs.next());
 
         rs = conn.getMetaData().getPrimaryKeys("", SCHEMA_NAME, INDEX_TABLE_NAME);
         assertTrue(rs.next());
@@ -467,6 +471,10 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         assertTrue(rs.next());
         assertEquals(IndexUtil.INDEX_COLUMN_NAME_SEP + "K2",rs.getString("COLUMN_NAME"));
         assertEquals(3, rs.getShort("KEY_SEQ"));
+        assertTrue(rs.next());
+        assertEquals(IndexUtil.INDEX_COLUMN_NAME_SEP + "K3",rs.getString("COLUMN_NAME"));
+        assertEquals(4, rs.getShort("KEY_SEQ"));
+        assertFalse(rs.next());
 
         query = "SELECT * FROM " + DATA_TABLE_FULL_NAME;
         rs = conn.createStatement().executeQuery(query);
@@ -478,19 +486,21 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         assertFalse(rs.next());
 
         // load some data into the table
-        stmt = conn.prepareStatement("UPSERT INTO " + DATA_TABLE_FULL_NAME + "(K,K2,V1,V2) VALUES(?,?,?,?)");
+        stmt = conn.prepareStatement("UPSERT INTO " + DATA_TABLE_FULL_NAME + "(K,K2,V1,V2,K3) VALUES(?,?,?,?,?)");
         stmt.setString(1, "b");
         stmt.setBigDecimal(2, BigDecimal.valueOf(2));
         stmt.setString(3, "y");
         stmt.setString(4, "2");
+        stmt.setBigDecimal(5, BigDecimal.valueOf(3));
         stmt.execute();
         conn.commit();
 
-        query = "SELECT k,k2 FROM " + DATA_TABLE_FULL_NAME + " WHERE v1='y'";
+        query = "SELECT k,k2,k3 FROM " + DATA_TABLE_FULL_NAME + " WHERE v1='y'";
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("b",rs.getString(1));
         assertEquals(BigDecimal.valueOf(2),rs.getBigDecimal(2));
+        assertEquals(BigDecimal.valueOf(3),rs.getBigDecimal(3));
         assertFalse(rs.next());
     }
 
@@ -2345,6 +2355,21 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         return false;
     }
     
+    private int getIndexOfPkColumn(PhoenixConnection conn, String columnName, String tableName) throws SQLException {
+        String normalizedTableName = SchemaUtil.normalizeIdentifier(tableName);
+        PTable table = conn.getMetaDataCache().getTable(new PTableKey(conn.getTenantId(), normalizedTableName));
+        List<PColumn> pkCols = table.getPKColumns();
+        String normalizedColumnName = SchemaUtil.normalizeIdentifier(columnName);
+        int i = 0;
+        for (PColumn pkCol : pkCols) {
+            if (pkCol.getName().getString().equals(normalizedColumnName)) {
+                return i;
+            }
+            i++;
+        }
+        return -1;
+    }
+    
     private Connection getTenantConnection(String tenantId) throws Exception {
         Properties tenantProps = new Properties();
         tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
@@ -2444,35 +2469,35 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
 
             ResultSet rs = tenantConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view1);
             PhoenixConnection phxConn = tenantConn.unwrap(PhoenixConnection.class);
-            assertTrue(checkColumnPartOfPk(phxConn, "k2", view1));
-            assertTrue(checkColumnPartOfPk(phxConn, "k3", view1));
+            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view1));
+            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view1));
             assertEquals(1, getTableSequenceNumber(phxConn, view1));
             assertEquals(4, getMaxKeySequenceNumber(phxConn, view1));
             verifyNewColumns(rs, "K2", "K3", "V3");
 
 
             rs = tenantConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view2);
-            assertTrue(checkColumnPartOfPk(phxConn, "k2", view2));
-            assertTrue(checkColumnPartOfPk(phxConn, "k3", view2));
+            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view2));
+            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view2));
             assertEquals(1, getTableSequenceNumber(phxConn, view2));
             assertEquals(4, getMaxKeySequenceNumber(phxConn, view2));
             verifyNewColumns(rs, "K2", "K3", "V3");
 
-            assertTrue(checkColumnPartOfPk(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view2Index));
-            assertTrue(checkColumnPartOfPk(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view2Index));
+            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view2Index));
+            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view2Index));
             assertEquals(1, getTableSequenceNumber(phxConn, view2Index));
             assertEquals(6, getMaxKeySequenceNumber(phxConn, view2Index));
         }
         try (Connection tenantConn = getTenantConnection(tenant2)) {
             ResultSet rs = tenantConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view3);
             PhoenixConnection phxConn = tenantConn.unwrap(PhoenixConnection.class);
-            assertTrue(checkColumnPartOfPk(phxConn, "k2", view3));
-            assertTrue(checkColumnPartOfPk(phxConn, "k3", view3));
+            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view3));
+            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view3));
             assertEquals(1, getTableSequenceNumber(phxConn, view3));
             verifyNewColumns(rs, "K22", "K33", "V33");
 
-            assertTrue(checkColumnPartOfPk(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view3Index));
-            assertTrue(checkColumnPartOfPk(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view3Index));
+            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view3Index));
+            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view3Index));
             assertEquals(1, getTableSequenceNumber(phxConn, view3Index));
             assertEquals(6, getMaxKeySequenceNumber(phxConn, view3Index));
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/05911ac5/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 5876290..76be38b 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
@@ -1173,13 +1173,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private static final byte[] PHYSICAL_TABLE_BYTES = new byte[] {PTable.LinkType.PHYSICAL_TABLE.getSerializedValue()};
-    private static final byte[] PARENT_TABLE_BYTES = new byte[] {PTable.LinkType.PARENT_TABLE.getSerializedValue()};
 
     /**
      * @param tableName parent table's name
      * Looks for whether child views exist for the table specified by table.
      * TODO: should we pass a timestamp here?
      */
+
     private TableViewFinderResult findChildViews(HRegion region, byte[] tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
         byte[] schemaName = table.getSchemaName().getBytes();
         byte[] tableName = table.getTableName().getBytes();
@@ -1462,7 +1462,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
           SQLException;
     }
 
-    @SuppressWarnings("deprecation")
     private MetaDataMutationResult
     mutateColumn(List<Mutation> tableMetadata, ColumnMutator mutator) throws IOException {
         byte[][] rowKeyMetaData = new byte[5][];

http://git-wip-us.apache.org/repos/asf/phoenix/blob/05911ac5/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 d77ded8..0ad9b56 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
@@ -2341,7 +2341,10 @@ public class MetaDataClient {
             while (true) {
                 ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 table = resolver.getTables().get(0).getTable();
-                List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize(2);
+                int nIndexes = table.getIndexes().size();
+                int nNewColumns = columnDefs.size();
+                List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize((1 + nNewColumns) * (nIndexes + 1));
+                List<Mutation> columnMetaData = Lists.newArrayListWithExpectedSize(nNewColumns * (nIndexes + 1));
                 if (logger.isDebugEnabled()) {
                     logger.debug(LogUtil.addCustomAnnotations("Resolved table to " + table.getName().getString() + " with seqNum " + table.getSequenceNumber() + " at timestamp " + table.getTimeStamp() + " with " + table.getColumns().size() + " columns: " + table.getColumns(), connection));
                 }
@@ -2453,7 +2456,7 @@ public class MetaDataClient {
                         }
                     }
 
-                    tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond());
+                    columnMetaData.addAll(connection.getMutationState().toMutations().next().getSecond());
                     connection.rollback();
                 } else {
                     // Check that HBase configured properly for mutable secondary indexing
@@ -2489,6 +2492,8 @@ public class MetaDataClient {
                 
                 // Force the table header row to be first
                 Collections.reverse(tableMetaData);
+                // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
+                tableMetaData.addAll(columnMetaData);
 
                 byte[] family = families.size() > 0 ? families.iterator().next().getBytes() : null;