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:10 UTC

[05/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/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 883f96d..29cf2a3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -91,8 +91,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0 = MIN_TABLE_TIMESTAMP + 28;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0 = MIN_TABLE_TIMESTAMP + 29;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
-    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0;
     // Version below which we should disallow usage of mutable secondary indexing.
     public static final int MUTABLE_SI_VERSION_THRESHOLD = VersionUtil.encodeVersion("0", "94", "10");
     public static final int MAX_LOCAL_SI_VERSION_DISALLOW = VersionUtil.encodeVersion("0", "98", "8");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableInfo.java
new file mode 100644
index 0000000..b1c5f65
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableInfo.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.coprocessor;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.util.SchemaUtil;
+
+public class TableInfo {
+
+    private final byte[] tenantId;
+    private final byte[] schema;
+    private final byte[] name;
+
+    public TableInfo(byte[] tenantId, byte[] schema, byte[] name) {
+        this.tenantId = tenantId;
+        this.schema = schema;
+        this.name = name;
+    }
+    
+    public byte[] getRowKeyPrefix() {
+        return SchemaUtil.getTableKey(tenantId, schema, name);
+    }
+
+    @Override
+    public String toString() {
+        return Bytes.toStringBinary(getRowKeyPrefix());
+    }
+    
+    public byte[] getTenantId() {
+        return tenantId;
+    }
+
+    public byte[] getSchemaName() {
+        return schema;
+    }
+
+    public byte[] getTableName() {
+        return name;
+    }
+    
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + Arrays.hashCode(name);
+        result = prime * result + Arrays.hashCode(schema);
+        result = prime * result + Arrays.hashCode(tenantId);
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        TableInfo other = (TableInfo) obj;
+        if (!Arrays.equals(name, other.name)) return false;
+        if (!Arrays.equals(schema, other.schema)) return false;
+        if (!Arrays.equals(tenantId, other.tenantId)) return false;
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java
new file mode 100644
index 0000000..269c5cb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.coprocessor;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+/**
+ * This class wraps the results of a scanning SYSTEM.CATALOG or SYSTEM.CHILD_LINK
+ * for child related tables or views.
+ */
+public class TableViewFinderResult {
+
+    private List<TableInfo> tableViewInfoList = Lists.newArrayList();
+
+    public TableViewFinderResult() {
+    }
+
+    public TableViewFinderResult(List<TableInfo> results) {
+        this.tableViewInfoList = results;
+    }
+
+    public boolean hasLinks() {
+        return !tableViewInfoList.isEmpty();
+    }
+
+    public List<TableInfo> getLinks() {
+        return tableViewInfoList;
+    }
+
+    void addResult(TableViewFinderResult result) {
+        this.tableViewInfoList.addAll(result.getLinks());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
new file mode 100644
index 0000000..5060d73
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
@@ -0,0 +1,144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.coprocessor;
+
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
+import static org.apache.phoenix.util.SchemaUtil.getVarChars;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.collect.Lists;
+
+public class ViewFinder {
+
+	// The PHYSICAL_TABLE link from view to the base table overwrites the PARENT_TABLE link (when namespace mapping is disabled)
+    static TableViewFinderResult findBaseTable(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
+        throws IOException {
+        return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.PHYSICAL_TABLE,
+            HConstants.LATEST_TIMESTAMP);
+    }
+    
+    static TableViewFinderResult findParentViewofIndex(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
+            throws IOException {
+            return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.VIEW_INDEX_PARENT_TABLE,
+                HConstants.LATEST_TIMESTAMP);
+        }
+
+    public static void findAllRelatives(Table systemTable, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, TableViewFinderResult result) throws IOException {
+        findAllRelatives(systemTable, tenantId, schema, table, linkType, HConstants.LATEST_TIMESTAMP, result);
+    }
+
+    static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, long timestamp, TableViewFinderResult result) throws IOException {
+        TableViewFinderResult currentResult =
+            findRelatedViews(systemCatalog, tenantId, schema, table, linkType, timestamp);
+        result.addResult(currentResult);
+        for (TableInfo viewInfo : currentResult.getLinks()) {
+            findAllRelatives(systemCatalog, viewInfo.getTenantId(), viewInfo.getSchemaName(), viewInfo.getTableName(), linkType, timestamp, result);
+        }
+    }
+
+    /**
+     * Runs a scan on SYSTEM.CATALOG or SYSTEM.CHILD_LINK to get the related tables/views
+     */
+    static TableViewFinderResult findRelatedViews(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, long timestamp) throws IOException {
+        if (linkType==PTable.LinkType.INDEX_TABLE || linkType==PTable.LinkType.EXCLUDED_COLUMN) {
+            throw new IllegalArgumentException("findAllRelatives does not support link type "+linkType);
+        }
+        byte[] key = SchemaUtil.getTableKey(tenantId, schema, table);
+		Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
+        SingleColumnValueFilter linkFilter =
+            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
+                linkType.getSerializedValueAsByteArray());
+        linkFilter.setFilterIfMissing(true);
+        scan.setFilter(linkFilter);
+        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
+        if (linkType==PTable.LinkType.PARENT_TABLE)
+            scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
+        if (linkType==PTable.LinkType.PHYSICAL_TABLE)
+            scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
+        List<TableInfo> tableInfoList = Lists.newArrayList();
+        try (ResultScanner scanner = systemCatalog.getScanner(scan))  {
+            for (Result result = scanner.next(); (result != null); result = scanner.next()) {
+                byte[][] rowKeyMetaData = new byte[5][];
+                byte[] viewTenantId = null;
+                getVarChars(result.getRow(), 5, rowKeyMetaData);
+                if (linkType==PTable.LinkType.PARENT_TABLE) {
+                    viewTenantId = result.getValue(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
+                } else if (linkType==PTable.LinkType.CHILD_TABLE) {
+                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
+                } else if (linkType==PTable.LinkType.VIEW_INDEX_PARENT_TABLE) {
+                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+                } 
+                else if (linkType==PTable.LinkType.PHYSICAL_TABLE && result.getValue(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES)!=null) {
+                    // do not links from indexes to their physical table
+                    continue;
+                }
+                byte[] viewSchemaName = SchemaUtil.getSchemaNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
+                byte[] viewName = SchemaUtil.getTableNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
+                tableInfoList.add(new TableInfo(viewTenantId, viewSchemaName, viewName));
+            }
+            return new TableViewFinderResult(tableInfoList);
+        } 
+    }
+    
+    /**
+     * @return true if the given table has at least one child view
+     * @throws IOException 
+     */
+    public static boolean hasChildViews(Table systemCatalog, byte[] tenantId, byte[] schemaName, byte[] tableName, long timestamp) throws IOException {
+        byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+        Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
+        SingleColumnValueFilter linkFilter =
+                new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES,
+                        CompareFilter.CompareOp.EQUAL,
+                        LinkType.CHILD_TABLE.getSerializedValueAsByteArray()) {
+                    // if we found a row with the CHILD_TABLE link type we are done and can
+                    // terminate the scan
+                    @Override
+                    public boolean filterAllRemaining() throws IOException {
+                        return matchedColumn;
+                    }
+                };
+        linkFilter.setFilterIfMissing(true);
+        scan.setFilter(linkFilter);
+        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
+        try (ResultScanner scanner = systemCatalog.getScanner(scan)) {
+            Result result = scanner.next();
+            return result!=null; 
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
new file mode 100644
index 0000000..4528f55
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.coprocessor;
+
+import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.compile.CreateTableCompiler;
+import org.apache.phoenix.compile.ExpressionCompiler;
+import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.compile.WhereCompiler;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.MetaDataUtil;
+
+import com.google.common.collect.Lists;
+
+
+public class WhereConstantParser {
+
+    static PTable addViewInfoToPColumnsIfNeeded(PTable view) throws SQLException {
+    	boolean[] viewColumnConstantsMatched = new boolean[view.getColumns().size()];
+        byte[][] viewColumnConstantsToBe = new byte[view.getColumns().size()][];
+        if (view.getViewStatement() == null) {
+        	return view;
+        }
+        SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
+        ParseNode whereNode = select.getWhere();
+        ColumnResolver resolver = FromCompiler.getResolver(new TableRef(view));
+        StatementContext context = new StatementContext(new PhoenixStatement(getConnectionlessConnection()), resolver);
+        Expression expression = null;
+        try {
+        	expression = WhereCompiler.compile(context, whereNode);
+        }
+        catch (ColumnNotFoundException e) {
+        	// if we could not find a column used in the view statement (which means its was dropped)
+        	// this view is not valid any more
+        	return null;
+        }
+        CreateTableCompiler.ViewWhereExpressionVisitor visitor =
+            new CreateTableCompiler.ViewWhereExpressionVisitor(view, viewColumnConstantsToBe);
+        expression.accept(visitor);
+        
+        BitSet isViewColumnReferencedToBe = new BitSet(view.getColumns().size());
+        // Used to track column references in a view
+        ExpressionCompiler expressionCompiler = new CreateTableCompiler.ColumnTrackingExpressionCompiler(context, isViewColumnReferencedToBe);
+        whereNode.accept(expressionCompiler);
+        
+        List<PColumn> result = Lists.newArrayList();
+        for (PColumn column : PTableImpl.getColumnsToClone(view)) {
+        	boolean isViewReferenced = isViewColumnReferencedToBe.get(column.getPosition());
+        	if ( (visitor.isUpdatable() || view.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(view)).equals(column)) 
+        			&& viewColumnConstantsToBe[column.getPosition()] != null) {
+				result.add(new PColumnImpl(column, viewColumnConstantsToBe[column.getPosition()], isViewReferenced));
+				viewColumnConstantsMatched[column.getPosition()]=true;
+            }
+        	// If view is not updatable, viewColumnConstants should be empty. We will still
+            // inherit our parent viewConstants, but we have no additional ones.
+        	else if(isViewReferenced ){
+        		result.add(new PColumnImpl(column, column.getViewConstant(), isViewReferenced));
+        	}
+        	else {
+                result.add(column);
+            }
+        }
+        return PTableImpl.makePTable(view, result);
+    }
+
+    private static PhoenixConnection getConnectionlessConnection() throws SQLException {
+        return DriverManager
+            .getConnection(JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + CONNECTIONLESS)
+            .unwrap(PhoenixConnection.class);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
index 8119c6e..744dc7e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
@@ -105,6 +105,14 @@ public final class MetaDataProtos {
      * <code>TOO_MANY_INDEXES = 22;</code>
      */
     TOO_MANY_INDEXES(22, 22),
+    /**
+     * <code>UNABLE_TO_CREATE_CHILD_LINK = 23;</code>
+     */
+    UNABLE_TO_CREATE_CHILD_LINK(23, 23),
+    /**
+     * <code>UNABLE_TO_UPDATE_PARENT_TABLE = 24;</code>
+     */
+    UNABLE_TO_UPDATE_PARENT_TABLE(24, 24),
     ;
 
     /**
@@ -199,6 +207,14 @@ public final class MetaDataProtos {
      * <code>TOO_MANY_INDEXES = 22;</code>
      */
     public static final int TOO_MANY_INDEXES_VALUE = 22;
+    /**
+     * <code>UNABLE_TO_CREATE_CHILD_LINK = 23;</code>
+     */
+    public static final int UNABLE_TO_CREATE_CHILD_LINK_VALUE = 23;
+    /**
+     * <code>UNABLE_TO_UPDATE_PARENT_TABLE = 24;</code>
+     */
+    public static final int UNABLE_TO_UPDATE_PARENT_TABLE_VALUE = 24;
 
 
     public final int getNumber() { return value; }
@@ -228,6 +244,8 @@ public final class MetaDataProtos {
         case 20: return AUTO_PARTITION_SEQUENCE_NOT_FOUND;
         case 21: return CANNOT_COERCE_AUTO_PARTITION_ID;
         case 22: return TOO_MANY_INDEXES;
+        case 23: return UNABLE_TO_CREATE_CHILD_LINK;
+        case 24: return UNABLE_TO_UPDATE_PARENT_TABLE;
         default: return null;
       }
     }
@@ -3948,6 +3966,40 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 6;</code>
      */
     int getClientVersion();
+
+    // optional bool skipAddingParentColumns = 7;
+    /**
+     * <code>optional bool skipAddingParentColumns = 7;</code>
+     */
+    boolean hasSkipAddingParentColumns();
+    /**
+     * <code>optional bool skipAddingParentColumns = 7;</code>
+     */
+    boolean getSkipAddingParentColumns();
+
+    // optional bool skipAddingIndexes = 8;
+    /**
+     * <code>optional bool skipAddingIndexes = 8;</code>
+     */
+    boolean hasSkipAddingIndexes();
+    /**
+     * <code>optional bool skipAddingIndexes = 8;</code>
+     */
+    boolean getSkipAddingIndexes();
+
+    // optional .PTable lockedAncestorTable = 9;
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    boolean hasLockedAncestorTable();
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable();
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder();
   }
   /**
    * Protobuf type {@code GetTableRequest}
@@ -4030,6 +4082,29 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
+            case 56: {
+              bitField0_ |= 0x00000040;
+              skipAddingParentColumns_ = input.readBool();
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000080;
+              skipAddingIndexes_ = input.readBool();
+              break;
+            }
+            case 74: {
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000100) == 0x00000100)) {
+                subBuilder = lockedAncestorTable_.toBuilder();
+              }
+              lockedAncestorTable_ = input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(lockedAncestorTable_);
+                lockedAncestorTable_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000100;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4166,6 +4241,60 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
+    // optional bool skipAddingParentColumns = 7;
+    public static final int SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER = 7;
+    private boolean skipAddingParentColumns_;
+    /**
+     * <code>optional bool skipAddingParentColumns = 7;</code>
+     */
+    public boolean hasSkipAddingParentColumns() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional bool skipAddingParentColumns = 7;</code>
+     */
+    public boolean getSkipAddingParentColumns() {
+      return skipAddingParentColumns_;
+    }
+
+    // optional bool skipAddingIndexes = 8;
+    public static final int SKIPADDINGINDEXES_FIELD_NUMBER = 8;
+    private boolean skipAddingIndexes_;
+    /**
+     * <code>optional bool skipAddingIndexes = 8;</code>
+     */
+    public boolean hasSkipAddingIndexes() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional bool skipAddingIndexes = 8;</code>
+     */
+    public boolean getSkipAddingIndexes() {
+      return skipAddingIndexes_;
+    }
+
+    // optional .PTable lockedAncestorTable = 9;
+    public static final int LOCKEDANCESTORTABLE_FIELD_NUMBER = 9;
+    private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable lockedAncestorTable_;
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    public boolean hasLockedAncestorTable() {
+      return ((bitField0_ & 0x00000100) == 0x00000100);
+    }
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable() {
+      return lockedAncestorTable_;
+    }
+    /**
+     * <code>optional .PTable lockedAncestorTable = 9;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder() {
+      return lockedAncestorTable_;
+    }
+
     private void initFields() {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       schemaName_ = com.google.protobuf.ByteString.EMPTY;
@@ -4173,6 +4302,9 @@ public final class MetaDataProtos {
       tableTimestamp_ = 0L;
       clientTimestamp_ = 0L;
       clientVersion_ = 0;
+      skipAddingParentColumns_ = false;
+      skipAddingIndexes_ = false;
+      lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4199,6 +4331,12 @@ public final class MetaDataProtos {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (hasLockedAncestorTable()) {
+        if (!getLockedAncestorTable().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -4224,6 +4362,15 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeInt32(6, clientVersion_);
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeBool(7, skipAddingParentColumns_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeBool(8, skipAddingIndexes_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeMessage(9, lockedAncestorTable_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4257,6 +4404,18 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(6, clientVersion_);
       }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(7, skipAddingParentColumns_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(8, skipAddingIndexes_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(9, lockedAncestorTable_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4310,6 +4469,21 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
+      result = result && (hasSkipAddingParentColumns() == other.hasSkipAddingParentColumns());
+      if (hasSkipAddingParentColumns()) {
+        result = result && (getSkipAddingParentColumns()
+            == other.getSkipAddingParentColumns());
+      }
+      result = result && (hasSkipAddingIndexes() == other.hasSkipAddingIndexes());
+      if (hasSkipAddingIndexes()) {
+        result = result && (getSkipAddingIndexes()
+            == other.getSkipAddingIndexes());
+      }
+      result = result && (hasLockedAncestorTable() == other.hasLockedAncestorTable());
+      if (hasLockedAncestorTable()) {
+        result = result && getLockedAncestorTable()
+            .equals(other.getLockedAncestorTable());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4347,6 +4521,18 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
+      if (hasSkipAddingParentColumns()) {
+        hash = (37 * hash) + SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipAddingParentColumns());
+      }
+      if (hasSkipAddingIndexes()) {
+        hash = (37 * hash) + SKIPADDINGINDEXES_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipAddingIndexes());
+      }
+      if (hasLockedAncestorTable()) {
+        hash = (37 * hash) + LOCKEDANCESTORTABLE_FIELD_NUMBER;
+        hash = (53 * hash) + getLockedAncestorTable().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4448,6 +4634,7 @@ public final class MetaDataProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getLockedAncestorTableFieldBuilder();
         }
       }
       private static Builder create() {
@@ -4468,6 +4655,16 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000020);
+        skipAddingParentColumns_ = false;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        skipAddingIndexes_ = false;
+        bitField0_ = (bitField0_ & ~0x00000080);
+        if (lockedAncestorTableBuilder_ == null) {
+          lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+        } else {
+          lockedAncestorTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000100);
         return this;
       }
 
@@ -4520,6 +4717,22 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000020;
         }
         result.clientVersion_ = clientVersion_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.skipAddingParentColumns_ = skipAddingParentColumns_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.skipAddingIndexes_ = skipAddingIndexes_;
+        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+          to_bitField0_ |= 0x00000100;
+        }
+        if (lockedAncestorTableBuilder_ == null) {
+          result.lockedAncestorTable_ = lockedAncestorTable_;
+        } else {
+          result.lockedAncestorTable_ = lockedAncestorTableBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -4554,6 +4767,15 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
+        if (other.hasSkipAddingParentColumns()) {
+          setSkipAddingParentColumns(other.getSkipAddingParentColumns());
+        }
+        if (other.hasSkipAddingIndexes()) {
+          setSkipAddingIndexes(other.getSkipAddingIndexes());
+        }
+        if (other.hasLockedAncestorTable()) {
+          mergeLockedAncestorTable(other.getLockedAncestorTable());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4579,6 +4801,12 @@ public final class MetaDataProtos {
           
           return false;
         }
+        if (hasLockedAncestorTable()) {
+          if (!getLockedAncestorTable().isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -4808,6 +5036,189 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional bool skipAddingParentColumns = 7;
+      private boolean skipAddingParentColumns_ ;
+      /**
+       * <code>optional bool skipAddingParentColumns = 7;</code>
+       */
+      public boolean hasSkipAddingParentColumns() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 7;</code>
+       */
+      public boolean getSkipAddingParentColumns() {
+        return skipAddingParentColumns_;
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 7;</code>
+       */
+      public Builder setSkipAddingParentColumns(boolean value) {
+        bitField0_ |= 0x00000040;
+        skipAddingParentColumns_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 7;</code>
+       */
+      public Builder clearSkipAddingParentColumns() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        skipAddingParentColumns_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional bool skipAddingIndexes = 8;
+      private boolean skipAddingIndexes_ ;
+      /**
+       * <code>optional bool skipAddingIndexes = 8;</code>
+       */
+      public boolean hasSkipAddingIndexes() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional bool skipAddingIndexes = 8;</code>
+       */
+      public boolean getSkipAddingIndexes() {
+        return skipAddingIndexes_;
+      }
+      /**
+       * <code>optional bool skipAddingIndexes = 8;</code>
+       */
+      public Builder setSkipAddingIndexes(boolean value) {
+        bitField0_ |= 0x00000080;
+        skipAddingIndexes_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool skipAddingIndexes = 8;</code>
+       */
+      public Builder clearSkipAddingIndexes() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        skipAddingIndexes_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional .PTable lockedAncestorTable = 9;
+      private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> lockedAncestorTableBuilder_;
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public boolean hasLockedAncestorTable() {
+        return ((bitField0_ & 0x00000100) == 0x00000100);
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable() {
+        if (lockedAncestorTableBuilder_ == null) {
+          return lockedAncestorTable_;
+        } else {
+          return lockedAncestorTableBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public Builder setLockedAncestorTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (lockedAncestorTableBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          lockedAncestorTable_ = value;
+          onChanged();
+        } else {
+          lockedAncestorTableBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000100;
+        return this;
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public Builder setLockedAncestorTable(
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder builderForValue) {
+        if (lockedAncestorTableBuilder_ == null) {
+          lockedAncestorTable_ = builderForValue.build();
+          onChanged();
+        } else {
+          lockedAncestorTableBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000100;
+        return this;
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public Builder mergeLockedAncestorTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (lockedAncestorTableBuilder_ == null) {
+          if (((bitField0_ & 0x00000100) == 0x00000100) &&
+              lockedAncestorTable_ != org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance()) {
+            lockedAncestorTable_ =
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.newBuilder(lockedAncestorTable_).mergeFrom(value).buildPartial();
+          } else {
+            lockedAncestorTable_ = value;
+          }
+          onChanged();
+        } else {
+          lockedAncestorTableBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000100;
+        return this;
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public Builder clearLockedAncestorTable() {
+        if (lockedAncestorTableBuilder_ == null) {
+          lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+          onChanged();
+        } else {
+          lockedAncestorTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000100);
+        return this;
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder getLockedAncestorTableBuilder() {
+        bitField0_ |= 0x00000100;
+        onChanged();
+        return getLockedAncestorTableFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder() {
+        if (lockedAncestorTableBuilder_ != null) {
+          return lockedAncestorTableBuilder_.getMessageOrBuilder();
+        } else {
+          return lockedAncestorTable_;
+        }
+      }
+      /**
+       * <code>optional .PTable lockedAncestorTable = 9;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> 
+          getLockedAncestorTableFieldBuilder() {
+        if (lockedAncestorTableBuilder_ == null) {
+          lockedAncestorTableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder>(
+                  lockedAncestorTable_,
+                  getParentForChildren(),
+                  isClean());
+          lockedAncestorTable_ = null;
+        }
+        return lockedAncestorTableBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:GetTableRequest)
     }
 
@@ -8780,6 +9191,16 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 4;</code>
      */
     int getClientVersion();
+
+    // optional bool skipAddingParentColumns = 5;
+    /**
+     * <code>optional bool skipAddingParentColumns = 5;</code>
+     */
+    boolean hasSkipAddingParentColumns();
+    /**
+     * <code>optional bool skipAddingParentColumns = 5;</code>
+     */
+    boolean getSkipAddingParentColumns();
   }
   /**
    * Protobuf type {@code DropTableRequest}
@@ -8855,6 +9276,11 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
+            case 40: {
+              bitField0_ |= 0x00000008;
+              skipAddingParentColumns_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -8996,11 +9422,28 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
+    // optional bool skipAddingParentColumns = 5;
+    public static final int SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER = 5;
+    private boolean skipAddingParentColumns_;
+    /**
+     * <code>optional bool skipAddingParentColumns = 5;</code>
+     */
+    public boolean hasSkipAddingParentColumns() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional bool skipAddingParentColumns = 5;</code>
+     */
+    public boolean getSkipAddingParentColumns() {
+      return skipAddingParentColumns_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       tableType_ = "";
       cascade_ = false;
       clientVersion_ = 0;
+      skipAddingParentColumns_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -9030,6 +9473,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeInt32(4, clientVersion_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(5, skipAddingParentColumns_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -9060,6 +9506,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(4, clientVersion_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(5, skipAddingParentColumns_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -9100,6 +9550,11 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
+      result = result && (hasSkipAddingParentColumns() == other.hasSkipAddingParentColumns());
+      if (hasSkipAddingParentColumns()) {
+        result = result && (getSkipAddingParentColumns()
+            == other.getSkipAddingParentColumns());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -9129,6 +9584,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
+      if (hasSkipAddingParentColumns()) {
+        hash = (37 * hash) + SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipAddingParentColumns());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -9246,6 +9705,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000004);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000008);
+        skipAddingParentColumns_ = false;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -9291,6 +9752,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000004;
         }
         result.clientVersion_ = clientVersion_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.skipAddingParentColumns_ = skipAddingParentColumns_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -9328,6 +9793,9 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
+        if (other.hasSkipAddingParentColumns()) {
+          setSkipAddingParentColumns(other.getSkipAddingParentColumns());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -9571,6 +10039,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional bool skipAddingParentColumns = 5;
+      private boolean skipAddingParentColumns_ ;
+      /**
+       * <code>optional bool skipAddingParentColumns = 5;</code>
+       */
+      public boolean hasSkipAddingParentColumns() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 5;</code>
+       */
+      public boolean getSkipAddingParentColumns() {
+        return skipAddingParentColumns_;
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 5;</code>
+       */
+      public Builder setSkipAddingParentColumns(boolean value) {
+        bitField0_ |= 0x00000010;
+        skipAddingParentColumns_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool skipAddingParentColumns = 5;</code>
+       */
+      public Builder clearSkipAddingParentColumns() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        skipAddingParentColumns_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:DropTableRequest)
     }
 
@@ -17158,83 +17659,88 @@ public final class MetaDataProtos {
       "\030\007 \001(\014\022\034\n\010function\030\010 \003(\0132\n.PFunction\022/\n\024" +
       "sharedTablesToDelete\030\t \003(\0132\021.SharedTable" +
       "State\022\030\n\006schema\030\n \001(\0132\010.PSchema\022\030\n\020autoP" +
-      "artitionNum\030\013 \001(\003\022\023\n\013viewIndexId\030\014 \001(\005\"\222" +
+      "artitionNum\030\013 \001(\003\022\023\n\013viewIndexId\030\014 \001(\005\"\364" +
       "\001\n\017GetTableRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\n" +
       "schemaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\026\n\016t" +
       "ableTimestamp\030\004 \002(\003\022\027\n\017clientTimestamp\030\005" +
-      " \002(\003\022\025\n\rclientVersion\030\006 \001(\005\"\212\001\n\023GetFunct" +
-      "ionsRequest\022\020\n\010tenantId\030\001 \002(\014\022\025\n\rfunctio" +
-      "nNames\030\002 \003(\014\022\032\n\022functionTimestamps\030\003 \003(\003",
-      "\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n\rclientVersi" +
-      "on\030\005 \001(\005\"V\n\020GetSchemaRequest\022\022\n\nschemaNa" +
-      "me\030\001 \002(\t\022\027\n\017clientTimestamp\030\002 \002(\003\022\025\n\rcli" +
-      "entVersion\030\003 \002(\005\"d\n\022CreateTableRequest\022\036" +
-      "\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rclient" +
-      "Version\030\002 \001(\005\022\027\n\017allocateIndexId\030\003 \001(\010\"r" +
-      "\n\025CreateFunctionRequest\022\036\n\026tableMetadata" +
-      "Mutations\030\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007re" +
-      "place\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\"`\n\023Cr" +
-      "eateSchemaRequest\022\036\n\026tableMetadataMutati",
-      "ons\030\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientV" +
-      "ersion\030\003 \002(\005\"m\n\020DropTableRequest\022\036\n\026tabl" +
-      "eMetadataMutations\030\001 \003(\014\022\021\n\ttableType\030\002 " +
-      "\002(\t\022\017\n\007cascade\030\003 \001(\010\022\025\n\rclientVersion\030\004 " +
-      "\001(\005\"_\n\021DropSchemaRequest\022\037\n\027schemaMetada" +
-      "taMutations\030\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n" +
-      "\rclientVersion\030\003 \002(\005\"I\n\020AddColumnRequest" +
-      "\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rclie" +
-      "ntVersion\030\002 \001(\005\"J\n\021DropColumnRequest\022\036\n\026" +
-      "tableMetadataMutations\030\001 \003(\014\022\025\n\rclientVe",
-      "rsion\030\002 \001(\005\"^\n\023DropFunctionRequest\022\036\n\026ta" +
-      "bleMetadataMutations\030\001 \003(\014\022\020\n\010ifExists\030\002" +
-      " \001(\010\022\025\n\rclientVersion\030\003 \001(\005\"P\n\027UpdateInd" +
-      "exStateRequest\022\036\n\026tableMetadataMutations" +
-      "\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"*\n\021ClearCa" +
-      "cheRequest\022\025\n\rclientVersion\030\001 \001(\005\"*\n\022Cle" +
-      "arCacheResponse\022\024\n\014unfreedBytes\030\001 \001(\003\"*\n" +
-      "\021GetVersionRequest\022\025\n\rclientVersion\030\001 \001(" +
-      "\005\"E\n\022GetVersionResponse\022\017\n\007version\030\001 \002(\003" +
-      "\022\036\n\026systemCatalogTimestamp\030\002 \001(\003\"\205\001\n\032Cle",
-      "arTableFromCacheRequest\022\020\n\010tenantId\030\001 \002(" +
-      "\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014" +
-      "\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n\rclientVersi" +
-      "on\030\005 \001(\005\"\035\n\033ClearTableFromCacheResponse*" +
-      "\365\004\n\014MutationCode\022\030\n\024TABLE_ALREADY_EXISTS" +
-      "\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FO" +
-      "UND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CONC" +
-      "URRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_" +
-      "REGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNALL" +
-      "OWED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010",
-      "\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t\022\033\n\027FUNCTION" +
-      "_ALREADY_EXISTS\020\n\022\026\n\022FUNCTION_NOT_FOUND\020" +
-      "\013\022\030\n\024NEWER_FUNCTION_FOUND\020\014\022\032\n\026FUNCTION_" +
-      "NOT_IN_REGION\020\r\022\031\n\025SCHEMA_ALREADY_EXISTS" +
-      "\020\016\022\026\n\022NEWER_SCHEMA_FOUND\020\017\022\024\n\020SCHEMA_NOT" +
-      "_FOUND\020\020\022\030\n\024SCHEMA_NOT_IN_REGION\020\021\022\032\n\026TA" +
-      "BLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UNALLOWED_SCHE" +
-      "MA_MUTATION\020\023\022%\n!AUTO_PARTITION_SEQUENCE" +
-      "_NOT_FOUND\020\024\022#\n\037CANNOT_COERCE_AUTO_PARTI" +
-      "TION_ID\020\025\022\024\n\020TOO_MANY_INDEXES\020\0262\345\006\n\017Meta",
+      " \002(\003\022\025\n\rclientVersion\030\006 \001(\005\022\037\n\027skipAddin" +
+      "gParentColumns\030\007 \001(\010\022\031\n\021skipAddingIndexe" +
+      "s\030\010 \001(\010\022$\n\023lockedAncestorTable\030\t \001(\0132\007.P",
+      "Table\"\212\001\n\023GetFunctionsRequest\022\020\n\010tenantI" +
+      "d\030\001 \002(\014\022\025\n\rfunctionNames\030\002 \003(\014\022\032\n\022functi" +
+      "onTimestamps\030\003 \003(\003\022\027\n\017clientTimestamp\030\004 " +
+      "\002(\003\022\025\n\rclientVersion\030\005 \001(\005\"V\n\020GetSchemaR" +
+      "equest\022\022\n\nschemaName\030\001 \002(\t\022\027\n\017clientTime" +
+      "stamp\030\002 \002(\003\022\025\n\rclientVersion\030\003 \002(\005\"d\n\022Cr" +
+      "eateTableRequest\022\036\n\026tableMetadataMutatio" +
+      "ns\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\022\027\n\017alloc" +
+      "ateIndexId\030\003 \001(\010\"r\n\025CreateFunctionReques" +
+      "t\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\021\n\ttem",
+      "porary\030\002 \002(\010\022\017\n\007replace\030\003 \001(\010\022\025\n\rclientV" +
+      "ersion\030\004 \001(\005\"`\n\023CreateSchemaRequest\022\036\n\026t" +
+      "ableMetadataMutations\030\001 \003(\014\022\022\n\nschemaNam" +
+      "e\030\002 \002(\t\022\025\n\rclientVersion\030\003 \002(\005\"\216\001\n\020DropT" +
+      "ableRequest\022\036\n\026tableMetadataMutations\030\001 " +
+      "\003(\014\022\021\n\ttableType\030\002 \002(\t\022\017\n\007cascade\030\003 \001(\010\022" +
+      "\025\n\rclientVersion\030\004 \001(\005\022\037\n\027skipAddingPare" +
+      "ntColumns\030\005 \001(\010\"_\n\021DropSchemaRequest\022\037\n\027" +
+      "schemaMetadataMutations\030\001 \003(\014\022\022\n\nschemaN" +
+      "ame\030\002 \002(\t\022\025\n\rclientVersion\030\003 \002(\005\"I\n\020AddC",
+      "olumnRequest\022\036\n\026tableMetadataMutations\030\001" +
+      " \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"J\n\021DropColum" +
+      "nRequest\022\036\n\026tableMetadataMutations\030\001 \003(\014" +
+      "\022\025\n\rclientVersion\030\002 \001(\005\"^\n\023DropFunctionR" +
+      "equest\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\020" +
+      "\n\010ifExists\030\002 \001(\010\022\025\n\rclientVersion\030\003 \001(\005\"" +
+      "P\n\027UpdateIndexStateRequest\022\036\n\026tableMetad" +
+      "ataMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(" +
+      "\005\"*\n\021ClearCacheRequest\022\025\n\rclientVersion\030" +
+      "\001 \001(\005\"*\n\022ClearCacheResponse\022\024\n\014unfreedBy",
+      "tes\030\001 \001(\003\"*\n\021GetVersionRequest\022\025\n\rclient" +
+      "Version\030\001 \001(\005\"E\n\022GetVersionResponse\022\017\n\007v" +
+      "ersion\030\001 \002(\003\022\036\n\026systemCatalogTimestamp\030\002" +
+      " \001(\003\"\205\001\n\032ClearTableFromCacheRequest\022\020\n\010t" +
+      "enantId\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttab" +
+      "leName\030\003 \002(\014\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n" +
+      "\rclientVersion\030\005 \001(\005\"\035\n\033ClearTableFromCa" +
+      "cheResponse*\271\005\n\014MutationCode\022\030\n\024TABLE_AL" +
+      "READY_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020C" +
+      "OLUMN_NOT_FOUND\020\002\022\031\n\025COLUMN_ALREADY_EXIS",
+      "TS\020\003\022\035\n\031CONCURRENT_TABLE_MUTATION\020\004\022\027\n\023T" +
+      "ABLE_NOT_IN_REGION\020\005\022\025\n\021NEWER_TABLE_FOUN" +
+      "D\020\006\022\034\n\030UNALLOWED_TABLE_MUTATION\020\007\022\021\n\rNO_" +
+      "PK_COLUMNS\020\010\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t" +
+      "\022\033\n\027FUNCTION_ALREADY_EXISTS\020\n\022\026\n\022FUNCTIO" +
+      "N_NOT_FOUND\020\013\022\030\n\024NEWER_FUNCTION_FOUND\020\014\022" +
+      "\032\n\026FUNCTION_NOT_IN_REGION\020\r\022\031\n\025SCHEMA_AL" +
+      "READY_EXISTS\020\016\022\026\n\022NEWER_SCHEMA_FOUND\020\017\022\024" +
+      "\n\020SCHEMA_NOT_FOUND\020\020\022\030\n\024SCHEMA_NOT_IN_RE" +
+      "GION\020\021\022\032\n\026TABLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UN",
+      "ALLOWED_SCHEMA_MUTATION\020\023\022%\n!AUTO_PARTIT" +
+      "ION_SEQUENCE_NOT_FOUND\020\024\022#\n\037CANNOT_COERC" +
+      "E_AUTO_PARTITION_ID\020\025\022\024\n\020TOO_MANY_INDEXE" +
+      "S\020\026\022\037\n\033UNABLE_TO_CREATE_CHILD_LINK\020\027\022!\n\035" +
+      "UNABLE_TO_UPDATE_PARENT_TABLE\020\0302\345\006\n\017Meta" +
       "DataService\022/\n\010getTable\022\020.GetTableReques" +
       "t\032\021.MetaDataResponse\0227\n\014getFunctions\022\024.G" +
       "etFunctionsRequest\032\021.MetaDataResponse\0221\n" +
       "\tgetSchema\022\021.GetSchemaRequest\032\021.MetaData" +
-      "Response\0225\n\013createTable\022\023.CreateTableReq" +
+      "Response\0225\n\013createTable\022\023.CreateTableReq",
       "uest\032\021.MetaDataResponse\022;\n\016createFunctio" +
       "n\022\026.CreateFunctionRequest\032\021.MetaDataResp" +
       "onse\0227\n\014createSchema\022\024.CreateSchemaReque" +
       "st\032\021.MetaDataResponse\0221\n\tdropTable\022\021.Dro" +
-      "pTableRequest\032\021.MetaDataResponse\0223\n\ndrop",
+      "pTableRequest\032\021.MetaDataResponse\0223\n\ndrop" +
       "Schema\022\022.DropSchemaRequest\032\021.MetaDataRes" +
       "ponse\0227\n\014dropFunction\022\024.DropFunctionRequ" +
       "est\032\021.MetaDataResponse\0221\n\taddColumn\022\021.Ad" +
       "dColumnRequest\032\021.MetaDataResponse\0223\n\ndro" +
-      "pColumn\022\022.DropColumnRequest\032\021.MetaDataRe" +
+      "pColumn\022\022.DropColumnRequest\032\021.MetaDataRe",
       "sponse\022?\n\020updateIndexState\022\030.UpdateIndex" +
       "StateRequest\032\021.MetaDataResponse\0225\n\nclear" +
       "Cache\022\022.ClearCacheRequest\032\023.ClearCacheRe" +
       "sponse\0225\n\ngetVersion\022\022.GetVersionRequest" +
-      "\032\023.GetVersionResponse\022P\n\023clearTableFromC",
+      "\032\023.GetVersionResponse\022P\n\023clearTableFromC" +
       "ache\022\033.ClearTableFromCacheRequest\032\034.Clea" +
       "rTableFromCacheResponseBB\n(org.apache.ph" +
       "oenix.coprocessor.generatedB\016MetaDataPro" +
@@ -17262,7 +17768,7 @@ public final class MetaDataProtos {
           internal_static_GetTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetTableRequest_descriptor,
-              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "TableTimestamp", "ClientTimestamp", "ClientVersion", });
+              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "TableTimestamp", "ClientTimestamp", "ClientVersion", "SkipAddingParentColumns", "SkipAddingIndexes", "LockedAncestorTable", });
           internal_static_GetFunctionsRequest_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_GetFunctionsRequest_fieldAccessorTable = new
@@ -17298,7 +17804,7 @@ public final class MetaDataProtos {
           internal_static_DropTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_DropTableRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", "ClientVersion", });
+              new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", "ClientVersion", "SkipAddingParentColumns", });
           internal_static_DropSchemaRequest_descriptor =
             getDescriptor().getMessageTypes().get(9);
           internal_static_DropSchemaRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index 8d500e8..ab61826 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -140,17 +140,17 @@ public final class PTableProtos {
      */
     com.google.protobuf.ByteString getFamilyNameBytes();
 
-    // required string dataType = 3;
+    // optional string dataType = 3;
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     boolean hasDataType();
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     java.lang.String getDataType();
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     com.google.protobuf.ByteString
         getDataTypeBytes();
@@ -195,13 +195,13 @@ public final class PTableProtos {
      */
     int getPosition();
 
-    // required int32 sortOrder = 8;
+    // optional int32 sortOrder = 8;
     /**
-     * <code>required int32 sortOrder = 8;</code>
+     * <code>optional int32 sortOrder = 8;</code>
      */
     boolean hasSortOrder();
     /**
-     * <code>required int32 sortOrder = 8;</code>
+     * <code>optional int32 sortOrder = 8;</code>
      */
     int getSortOrder();
 
@@ -279,6 +279,26 @@ public final class PTableProtos {
      * <code>optional bytes columnQualifierBytes = 15;</code>
      */
     com.google.protobuf.ByteString getColumnQualifierBytes();
+
+    // optional int64 timestamp = 16;
+    /**
+     * <code>optional int64 timestamp = 16;</code>
+     */
+    boolean hasTimestamp();
+    /**
+     * <code>optional int64 timestamp = 16;</code>
+     */
+    long getTimestamp();
+
+    // optional bool derived = 17 [default = false];
+    /**
+     * <code>optional bool derived = 17 [default = false];</code>
+     */
+    boolean hasDerived();
+    /**
+     * <code>optional bool derived = 17 [default = false];</code>
+     */
+    boolean getDerived();
   }
   /**
    * Protobuf type {@code PColumn}
@@ -406,6 +426,16 @@ public final class PTableProtos {
               columnQualifierBytes_ = input.readBytes();
               break;
             }
+            case 128: {
+              bitField0_ |= 0x00008000;
+              timestamp_ = input.readInt64();
+              break;
+            }
+            case 136: {
+              bitField0_ |= 0x00010000;
+              derived_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -478,17 +508,17 @@ public final class PTableProtos {
       return familyNameBytes_;
     }
 
-    // required string dataType = 3;
+    // optional string dataType = 3;
     public static final int DATATYPE_FIELD_NUMBER = 3;
     private java.lang.Object dataType_;
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     public boolean hasDataType() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     public java.lang.String getDataType() {
       java.lang.Object ref = dataType_;
@@ -505,7 +535,7 @@ public final class PTableProtos {
       }
     }
     /**
-     * <code>required string dataType = 3;</code>
+     * <code>optional string dataType = 3;</code>
      */
     public com.google.protobuf.ByteString
         getDataTypeBytes() {
@@ -585,17 +615,17 @@ public final class PTableProtos {
       return position_;
     }
 
-    // required int32 sortOrder = 8;
+    // optional int32 sortOrder = 8;
     public static final int SORTORDER_FIELD_NUMBER = 8;
     private int sortOrder_;
     /**
-     * <code>required int32 sortOrder = 8;</code>
+     * <code>optional int32 sortOrder = 8;</code>
      */
     public boolean hasSortOrder() {
       return ((bitField0_ & 0x00000080) == 0x00000080);
     }
     /**
-     * <code>required int32 sortOrder = 8;</code>
+     * <code>optional int32 sortOrder = 8;</code>
      */
     public int getSortOrder() {
       return sortOrder_;
@@ -740,6 +770,38 @@ public final class PTableProtos {
       return columnQualifierBytes_;
     }
 
+    // optional int64 timestamp = 16;
+    public static final int TIMESTAMP_FIELD_NUMBER = 16;
+    private long timestamp_;
+    /**
+     * <code>optional int64 timestamp = 16;</code>
+     */
+    public boolean hasTimestamp() {
+      return ((bitField0_ & 0x00008000) == 0x00008000);
+    }
+    /**
+     * <code>optional int64 timestamp = 16;</code>
+     */
+    public long getTimestamp() {
+      return timestamp_;
+    }
+
+    // optional bool derived = 17 [default = false];
+    public static final int DERIVED_FIELD_NUMBER = 17;
+    private boolean derived_;
+    /**
+     * <code>optional bool derived = 17 [default = false];</code>
+     */
+    public boolean hasDerived() {
+      return ((bitField0_ & 0x00010000) == 0x00010000);
+    }
+    /**
+     * <code>optional bool derived = 17 [default = false];</code>
+     */
+    public boolean getDerived() {
+      return derived_;
+    }
+
     private void initFields() {
       columnNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       familyNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -756,6 +818,8 @@ public final class PTableProtos {
       isRowTimestamp_ = false;
       isDynamic_ = false;
       columnQualifierBytes_ = com.google.protobuf.ByteString.EMPTY;
+      timestamp_ = 0L;
+      derived_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -766,10 +830,6 @@ public final class PTableProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasDataType()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
       if (!hasNullable()) {
         memoizedIsInitialized = 0;
         return false;
@@ -778,10 +838,6 @@ public final class PTableProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasSortOrder()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -834,6 +890,12 @@ public final class PTableProtos {
       if (((bitField0_ & 0x00004000) == 0x00004000)) {
         output.writeBytes(15, columnQualifierBytes_);
       }
+      if (((bitField0_ & 0x00008000) == 0x00008000)) {
+        output.writeInt64(16, timestamp_);
+      }
+      if (((bitField0_ & 0x00010000) == 0x00010000)) {
+        output.writeBool(17, derived_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -903,6 +965,14 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(15, columnQualifierBytes_);
       }
+      if (((bitField0_ & 0x00008000) == 0x00008000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(16, timestamp_);
+      }
+      if (((bitField0_ & 0x00010000) == 0x00010000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(17, derived_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -1001,6 +1071,16 @@ public final class PTableProtos {
         result = result && getColumnQualifierBytes()
             .equals(other.getColumnQualifierBytes());
       }
+      result = result && (hasTimestamp() == other.hasTimestamp());
+      if (hasTimestamp()) {
+        result = result && (getTimestamp()
+            == other.getTimestamp());
+      }
+      result = result && (hasDerived() == other.hasDerived());
+      if (hasDerived()) {
+        result = result && (getDerived()
+            == other.getDerived());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -1074,6 +1154,14 @@ public final class PTableProtos {
         hash = (37 * hash) + COLUMNQUALIFIERBYTES_FIELD_NUMBER;
         hash = (53 * hash) + getColumnQualifierBytes().hashCode();
       }
+      if (hasTimestamp()) {
+        hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getTimestamp());
+      }
+      if (hasDerived()) {
+        hash = (37 * hash) + DERIVED_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getDerived());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1213,6 +1301,10 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x00002000);
         columnQualifierBytes_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x00004000);
+        timestamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00008000);
+        derived_ = false;
+        bitField0_ = (bitField0_ & ~0x00010000);
         return this;
       }
 
@@ -1301,6 +1393,14 @@ public final class PTableProtos {
           to_bitField0_ |= 0x00004000;
         }
         result.columnQualifierBytes_ = columnQualifierBytes_;
+        if (((from_bitField0_ & 0x00008000) == 0x00008000)) {
+          to_bitField0_ |= 0x00008000;
+        }
+        result.timestamp_ = timestamp_;
+        if (((from_bitField0_ & 0x00010000) == 0x00010000)) {
+          to_bitField0_ |= 0x00010000;
+        }
+        result.derived_ = derived_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1366,6 +1466,12 @@ public final class PTableProtos {
         if (other.hasColumnQualifierBytes()) {
           setColumnQualifierBytes(other.getColumnQualifierBytes());
         }
+        if (other.hasTimestamp()) {
+          setTimestamp(other.getTimestamp());
+        }
+        if (other.hasDerived()) {
+          setDerived(other.getDerived());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1375,10 +1481,6 @@ public final class PTableProtos {
           
           return false;
         }
-        if (!hasDataType()) {
-          
-          return false;
-        }
         if (!hasNullable()) {
           
           return false;
@@ -1387,10 +1489,6 @@ public final class PTableProtos {
           
           return false;
         }
-        if (!hasSortOrder()) {
-          
-          return false;
-        }
         return true;
       }
 
@@ -1485,16 +1583,16 @@ public final class PTableProtos {
         return this;
       }
 
-      // required string dataType = 3;
+      // optional string dataType = 3;
       private java.lang.Object dataType_ = "";
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public boolean hasDataType() {
         return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public java.lang.String getDataType() {
         java.lang.Object ref = dataType_;
@@ -1508,7 +1606,7 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public com.google.protobuf.ByteString
           getDataTypeBytes() {
@@ -1524,7 +1622,7 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public Builder setDataType(
           java.lang.String value) {
@@ -1537,7 +1635,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public Builder clearDataType() {
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -1546,7 +1644,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>required string dataType = 3;</code>
+       * <code>optional string dataType = 3;</code>
        */
       public Builder setDataTypeBytes(
           com.google.protobuf.ByteString value) {
@@ -1691,22 +1789,22 @@ public final class PTableProtos {
         return this;
       }
 
-      // required int32 sortOrder = 8;
+      // optional int32 sortOrder = 8;
       private int sortOrder_ ;
       /**
-       * <code>required int32 sortOrder = 8;</code>
+       * <code>optional int32 sortOrder = 8;</code>
        */
       public boolean hasSortOrder() {
         return ((bitField0_ & 0x00000080) == 0x00000080);
       }
       /**
-       * <code>required int32 sortOrder = 8;</code>
+       * <code>optional int32 sortOrder = 8;</code>
        */
       public int getSortOrder() {
         return sortOrder_;
       }
       /**
-       * <code>required int32 sortOrder = 8;</code>
+       * <code>optional int32 sortOrder = 8;</code>
        */
       public Builder setSortOrder(int value) {
         bitField0_ |= 0x00000080;
@@ -1715,7 +1813,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>required int32 sortOrder = 8;</code>
+       * <code>optional int32 sortOrder = 8;</code>
        */
       public Builder clearSortOrder() {
         bitField0_ = (bitField0_ & ~0x00000080);
@@ -2002,6 +2100,72 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional int64 timestamp = 16;
+      private long timestamp_ ;
+      /**
+       * <code>optional int64 timestamp = 16;</code>
+       */
+      public boolean hasTimestamp() {
+        return ((bitField0_ & 0x00008000) == 0x00008000);
+      }
+      /**
+       * <code>optional int64 timestamp = 16;</code>
+       */
+      public long getTimestamp() {
+        return timestamp_;
+      }
+      /**
+       * <code>optional int64 timestamp = 16;</code>
+       */
+      public Builder setTimestamp(long value) {
+        bitField0_ |= 0x00008000;
+        timestamp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 timestamp = 16;</code>
+       */
+      public Builder clearTimestamp() {
+        bitField0_ = (bitField0_ & ~0x00008000);
+        timestamp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional bool derived = 17 [default = false];
+      private boolean derived_ ;
+      /**
+       * <code>optional bool derived = 17 [default = false];</code>
+       */
+      public boolean hasDerived() {
+        return ((bitField0_ & 0x00010000) == 0x00010000);
+      }
+      /**
+       * <code>optional bool derived = 17 [default = false];</code>
+       */
+      public boolean getDerived() {
+        return derived_;
+      }
+      /**
+       * <code>optional bool derived = 17 [default = false];</code>
+       */
+      public Builder setDerived(boolean value) {
+        bitField0_ |= 0x00010000;
+        derived_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool derived = 17 [default = false];</code>
+       */
+      public Builder clearDerived() {
+        bitField0_ = (bitField0_ & ~0x00010000);
+        derived_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PColumn)
     }
 
@@ -8668,48 +8832,49 @@ public final class PTableProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\014PTable.proto\032\021PGuidePosts.proto\"\304\002\n\007PC" +
+      "\n\014PTable.proto\032\021PGuidePosts.proto\"\357\002\n\007PC" +
       "olumn\022\027\n\017columnNameBytes\030\001 \002(\014\022\027\n\017family" +
-      "NameBytes\030\002 \001(\014\022\020\n\010dataType\030\003 \002(\t\022\021\n\tmax" +
+      "NameBytes\030\002 \001(\014\022\020\n\010dataType\030\003 \001(\t\022\021\n\tmax" +
       "Length\030\004 \001(\005\022\r\n\005scale\030\005 \001(\005\022\020\n\010nullable\030" +
-      "\006 \002(\010\022\020\n\010position\030\007 \002(\005\022\021\n\tsortOrder\030\010 \002" +
+      "\006 \002(\010\022\020\n\010position\030\007 \002(\005\022\021\n\tsortOrder\030\010 \001" +
       "(\005\022\021\n\tarraySize\030\t \001(\005\022\024\n\014viewConstant\030\n " +
       "\001(\014\022\026\n\016viewReferenced\030\013 \001(\010\022\022\n\nexpressio" +
       "n\030\014 \001(\t\022\026\n\016isRowTimestamp\030\r \001(\010\022\021\n\tisDyn" +
       "amic\030\016 \001(\010\022\034\n\024columnQualifierBytes\030\017 \001(\014" +
-      "\"\232\001\n\013PTableStats\022\013\n\003key\030\001 \002(\014\022\016\n\006values\030",
-      "\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n\rke" +
-      "yBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030\005 \001" +
-      "(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts\"\255\007" +
-      "\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016tab" +
-      "leNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162\013.P" +
-      "TableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequenc" +
-      "eNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013pkNa" +
-      "meBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007colu" +
-      "mns\030\t \003(\0132\010.PColumn\022\030\n\007indexes\030\n \003(\0132\007.P" +
-      "Table\022\027\n\017isImmutableRows\030\013 \002(\010\022\032\n\022dataTa",
-      "bleNameBytes\030\r \001(\014\022\031\n\021defaultFamilyName\030" +
-      "\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTenant" +
-      "\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewStatemen" +
-      "t\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010tenant" +
-      "Id\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tindexTy" +
-      "pe\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\nstor" +
-      "eNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(\005\022\036\n" +
-      "\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtransac" +
-      "tional\030\033 \001(\010\022\034\n\024updateCacheFrequency\030\034 \001" +
-      "(\003\022\035\n\025indexDisableTimestamp\030\035 \001(\003\022\031\n\021isN",
-      "amespaceMapped\030\036 \001(\010\022\034\n\024autoParititonSeq" +
-      "Name\030\037 \001(\t\022\032\n\022isAppendOnlySchema\030  \001(\010\022\027" +
-      "\n\017parentNameBytes\030! \001(\014\022\025\n\rstorageScheme" +
-      "\030\" \001(\014\022\026\n\016encodingScheme\030# \001(\014\022,\n\021encode" +
-      "dCQCounters\030$ \003(\0132\021.EncodedCQCounter\022\"\n\032" +
-      "useStatsForParallelization\030% \001(\010\022\033\n\023tran" +
-      "sactionProvider\030& \001(\005\"6\n\020EncodedCQCounte" +
-      "r\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007counter\030\002 \002(\005*A\n" +
-      "\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIE" +
-      "W\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.p",
-      "hoenix.coprocessor.generatedB\014PTableProt" +
-      "osH\001\210\001\001\240\001\001"
+      "\022\021\n\ttimestamp\030\020 \001(\003\022\026\n\007derived\030\021 \001(\010:\005fa",
+      "lse\"\232\001\n\013PTableStats\022\013\n\003key\030\001 \002(\014\022\016\n\006valu" +
+      "es\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n" +
+      "\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030" +
+      "\005 \001(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts" +
+      "\"\255\007\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016" +
+      "tableNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162" +
+      "\013.PTableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequ" +
+      "enceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013p" +
+      "kNameBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007c" +
+      "olumns\030\t \003(\0132\010.PColumn\022\030\n\007indexes\030\n \003(\0132",
+      "\007.PTable\022\027\n\017isImmutableRows\030\013 \002(\010\022\032\n\022dat" +
+      "aTableNameBytes\030\r \001(\014\022\031\n\021defaultFamilyNa" +
+      "me\030\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTen" +
+      "ant\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewState" +
+      "ment\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010ten" +
+      "antId\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tinde" +
+      "xType\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\ns" +
+      "toreNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(\005" +
+      "\022\036\n\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtran" +
+      "sactional\030\033 \001(\010\022\034\n\024updateCacheFrequency\030",
+      "\034 \001(\003\022\035\n\025indexDisableTimestamp\030\035 \001(\003\022\031\n\021" +
+      "isNamespaceMapped\030\036 \001(\010\022\034\n\024autoParititon" +
+      "SeqName\030\037 \001(\t\022\032\n\022isAppendOnlySchema\030  \001(" +
+      "\010\022\027\n\017parentNameBytes\030! \001(\014\022\025\n\rstorageSch" +
+      "eme\030\" \001(\014\022\026\n\016encodingScheme\030# \001(\014\022,\n\021enc" +
+      "odedCQCounters\030$ \003(\0132\021.EncodedCQCounter\022" +
+      "\"\n\032useStatsForParallelization\030% \001(\010\022\033\n\023t" +
+      "ransactionProvider\030& \001(\005\"6\n\020EncodedCQCou" +
+      "nter\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007counter\030\002 \002(\005" +
+      "*A\n\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004",
+      "VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apach" +
+      "e.phoenix.coprocessor.generatedB\014PTableP" +
+      "rotosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8721,7 +8886,7 @@ public final class PTableProtos {
           internal_static_PColumn_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PColumn_descriptor,
-              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", "ColumnQualifierBytes", });
+              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", "ColumnQualifierBytes", "Timestamp", "Derived", });
           internal_static_PTableStats_descriptor =
             getDescriptor().getMessageTypes().get(1);
           internal_static_PTableStats_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
index ec4aa3a..532012f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
@@ -147,7 +147,7 @@ public abstract class LikeExpression extends BaseCompoundExpression {
         }
     }
 
-    private static String toPattern(String s) {
+    public static String toPattern(String s) {
         StringBuilder sb = new StringBuilder(s.length());
 
         // From the JDK doc: \Q and \E protect everything between them

http://git-wip-us.apache.org/repos/asf/phoenix/blob/93fdd5ba/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 312d17b..3d9b261 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -209,13 +209,11 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
 
     public PhoenixConnection(PhoenixConnection connection, long scn)
             throws SQLException {
-        this(connection.getQueryServices(), connection, scn);
+        this(connection, newPropsWithSCN(scn, connection.getClientInfo()));
     }
 
-    public PhoenixConnection(ConnectionQueryServices services,
-            PhoenixConnection connection, long scn) throws SQLException {
-        this(services, connection.getURL(), newPropsWithSCN(scn,
-                connection.getClientInfo()), connection.metaData, connection
+	public PhoenixConnection(PhoenixConnection connection, Properties props) throws SQLException {
+        this(connection.getQueryServices(), connection.getURL(), props, connection.metaData, connection
                 .getMutationState(), connection.isDescVarLengthRowKeyUpgrade(),
                 connection.isRunningUpgrade(), connection.buildingIndex);
         this.isAutoCommit = connection.isAutoCommit;