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 2018/09/24 15:28:08 UTC

[03/50] [abbrv] phoenix git commit: PHOENIX-3534 Support multi region SYSTEM.CATALOG table (Thomas D'Silva and Rahul Gidwani)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
index 45aca98..a267629 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.schema;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.phoenix.coprocessor.generated.PTableProtos;
 import org.apache.phoenix.query.QueryConstants;
@@ -42,36 +43,63 @@ public class PColumnImpl implements PColumn {
     private boolean isRowTimestamp;
     private boolean isDynamic;
     private byte[] columnQualifierBytes;
-    
+    private boolean derived;
+    private long timestamp;
+
     public PColumnImpl() {
     }
 
-    public PColumnImpl(PName name,
-                       PName familyName,
-                       PDataType dataType,
-                       Integer maxLength,
-                       Integer scale,
-                       boolean nullable,
-                       int position,
-                       SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic, byte[] columnQualifierBytes) {
-        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes);
+    public PColumnImpl(PColumn column, int position) {
+        this(column, column.isDerived(), position);
     }
 
-    public PColumnImpl(PColumn column, int position) {
+    public PColumnImpl(PColumn column, byte[] viewConstant, boolean isViewReferenced) {
+        this(column.getName(), column.getFamilyName(), column.getDataType(), column.getMaxLength(),
+                column.getScale(), column.isNullable(), column.getPosition(), column.getSortOrder(), column.getArraySize(), viewConstant, isViewReferenced, column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifierBytes(),
+                column.getTimestamp(), column.isDerived());
+    }
+
+    public PColumnImpl(PColumn column, boolean derivedColumn, int position) {
+        this(column, derivedColumn, position, column.getViewConstant());
+    }
+
+    public PColumnImpl(PColumn column, boolean derivedColumn, int position, byte[] viewConstant) {
         this(column.getName(), column.getFamilyName(), column.getDataType(), column.getMaxLength(),
-                column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifierBytes());
+            column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize(), viewConstant, column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifierBytes(),
+            column.getTimestamp(), derivedColumn);
+    }
+    
+    public PColumnImpl(PName name, PName familyName, PDataType dataType, Integer maxLength, Integer scale, boolean nullable,
+        int position, SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic,
+        byte[] columnQualifierBytes, long timestamp) {
+        this(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes, timestamp, false);
+    }
+
+    public PColumnImpl(PName name, PName familyName, PDataType dataType, Integer maxLength, Integer scale, boolean nullable,
+        int position, SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic,
+        byte[] columnQualifierBytes, long timestamp, boolean derived) {
+        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes, timestamp, derived);
+    }
+
+    private PColumnImpl(PName familyName, PName columnName, Long timestamp) {
+        this.familyName = familyName;
+        this.name = columnName;
+        this.derived = true;
+        if (timestamp!=null) {
+            this.timestamp = timestamp;
+        }
     }
 
-    private void init(PName name,
-            PName familyName,
-            PDataType dataType,
-            Integer maxLength,
-            Integer scale,
-            boolean nullable,
-            int position,
-            SortOrder sortOrder,
-            Integer arrSize,
-            byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic, byte[] columnQualifierBytes) {
+    // a derived column has null type
+    public static PColumnImpl createExcludedColumn(PName familyName, PName columnName, Long timestamp) {
+        return new PColumnImpl(familyName, columnName, timestamp);
+    }
+    
+    private void init(PName name, PName familyName, PDataType dataType, Integer maxLength,
+            Integer scale, boolean nullable, int position, SortOrder sortOrder, Integer arrSize,
+            byte[] viewConstant, boolean isViewReferenced, String expressionStr,
+            boolean isRowTimestamp, boolean isDynamic, byte[] columnQualifierBytes, long timestamp,
+            boolean derived) {
     	Preconditions.checkNotNull(sortOrder);
         this.dataType = dataType;
         if (familyName == null) {
@@ -97,6 +125,8 @@ public class PColumnImpl implements PColumn {
         this.isRowTimestamp = isRowTimestamp;
         this.isDynamic = isDynamic;
         this.columnQualifierBytes = columnQualifierBytes;
+        this.timestamp = timestamp;
+        this.derived = derived;
     }
 
     @Override
@@ -137,6 +167,16 @@ public class PColumnImpl implements PColumn {
     }
 
     @Override
+    public long getTimestamp() {
+        return timestamp;
+    }
+
+    @Override
+    public boolean isExcluded() {
+        return dataType == null;
+    }
+
+    @Override
     public boolean isNullable() {
         return nullable;
     }
@@ -226,7 +266,7 @@ public class PColumnImpl implements PColumn {
         if (column.hasFamilyNameBytes()) {
             familyName = PNameFactory.newName(column.getFamilyNameBytes().toByteArray());
         }
-        PDataType dataType = PDataType.fromSqlTypeName(column.getDataType());
+        PDataType dataType = column.hasDataType() ? PDataType.fromSqlTypeName(column.getDataType()) : null;
         Integer maxLength = null;
         if (column.hasMaxLength()) {
             maxLength = column.getMaxLength();
@@ -263,8 +303,17 @@ public class PColumnImpl implements PColumn {
         if (column.hasColumnQualifierBytes()) {
             columnQualifierBytes = column.getColumnQualifierBytes().toByteArray();
         }
+        long timestamp = HConstants.LATEST_TIMESTAMP;
+        if (column.hasTimestamp()) {
+            timestamp = column.getTimestamp();
+        }
+        boolean derived = false;
+        if (column.hasDerived()) {
+            derived = column.getDerived();
+        }
         return new PColumnImpl(columnName, familyName, dataType, maxLength, scale, nullable, position, sortOrder,
-                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes);
+                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes,
+            timestamp, derived);
     }
 
     public static PTableProtos.PColumn toProto(PColumn column) {
@@ -273,7 +322,9 @@ public class PColumnImpl implements PColumn {
         if (column.getFamilyName() != null) {
             builder.setFamilyNameBytes(ByteStringer.wrap(column.getFamilyName().getBytes()));
         }
-        builder.setDataType(column.getDataType().getSqlTypeName());
+        if (column.getDataType()!=null) {
+            builder.setDataType(column.getDataType().getSqlTypeName());
+        }
         if (column.getMaxLength() != null) {
             builder.setMaxLength(column.getMaxLength());
         }
@@ -282,7 +333,9 @@ public class PColumnImpl implements PColumn {
         }
         builder.setNullable(column.isNullable());
         builder.setPosition(column.getPosition());
-        builder.setSortOrder(column.getSortOrder().getSystemValue());
+        if (column.getSortOrder()!=null) {
+            builder.setSortOrder(column.getSortOrder().getSystemValue());
+        }
         if (column.getArraySize() != null) {
             builder.setArraySize(column.getArraySize());
         }
@@ -298,6 +351,14 @@ public class PColumnImpl implements PColumn {
         if (column.getColumnQualifierBytes() != null) {
             builder.setColumnQualifierBytes(ByteStringer.wrap(column.getColumnQualifierBytes()));
         }
+        if (column.getTimestamp() != HConstants.LATEST_TIMESTAMP) {
+            builder.setTimestamp(column.getTimestamp());
+        }
+        builder.setDerived(column.isDerived());
         return builder.build();
     }
+
+    public boolean isDerived() {
+        return derived;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 17f929f..598cc79 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -214,7 +214,8 @@ public class PMetaDataImpl implements PMetaData {
             // Update position of columns that follow removed column
             for (int i = position+1; i < oldColumns.size(); i++) {
                 PColumn oldColumn = oldColumns.get(i);
-                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), oldColumn.getExpressionStr(), oldColumn.isRowTimestamp(), oldColumn.isDynamic(), oldColumn.getColumnQualifierBytes());
+                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), oldColumn.getExpressionStr(), oldColumn.isRowTimestamp(), oldColumn.isDynamic(), oldColumn.getColumnQualifierBytes(),
+                    oldColumn.getTimestamp());
                 columns.add(newColumn);
             }
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index af78612..bb81d76 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -151,15 +151,26 @@ public interface PTable extends PMetaDataEntity {
         PARENT_TABLE((byte)3),
         /**
          * Link from a parent table to its child view
+         * (these are stored in SYSTEM.CHILD_LINK for scalability)
          */
-        CHILD_TABLE((byte)4);
+        CHILD_TABLE((byte)4),
+        /**
+         * Link for an excluded (dropped) column
+         */
+        EXCLUDED_COLUMN((byte)5),
+        /**
+         * Link from an index on a view to its parent table
+         */
+        VIEW_INDEX_PARENT_TABLE((byte)6);
 
         private final byte[] byteValue;
         private final byte serializedValue;
+        private final byte[] serializedByteArrayValue;
 
         LinkType(byte serializedValue) {
             this.serializedValue = serializedValue;
             this.byteValue = Bytes.toBytes(this.name());
+            this.serializedByteArrayValue = new byte[] { serializedValue };
         }
 
         public byte[] getBytes() {
@@ -170,6 +181,10 @@ public interface PTable extends PMetaDataEntity {
             return this.serializedValue;
         }
 
+        public byte[] getSerializedValueAsByteArray() {
+            return serializedByteArrayValue;
+        }
+
         public static LinkType fromSerializedValue(byte serializedValue) {
             if (serializedValue < 1 || serializedValue > LinkType.values().length) {
                 return null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 082a58b..d575902 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -28,6 +28,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -258,125 +259,81 @@ public class PTableImpl implements PTable {
     }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes) throws SQLException {
-        return makePTable(table, timeStamp, indexes, table.getParentSchemaName(), table.getViewStatement());
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), timeStamp,
+                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), indexes);
     }
 
-    public static PTable makePTable(PTable index, PName indexName, String viewStatement, long updateCacheFrequency, PName tenantId) throws SQLException {
-        return Objects.equal(viewStatement, index.getViewStatement()) ? index : makePTable(index, indexName, index.getTimeStamp(), Lists.newArrayList(index.getPhysicalName()), index.getIndexes(), viewStatement, updateCacheFrequency, tenantId);
+    public static PTable makePTable(PTable index, PName indexName, String viewStatement, long updateCacheFrequency,
+            PName tenantId) throws SQLException {
+        return Objects.equal(viewStatement, index.getViewStatement()) ? index
+                : new PTableImpl(index, index.rowKeyOrderOptimizable(), index.getIndexState(), index.getTimeStamp(),
+                        index.getSequenceNumber(), index.getColumns(), index.getDefaultFamilyName(), index.getType(),
+                        index.getBaseColumnCount(), index.getSchemaName(), indexName,
+                        viewStatement, updateCacheFrequency, tenantId,
+                        index.getIndexes());
     }
     
-    public static PTableImpl makePTable(PTable table, PName tableName, long timeStamp, List<PName> physicalNames, List<PTable> indexes, String viewStatement, long updateCacheFrequency, PName tenantId) throws SQLException {
-            return new PTableImpl(
-                    tenantId, table.getSchemaName(), tableName, table.getType(), table.getIndexState(), timeStamp,
-                    table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), table.getParentTableName(),
-                    indexes, table.isImmutableRows(), physicalNames, table.getDefaultFamilyName(), viewStatement,
-                    table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                    table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), updateCacheFrequency,
-                    table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
-        }
-
-    public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes, PName parentSchemaName, String viewStatement) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp,
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), parentSchemaName, table.getParentTableName(),
-                indexes, table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), viewStatement,
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+    public static PTableImpl makePTable(PTable table, Collection<PColumn> columns) throws SQLException {
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), columns, table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
-    public static PTableImpl makePTable(PTable table, Collection<PColumn> columns) throws SQLException {
+    /**
+     * Used to create a PTable for views or view indexes, the basePTable is for attributes we inherit from the physical table
+     */
+    public static PTableImpl makePTable(PTable view, PTable baseTable, Collection<PColumn> columns, long timestamp, int baseTableColumnCount) throws SQLException {
+        // if a TableProperty is not valid on a view we set it to the base table value
+        // if a TableProperty is valid on a view and is not mutable on a view we set it to the base table value
+        // if a TableProperty is valid on a view and is mutable on a view we use the value set on the view 
         return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
-                table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+            view.getTenantId(), view.getSchemaName(), view.getTableName(), view.getType(), view.getIndexState(), timestamp,
+            view.getSequenceNumber(), view.getPKName(), view.getBucketNum(), columns, view.getParentSchemaName(), view.getParentTableName(),
+            view.getIndexes(), baseTable.isImmutableRows(), view.getPhysicalNames(), view.getDefaultFamilyName(), view.getViewStatement(),
+            baseTable.isWALDisabled(), baseTable.isMultiTenant(), baseTable.getStoreNulls(), view.getViewType(), view.getViewIndexId(), view.getIndexType(),
+            baseTableColumnCount, view.rowKeyOrderOptimizable(), baseTable.getTransactionProvider(), view.getUpdateCacheFrequency(),
+            view.getIndexDisableTimestamp(), view.isNamespaceMapped(), baseTable.getAutoPartitionSeqName(), baseTable.isAppendOnlySchema(),
+            baseTable.getImmutableStorageScheme(), baseTable.getEncodingScheme(), view.getEncodedCQCounter(), view.useStatsForParallelization());
     }
     
     public static PTableImpl makePTable(PTable table, PTableType type, Collection<PColumn> columns) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), type, table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
-                table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), columns, table.getDefaultFamilyName(), type,
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
-    public static PTableImpl makePTable(PTable table, Collection<PColumn> columns, PName defaultFamily) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
-                table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), defaultFamily, table.getViewStatement(),
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+    public static PTableImpl makePTable(PTable table, Collection<PColumn> columns, PName defaultFamily)
+            throws SQLException {
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), columns, defaultFamily, table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
-    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp,
-                sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
-                table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(),
-                table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
-                table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns)
+            throws SQLException {
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), table.getIndexState(), timeStamp,
+                sequenceNumber, columns, table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
-    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns, boolean isImmutableRows) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp,
-                sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
-                table.getIndexes(), isImmutableRows, table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                table.getIndexType(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(),
-                table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
-    }
-    
-    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns, boolean isImmutableRows, boolean isWalDisabled,
-            boolean isMultitenant, boolean storeNulls, TransactionFactory.Provider transactionProvider, long updateCacheFrequency, boolean isNamespaceMapped) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp,
-                sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
-                table.getIndexes(), isImmutableRows, table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
-                isWalDisabled, isMultitenant, storeNulls, table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), transactionProvider, updateCacheFrequency, table.getIndexDisableTimestamp(), 
-                isNamespaceMapped, table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
-    }
-    
     public static PTableImpl makePTable(PTable table, PIndexState state) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), state, table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table),
-                table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
-                table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+        return new PTableImpl(table, table.rowKeyOrderOptimizable(), state, table.getTimeStamp(),
+                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
     public static PTableImpl makePTable(PTable table, boolean rowKeyOrderOptimizable) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table),
-                table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
-                table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), rowKeyOrderOptimizable, table.getTransactionProvider(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
-                table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
-    }
-
-    public static PTableImpl makePTable(PTable table) throws SQLException {
-        return new PTableImpl(
-                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
-                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table),
-                table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
-                table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
-                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.getTransactionProvider(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
-                table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+        return new PTableImpl(table, rowKeyOrderOptimizable, table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), getColumnsToClone(table), table.getDefaultFamilyName(), table.getType(),
+                table.getBaseColumnCount(), table.getSchemaName(), table.getTableName(), table.getViewStatement(),
+                table.getUpdateCacheFrequency(), table.getTenantId(), table.getIndexes());
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
@@ -384,13 +341,18 @@ public class PTableImpl implements PTable {
             Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
-            IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider, long updateCacheFrequency,
-            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
-        return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns, dataSchemaName,
-                dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
-                viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
+            IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider,
+            long updateCacheFrequency, long indexDisableTimestamp, boolean isNamespaceMapped,
+            String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
+            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter,
+            Boolean useStatsForParallelization) throws SQLException {
+        return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
+                bucketNum, columns, dataSchemaName, dataTableName, indexes, isImmutableRows, physicalNames,
+                defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
                 indexType, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, rowKeyOrderOptimizable, transactionProvider,
-                updateCacheFrequency,indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
+                updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName,
+                isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter,
+                useStatsForParallelization);
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
@@ -398,16 +360,31 @@ public class PTableImpl implements PTable {
             Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
-            IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider, long updateCacheFrequency,
-            int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
+            IndexType indexType, boolean rowKeyOrderOptimizable, TransactionFactory.Provider transactionProvider,
+            long updateCacheFrequency, int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
             String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
-            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization)
-            throws SQLException {
+            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter,
+            Boolean useStatsForParallelization) throws SQLException {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
                 bucketNum, columns, dataSchemaName, dataTableName, indexes, isImmutableRows, physicalNames,
                 defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
-                indexType, baseColumnCount, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, 
-                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
+                indexType, baseColumnCount, rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency,
+                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme,
+                qualifierEncodingScheme, encodedCQCounter, useStatsForParallelization);
+    }
+
+    private PTableImpl(PTable table, boolean rowKeyOrderOptimizable, PIndexState state, long timeStamp,
+            long sequenceNumber, Collection<PColumn> columns, PName defaultFamily, PTableType type,
+            int baseTableColumnCount, PName schemaName, PName tableName, String viewStatement,
+            long updateCacheFrequency, PName tenantId, List<PTable> indexes) throws SQLException {
+        init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, table.getPKName(),
+                table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(), indexes,
+                table.isImmutableRows(), table.getPhysicalNames(), defaultFamily, viewStatement, table.isWALDisabled(),
+                table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
+                table.getIndexType(), baseTableColumnCount, rowKeyOrderOptimizable, table.getTransactionProvider(),
+                updateCacheFrequency, table.getIndexDisableTimestamp(), table.isNamespaceMapped(),
+                table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(),
+                table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
 
     private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
@@ -513,8 +490,23 @@ public class PTableImpl implements PTable {
             allColumns = new PColumn[columns.size()];
             pkColumns = Lists.newArrayListWithExpectedSize(columns.size());
         }
-        for (PColumn column : columns) {
-            allColumns[column.getPosition()] = column;
+        // Must do this as with the new method of storing diffs, we just care about ordinal position
+        // relative order and not the true ordinal value itself.
+        List<PColumn> sortedColumns = Lists.newArrayList(columns);
+        Collections.sort(sortedColumns, new Comparator<PColumn>() {
+            @Override
+            public int compare(PColumn o1, PColumn o2) {
+                return Integer.valueOf(o1.getPosition()).compareTo(o2.getPosition());
+            }
+        });
+
+        int position = 0;
+        if (bucketNum != null) {
+            position = 1;
+        }
+        for (PColumn column : sortedColumns) {
+            allColumns[position] = column;
+            position++;
             PName familyName = column.getFamilyName();
             if (familyName == null) {
                 ++numPKColumns;
@@ -566,10 +558,10 @@ public class PTableImpl implements PTable {
                                 || column.getDataType() == PDouble.INSTANCE 
                                 || column.getDataType() == PBinary.INSTANCE) )
                         || (column.getSortOrder() == SortOrder.ASC && column.getDataType() == PBinary.INSTANCE && column.getMaxLength() != null && column.getMaxLength() > 1);
-            	pkColumns.add(column);
-            	if (column.isRowTimestamp()) {
-            	    rowTimestampCol = column;
-            	}
+                pkColumns.add(column);
+                if (column.isRowTimestamp()) {
+                    rowTimestampCol = column;
+                }
             }
             if (familyName == null) {
                 estimatedSize += column.getEstimatedSize(); // PK columns
@@ -752,7 +744,7 @@ public class PTableImpl implements PTable {
                 if (byteValue.length == 0 && !column.isNullable()) {
                     throw new ConstraintViolationException(name.getString() + "." + column.getName().getString() + " may not be null");
                 }
-                Integer	maxLength = column.getMaxLength();
+                Integer    maxLength = column.getMaxLength();
                 Integer scale = column.getScale();
                 key.set(byteValue);
                 if (!type.isSizeCompatible(key, null, type, sortOrder, null, null, maxLength, scale)) {
@@ -933,10 +925,10 @@ public class PTableImpl implements PTable {
                         }
                         Expression[] colValues = EncodedColumnsUtil.createColumnExpressionArray(maxEncodedColumnQualifier);
                         for (PColumn column : columns) {
-                        	if (columnToValueMap.containsKey(column)) {
-                        	    int colIndex = qualifierEncodingScheme.decode(column.getColumnQualifierBytes())-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1;
-                        	    colValues[colIndex] = new LiteralExpression(columnToValueMap.get(column));
-                        	}
+                            if (columnToValueMap.containsKey(column)) {
+                                int colIndex = qualifierEncodingScheme.decode(column.getColumnQualifierBytes())-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1;
+                                colValues[colIndex] = new LiteralExpression(columnToValueMap.get(column));
+                            }
                         }
                         
                         List<Expression> children = Arrays.asList(colValues);
@@ -1010,7 +1002,7 @@ public class PTableImpl implements PTable {
                             .getFamilyName().getBytesPtr(), qualifierPtr, ts));
             } else {
                 ImmutableBytesWritable ptr = new ImmutableBytesWritable(byteValue);
-                Integer	maxLength = column.getMaxLength();
+                Integer    maxLength = column.getMaxLength();
                 Integer scale = column.getScale();
                 SortOrder sortOrder = column.getSortOrder();
                 if (!type.isSizeCompatible(ptr, null, type, sortOrder, null, null, maxLength, scale)) {
@@ -1192,7 +1184,7 @@ public class PTableImpl implements PTable {
 
     @Override
     public List<PName> getPhysicalNames() {
-        return physicalNames;
+        return !physicalNames.isEmpty() ? physicalNames : Lists.newArrayList(getPhysicalName());
     }
 
     @Override
@@ -1230,6 +1222,8 @@ public class PTableImpl implements PTable {
      * @param table
      */
     public static PTable createFromProto(PTableProtos.PTable table) {
+        if (table==null)
+            return null;
         PName tenantId = null;
         if(table.hasTenantId()){
             tenantId = PNameFactory.newName(table.getTenantId().toByteArray());
@@ -1339,15 +1333,15 @@ public class PTableImpl implements PTable {
         }
         EncodedCQCounter encodedColumnQualifierCounter = null;
         if ((!EncodedColumnsUtil.usesEncodedColumnNames(qualifierEncodingScheme) || tableType == PTableType.VIEW)) {
-        	encodedColumnQualifierCounter = PTable.EncodedCQCounter.NULL_COUNTER;
+            encodedColumnQualifierCounter = PTable.EncodedCQCounter.NULL_COUNTER;
         }
         else {
-        	encodedColumnQualifierCounter = new EncodedCQCounter();
-        	if (table.getEncodedCQCountersList() != null) {
-        		for (org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter cqCounterFromProto : table.getEncodedCQCountersList()) {
-        			encodedColumnQualifierCounter.setValue(cqCounterFromProto.getColFamily(), cqCounterFromProto.getCounter());
-        		}
-        	}
+            encodedColumnQualifierCounter = new EncodedCQCounter();
+            if (table.getEncodedCQCountersList() != null) {
+                for (org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter cqCounterFromProto : table.getEncodedCQCountersList()) {
+                    encodedColumnQualifierCounter.setValue(cqCounterFromProto.getColFamily(), cqCounterFromProto.getCounter());
+                }
+            }
         }
         Boolean useStatsForParallelization = null;
         if (table.hasUseStatsForParallelization()) {
@@ -1376,15 +1370,15 @@ public class PTableImpl implements PTable {
       builder.setTableNameBytes(ByteStringer.wrap(table.getTableName().getBytes()));
       builder.setTableType(ProtobufUtil.toPTableTypeProto(table.getType()));
       if (table.getType() == PTableType.INDEX) {
-    	if(table.getIndexState() != null) {
-    	  builder.setIndexState(table.getIndexState().getSerializedValue());
-    	}
-    	if(table.getViewIndexId() != null) {
-    	  builder.setViewIndexId(table.getViewIndexId());
-    	}
-    	if(table.getIndexType() != null) {
-    	    builder.setIndexType(ByteStringer.wrap(new byte[]{table.getIndexType().getSerializedValue()}));
-    	}
+        if(table.getIndexState() != null) {
+          builder.setIndexState(table.getIndexState().getSerializedValue());
+        }
+        if(table.getViewIndexId() != null) {
+          builder.setViewIndexId(table.getViewIndexId());
+        }
+        if(table.getIndexType() != null) {
+            builder.setIndexType(ByteStringer.wrap(new byte[]{table.getIndexType().getSerializedValue()}));
+        }
       }
       builder.setSequenceNumber(table.getSequenceNumber());
       builder.setTimeStamp(table.getTimeStamp());
@@ -1403,10 +1397,9 @@ public class PTableImpl implements PTable {
       List<PColumn> columns = table.getColumns();
       int columnSize = columns.size();
       for (int i = offset; i < columnSize; i++) {
-        PColumn column = columns.get(i);
-        builder.addColumns(PColumnImpl.toProto(column));
+          PColumn column = columns.get(i);
+          builder.addColumns(PColumnImpl.toProto(column));
       }
-
       List<PTable> indexes = table.getIndexes();
       for (PTable curIndex : indexes) {
         builder.addIndexes(toProto(curIndex));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
index 017c75d..4b44595 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
@@ -45,14 +45,14 @@ public class PTableKey {
     
     @Override
     public String toString() {
-        return name + (tenantId == null ? "" : " for " + tenantId.getString());
+        return name + ((tenantId == null || tenantId.getBytes().length==0) ? "" : " for " + tenantId.getString());
     }
     
     @Override
     public int hashCode() {
         final int prime = 31;
         int result = 1;
-        result = prime * result + ((tenantId == null) ? 0 : tenantId.hashCode());
+        result = prime * result + ((tenantId == null || tenantId.getBytes().length==0) ? 0 : tenantId.hashCode());
         result = prime * result + name.hashCode();
         return result;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
new file mode 100644
index 0000000..27446e0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
@@ -0,0 +1,30 @@
+package org.apache.phoenix.schema;
+
+import org.apache.phoenix.coprocessor.TableInfo;
+
+public class ParentTableNotFoundException extends TableNotFoundException {
+    private static final long serialVersionUID = 1L;
+    private final byte[] parentTenantId;
+    private final byte[] parentSchemaName;
+    private final byte[] parentTableName;
+
+    public ParentTableNotFoundException(TableInfo parentTableInfo, String tableName) {
+        super(tableName);
+        this.parentTenantId = parentTableInfo.getTenantId();
+        this.parentSchemaName = parentTableInfo.getSchemaName();
+        this.parentTableName = parentTableInfo.getTableName();
+    }
+
+    public byte[] getParentTenantId() {
+        return parentTenantId;
+    }
+
+    public byte[] getParentSchemaName() {
+        return parentSchemaName;
+    }
+
+    public byte[] getParentTableName() {
+        return parentTableName;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
index 23cfd1b..7bbd620 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.schema;
 
 import java.util.List;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.query.KeyRange;
@@ -38,7 +39,8 @@ public class SaltingUtil {
     public static final String SALTING_COLUMN_NAME = "_SALT";
     public static final String SALTED_ROW_KEY_NAME = "_SALTED_KEY";
     public static final PColumnImpl SALTING_COLUMN = new PColumnImpl(
-            PNameFactory.newName(SALTING_COLUMN_NAME), null, PBinary.INSTANCE, 1, 0, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false, null);
+            PNameFactory.newName(SALTING_COLUMN_NAME), null, PBinary.INSTANCE, 1, 0, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false, null,
+        HConstants.LATEST_TIMESTAMP);
     public static final RowKeySchema VAR_BINARY_SALTED_SCHEMA = new RowKeySchemaBuilder(2)
         .addField(SALTING_COLUMN, false, SortOrder.getDefault())
         .addField(SchemaUtil.VAR_BINARY_DATUM, false, SortOrder.getDefault()).build();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/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 3d473c4..3d2e84e 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
@@ -152,7 +152,7 @@ public enum TableProperty {
             return table.isAppendOnlySchema();
         }
     },
-    GUIDE_POSTS_WIDTH(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH, true, false, false, false) {
+    GUIDE_POSTS_WIDTH(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH, true, false, false) {
         @Override
         public Object getValue(Object value) {
             return value == null ? null : ((Number) value).longValue();
@@ -231,36 +231,26 @@ public enum TableProperty {
     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);
+        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, true);
+        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, true);
+        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, 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) {
@@ -326,10 +316,6 @@ public enum TableProperty {
         return isMutableOnView;
     }
 
-    public boolean propagateToViews() {
-        return propagateToViews;
-    }
-
     abstract public Object getPTableValue(PTable table);
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 3fe5438..b63b053 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -100,7 +100,10 @@ import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
@@ -139,7 +142,7 @@ public class IndexUtil {
     
     // Since we cannot have nullable fixed length in a row key
     // we need to translate to variable length. The verification that we have a valid index
-    // row key was already done, so here we just need to covert from one built-in type to
+    // row key was already done, so here we just need to convert from one built-in type to
     // another.
     public static PDataType getIndexColumnDataType(boolean isNullable, PDataType dataType) {
         if (dataType == null || !isNullable || !dataType.isFixedWidth()) {
@@ -194,6 +197,17 @@ public class IndexUtil {
         String dataColumnFamilyName = SchemaUtil.isPKColumn(dataColumn) ? null : dataColumn.getFamilyName().getString();
         return getIndexColumnName(dataColumnFamilyName, dataColumn.getName().getString());
     }
+    
+	public static PColumn getIndexPKColumn(int position, PColumn dataColumn) {
+		assert (SchemaUtil.isPKColumn(dataColumn));
+		PName indexColumnName = PNameFactory.newName(getIndexColumnName(null, dataColumn.getName().getString()));
+		PColumn column = new PColumnImpl(indexColumnName, null, dataColumn.getDataType(), dataColumn.getMaxLength(),
+				dataColumn.getScale(), dataColumn.isNullable(), position, dataColumn.getSortOrder(),
+				dataColumn.getArraySize(), null, false, dataColumn.getExpressionStr(), dataColumn.isRowTimestamp(), false,
+				// TODO set the columnQualifierBytes correctly
+				/*columnQualifierBytes*/null, HConstants.LATEST_TIMESTAMP); 
+		return column;
+	}
 
     public static String getLocalIndexColumnFamily(String dataColumnFamilyName) {
         return dataColumnFamilyName == null ? null

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 1dda818..f7b7b4a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -24,7 +24,9 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
+import java.util.NavigableMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -34,10 +36,12 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -61,6 +65,7 @@ import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SortOrder;
@@ -68,12 +73,15 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableProperty;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PSmallint;
 import org.apache.phoenix.schema.types.PUnsignedTinyint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 import com.google.protobuf.ServiceException;
 
 
@@ -176,28 +184,104 @@ public class MetaDataUtil {
         return version;
     }
     
+    public static byte[] getTenantIdAndSchemaAndTableName(Mutation someRow) {
+        byte[][] rowKeyMetaData = new byte[3][];
+        getVarChars(someRow.getRow(), 3, rowKeyMetaData);
+        return ByteUtil.concat(rowKeyMetaData[0], rowKeyMetaData[1], rowKeyMetaData[2]);
+    }
+
+    public static byte[] getTenantIdAndSchemaAndTableName(Result result) {
+        byte[][] rowKeyMetaData = new byte[3][];
+        getVarChars(result.getRow(), 3, rowKeyMetaData);
+        return ByteUtil.concat(rowKeyMetaData[0], rowKeyMetaData[1], rowKeyMetaData[2]);
+    }
+
     public static void getTenantIdAndSchemaAndTableName(List<Mutation> tableMetadata, byte[][] rowKeyMetaData) {
         Mutation m = getTableHeaderRow(tableMetadata);
         getVarChars(m.getRow(), 3, rowKeyMetaData);
     }
 
+    public static int getBaseColumnCount(List<Mutation> tableMetadata) {
+        int result = -1;
+        for (Mutation mutation : tableMetadata) {
+            for (List<Cell> cells : mutation.getFamilyCellMap().values()) {
+                for (Cell cell : cells) {
+                    // compare using offsets
+                    if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES, 0,
+                        PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES.length) == 0)
+                    if (Bytes.contains(cell.getQualifierArray(), PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES)) {
+                        result = PInteger.INSTANCE.getCodec()
+                            .decodeInt(cell.getValueArray(), cell.getValueOffset(), SortOrder.ASC);
+                    }
+                }
+            }
+        }
+        return result;
+    }
+
+    public static void mutatePutValue(Put somePut, byte[] family, byte[] qualifier, byte[] newValue) {
+        NavigableMap<byte[], List<Cell>> familyCellMap = somePut.getFamilyCellMap();
+        List<Cell> cells = familyCellMap.get(family);
+        List<Cell> newCells = Lists.newArrayList();
+        if (cells != null) {
+            for (Cell cell : cells) {
+                if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
+                    qualifier, 0, qualifier.length) == 0) {
+                    Cell replacementCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
+                        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), cell.getQualifierArray(),
+                        cell.getQualifierOffset(), cell.getQualifierLength(), cell.getTimestamp(),
+                        KeyValue.Type.codeToType(cell.getTypeByte()), newValue, 0, newValue.length);
+                    newCells.add(replacementCell);
+                } else {
+                    newCells.add(cell);
+                }
+            }
+            familyCellMap.put(family, newCells);
+        }
+    }
+
+    public static Put cloneDeleteToPutAndAddColumn(Delete delete, byte[] family, byte[] qualifier, byte[] value) {
+        NavigableMap<byte[], List<Cell>> familyCellMap = delete.getFamilyCellMap();
+        List<Cell> cells = familyCellMap.get(family);
+        Cell cell = Iterables.getFirst(cells, null);
+        if (cell == null) {
+            throw new RuntimeException("Empty cells for delete for family: " + Bytes.toStringBinary(family));
+        }
+        byte[] rowArray = new byte[cell.getRowLength()];
+        System.arraycopy(cell.getRowArray(), cell.getRowOffset(), rowArray, 0, cell.getRowLength());
+        Put put = new Put(rowArray, delete.getTimeStamp());
+        put.addColumn(family, qualifier, delete.getTimeStamp(), value);
+        return put;
+    }
+
+
     public static void getTenantIdAndFunctionName(List<Mutation> functionMetadata, byte[][] rowKeyMetaData) {
         Mutation m = getTableHeaderRow(functionMetadata);
         getVarChars(m.getRow(), 2, rowKeyMetaData);
     }
 
+    /**
+     * Only return the parent table name if it has the same tenant id and schema name as the current
+     * table (this is only used to lock the parent table of indexes)
+     */
     public static byte[] getParentTableName(List<Mutation> tableMetadata) {
         if (tableMetadata.size() == 1) {
             return null;
         }
         byte[][] rowKeyMetaData = new byte[3][];
+        // get the tenantId, schema name and table name for the current table
         getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
+        byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
         byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+        // get the tenantId, schema name and table name for the parent table
         Mutation m = getParentTableHeaderRow(tableMetadata);
         getVarChars(m.getRow(), 3, rowKeyMetaData);
-        if (   Bytes.compareTo(schemaName, rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX]) == 0
-            && Bytes.compareTo(tableName, rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]) == 0) {
+        if (Bytes.compareTo(tenantId, rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX]) == 0
+                && Bytes.compareTo(schemaName,
+                    rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX]) == 0
+                && Bytes.compareTo(tableName,
+                    rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]) == 0) {
             return null;
         }
         return rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
@@ -238,6 +322,24 @@ public class MetaDataUtil {
     }
 
     
+    public static ViewType getViewType(List<Mutation> tableMetaData, KeyValueBuilder builder,
+    	      ImmutableBytesWritable value) {
+    	        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
+    	            PhoenixDatabaseMetaData.VIEW_TYPE_BYTES, builder, value)) {
+    	            return ViewType.fromSerializedValue(value.get()[value.getOffset()]);
+    	        }
+    	        return null;
+    	    }
+    
+    public static int getSaltBuckets(List<Mutation> tableMetaData, KeyValueBuilder builder,
+      ImmutableBytesWritable value) {
+        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
+            PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES, builder, value)) {
+            return PInteger.INSTANCE.getCodec().decodeInt(value, SortOrder.getDefault());
+        }
+        return 0;
+    }
+    
     public static long getParentSequenceNumber(List<Mutation> tableMetaData) {
         return getSequenceNumber(getParentTableHeaderRow(tableMetaData));
     }
@@ -273,6 +375,35 @@ public class MetaDataUtil {
         return false;
     }
 
+    public static KeyValue getMutationValue(Mutation headerRow, byte[] key,
+        KeyValueBuilder builder) {
+        List<Cell> kvs = headerRow.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        if (kvs != null) {
+            for (Cell cell : kvs) {
+                KeyValue kv = org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(cell);
+                if (builder.compareQualifier(kv, key, 0, key.length) ==0) {
+                    return kv;
+                }
+            }
+        }
+        return null;
+    }
+
+    public static boolean setMutationValue(Mutation headerRow, byte[] key,
+        KeyValueBuilder builder, KeyValue keyValue) {
+        List<Cell> kvs = headerRow.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        if (kvs != null) {
+            for (Cell cell : kvs) {
+                KeyValue kv = org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(cell);
+                if (builder.compareQualifier(kv, key, 0, key.length) ==0) {
+                    KeyValueBuilder.addQuietly(headerRow, builder, keyValue);
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
     /**
      * Returns the first Put element in <code>tableMetaData</code>. There could be leading Delete elements before the
      * table header row
@@ -601,14 +732,17 @@ public class MetaDataUtil {
 	}
 
     public static LinkType getLinkType(Mutation tableMutation) {
-        List<Cell> kvs = tableMutation.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        return getLinkType(tableMutation.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES));
+    }
+
+    public static LinkType getLinkType(Collection<Cell> kvs) {
         if (kvs != null) {
             for (Cell kv : kvs) {
                 if (Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
-                        PhoenixDatabaseMetaData.LINK_TYPE_BYTES, 0,
-                        PhoenixDatabaseMetaData.LINK_TYPE_BYTES.length) == 0) { return LinkType
-                                .fromSerializedValue(PUnsignedTinyint.INSTANCE.getCodec().decodeByte(kv.getValueArray(),
-                                        kv.getValueOffset(), SortOrder.getDefault())); }
+                    PhoenixDatabaseMetaData.LINK_TYPE_BYTES, 0,
+                    PhoenixDatabaseMetaData.LINK_TYPE_BYTES.length) == 0) { return LinkType
+                    .fromSerializedValue(PUnsignedTinyint.INSTANCE.getCodec().decodeByte(kv.getValueArray(),
+                        kv.getValueOffset(), SortOrder.getDefault())); }
             }
         }
         return null;
@@ -674,7 +808,28 @@ public class MetaDataUtil {
         return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, physicalTableSchemaName, physicalTableName);
     }
     
-    public static IndexType getIndexType(List<Mutation> tableMetaData, KeyValueBuilder builder,
+	public static List<Mutation> removeChildLinks(List<Mutation> catalogMutations) {
+		List<Mutation> childLinks = Lists.newArrayList();
+		Iterator<Mutation> iter = catalogMutations.iterator();
+		while (iter.hasNext()) {
+			Mutation m = iter.next();
+			for (KeyValue kv : m.getFamilyMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES)) {
+				// remove mutations of link type LinkType.CHILD_TABLE
+				if ((Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
+						PhoenixDatabaseMetaData.LINK_TYPE_BYTES, 0,
+						PhoenixDatabaseMetaData.LINK_TYPE_BYTES.length) == 0)
+						&& ((Bytes.compareTo(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(),
+								LinkType.CHILD_TABLE.getSerializedValueAsByteArray(), 0,
+								LinkType.CHILD_TABLE.getSerializedValueAsByteArray().length) == 0))) {
+					childLinks.add(m);
+					iter.remove();
+				}
+			}
+		}
+		return childLinks;
+	}
+
+	public static IndexType getIndexType(List<Mutation> tableMetaData, KeyValueBuilder builder,
             ImmutableBytesWritable value) {
         if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData), PhoenixDatabaseMetaData.INDEX_TYPE_BYTES, builder,
                 value)) { return IndexType.fromSerializedValue(value.get()[value.getOffset()]); }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index 6cf6e56..aee66aa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -263,7 +263,6 @@ public class PhoenixRuntime {
                 String srcTable = execCmd.getSrcTable();
                 System.out.println("Starting upgrading table:" + srcTable + "... please don't kill it in between!!");
                 UpgradeUtil.upgradeTable(conn, srcTable);
-                UpgradeUtil.mapChildViewsToNamespace(conn, srcTable,props);
             } else if (execCmd.isUpgrade()) {
                 if (conn.getClientInfo(PhoenixRuntime.CURRENT_SCN_ATTRIB) != null) { throw new SQLException(
                         "May not specify the CURRENT_SCN property when upgrading"); }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index dd00a69..48d2a5c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -22,13 +22,17 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Strings.isNullOrEmpty;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE;
+import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
@@ -274,7 +278,7 @@ public class SchemaUtil {
      * @param tableName
      */
     public static byte[] getTableKey(byte[] tenantId, byte[] schemaName, byte[] tableName) {
-        return ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName, QueryConstants.SEPARATOR_BYTE_ARRAY, tableName);
+        return ByteUtil.concat(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId, SEPARATOR_BYTE_ARRAY, schemaName, SEPARATOR_BYTE_ARRAY, tableName);
     }
 
     /**
@@ -283,32 +287,32 @@ public class SchemaUtil {
      * @param functionName
      */
     public static byte[] getFunctionKey(byte[] tenantId, byte[] functionName) {
-        return ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY, functionName);
+        return ByteUtil.concat(tenantId, SEPARATOR_BYTE_ARRAY, functionName);
     }
 
     public static byte[] getKeyForSchema(String tenantId, String schemaName) {
         return ByteUtil.concat(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId),
-                QueryConstants.SEPARATOR_BYTE_ARRAY,
+                SEPARATOR_BYTE_ARRAY,
                 schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName));
     }
 
     public static byte[] getTableKey(String tenantId, String schemaName, String tableName) {
-        return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId), QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName));
+        return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId), SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName));
     }
 
     public static byte[] getColumnKey(String tenantId, String schemaName, String tableName, String columnName, String familyName) {
         Preconditions.checkNotNull(columnName,"Column name cannot be null");
         if (familyName == null) {
             return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId),
-                    QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), 
-                    QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
-                    QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName));
+                    SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName),
+                    SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
+                    SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName));
         }
         return ByteUtil.concat(tenantId == null  ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(tenantId),
-                QueryConstants.SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName), 
-                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
-                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName),
-                QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(familyName));
+                SEPARATOR_BYTE_ARRAY, schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : Bytes.toBytes(schemaName),
+                SEPARATOR_BYTE_ARRAY, Bytes.toBytes(tableName),
+                SEPARATOR_BYTE_ARRAY, Bytes.toBytes(columnName),
+                SEPARATOR_BYTE_ARRAY, Bytes.toBytes(familyName));
     }
 
     public static String getTableName(String schemaName, String tableName) {
@@ -536,6 +540,11 @@ public class SchemaUtil {
                 || Bytes.compareTo(tableName, SchemaUtil
                         .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES, true).getName()) == 0;
     }
+    
+    public static boolean isChildLinkTable(byte[] tableName) {
+        return Bytes.compareTo(tableName, SYSTEM_CHILD_LINK_NAME_BYTES) == 0 || Bytes.compareTo(tableName,
+                SchemaUtil.getPhysicalTableName(SYSTEM_CHILD_LINK_NAME_BYTES, true).getName()) == 0;
+    }
 
     public static boolean isSequenceTable(PTable table) {
         return PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME.equals(table.getName().getString());
@@ -603,7 +612,7 @@ public class SchemaUtil {
                 if (pos == pkColumns.size() - 1) {
                     break;
                 }
-                while (offset < maxOffset && split[offset] != QueryConstants.SEPARATOR_BYTE) {
+                while (offset < maxOffset && split[offset] != SEPARATOR_BYTE) {
                     offset++;
                 }
                 if (offset == maxOffset) {
@@ -645,7 +654,7 @@ public class SchemaUtil {
         PhoenixConnection metaConnection = null;
         Statement stmt = null;
         try {
-            metaConnection = new PhoenixConnection(conn.getQueryServices(), conn, scn);
+            metaConnection = new PhoenixConnection(conn, scn);
             try {
                 stmt = metaConnection.createStatement();
                 stmt.executeUpdate("ALTER TABLE SYSTEM.\"TABLE\" ADD IF NOT EXISTS " + columnDef);
@@ -893,7 +902,7 @@ public class SchemaUtil {
      * @return the byte to use as the separator
      */
     public static byte getSeparatorByte(boolean rowKeyOrderOptimizable, boolean isNullValue, SortOrder sortOrder) {
-        return !rowKeyOrderOptimizable || isNullValue || sortOrder == SortOrder.ASC ? QueryConstants.SEPARATOR_BYTE : QueryConstants.DESC_SEPARATOR_BYTE;
+        return !rowKeyOrderOptimizable || isNullValue || sortOrder == SortOrder.ASC ? SEPARATOR_BYTE : QueryConstants.DESC_SEPARATOR_BYTE;
     }
     
     public static byte getSeparatorByte(boolean rowKeyOrderOptimizable, boolean isNullValue, Field f) {
@@ -1157,7 +1166,11 @@ public class SchemaUtil {
         }
     }
 
-    public static boolean hasGlobalIndex(PTable table) {
+	public static int getIsNullableInt(boolean isNullable) {
+		return isNullable ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls;
+	}
+
+	public static boolean hasGlobalIndex(PTable table) {
         for (PTable index : table.getIndexes()) {
             if (index.getIndexType() == IndexType.GLOBAL) {
                 return true;