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

[1/3] phoenix git commit: PHOENIX-2111 Race condition on creation of new view and adding of column to base table

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 1928ba03c -> 8ffd6d8d6


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8ffd6d8d/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 1c24b2c..cb405b1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -18,6 +18,9 @@
 package org.apache.phoenix.query;
 
 import static org.apache.hadoop.hbase.HColumnDescriptor.TTL;
+import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_MAJOR_VERSION;
+import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_MINOR_VERSION;
+import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_PATCH_NUMBER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
 import static org.apache.phoenix.util.UpgradeUtil.upgradeTo4_5_0;
@@ -110,6 +113,7 @@ import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
+import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.index.PhoenixIndexBuilder;
 import org.apache.phoenix.index.PhoenixIndexCodec;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -966,7 +970,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             BlockingRpcCallback<GetVersionResponse> rpcCallback =
                                     new BlockingRpcCallback<GetVersionResponse>();
                             GetVersionRequest.Builder builder = GetVersionRequest.newBuilder();
-
+                            builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                             instance.getVersion(controller, builder.build(), rpcCallback);
                             if(controller.getFailedOn() != null) {
                                 throw controller.getFailedOn();
@@ -1265,6 +1269,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             MutationProto mp = ProtobufUtil.toProto(m);
                             builder.addTableMetadataMutations(mp.toByteString());
                         }
+                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                         instance.createTable(controller, builder.build(), rpcCallback);
                         if(controller.getFailedOn() != null) {
                             throw controller.getFailedOn();
@@ -1293,12 +1298,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     builder.setTableName(ByteStringer.wrap(tableBytes));
                     builder.setTableTimestamp(tableTimestamp);
                     builder.setClientTimestamp(clientTimestamp);
-
-                   instance.getTable(controller, builder.build(), rpcCallback);
-                   if(controller.getFailedOn() != null) {
-                       throw controller.getFailedOn();
-                   }
-                   return rpcCallback.get();
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.getTable(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
                 }
             });
     }
@@ -1325,7 +1330,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                         builder.setTableType(tableType.getSerializedValue());
                         builder.setCascade(cascade);
-
+                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                         instance.dropTable(controller, builder.build(), rpcCallback);
                         if(controller.getFailedOn() != null) {
                             throw controller.getFailedOn();
@@ -1379,6 +1384,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             builder.addTableMetadataMutations(mp.toByteString());
                         }
                         builder.setIfExists(ifExists);
+                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                         instance.dropFunction(controller, builder.build(), rpcCallback);
                         if(controller.getFailedOn() != null) {
                             throw controller.getFailedOn();
@@ -1553,7 +1559,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         MutationProto mp = ProtobufUtil.toProto(m);
                         builder.addTableMetadataMutations(mp.toByteString());
                     }
-
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                     instance.addColumn(controller, builder.build(), rpcCallback);
                     if(controller.getFailedOn() != null) {
                         throw controller.getFailedOn();
@@ -1804,6 +1810,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         MutationProto mp = ProtobufUtil.toProto(m);
                         builder.addTableMetadataMutations(mp.toByteString());
                     }
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                     instance.dropColumn(controller, builder.build(), rpcCallback);
                     if(controller.getFailedOn() != null) {
                         throw controller.getFailedOn();
@@ -2125,6 +2132,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         BlockingRpcCallback<ClearCacheResponse> rpcCallback =
                                 new BlockingRpcCallback<ClearCacheResponse>();
                         ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
+                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                         instance.clearCache(controller, builder.build(), rpcCallback);
                         if(controller.getFailedOn() != null) {
                             throw controller.getFailedOn();
@@ -2189,23 +2197,24 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX], rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]);
         return metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        UpdateIndexStateRequest.Builder builder = UpdateIndexStateRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        instance.updateIndexState(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                });
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                UpdateIndexStateRequest.Builder builder = UpdateIndexStateRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.updateIndexState(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
     }
 
     @Override
@@ -2408,6 +2417,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 builder.setTableName(ByteStringer.wrap(tableName));
                                 builder.setSchemaName(ByteStringer.wrap(schemaName));
                                 builder.setClientTimestamp(clientTS);
+                                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                                 instance.clearTableFromCache(controller, builder.build(), rpcCallback);
                                 if (controller.getFailedOn() != null) { throw controller.getFailedOn(); }
                                 return rpcCallback.get();
@@ -2696,12 +2706,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         builder.addFunctionTimestamps(function.getSecond().longValue());
                     }
                     builder.setClientTimestamp(clientTimestamp);
-
-                   instance.getFunctions(controller, builder.build(), rpcCallback);
-                   if(controller.getFailedOn() != null) {
-                       throw controller.getFailedOn();
-                   }
-                   return rpcCallback.get();
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.getFunctions(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
                 }
             }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
 
@@ -2732,6 +2742,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                         builder.setTemporary(temporary);
                         builder.setReplace(function.isReplace());
+                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
                         instance.createFunction(controller, builder.build(), rpcCallback);
                         if(controller.getFailedOn() != null) {
                             throw controller.getFailedOn();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8ffd6d8d/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index b1fcf30..f74133a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -234,8 +234,9 @@ public class MetaDataClient {
             TABLE_SCHEM + "," +
             TABLE_NAME + "," +
             COLUMN_FAMILY + "," +
-            LINK_TYPE +
-            ") VALUES (?, ?, ?, ?, ?)";
+            LINK_TYPE + "," +
+            TABLE_SEQ_NUM + // this is actually set to the parent table's sequence number
+            ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
             TENANT_ID + "," +
@@ -1490,6 +1491,7 @@ public class MetaDataClient {
                 linkStatement.setString(3, parentTableName);
                 linkStatement.setString(4, tableName);
                 linkStatement.setByte(5, LinkType.INDEX_TABLE.getSerializedValue());
+                linkStatement.setLong(6, parent.getSequenceNumber());
                 linkStatement.execute();
             }
 
@@ -1649,6 +1651,12 @@ public class MetaDataClient {
                         linkStatement.setString(3, tableName);
                         linkStatement.setString(4, physicalName.getString());
                         linkStatement.setByte(5, LinkType.PHYSICAL_TABLE.getSerializedValue());
+                        if (tableType == PTableType.VIEW) {
+                            PTable physicalTable = connection.getMetaDataCache().getTable(new PTableKey(null, physicalName.getString()));
+                            linkStatement.setLong(6, physicalTable.getSequenceNumber());
+                        } else {
+                            linkStatement.setLong(6, parent.getSequenceNumber());
+                        }
                         linkStatement.execute();
                     }
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8ffd6d8d/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 fbc15be..ee73a58 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
@@ -51,6 +51,7 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SortOrder;
@@ -59,6 +60,7 @@ import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
 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;
 
@@ -437,4 +439,16 @@ public class MetaDataUtil {
         scan.setStopRow(stopKey);
         return scan;
     }
+    
+    public static LinkType getLinkType(Mutation tableMutation) {
+        List<Cell> kvs = tableMutation.getFamilyCellMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES);
+        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()));
+                }
+            }
+        }
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8ffd6d8d/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 f321eb2..d0ad34c 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
@@ -83,8 +83,6 @@ import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.ValueBitSet;
 import org.apache.phoenix.schema.types.PDataType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Splitter;
@@ -99,8 +97,6 @@ import com.google.common.collect.Lists;
  * @since 0.1
  */
 public class PhoenixRuntime {
-    private static final Logger logger = LoggerFactory.getLogger(PhoenixRuntime.class);
-
     /**
      * Use this connection property to control HBase timestamps
      * by specifying your own long timestamp value at connection time. All

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8ffd6d8d/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index c7baf43..d7e74ca 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -466,8 +466,8 @@ public class UpgradeUtil {
     public static void upgradeTo4_5_0(PhoenixConnection oldMetaConnection) throws SQLException {
         PhoenixConnection metaConnection = null;
         try {
-            // Need to use own connection without any SCN to be able to read all data from SYSTEM.CATALOG 
-            metaConnection = new PhoenixConnection(oldMetaConnection);
+            // Need to use own connection with max time stamp to be able to read all data from SYSTEM.CATALOG 
+            metaConnection = new PhoenixConnection(oldMetaConnection, HConstants.LATEST_TIMESTAMP);
             logger.info("Upgrading metadata to support adding columns to tables with views");
             String getBaseTableAndViews = "SELECT "
                     + COLUMN_FAMILY + " AS BASE_PHYSICAL_TABLE, "

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8ffd6d8d/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index e79f846..c265158 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -61,6 +61,7 @@ message GetTableRequest {
   required bytes tableName = 3;
   required int64 tableTimestamp = 4;
   required int64 clientTimestamp = 5;
+  optional int32 clientVersion = 6;
 }
 
 message GetFunctionsRequest {
@@ -68,11 +69,13 @@ message GetFunctionsRequest {
   repeated bytes functionNames = 2;
   repeated int64 functionTimestamps = 3;
   required int64 clientTimestamp = 4;
+  optional int32 clientVersion = 5;
 }
 
 // each byte array represents a MutationProto instance
 message CreateTableRequest {
-  repeated bytes tableMetadataMutations = 1; 
+  repeated bytes tableMetadataMutations = 1;
+  optional int32 clientVersion = 2;
 }
 
 // each byte array represents a MutationProto instance
@@ -80,38 +83,46 @@ message CreateFunctionRequest {
   repeated bytes tableMetadataMutations = 1;
   required bool temporary = 2;
   optional bool replace = 3;
+  optional int32 clientVersion = 4;
 }
 
 message DropTableRequest {
   repeated bytes tableMetadataMutations = 1;
   required string tableType = 2;
   optional bool cascade = 3;
+  optional int32 clientVersion = 4;
 }
 
 message AddColumnRequest {
   repeated bytes tableMetadataMutations = 1;
+  optional int32 clientVersion = 2;
 }
 
 message DropColumnRequest {
   repeated bytes tableMetadataMutations = 1;
+  optional int32 clientVersion = 2;
 }
 
 message DropFunctionRequest {
   repeated bytes tableMetadataMutations = 1;
   optional bool ifExists = 2;
+  optional int32 clientVersion = 3;
 }
 
 message UpdateIndexStateRequest {
   repeated bytes tableMetadataMutations = 1;
+  optional int32 clientVersion = 2;
 }
 
 message ClearCacheRequest {
+  optional int32 clientVersion = 1;
 }
 
 message ClearCacheResponse {
 }
 
 message GetVersionRequest {
+  optional int32 clientVersion = 1;
 }
 
 message GetVersionResponse {
@@ -123,6 +134,7 @@ message ClearTableFromCacheRequest {
   required bytes schemaName  = 2;
   required bytes tableName = 3;
   required int64 clientTimestamp = 4;
+  optional int32 clientVersion = 5;
 }
 
 message ClearTableFromCacheResponse {


[2/3] phoenix git commit: PHOENIX-2111 Race condition on creation of new view and adding of column to base table

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/8ffd6d8d/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 acb32d2..a121d28 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
@@ -1811,6 +1811,16 @@ public final class MetaDataProtos {
      * <code>required int64 clientTimestamp = 5;</code>
      */
     long getClientTimestamp();
+
+    // optional int32 clientVersion = 6;
+    /**
+     * <code>optional int32 clientVersion = 6;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 6;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code GetTableRequest}
@@ -1888,6 +1898,11 @@ public final class MetaDataProtos {
               clientTimestamp_ = input.readInt64();
               break;
             }
+            case 48: {
+              bitField0_ |= 0x00000020;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2008,12 +2023,29 @@ public final class MetaDataProtos {
       return clientTimestamp_;
     }
 
+    // optional int32 clientVersion = 6;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 6;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 6;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional int32 clientVersion = 6;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       schemaName_ = com.google.protobuf.ByteString.EMPTY;
       tableName_ = com.google.protobuf.ByteString.EMPTY;
       tableTimestamp_ = 0L;
       clientTimestamp_ = 0L;
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2062,6 +2094,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeInt64(5, clientTimestamp_);
       }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeInt32(6, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -2091,6 +2126,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(5, clientTimestamp_);
       }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(6, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -2139,6 +2178,11 @@ public final class MetaDataProtos {
         result = result && (getClientTimestamp()
             == other.getClientTimestamp());
       }
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -2172,6 +2216,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTTIMESTAMP_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getClientTimestamp());
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -2291,6 +2339,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000008);
         clientTimestamp_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000010);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000020);
         return this;
       }
 
@@ -2339,6 +2389,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000010;
         }
         result.clientTimestamp_ = clientTimestamp_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.clientVersion_ = clientVersion_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -2370,6 +2424,9 @@ public final class MetaDataProtos {
         if (other.hasClientTimestamp()) {
           setClientTimestamp(other.getClientTimestamp());
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -2591,6 +2648,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 6;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 6;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional int32 clientVersion = 6;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 6;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000020;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 6;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:GetTableRequest)
     }
 
@@ -2652,6 +2742,16 @@ public final class MetaDataProtos {
      * <code>required int64 clientTimestamp = 4;</code>
      */
     long getClientTimestamp();
+
+    // optional int32 clientVersion = 5;
+    /**
+     * <code>optional int32 clientVersion = 5;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 5;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code GetFunctionsRequest}
@@ -2743,6 +2843,11 @@ public final class MetaDataProtos {
               clientTimestamp_ = input.readInt64();
               break;
             }
+            case 40: {
+              bitField0_ |= 0x00000004;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2867,11 +2972,28 @@ public final class MetaDataProtos {
       return clientTimestamp_;
     }
 
+    // optional int32 clientVersion = 5;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 5;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 5;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int32 clientVersion = 5;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       functionNames_ = java.util.Collections.emptyList();
       functionTimestamps_ = java.util.Collections.emptyList();
       clientTimestamp_ = 0L;
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2905,6 +3027,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeInt64(4, clientTimestamp_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt32(5, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -2940,6 +3065,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(4, clientTimestamp_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(5, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -2977,6 +3106,11 @@ public final class MetaDataProtos {
         result = result && (getClientTimestamp()
             == other.getClientTimestamp());
       }
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -3006,6 +3140,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTTIMESTAMP_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getClientTimestamp());
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3123,6 +3261,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000004);
         clientTimestamp_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -3169,6 +3309,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000002;
         }
         result.clientTimestamp_ = clientTimestamp_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.clientVersion_ = clientVersion_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3211,6 +3355,9 @@ public final class MetaDataProtos {
         if (other.hasClientTimestamp()) {
           setClientTimestamp(other.getClientTimestamp());
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -3453,6 +3600,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 5;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 5;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional int32 clientVersion = 5;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 5;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000010;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 5;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:GetFunctionsRequest)
     }
 
@@ -3480,6 +3660,16 @@ public final class MetaDataProtos {
      * <code>repeated bytes tableMetadataMutations = 1;</code>
      */
     com.google.protobuf.ByteString getTableMetadataMutations(int index);
+
+    // optional int32 clientVersion = 2;
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code CreateTableRequest}
@@ -3544,6 +3734,11 @@ public final class MetaDataProtos {
               tableMetadataMutations_.add(input.readBytes());
               break;
             }
+            case 16: {
+              bitField0_ |= 0x00000001;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3586,6 +3781,7 @@ public final class MetaDataProtos {
       return PARSER;
     }
 
+    private int bitField0_;
     // repeated bytes tableMetadataMutations = 1;
     public static final int TABLEMETADATAMUTATIONS_FIELD_NUMBER = 1;
     private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_;
@@ -3609,8 +3805,25 @@ public final class MetaDataProtos {
       return tableMetadataMutations_.get(index);
     }
 
+    // optional int32 clientVersion = 2;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 2;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3627,6 +3840,9 @@ public final class MetaDataProtos {
       for (int i = 0; i < tableMetadataMutations_.size(); i++) {
         output.writeBytes(1, tableMetadataMutations_.get(i));
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(2, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -3645,6 +3861,10 @@ public final class MetaDataProtos {
         size += dataSize;
         size += 1 * getTableMetadataMutationsList().size();
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(2, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -3670,6 +3890,11 @@ public final class MetaDataProtos {
       boolean result = true;
       result = result && getTableMetadataMutationsList()
           .equals(other.getTableMetadataMutationsList());
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -3687,6 +3912,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + TABLEMETADATAMUTATIONS_FIELD_NUMBER;
         hash = (53 * hash) + getTableMetadataMutationsList().hashCode();
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -3802,6 +4031,8 @@ public final class MetaDataProtos {
         super.clear();
         tableMetadataMutations_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000001);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
@@ -3829,11 +4060,17 @@ public final class MetaDataProtos {
       public org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest buildPartial() {
         org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest(this);
         int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
           bitField0_ = (bitField0_ & ~0x00000001);
         }
         result.tableMetadataMutations_ = tableMetadataMutations_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.clientVersion_ = clientVersion_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -3859,6 +4096,9 @@ public final class MetaDataProtos {
           }
           onChanged();
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -3958,6 +4198,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 2;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000002;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:CreateTableRequest)
     }
 
@@ -4005,6 +4278,16 @@ public final class MetaDataProtos {
      * <code>optional bool replace = 3;</code>
      */
     boolean getReplace();
+
+    // optional int32 clientVersion = 4;
+    /**
+     * <code>optional int32 clientVersion = 4;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 4;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code CreateFunctionRequest}
@@ -4079,6 +4362,11 @@ public final class MetaDataProtos {
               replace_ = input.readBool();
               break;
             }
+            case 32: {
+              bitField0_ |= 0x00000004;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4177,10 +4465,27 @@ public final class MetaDataProtos {
       return replace_;
     }
 
+    // optional int32 clientVersion = 4;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 4;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 4;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int32 clientVersion = 4;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       temporary_ = false;
       replace_ = false;
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4207,6 +4512,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBool(3, replace_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt32(4, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4233,6 +4541,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, replace_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(4, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4268,6 +4580,11 @@ public final class MetaDataProtos {
         result = result && (getReplace()
             == other.getReplace());
       }
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4293,6 +4610,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + REPLACE_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getReplace());
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4412,6 +4733,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         replace_ = false;
         bitField0_ = (bitField0_ & ~0x00000004);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -4453,6 +4776,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000002;
         }
         result.replace_ = replace_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.clientVersion_ = clientVersion_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -4485,6 +4812,9 @@ public final class MetaDataProtos {
         if (other.hasReplace()) {
           setReplace(other.getReplace());
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4654,6 +4984,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 4;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 4;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional int32 clientVersion = 4;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 4;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000008;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 4;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:CreateFunctionRequest)
     }
 
@@ -4706,6 +5069,16 @@ public final class MetaDataProtos {
      * <code>optional bool cascade = 3;</code>
      */
     boolean getCascade();
+
+    // optional int32 clientVersion = 4;
+    /**
+     * <code>optional int32 clientVersion = 4;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 4;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code DropTableRequest}
@@ -4776,6 +5149,11 @@ public final class MetaDataProtos {
               cascade_ = input.readBool();
               break;
             }
+            case 32: {
+              bitField0_ |= 0x00000004;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4901,10 +5279,27 @@ public final class MetaDataProtos {
       return cascade_;
     }
 
+    // optional int32 clientVersion = 4;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 4;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 4;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int32 clientVersion = 4;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       tableType_ = "";
       cascade_ = false;
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4931,6 +5326,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBool(3, cascade_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt32(4, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4957,6 +5355,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, cascade_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(4, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4992,6 +5394,11 @@ public final class MetaDataProtos {
         result = result && (getCascade()
             == other.getCascade());
       }
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -5017,6 +5424,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CASCADE_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getCascade());
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5132,6 +5543,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         cascade_ = false;
         bitField0_ = (bitField0_ & ~0x00000004);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -5173,6 +5586,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000002;
         }
         result.cascade_ = cascade_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.clientVersion_ = clientVersion_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5207,6 +5624,9 @@ public final class MetaDataProtos {
         if (other.hasCascade()) {
           setCascade(other.getCascade());
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -5417,6 +5837,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 4;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 4;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional int32 clientVersion = 4;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 4;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000008;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 4;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:DropTableRequest)
     }
 
@@ -5444,6 +5897,16 @@ public final class MetaDataProtos {
      * <code>repeated bytes tableMetadataMutations = 1;</code>
      */
     com.google.protobuf.ByteString getTableMetadataMutations(int index);
+
+    // optional int32 clientVersion = 2;
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code AddColumnRequest}
@@ -5504,6 +5967,11 @@ public final class MetaDataProtos {
               tableMetadataMutations_.add(input.readBytes());
               break;
             }
+            case 16: {
+              bitField0_ |= 0x00000001;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -5546,6 +6014,7 @@ public final class MetaDataProtos {
       return PARSER;
     }
 
+    private int bitField0_;
     // repeated bytes tableMetadataMutations = 1;
     public static final int TABLEMETADATAMUTATIONS_FIELD_NUMBER = 1;
     private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_;
@@ -5569,8 +6038,25 @@ public final class MetaDataProtos {
       return tableMetadataMutations_.get(index);
     }
 
+    // optional int32 clientVersion = 2;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 2;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -5587,6 +6073,9 @@ public final class MetaDataProtos {
       for (int i = 0; i < tableMetadataMutations_.size(); i++) {
         output.writeBytes(1, tableMetadataMutations_.get(i));
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(2, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -5605,6 +6094,10 @@ public final class MetaDataProtos {
         size += dataSize;
         size += 1 * getTableMetadataMutationsList().size();
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(2, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -5630,6 +6123,11 @@ public final class MetaDataProtos {
       boolean result = true;
       result = result && getTableMetadataMutationsList()
           .equals(other.getTableMetadataMutationsList());
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -5647,6 +6145,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + TABLEMETADATAMUTATIONS_FIELD_NUMBER;
         hash = (53 * hash) + getTableMetadataMutationsList().hashCode();
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5758,6 +6260,8 @@ public final class MetaDataProtos {
         super.clear();
         tableMetadataMutations_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000001);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
@@ -5785,11 +6289,17 @@ public final class MetaDataProtos {
       public org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest buildPartial() {
         org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest(this);
         int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
           bitField0_ = (bitField0_ & ~0x00000001);
         }
         result.tableMetadataMutations_ = tableMetadataMutations_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.clientVersion_ = clientVersion_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -5815,6 +6325,9 @@ public final class MetaDataProtos {
           }
           onChanged();
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -5914,6 +6427,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 2;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000002;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:AddColumnRequest)
     }
 
@@ -5941,6 +6487,16 @@ public final class MetaDataProtos {
      * <code>repeated bytes tableMetadataMutations = 1;</code>
      */
     com.google.protobuf.ByteString getTableMetadataMutations(int index);
+
+    // optional int32 clientVersion = 2;
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code DropColumnRequest}
@@ -6001,6 +6557,11 @@ public final class MetaDataProtos {
               tableMetadataMutations_.add(input.readBytes());
               break;
             }
+            case 16: {
+              bitField0_ |= 0x00000001;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -6043,6 +6604,7 @@ public final class MetaDataProtos {
       return PARSER;
     }
 
+    private int bitField0_;
     // repeated bytes tableMetadataMutations = 1;
     public static final int TABLEMETADATAMUTATIONS_FIELD_NUMBER = 1;
     private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_;
@@ -6066,8 +6628,25 @@ public final class MetaDataProtos {
       return tableMetadataMutations_.get(index);
     }
 
+    // optional int32 clientVersion = 2;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 2;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -6084,6 +6663,9 @@ public final class MetaDataProtos {
       for (int i = 0; i < tableMetadataMutations_.size(); i++) {
         output.writeBytes(1, tableMetadataMutations_.get(i));
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(2, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -6102,6 +6684,10 @@ public final class MetaDataProtos {
         size += dataSize;
         size += 1 * getTableMetadataMutationsList().size();
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(2, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -6127,6 +6713,11 @@ public final class MetaDataProtos {
       boolean result = true;
       result = result && getTableMetadataMutationsList()
           .equals(other.getTableMetadataMutationsList());
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -6144,6 +6735,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + TABLEMETADATAMUTATIONS_FIELD_NUMBER;
         hash = (53 * hash) + getTableMetadataMutationsList().hashCode();
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -6255,6 +6850,8 @@ public final class MetaDataProtos {
         super.clear();
         tableMetadataMutations_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000001);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
@@ -6282,11 +6879,17 @@ public final class MetaDataProtos {
       public org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropColumnRequest buildPartial() {
         org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropColumnRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropColumnRequest(this);
         int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
           bitField0_ = (bitField0_ & ~0x00000001);
         }
         result.tableMetadataMutations_ = tableMetadataMutations_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.clientVersion_ = clientVersion_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -6312,6 +6915,9 @@ public final class MetaDataProtos {
           }
           onChanged();
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -6411,6 +7017,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 2;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000002;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:DropColumnRequest)
     }
 
@@ -6448,6 +7087,16 @@ public final class MetaDataProtos {
      * <code>optional bool ifExists = 2;</code>
      */
     boolean getIfExists();
+
+    // optional int32 clientVersion = 3;
+    /**
+     * <code>optional int32 clientVersion = 3;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 3;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code DropFunctionRequest}
@@ -6513,6 +7162,11 @@ public final class MetaDataProtos {
               ifExists_ = input.readBool();
               break;
             }
+            case 24: {
+              bitField0_ |= 0x00000002;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -6595,9 +7249,26 @@ public final class MetaDataProtos {
       return ifExists_;
     }
 
+    // optional int32 clientVersion = 3;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 3;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 3;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional int32 clientVersion = 3;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       ifExists_ = false;
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -6617,6 +7288,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(2, ifExists_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeInt32(3, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -6639,6 +7313,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(2, ifExists_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(3, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -6669,6 +7347,11 @@ public final class MetaDataProtos {
         result = result && (getIfExists()
             == other.getIfExists());
       }
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -6690,6 +7373,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + IFEXISTS_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getIfExists());
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -6803,6 +7490,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         ifExists_ = false;
         bitField0_ = (bitField0_ & ~0x00000002);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -6840,6 +7529,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000001;
         }
         result.ifExists_ = ifExists_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.clientVersion_ = clientVersion_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -6869,6 +7562,9 @@ public final class MetaDataProtos {
         if (other.hasIfExists()) {
           setIfExists(other.getIfExists());
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -7001,6 +7697,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 3;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 3;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional int32 clientVersion = 3;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 3;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000004;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 3;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:DropFunctionRequest)
     }
 
@@ -7028,6 +7757,16 @@ public final class MetaDataProtos {
      * <code>repeated bytes tableMetadataMutations = 1;</code>
      */
     com.google.protobuf.ByteString getTableMetadataMutations(int index);
+
+    // optional int32 clientVersion = 2;
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code UpdateIndexStateRequest}
@@ -7088,6 +7827,11 @@ public final class MetaDataProtos {
               tableMetadataMutations_.add(input.readBytes());
               break;
             }
+            case 16: {
+              bitField0_ |= 0x00000001;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7130,6 +7874,7 @@ public final class MetaDataProtos {
       return PARSER;
     }
 
+    private int bitField0_;
     // repeated bytes tableMetadataMutations = 1;
     public static final int TABLEMETADATAMUTATIONS_FIELD_NUMBER = 1;
     private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_;
@@ -7153,8 +7898,25 @@ public final class MetaDataProtos {
       return tableMetadataMutations_.get(index);
     }
 
+    // optional int32 clientVersion = 2;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 2;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 clientVersion = 2;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -7171,6 +7933,9 @@ public final class MetaDataProtos {
       for (int i = 0; i < tableMetadataMutations_.size(); i++) {
         output.writeBytes(1, tableMetadataMutations_.get(i));
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(2, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -7189,6 +7954,10 @@ public final class MetaDataProtos {
         size += dataSize;
         size += 1 * getTableMetadataMutationsList().size();
       }
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(2, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -7214,6 +7983,11 @@ public final class MetaDataProtos {
       boolean result = true;
       result = result && getTableMetadataMutationsList()
           .equals(other.getTableMetadataMutationsList());
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -7231,6 +8005,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + TABLEMETADATAMUTATIONS_FIELD_NUMBER;
         hash = (53 * hash) + getTableMetadataMutationsList().hashCode();
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -7342,6 +8120,8 @@ public final class MetaDataProtos {
         super.clear();
         tableMetadataMutations_ = java.util.Collections.emptyList();
         bitField0_ = (bitField0_ & ~0x00000001);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
@@ -7369,11 +8149,17 @@ public final class MetaDataProtos {
       public org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest buildPartial() {
         org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest(this);
         int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
         if (((bitField0_ & 0x00000001) == 0x00000001)) {
           tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
           bitField0_ = (bitField0_ & ~0x00000001);
         }
         result.tableMetadataMutations_ = tableMetadataMutations_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.clientVersion_ = clientVersion_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -7399,6 +8185,9 @@ public final class MetaDataProtos {
           }
           onChanged();
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -7498,6 +8287,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 2;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000002;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 2;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:UpdateIndexStateRequest)
     }
 
@@ -7511,6 +8333,16 @@ public final class MetaDataProtos {
 
   public interface ClearCacheRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // optional int32 clientVersion = 1;
+    /**
+     * <code>optional int32 clientVersion = 1;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 1;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code ClearCacheRequest}
@@ -7545,6 +8377,7 @@ public final class MetaDataProtos {
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -7562,6 +8395,11 @@ public final class MetaDataProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7601,7 +8439,25 @@ public final class MetaDataProtos {
       return PARSER;
     }
 
+    private int bitField0_;
+    // optional int32 clientVersion = 1;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 1;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 1;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 clientVersion = 1;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -7615,6 +8471,9 @@ public final class MetaDataProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(1, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -7624,6 +8483,10 @@ public final class MetaDataProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(1, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -7647,6 +8510,11 @@ public final class MetaDataProtos {
       org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest) obj;
 
       boolean result = true;
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -7660,6 +8528,10 @@ public final class MetaDataProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -7769,6 +8641,8 @@ public final class MetaDataProtos {
 
       public Builder clear() {
         super.clear();
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -7795,6 +8669,13 @@ public final class MetaDataProtos {
 
       public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest buildPartial() {
         org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.clientVersion_ = clientVersion_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -7810,6 +8691,9 @@ public final class MetaDataProtos {
 
       public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest other) {
         if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest.getDefaultInstance()) return this;
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -7835,6 +8719,40 @@ public final class MetaDataProtos {
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional int32 clientVersion = 1;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 1;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional int32 clientVersion = 1;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 1;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000001;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 1;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
 
       // @@protoc_insertion_point(builder_scope:ClearCacheRequest)
     }
@@ -8187,6 +9105,16 @@ public final class MetaDataProtos {
 
   public interface GetVersionRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // optional int32 clientVersion = 1;
+    /**
+     * <code>optional int32 clientVersion = 1;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 1;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code GetVersionRequest}
@@ -8221,6 +9149,7 @@ public final class MetaDataProtos {
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -8238,6 +9167,11 @@ public final class MetaDataProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -8277,7 +9211,25 @@ public final class MetaDataProtos {
       return PARSER;
     }
 
+    private int bitField0_;
+    // optional int32 clientVersion = 1;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 1;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 1;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 clientVersion = 1;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -8291,6 +9243,9 @@ public final class MetaDataProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(1, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -8300,6 +9255,10 @@ public final class MetaDataProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(1, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -8323,6 +9282,11 @@ public final class MetaDataProtos {
       org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest) obj;
 
       boolean result = true;
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -8336,6 +9300,10 @@ public final class MetaDataProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -8445,6 +9413,8 @@ public final class MetaDataProtos {
 
       public Builder clear() {
         super.clear();
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -8471,6 +9441,13 @@ public final class MetaDataProtos {
 
       public org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest buildPartial() {
         org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.clientVersion_ = clientVersion_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -8486,6 +9463,9 @@ public final class MetaDataProtos {
 
       public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest other) {
         if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest.getDefaultInstance()) return this;
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -8511,6 +9491,40 @@ public final class MetaDataProtos {
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional int32 clientVersion = 1;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 1;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional int32 clientVersion = 1;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 1;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000001;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 1;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
 
       // @@protoc_insertion_point(builder_scope:GetVersionRequest)
     }
@@ -9007,6 +10021,16 @@ public final class MetaDataProtos {
      * <code>required int64 clientTimestamp = 4;</code>
      */
     long getClientTimestamp();
+
+    // optional int32 clientVersion = 5;
+    /**
+     * <code>optional int32 clientVersion = 5;</code>
+     */
+    boolean hasClientVersion();
+    /**
+     * <code>optional int32 clientVersion = 5;</code>
+     */
+    int getClientVersion();
   }
   /**
    * Protobuf type {@code ClearTableFromCacheRequest}
@@ -9079,6 +10103,11 @@ public final class MetaDataProtos {
               clientTimestamp_ = input.readInt64();
               break;
             }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              clientVersion_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -9183,11 +10212,28 @@ public final class MetaDataProtos {
       return clientTimestamp_;
     }
 
+    // optional int32 clientVersion = 5;
+    public static final int CLIENTVERSION_FIELD_NUMBER = 5;
+    private int clientVersion_;
+    /**
+     * <code>optional int32 clientVersion = 5;</code>
+     */
+    public boolean hasClientVersion() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional int32 clientVersion = 5;</code>
+     */
+    public int getClientVersion() {
+      return clientVersion_;
+    }
+
     private void initFields() {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       schemaName_ = com.google.protobuf.ByteString.EMPTY;
       tableName_ = com.google.protobuf.ByteString.EMPTY;
       clientTimestamp_ = 0L;
+      clientVersion_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -9229,6 +10275,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeInt64(4, clientTimestamp_);
       }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeInt32(5, clientVersion_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -9254,6 +10303,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(4, clientTimestamp_);
       }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(5, clientVersion_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -9297,6 +10350,11 @@ public final class MetaDataProtos {
         result = result && (getClientTimestamp()
             == other.getClientTimestamp());
       }
+      result = result && (hasClientVersion() == other.hasClientVersion());
+      if (hasClientVersion()) {
+        result = result && (getClientVersion()
+            == other.getClientVersion());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -9326,6 +10384,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTTIMESTAMP_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getClientTimestamp());
       }
+      if (hasClientVersion()) {
+        hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
+        hash = (53 * hash) + getClientVersion();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -9443,6 +10505,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000004);
         clientTimestamp_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
+        clientVersion_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -9487,6 +10551,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000008;
         }
         result.clientTimestamp_ = clientTimestamp_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.clientVersion_ = clientVersion_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -9515,6 +10583,9 @@ public final class MetaDataProtos {
         if (other.hasClientTimestamp()) {
           setClientTimestamp(other.getClientTimestamp());
         }
+        if (other.hasClientVersion()) {
+          setClientVersion(other.getClientVersion());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -9699,6 +10770,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 clientVersion = 5;
+      private int clientVersion_ ;
+      /**
+       * <code>optional int32 clientVersion = 5;</code>
+       */
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional int32 clientVersion = 5;</code>
+       */
+      public int getClientVersion() {
+        return clientVersion_;
+      }
+      /**
+       * <code>optional int32 clientVersion = 5;</code>
+       */
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000010;
+        clientVersion_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 clientVersion = 5;</code>
+       */
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        clientVersion_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:ClearTableFromCacheRequest)
     }
 
@@ -11156,60 +12260,67 @@ public final class MetaDataProtos {
       "onTime\030\002 \001(\003\022\026\n\005table\030\003 \001(\0132\007.PTable\022\026\n\016" +
       "tablesToDelete\030\004 \003(\014\022\022\n\ncolumnName\030\005 \001(\014" +
       "\022\022\n\nfamilyName\030\006 \001(\014\022\024\n\014functionName\030\007 \001" +
-      "(\014\022\034\n\010function\030\010 \003(\0132\n.PFunction\"{\n\017GetT" +
-      "ableRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\nschemaN" +
-      "ame\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\026\n\016tableTim" +
-      "estamp\030\004 \002(\003\022\027\n\017clientTimestamp\030\005 \002(\003\"s\n",
-      "\023GetFunctionsRequest\022\020\n\010tenantId\030\001 \002(\014\022\025" +
-      "\n\rfunctionNames\030\002 \003(\014\022\032\n\022functionTimesta" +
-      "mps\030\003 \003(\003\022\027\n\017clientTimestamp\030\004 \002(\003\"4\n\022Cr" +
-      "eateTableRequest\022\036\n\026tableMetadataMutatio" +
-      "ns\030\001 \003(\014\"[\n\025CreateFunctionRequest\022\036\n\026tab" +
-      "leMetadataMutations\030\001 \003(\014\022\021\n\ttemporary\030\002" +
-      " \002(\010\022\017\n\007replace\030\003 \001(\010\"V\n\020DropTableReques" +
-      "t\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\021\n\ttab" +
-      "leType\030\002 \002(\t\022\017\n\007cascade\030\003 \001(\010\"2\n\020AddColu" +
-      "mnRequest\022\036\n\026tableMetadataMutations\030\001 \003(",
-      "\014\"3\n\021DropColumnRequest\022\036\n\026tableMetadataM" +
-      "utations\030\001 \003(\014\"G\n\023DropFunctionRequest\022\036\n" +
-      "\026tableMetadataMutations\030\001 \003(\014\022\020\n\010ifExist" +
-      "s\030\002 \001(\010\"9\n\027UpdateIndexStateRequest\022\036\n\026ta" +
-      "bleMetadataMutations\030\001 \003(\014\"\023\n\021ClearCache" +
-      "Request\"\024\n\022ClearCacheResponse\"\023\n\021GetVers" +
-      "ionRequest\"%\n\022GetVersionResponse\022\017\n\007vers" +
-      "ion\030\001 \002(\003\"n\n\032ClearTableFromCacheRequest\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\"\035\n\033ClearTableFromCacheResponse*\365\002\n\014Mut" +
-      "ationCode\022\030\n\024TABLE_ALREADY_EXISTS\020\000\022\023\n\017T" +
-      "ABLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031" +
-      "\n\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CONCURRENT_" +
-      "TABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_REGION\020" +
-      "\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNALLOWED_TA" +
-      "BLE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010\022\032\n\026PAR" +
-      "ENT_TABLE_NOT_FOUND\020\t\022\033\n\027FUNCTION_ALREAD" +
-      "Y_EXISTS\020\n\022\026\n\022FUNCTION_NOT_FOUND\020\013\022\030\n\024NE" +
-      "WER_FUNCTION_FOUND\020\014\022\032\n\026FUNCTION_NOT_IN_",
-      "REGION\020\r2\304\005\n\017MetaDataService\022/\n\010getTable" +
-      "\022\020.GetTableRequest\032\021.MetaDataResponse\0227\n" +
-      "\014getFunctions\022\024.GetFunctionsRequest\032\021.Me" +
-      "taDataResponse\0225\n\013createTable\022\023.CreateTa" +
-      "bleRequest\032\021.MetaDataResponse\022;\n\016createF" +
-      "unction\022\026.CreateFunctionRequest\032\021.MetaDa" +
-      "taResponse\0221\n\tdropTable\022\021.DropTableReque" +
-      "st\032\021.MetaDataResponse\0227\n\014dropFunction\022\024." +
-      "DropFunctionRequest\032\021.MetaDataResponse\0221" +
-      "\n\taddColumn\022\021.AddColumnRequest\032\021.MetaDat",
-      "aResponse\0223\n\ndropColumn\022\022.DropColumnRequ" +
-      "est\032\021.MetaDataResponse\022?\n\020updateIndexSta" +
-      "te\022\030.UpdateIndexStateRequest\032\021.MetaDataR" +
-      "esponse\0225\n\nclearCache\022\022.ClearCacheReques" +
-      "t\032\023.ClearCacheResponse\0225\n\ngetVersion\022\022.G" +
-      "etVersionRequest\032\023.GetVersionResponse\022P\n" +
-      "\023clearTableFromCache\022\033.ClearTableFromCac" +
-      "heRequest\032\034.ClearTableFromCacheResponseB" +
-      "B\n(org.apache.phoenix.coprocessor.genera" +
-      "tedB\016MetaDataProtosH\001\210\001\001\240\001\001"
+      "(\014\022\034\n\010function\030\010 \003(\0132\n.PFunction\"\222\001\n\017Get" +
+      "TableRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\nschema" +
+      "Name\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\026\n\016tableTi" +
+      "mestamp\030\004 \002(\003\022\027\n\017clientTimestamp\030\005 \002(\003\022\025",
+      "\n\rclientVersion\030\006 \001(\005\"\212\001\n\023GetFunctionsRe" +
+      "quest\022\020\n\010tenantId\030\001 \002(\014\022\025\n\rfunctionNames" +
+      "\030\002 \003(\014\022\032\n\022functionTimestamps\030\003 \003(\003\022\027\n\017cl" +
+      "ientTimestamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001" +
+      "(\005\"K\n\022CreateTableRequest\022\036\n\026tableMetadat" +
+      "aMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"" +
+      "r\n\025CreateFunctionRequest\022\036\n\026tableMetadat" +
+      "aMutations\030\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007r" +
+      "eplace\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\"m\n\020D" +
+      "ropTableRequest\022\036\n\026tableMetadataMutation",
+      "s\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\"I\n\020AddColumnR" +
+      "equest\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\025" +
+      "\n\rclientVersion\030\002 \001(\005\"J\n\021DropColumnReque" +
+      "st\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rcl" +
+      "ientVersion\030\002 \001(\005\"^\n\023DropFunctionRequest" +
+      "\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\020\n\010ifEx" +
+      "ists\030\002 \001(\010\022\025\n\rclientVersion\030\003 \001(\005\"P\n\027Upd" +
+      "ateIndexStateRequest\022\036\n\026tableMetadataMut" +
+      "ations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"*\n\021C",
+      "learCacheRequest\022\025\n\rclientVersion\030\001 \001(\005\"" +
+      "\024\n\022ClearCacheResponse\"*\n\021GetVersionReque" +
+      "st\022\025\n\rclientVersion\030\001 \001(\005\"%\n\022GetVersionR" +
+      "esponse\022\017\n\007version\030\001 \002(\003\"\205\001\n\032ClearTableF" +
+      "romCacheRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\nsch" +
+      "emaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\027\n\017clie" +
+      "ntTimestamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001(\005" +
+      "\"\035\n\033ClearTableFromCacheResponse*\365\002\n\014Muta" +
+      "tionCode\022\030\n\024TABLE_ALREADY_EXISTS\020\000\022\023\n\017TA" +
+      "BLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031\n",
+      "\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CONCURRENT_T" +
+      "ABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_REGION\020\005" +
+      "\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNALLOWED_TAB" +
+      "LE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010\022\032\n\026PARE" +
+      "NT_TABLE_NOT_FOUND\020\t\022\033\n\027FUNCTION_ALREADY" +
+      "_EXISTS\020\n\022\026\n\022FUNCTION_NOT_FOUND\020\013\022\030\n\024NEW" +
+      "ER_FUNCTION_FOUND\020\014\022\032\n\026FUNCTION_NOT_IN_R" +
+      "EGION\020\r2\304\005\n\017MetaDataService\022/\n\010getTable\022" +
+      "\020.GetTableRequest\032\021.MetaDataResponse\0227\n\014" +
+      "getFunctions\022\024.GetFunctionsRequest\032\021.Met",
+      "aDataResponse\0225\n\013createTable\022\023.CreateTab" +
+      "leRequest\032\021.MetaDataResponse\022;\n\016createFu" +
+      "nction\022\026.CreateFunctionRequest\032\021.MetaDat" +
+      "aResponse\0221\n\tdropTable\022\021.DropTableReques" +
+      "t\032\021.MetaDataResponse\0227\n\014dropFunction\022\024.D" +
+      "ropFunctionRequest\032\021.MetaDataResponse\0221\n" +
+      "\taddColumn\022\021.AddColumnRequest\032\021.MetaData" +
+      "Response\0223\n\ndropColumn\022\022.DropColumnReque" +
+      "st\032\021.MetaDataResponse\022?\n\020updateIndexStat" +
+      "e\022\030.UpdateIndexStateRequest\032\021.MetaDataRe",
+      "sponse\0225\n\nclearCache\022\022.ClearCacheRequest" +
+      "\032\023.ClearCacheResponse\0225\n\ngetVersion\022\022.Ge" +
+      "tVersionRequest\032\023.GetVersionResponse\022P\n\023" +
+      "clearTableFromCache\022\033.ClearTableFromCach" +
+      "eRequest\032\034.ClearTableFromCacheResponseBB" +
+      "\n(org.apache.phoenix.coprocessor.generat" +
+      "edB\016MetaDataProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -11227,61 +12338,61 @@ 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", });
+              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "TableTimestamp", "ClientTimestamp", "ClientVersion", });
           internal_static_GetFunctionsRequest_descriptor =
             getDescriptor().getMessageTypes().get(2);
           internal_static_GetFunctionsRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetFunctionsRequest_descriptor,
-              new java.lang.String[] { "TenantId", "FunctionNames", "FunctionTimestamps", "ClientTimestamp", });
+              new java.lang.String[] { "TenantId", "FunctionNames", "FunctionTimestamps", "ClientTimestamp", "ClientVersion", });
           internal_static_CreateTableRequest_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_CreateTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CreateTableRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", });
+              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", });
           internal_static_CreateFunctionRequest_descriptor =
             getDescriptor().getMessageTypes().get(4);
           internal_static_CreateFunctionRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CreateFunctionRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "Temporary", "Replace", });
+              new java.lang.String[] { "TableMetadataMutations", "Temporary", "Replace", "ClientVersion", });
           internal_static_DropTableRequest_descriptor =
             getDescriptor().getMessageTypes().get(5);
           internal_static_DropTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_DropTableRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", });
+              new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", "ClientVersion", });
           internal_static_AddColumnRequest_descriptor =
             getDescriptor().getMessageTypes().get(6);
           internal_static_AddColumnRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_AddColumnRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", });
+              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", });
           internal_static_DropColumnRequest_descriptor =
             getDescriptor().getMessageTypes().get(7);
           internal_static_DropColumnRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_DropColumnRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", });
+              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", });
           internal_static_DropFunctionRequest_descriptor =
             getDescriptor().getMessageTypes().get(8);
           internal_static_DropFunctionRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_DropFunctionRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "IfExists", });
+              new java.lang.String[] { "TableMetadataMutations", "IfExists", "ClientVersion", });
           internal_static_UpdateIndexStateRequest_descriptor =
             getDescriptor().getMessageTypes().get(9);
           internal_static_UpdateIndexStateRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_UpdateIndexStateRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", });
+              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", });
           internal_static_ClearCacheRequest_descriptor =
             getDescriptor().getMessageTypes().get(10);
           internal_static_ClearCacheRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ClearCacheRequest_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "ClientVersion", });
           internal_static_ClearCacheResponse_descriptor =
             getDescriptor().getMessageTypes().get(11);
           internal_static_ClearCacheResponse_fieldAccessorTable = new
@@ -11293,7 +12404,7 @@ public final class MetaDataProtos {
           internal_static_GetVersionRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetVersionRequest_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "ClientVersion", });
           internal_static_GetVersionResponse_descriptor =
             getDescriptor().getMessageTypes().get(13);
           internal_static_GetVersionResponse_fieldAccessorTable = new
@@ -11305,7 +12416,7 @@ public final class MetaDataProtos {
           internal_static_ClearTableFromCacheRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ClearTableFromCacheRequest_descriptor,
-              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "ClientTimestamp", });
+              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "ClientTimestamp", "ClientVersion", });
           internal_static_ClearTableFromCacheResponse_descriptor =
             getDescriptor().getMessageTypes().get(15);
           internal_static_ClearTableFromCacheResponse_fieldAccessorTable = new


[3/3] phoenix git commit: PHOENIX-2111 Race condition on creation of new view and adding of column to base table

Posted by sa...@apache.org.
PHOENIX-2111 Race condition on creation of new view and adding of column to base table


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 8ffd6d8d609f5f82a8a20f97f4d7c7347504abc5
Parents: 1928ba0
Author: Samarth <sa...@salesforce.com>
Authored: Tue Jul 14 15:13:13 2015 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Tue Jul 14 15:13:13 2015 -0700

----------------------------------------------------------------------
 .../coprocessor/MetaDataEndpointImpl.java       |  221 +++-
 .../coprocessor/generated/MetaDataProtos.java   | 1243 +++++++++++++++++-
 .../query/ConnectionQueryServicesImpl.java      |   75 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   12 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |   14 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |    4 -
 .../org/apache/phoenix/util/UpgradeUtil.java    |    4 +-
 phoenix-protocol/src/main/MetaDataService.proto |   14 +-
 8 files changed, 1413 insertions(+), 174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8ffd6d8d/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index defc7af..6372700 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -1066,7 +1066,64 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             return null;
         }
-
+    /**
+     * 
+     * @return null if the physical table row information is not present.
+     * 
+     */
+    private static Mutation getPhysicalTableForView(List<Mutation> tableMetadata, byte[][] parentSchemaTableNames) {
+        int size = tableMetadata.size();
+        byte[][] rowKeyMetaData = new byte[3][];
+        MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
+        Mutation physicalTableRow = null;
+        boolean physicalTableLinkFound = false;
+        if (size >= 2) {
+            int i = size - 1;
+            while (i >= 1) {
+                Mutation m = tableMetadata.get(i);
+                if (m instanceof Put) {
+                    LinkType linkType = MetaDataUtil.getLinkType(m);
+                    if (linkType == LinkType.PHYSICAL_TABLE) {
+                        physicalTableRow = m;
+                        physicalTableLinkFound = true;
+                        break;
+                    }
+                }
+                i--;
+            }
+        }
+        if (!physicalTableLinkFound) {
+            parentSchemaTableNames[0] = null;
+            parentSchemaTableNames[1] = null;
+            return null;
+        }
+        rowKeyMetaData = new byte[5][];
+        getVarChars(physicalTableRow.getRow(), 5, rowKeyMetaData);
+        byte[] colBytes = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
+        byte[] famBytes = rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX];
+        if ((colBytes == null || colBytes.length == 0) && (famBytes != null && famBytes.length > 0)) {
+            byte[] sName = SchemaUtil.getSchemaNameFromFullName(famBytes).getBytes();
+            byte[] tName = SchemaUtil.getTableNameFromFullName(famBytes).getBytes();
+            parentSchemaTableNames[0] = sName;
+            parentSchemaTableNames[1] = tName;
+        }
+        return physicalTableRow;
+    }
+    
+    private long getSequenceNumberForTable(byte[] headerRowKey) throws IOException {
+        Get get = new Get(headerRowKey);
+        get.addColumn(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
+        byte[] b;
+        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES)) {
+            Result result = hTable.get(get);
+            b = result.getValue(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
+        }
+        if (b == null) {
+            throw new IllegalArgumentException("No rows returned for the row key: " + Bytes.toString(headerRowKey));
+        }
+        return PLong.INSTANCE.getCodec().decodeLong(new ImmutableBytesWritable(b), SortOrder.getDefault());
+    }
+    
     @Override
     public void createTable(RpcController controller, CreateTableRequest request,
             RpcCallback<MetaDataResponse> done) {
@@ -1074,66 +1131,101 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         byte[][] rowKeyMetaData = new byte[3][];
         byte[] schemaName = null;
         byte[] tableName = null;
-
         try {
             List<Mutation> tableMetadata = ProtobufUtil.getMutations(request);
             MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
             byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
             schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
-            byte[] parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
-            byte[] lockTableName = parentTableName == null ? tableName : parentTableName;
-            byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, lockTableName);
-            byte[] key =
-                    parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
-                        schemaName, tableName);
-            byte[] parentKey = parentTableName == null ? null : lockKey;
 
-            HRegion region = env.getRegion();
-            MetaDataMutationResult result = checkTableKeyInRegion(lockKey, region);
-            if (result != null) {
-                done.run(MetaDataMutationResult.toProto(result));
-                return;
+            byte[] parentSchemaName = null;
+            byte[] parentTableName = null;
+            PTableType tableType = MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
+            byte[] parentTableKey = null;
+            Mutation viewPhysicalTableRow = null;
+            if (tableType == PTableType.VIEW) {
+                byte[][] parentSchemaTableNames = new byte[2][];
+                /*
+                 * For a view, we lock the base physical table row. For a mapped view, there is 
+                 * no link present to the physical table. So the viewPhysicalTableRow is null
+                 * in that case.
+                 */
+                viewPhysicalTableRow = getPhysicalTableForView(tableMetadata, parentSchemaTableNames);
+                parentSchemaName = parentSchemaTableNames[0];
+                parentTableName = parentSchemaTableNames[1];
+                if (parentTableName != null) {
+                    parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, parentSchemaName, parentTableName);
+                }
+            } else if (tableType == PTableType.INDEX) {
+                parentSchemaName = schemaName;
+                /* 
+                 * For an index we lock the parent table's row which could be a physical table or a view.
+                 * If the parent table is a physical table, then the tenantIdBytes is empty because
+                 * we allow creating an index with a tenant connection only if the parent table is a view.
+                 */ 
+                parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
+                parentTableKey = SchemaUtil.getTableKey(tenantIdBytes, parentSchemaName, parentTableName);
             }
+
+            HRegion region = env.getRegion();
             List<RowLock> locks = Lists.newArrayList();
-            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+            // Place a lock using key for the table to be created
+            byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
             try {
-                acquireLock(region, lockKey, locks);
-                if (key != lockKey) {
-                    acquireLock(region, key, locks);
+                acquireLock(region, tableKey, locks);
+
+                // If the table key resides outside the region, return without doing anything
+                MetaDataMutationResult result = checkTableKeyInRegion(tableKey, region);
+                if (result != null) {
+                    done.run(MetaDataMutationResult.toProto(result));
+                    return;
                 }
-                // Load parent table first
-                PTable parentTable = null;
+
+                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 ImmutableBytesPtr parentCacheKey = null;
-                if (parentKey != null) {
-                    parentCacheKey = new ImmutableBytesPtr(parentKey);
-                    parentTable =
-                            loadTable(env, parentKey, parentCacheKey, clientTimeStamp,
+                if (parentTableName != null) {
+                    // Check if the parent table resides in the same region. If not, don't worry about locking the parent table row
+                    // or loading the parent table. For a view, the parent table that needs to be locked is the base physical table.
+                    // For an index on view, the view header row needs to be locked. 
+                    result = checkTableKeyInRegion(parentTableKey, region);
+                    if (result == null) {
+                        acquireLock(region, parentTableKey, locks);
+                        parentCacheKey = new ImmutableBytesPtr(parentTableKey);
+                        PTable parentTable = loadTable(env, parentTableKey, parentCacheKey, clientTimeStamp,
                                 clientTimeStamp);
-                    if (parentTable == null || isTableDeleted(parentTable)) {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
-                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        builder.setTable(PTableImpl.toProto(parentTable));
-                        done.run(builder.build());
-                        return;
-                    }
-                    // If parent table isn't at the expected sequence number, then return
-                    if (parentTable.getSequenceNumber() != MetaDataUtil
-                            .getParentSequenceNumber(tableMetadata)) {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.CONCURRENT_TABLE_MUTATION);
-                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        builder.setTable(PTableImpl.toProto(parentTable));
-                        done.run(builder.build());
-                        return;
+                        if (parentTable == null || isTableDeleted(parentTable)) {
+                            builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
+                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                            done.run(builder.build());
+                            return;
+                        }
+                        long parentTableSeqNumber;
+                        if (tableType == PTableType.VIEW && viewPhysicalTableRow != null && request.hasClientVersion()) {
+                            // Starting 4.5, the client passes the sequence number of the physical table in the table metadata.
+                            parentTableSeqNumber = MetaDataUtil.getSequenceNumber(viewPhysicalTableRow);
+                        } else if (tableType == PTableType.VIEW) {
+                            // Before 4.5, due to a bug, the parent table key wasn't available. Using get to 
+                            // figure out the parent table sequence number.
+                            parentTableSeqNumber = getSequenceNumberForTable(parentTableKey);
+                        } else {
+                            parentTableSeqNumber = MetaDataUtil.getParentSequenceNumber(tableMetadata);
+                        }
+                        // If parent table isn't at the expected sequence number, then return
+                        if (parentTable.getSequenceNumber() != parentTableSeqNumber) {
+                            builder.setReturnCode(MetaDataProtos.MutationCode.CONCURRENT_TABLE_MUTATION);
+                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                            builder.setTable(PTableImpl.toProto(parentTable));
+                            done.run(builder.build());
+                            return;
+                        }
                     }
                 }
                 // Load child table next
-                ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
+                ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
                 // Get as of latest timestamp so we can detect if we have a newer table that already
-                // exists
-                // without making an additional query
+                // exists without making an additional query
                 PTable table =
-                        loadTable(env, key, cacheKey, clientTimeStamp, HConstants.LATEST_TIMESTAMP);
+                        loadTable(env, tableKey, cacheKey, clientTimeStamp, HConstants.LATEST_TIMESTAMP);
                 if (table != null) {
                     if (table.getTimeStamp() < clientTimeStamp) {
                         // If the table is older than the client time stamp and it's deleted,
@@ -1153,24 +1245,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         return;
                     }
                 }
-                PTableType tableType = MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
-                        new ImmutableBytesWritable());
                 // Add cell for ROW_KEY_ORDER_OPTIMIZABLE = true, as we know that new tables
                 // conform the correct row key. The exception is for a VIEW, which the client
                 // sends over depending on its base physical table.
                 if (tableType != PTableType.VIEW) {
-                    UpgradeUtil.addRowKeyOrderOptimizableCell(tableMetadata, key, clientTimeStamp);
+                    UpgradeUtil.addRowKeyOrderOptimizableCell(tableMetadata, tableKey, clientTimeStamp);
                 }
-                // TODO: Switch this to Region#batchMutate when we want to support indexes on the
-                // system
-                // table. Basically, we get all the locks that we don't already hold for all the
+                // TODO: Switch this to HRegion#batchMutate when we want to support indexes on the
+                // system table. Basically, we get all the locks that we don't already hold for all the
                 // tableMetadata rows. This ensures we don't have deadlock situations (ensuring
-                // primary and
-                // then index table locks are held, in that order). For now, we just don't support
-                // indexing
-                // on the system table. This is an issue because of the way we manage batch mutation
-                // in the
-                // Indexer.
+                // primary and then index table locks are held, in that order). For now, we just don't support
+                // indexing on the system table. This is an issue because of the way we manage batch mutation
+                // in the Indexer.
                 region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
 
                 // Invalidate the cache - the next getTable call will add it
@@ -1190,9 +1276,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 region.releaseRowLocks(locks);
             }
         } catch (Throwable t) {
-          logger.error("createTable failed", t);
+            logger.error("createTable failed", t);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
+                    ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
     }
 
@@ -1606,10 +1692,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
     
-    private boolean isvalidAttribute(Object obj1, Object obj2) {
-    	return (obj1!=null && obj1.equals(obj2)) || (obj1==null && obj2==null); 
-    }
-
     private MetaDataMutationResult addRowsToChildViews(PTable table, List<Mutation> tableMetadata, List<Mutation> mutationsForAddingColumnsToViews, byte[] schemaName, byte[] tableName,
             List<ImmutableBytesPtr> invalidateList, long clientTimeStamp, TableViewFinderResult childViewsResult,
             HRegion region, List<RowLock> locks) throws IOException, SQLException {
@@ -1922,7 +2004,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
     
     @Override
-    public void addColumn(RpcController controller, AddColumnRequest request,
+    public void addColumn(RpcController controller, final AddColumnRequest request,
             RpcCallback<MetaDataResponse> done) {
         try {
             List<Mutation> tableMetaData = ProtobufUtil.getMutations(request);
@@ -1952,13 +2034,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         TableViewFinderResult childViewsResult = findChildViews(region, tenantId, table, PHYSICAL_TABLE_BYTES);
                         if (childViewsResult.hasViews()) {
                            /* 
-                            * Adding a column is not allowed if the meta-data for child view/s spans over
+                            * Adding a column is not allowed if
+                            * 1) The meta-data for child view/s spans over
                             * more than one region (since the changes cannot be made in a transactional fashion)
-                            * A base column count of 0 means that the metadata hasn't been upgraded yet or
-                            * the upgrade is currently in progress. If that is the case, disallow adding columns
-                            *  for tables with views.
+                            * 
+                            * 2) The base column count is 0 which means that the metadata hasn't been upgraded yet or
+                            * the upgrade is currently in progress.
+                            * 
+                            * 3) If the request is from a client that is older than 4.5 version of phoenix. 
+                            * Starting from 4.5, metadata requests have the client version included in them. 
+                            * We don't want to allow clients before 4.5 to add a column to the base table if it has views.
                             */
-                            if (!childViewsResult.allViewsInSingleRegion() || table.getBaseColumnCount() == 0) {
+                            if (!childViewsResult.allViewsInSingleRegion() || table.getBaseColumnCount() == 0 || !request.hasClientVersion()) {
                                 return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
                                         EnvironmentEdgeManager.currentTimeMillis(), null);
                             } else {