You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by an...@apache.org on 2016/09/29 17:34:39 UTC

phoenix git commit: PHOENIX-3254 IndexId Sequence is incremented even if index exists already.

Repository: phoenix
Updated Branches:
  refs/heads/master 15a2f9a62 -> 655fe2be3


PHOENIX-3254 IndexId Sequence is incremented even if index exists already.


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

Branch: refs/heads/master
Commit: 655fe2be3d30f2b8277563920c15a357eb74bcde
Parents: 15a2f9a
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Sep 29 23:04:28 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Thu Sep 29 23:04:28 2016 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/AppendOnlySchemaIT.java     |   2 +-
 .../end2end/TenantSpecificViewIndexIT.java      |  12 +-
 .../phoenix/end2end/index/LocalIndexIT.java     |   5 +-
 .../phoenix/end2end/index/ViewIndexIT.java      |   9 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  53 +++
 .../phoenix/coprocessor/MetaDataProtocol.java   |  17 +
 .../coprocessor/generated/MetaDataProtos.java   | 353 ++++++++++++++-----
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java      |   7 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   8 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  88 ++---
 .../TenantSpecificViewIndexCompileTest.java     |   4 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  15 +-
 phoenix-protocol/src/main/MetaDataService.proto |   3 +
 15 files changed, 413 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index ed9fa74..93341b7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -114,7 +114,7 @@ public class AppendOnlySchemaIT extends ParallelStatsDisabledIT {
             // verify no create table rpcs
             verify(connectionQueryServices, never()).createTable(anyListOf(Mutation.class),
                 any(byte[].class), any(PTableType.class), anyMap(), anyList(), any(byte[][].class),
-                eq(false));
+                eq(false), eq(false));
             reset(connectionQueryServices);
             
             // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index f47e078..b7b8902 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -28,12 +28,9 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.sql.SQLException;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PNameFactory;
@@ -42,7 +39,6 @@ import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
-import org.junit.Ignore;
 import org.junit.Test;
 
 
@@ -126,8 +122,8 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         String sequenceNameA = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName("a"), isNamespaceEnabled);
         String sequenceNameB = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName("b"), isNamespaceEnabled);
         String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), isNamespaceEnabled);
-        verifySequence(isNamespaceEnabled? "a" : null, sequenceNameA, sequenceSchemaName, true);
-        verifySequence(isNamespaceEnabled? "b" : null, sequenceNameB, sequenceSchemaName, true);
+        verifySequenceValue(isNamespaceEnabled? "a" : null, sequenceNameA, sequenceSchemaName, -32767);
+        verifySequenceValue(isNamespaceEnabled? "b" : null, sequenceNameB, sequenceSchemaName, -32767);
 
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "a");
@@ -140,8 +136,8 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         }
         DriverManager.getConnection(getUrl()).createStatement().execute("DROP TABLE " + tableName + " CASCADE");
 
-        verifySequence(isNamespaceEnabled? "a" : null, sequenceNameA, sequenceSchemaName, false);
-        verifySequence(isNamespaceEnabled? "b" : null, sequenceNameB, sequenceSchemaName, false);
+        verifySequenceNotExists(isNamespaceEnabled? "a" : null, sequenceNameA, sequenceSchemaName);
+        verifySequenceNotExists(isNamespaceEnabled? "b" : null, sequenceNameB, sequenceSchemaName);
     }
 
     private void createViewAndIndexesWithTenantId(String tableName,String baseViewName, boolean localIndex, String tenantId,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index 9d3da8e..234a466 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -210,11 +210,10 @@ public class LocalIndexIT extends ParallelStatsDisabledIT {
         Connection conn1 = getConnection();
         Connection conn2 = getConnection();
         conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
-        verifySequence(null, sequenceName, sequenceSchemaName, true);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName,-32767);
         conn2.createStatement().executeQuery("SELECT * FROM " + tableName).next();
         conn1.createStatement().execute("DROP TABLE "+ tableName);
-
-        verifySequence(null, sequenceName, sequenceSchemaName, false);
+        verifySequenceNotExists(null, sequenceName, sequenceSchemaName);
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
index 1abbc87..ca50aab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
@@ -108,14 +108,17 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
         String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), isNamespaceMapped);
         String seqName = getViewIndexSequenceName(PNameFactory.newName(tableName), null, !isNamespaceMapped);
         String seqSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), !isNamespaceMapped);
-        verifySequence(null, sequenceName, sequenceSchemaName, true);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName, -32767);
+        verifySequenceValue(null, sequenceName, sequenceSchemaName, -32767);
+        conn1.createStatement().execute("CREATE INDEX " + indexName + "_2 ON " + viewName + " (v1)");
+        verifySequenceValue(null, sequenceName, sequenceSchemaName, -32766);
         // Check other format of sequence is not there as Sequences format is different for views/indexes created on
         // table which are namespace mapped and which are not.
-        verifySequence(null, seqName, seqSchemaName, false);
+        verifySequenceNotExists(null, seqName, seqSchemaName);
         conn1.createStatement().execute("DROP VIEW " + viewName);
         conn1.createStatement().execute("DROP TABLE "+ tableName);
         
-        verifySequence(null, sequenceName, sequenceSchemaName, false);
+        verifySequenceNotExists(null, sequenceName, sequenceSchemaName);
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/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 142f700..65da298 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
@@ -195,6 +195,9 @@ import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SequenceAllocation;
+import org.apache.phoenix.schema.SequenceAlreadyExistsException;
+import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SequenceNotFoundException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -1499,6 +1502,53 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
                     cells.add(viewConstantCell);
                 }
+                Short indexId = null;
+                if (request.hasAllocateIndexId() && request.getAllocateIndexId()) {
+                    String tenantIdStr = tenantIdBytes.length == 0 ? null : Bytes.toString(tenantIdBytes);
+                    final Properties props = new Properties();
+                    UpgradeUtil.doNotUpgradeOnFirstConnection(props);
+                    try (PhoenixConnection connection = DriverManager.getConnection(MetaDataUtil.getJdbcUrl(env), props).unwrap(PhoenixConnection.class)){
+                    PName physicalName = parentTable.getPhysicalName();
+                    int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
+                    SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
+                            nSequenceSaltBuckets, parentTable.isNamespaceMapped() );
+                        // TODO Review Earlier sequence was created at (SCN-1/LATEST_TIMESTAMP) and incremented at the client max(SCN,dataTable.getTimestamp), but it seems we should
+                        // use always LATEST_TIMESTAMP to avoid seeing wrong sequence values by different connection having SCN
+                        // or not. 
+                    long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
+                    try {
+                        connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
+                                Short.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
+                    } catch (SequenceAlreadyExistsException e) {
+                    }
+                    long[] seqValues = new long[1];
+                    SQLException[] sqlExceptions = new SQLException[1];
+                    connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
+                            HConstants.LATEST_TIMESTAMP, seqValues, sqlExceptions);
+                    if (sqlExceptions[0] != null) {
+                        throw sqlExceptions[0];
+                    }
+                    long seqValue = seqValues[0];
+                    if (seqValue > Short.MAX_VALUE) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        done.run(builder.build());
+                        return;
+                    }
+                    Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
+                    NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
+                    List<Cell> cells = familyCellMap.get(TABLE_FAMILY_BYTES);
+                    Cell cell = cells.get(0);
+                    PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
+                    Object val = dataType.toObject(seqValue, PLong.INSTANCE);
+                    byte[] bytes = new byte [dataType.getByteSize() + 1];
+                    dataType.toBytes(val, bytes, 0);
+                    Cell indexIdCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_INDEX_ID_BYTES,
+                            cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
+                    cells.add(indexIdCell);
+                    indexId = (short) seqValue;
+                    }
+                }
                 
                 // 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
@@ -1518,6 +1568,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // Get timeStamp from mutations - the above method sets it if it's unset
                 long currentTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+                if (indexId != null) {
+                    builder.setViewIndexId(indexId);
+                }
                 builder.setMutationTime(currentTimeStamp);
                 done.run(builder.build());
                 return;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 3b57981..bb720ee 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -130,6 +130,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
         UNALLOWED_SCHEMA_MUTATION,
         AUTO_PARTITION_SEQUENCE_NOT_FOUND,
         CANNOT_COERCE_AUTO_PARTITION_ID,
+        TOO_MANY_INDEXES,
         NO_OP
     };
 
@@ -208,6 +209,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
         private byte[] familyName;
         private boolean wasUpdated;
         private PSchema schema;
+        private Short viewIndexId;
 
         private List<PFunction> functions = new ArrayList<PFunction>(1);
         private long autoPartitionNum;
@@ -253,6 +255,11 @@ public abstract class MetaDataProtocol extends MetaDataService {
             this.tableNamesToDelete = tableNamesToDelete;
         }
         
+        public MetaDataMutationResult(MutationCode returnCode, int currentTime, PTable table, int viewIndexId) {
+            this(returnCode, currentTime, table, Collections.<byte[]> emptyList());
+            this.viewIndexId = (short)viewIndexId;
+        }
+        
         public MetaDataMutationResult(MutationCode returnCode, long currentTime, PTable table, List<byte[]> tableNamesToDelete, List<SharedTableState> sharedTablesToDelete) {
             this(returnCode, currentTime, table, tableNamesToDelete);
             this.sharedTablesToDelete = sharedTablesToDelete;
@@ -305,6 +312,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
         public long getAutoPartitionNum() {
             return autoPartitionNum;
         }
+        
+        public Short getViewIndexId() {
+            return viewIndexId;
+        }
 
         public static MetaDataMutationResult constructFromProto(MetaDataResponse proto) {
           MetaDataMutationResult result = new MetaDataMutationResult();
@@ -347,6 +358,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
           if (proto.hasAutoPartitionNum()) {
               result.autoPartitionNum = proto.getAutoPartitionNum();
           }
+            if (proto.hasViewIndexId()) {
+                result.viewIndexId = (short)proto.getViewIndexId();
+            }
           return result;
         }
 
@@ -394,6 +408,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
               builder.setSchema(PSchema.toProto(result.schema));
             }
             builder.setAutoPartitionNum(result.getAutoPartitionNum());
+                if (result.getViewIndexId() != null) {
+                    builder.setViewIndexId(result.getViewIndexId());
+                }
           }
           return builder.build();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/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 44dd9e0..e041abd 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
@@ -101,6 +101,10 @@ public final class MetaDataProtos {
      * <code>CANNOT_COERCE_AUTO_PARTITION_ID = 21;</code>
      */
     CANNOT_COERCE_AUTO_PARTITION_ID(21, 21),
+    /**
+     * <code>TOO_MANY_INDEXES = 22;</code>
+     */
+    TOO_MANY_INDEXES(22, 22),
     ;
 
     /**
@@ -191,6 +195,10 @@ public final class MetaDataProtos {
      * <code>CANNOT_COERCE_AUTO_PARTITION_ID = 21;</code>
      */
     public static final int CANNOT_COERCE_AUTO_PARTITION_ID_VALUE = 21;
+    /**
+     * <code>TOO_MANY_INDEXES = 22;</code>
+     */
+    public static final int TOO_MANY_INDEXES_VALUE = 22;
 
 
     public final int getNumber() { return value; }
@@ -219,6 +227,7 @@ public final class MetaDataProtos {
         case 19: return UNALLOWED_SCHEMA_MUTATION;
         case 20: return AUTO_PARTITION_SEQUENCE_NOT_FOUND;
         case 21: return CANNOT_COERCE_AUTO_PARTITION_ID;
+        case 22: return TOO_MANY_INDEXES;
         default: return null;
       }
     }
@@ -1697,6 +1706,16 @@ public final class MetaDataProtos {
      * <code>optional int64 autoPartitionNum = 11;</code>
      */
     long getAutoPartitionNum();
+
+    // optional int32 viewIndexId = 12;
+    /**
+     * <code>optional int32 viewIndexId = 12;</code>
+     */
+    boolean hasViewIndexId();
+    /**
+     * <code>optional int32 viewIndexId = 12;</code>
+     */
+    int getViewIndexId();
   }
   /**
    * Protobuf type {@code MetaDataResponse}
@@ -1835,6 +1854,11 @@ public final class MetaDataProtos {
               autoPartitionNum_ = input.readInt64();
               break;
             }
+            case 96: {
+              bitField0_ |= 0x00000100;
+              viewIndexId_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2119,6 +2143,22 @@ public final class MetaDataProtos {
       return autoPartitionNum_;
     }
 
+    // optional int32 viewIndexId = 12;
+    public static final int VIEWINDEXID_FIELD_NUMBER = 12;
+    private int viewIndexId_;
+    /**
+     * <code>optional int32 viewIndexId = 12;</code>
+     */
+    public boolean hasViewIndexId() {
+      return ((bitField0_ & 0x00000100) == 0x00000100);
+    }
+    /**
+     * <code>optional int32 viewIndexId = 12;</code>
+     */
+    public int getViewIndexId() {
+      return viewIndexId_;
+    }
+
     private void initFields() {
       returnCode_ = org.apache.phoenix.coprocessor.generated.MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS;
       mutationTime_ = 0L;
@@ -2131,6 +2171,7 @@ public final class MetaDataProtos {
       sharedTablesToDelete_ = java.util.Collections.emptyList();
       schema_ = org.apache.phoenix.coprocessor.generated.PSchemaProtos.PSchema.getDefaultInstance();
       autoPartitionNum_ = 0L;
+      viewIndexId_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2201,6 +2242,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000080) == 0x00000080)) {
         output.writeInt64(11, autoPartitionNum_);
       }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeInt32(12, viewIndexId_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -2259,6 +2303,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(11, autoPartitionNum_);
       }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(12, viewIndexId_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -2328,6 +2376,11 @@ public final class MetaDataProtos {
         result = result && (getAutoPartitionNum()
             == other.getAutoPartitionNum());
       }
+      result = result && (hasViewIndexId() == other.hasViewIndexId());
+      if (hasViewIndexId()) {
+        result = result && (getViewIndexId()
+            == other.getViewIndexId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -2385,6 +2438,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + AUTOPARTITIONNUM_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getAutoPartitionNum());
       }
+      if (hasViewIndexId()) {
+        hash = (37 * hash) + VIEWINDEXID_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexId();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -2536,6 +2593,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000200);
         autoPartitionNum_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000400);
+        viewIndexId_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000800);
         return this;
       }
 
@@ -2627,6 +2686,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000080;
         }
         result.autoPartitionNum_ = autoPartitionNum_;
+        if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
+          to_bitField0_ |= 0x00000100;
+        }
+        result.viewIndexId_ = viewIndexId_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -2729,6 +2792,9 @@ public final class MetaDataProtos {
         if (other.hasAutoPartitionNum()) {
           setAutoPartitionNum(other.getAutoPartitionNum());
         }
+        if (other.hasViewIndexId()) {
+          setViewIndexId(other.getViewIndexId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -3776,6 +3842,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional int32 viewIndexId = 12;
+      private int viewIndexId_ ;
+      /**
+       * <code>optional int32 viewIndexId = 12;</code>
+       */
+      public boolean hasViewIndexId() {
+        return ((bitField0_ & 0x00000800) == 0x00000800);
+      }
+      /**
+       * <code>optional int32 viewIndexId = 12;</code>
+       */
+      public int getViewIndexId() {
+        return viewIndexId_;
+      }
+      /**
+       * <code>optional int32 viewIndexId = 12;</code>
+       */
+      public Builder setViewIndexId(int value) {
+        bitField0_ |= 0x00000800;
+        viewIndexId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 viewIndexId = 12;</code>
+       */
+      public Builder clearViewIndexId() {
+        bitField0_ = (bitField0_ & ~0x00000800);
+        viewIndexId_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:MetaDataResponse)
     }
 
@@ -6411,6 +6510,16 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 2;</code>
      */
     int getClientVersion();
+
+    // optional bool allocateIndexId = 3;
+    /**
+     * <code>optional bool allocateIndexId = 3;</code>
+     */
+    boolean hasAllocateIndexId();
+    /**
+     * <code>optional bool allocateIndexId = 3;</code>
+     */
+    boolean getAllocateIndexId();
   }
   /**
    * Protobuf type {@code CreateTableRequest}
@@ -6480,6 +6589,11 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
+            case 24: {
+              bitField0_ |= 0x00000002;
+              allocateIndexId_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -6562,9 +6676,26 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
+    // optional bool allocateIndexId = 3;
+    public static final int ALLOCATEINDEXID_FIELD_NUMBER = 3;
+    private boolean allocateIndexId_;
+    /**
+     * <code>optional bool allocateIndexId = 3;</code>
+     */
+    public boolean hasAllocateIndexId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bool allocateIndexId = 3;</code>
+     */
+    public boolean getAllocateIndexId() {
+      return allocateIndexId_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       clientVersion_ = 0;
+      allocateIndexId_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -6584,6 +6715,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(2, clientVersion_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(3, allocateIndexId_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -6606,6 +6740,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, clientVersion_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, allocateIndexId_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -6636,6 +6774,11 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
+      result = result && (hasAllocateIndexId() == other.hasAllocateIndexId());
+      if (hasAllocateIndexId()) {
+        result = result && (getAllocateIndexId()
+            == other.getAllocateIndexId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -6657,6 +6800,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
+      if (hasAllocateIndexId()) {
+        hash = (37 * hash) + ALLOCATEINDEXID_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getAllocateIndexId());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -6774,6 +6921,8 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
+        allocateIndexId_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -6811,6 +6960,10 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000001;
         }
         result.clientVersion_ = clientVersion_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.allocateIndexId_ = allocateIndexId_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -6840,6 +6993,9 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
+        if (other.hasAllocateIndexId()) {
+          setAllocateIndexId(other.getAllocateIndexId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -6972,6 +7128,39 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional bool allocateIndexId = 3;
+      private boolean allocateIndexId_ ;
+      /**
+       * <code>optional bool allocateIndexId = 3;</code>
+       */
+      public boolean hasAllocateIndexId() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional bool allocateIndexId = 3;</code>
+       */
+      public boolean getAllocateIndexId() {
+        return allocateIndexId_;
+      }
+      /**
+       * <code>optional bool allocateIndexId = 3;</code>
+       */
+      public Builder setAllocateIndexId(boolean value) {
+        bitField0_ |= 0x00000004;
+        allocateIndexId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool allocateIndexId = 3;</code>
+       */
+      public Builder clearAllocateIndexId() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        allocateIndexId_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:CreateTableRequest)
     }
 
@@ -16871,7 +17060,7 @@ public final class MetaDataProtos {
       "TableState\022\020\n\010tenantId\030\001 \001(\014\022\022\n\nschemaNa" +
       "me\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\031\n\007columns\030\004" +
       " \003(\0132\010.PColumn\022\025\n\rphysicalNames\030\005 \003(\014\022\023\n" +
-      "\013viewIndexId\030\006 \002(\005\"\274\002\n\020MetaDataResponse\022" +
+      "\013viewIndexId\030\006 \002(\005\"\321\002\n\020MetaDataResponse\022" +
       "!\n\nreturnCode\030\001 \001(\0162\r.MutationCode\022\024\n\014mu" +
       "tationTime\030\002 \001(\003\022\026\n\005table\030\003 \001(\0132\007.PTable" +
       "\022\026\n\016tablesToDelete\030\004 \003(\014\022\022\n\ncolumnName\030\005" +
@@ -16879,84 +17068,86 @@ public final class MetaDataProtos {
       "\030\007 \001(\014\022\034\n\010function\030\010 \003(\0132\n.PFunction\022/\n\024" +
       "sharedTablesToDelete\030\t \003(\0132\021.SharedTable" +
       "State\022\030\n\006schema\030\n \001(\0132\010.PSchema\022\030\n\020autoP" +
-      "artitionNum\030\013 \001(\003\"\222\001\n\017GetTableRequest\022\020\n" +
-      "\010tenantId\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\tt" +
-      "ableName\030\003 \002(\014\022\026\n\016tableTimestamp\030\004 \002(\003\022\027" +
-      "\n\017clientTimestamp\030\005 \002(\003\022\025\n\rclientVersion" +
-      "\030\006 \001(\005\"\212\001\n\023GetFunctionsRequest\022\020\n\010tenant" +
-      "Id\030\001 \002(\014\022\025\n\rfunctionNames\030\002 \003(\014\022\032\n\022funct" +
-      "ionTimestamps\030\003 \003(\003\022\027\n\017clientTimestamp\030\004",
-      " \002(\003\022\025\n\rclientVersion\030\005 \001(\005\"V\n\020GetSchema" +
-      "Request\022\022\n\nschemaName\030\001 \002(\t\022\027\n\017clientTim" +
-      "estamp\030\002 \002(\003\022\025\n\rclientVersion\030\003 \002(\005\"K\n\022C" +
-      "reateTableRequest\022\036\n\026tableMetadataMutati" +
-      "ons\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"r\n\025Crea" +
-      "teFunctionRequest\022\036\n\026tableMetadataMutati" +
-      "ons\030\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007replace\030" +
-      "\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\"`\n\023CreateSc" +
-      "hemaRequest\022\036\n\026tableMetadataMutations\030\001 " +
-      "\003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersion",
-      "\030\003 \002(\005\"m\n\020DropTableRequest\022\036\n\026tableMetad" +
-      "ataMutations\030\001 \003(\014\022\021\n\ttableType\030\002 \002(\t\022\017\n" +
-      "\007cascade\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\"_\n" +
-      "\021DropSchemaRequest\022\037\n\027schemaMetadataMuta" +
-      "tions\030\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclien" +
-      "tVersion\030\003 \002(\005\"I\n\020AddColumnRequest\022\036\n\026ta" +
-      "bleMetadataMutations\030\001 \003(\014\022\025\n\rclientVers" +
-      "ion\030\002 \001(\005\"J\n\021DropColumnRequest\022\036\n\026tableM" +
-      "etadataMutations\030\001 \003(\014\022\025\n\rclientVersion\030" +
-      "\002 \001(\005\"^\n\023DropFunctionRequest\022\036\n\026tableMet",
-      "adataMutations\030\001 \003(\014\022\020\n\010ifExists\030\002 \001(\010\022\025" +
-      "\n\rclientVersion\030\003 \001(\005\"P\n\027UpdateIndexStat" +
-      "eRequest\022\036\n\026tableMetadataMutations\030\001 \003(\014" +
-      "\022\025\n\rclientVersion\030\002 \001(\005\"*\n\021ClearCacheReq" +
-      "uest\022\025\n\rclientVersion\030\001 \001(\005\"*\n\022ClearCach" +
-      "eResponse\022\024\n\014unfreedBytes\030\001 \001(\003\"*\n\021GetVe" +
-      "rsionRequest\022\025\n\rclientVersion\030\001 \001(\005\"%\n\022G" +
-      "etVersionResponse\022\017\n\007version\030\001 \002(\003\"\205\001\n\032C" +
-      "learTableFromCacheRequest\022\020\n\010tenantId\030\001 " +
-      "\002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002",
-      "(\014\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n\rclientVer" +
-      "sion\030\005 \001(\005\"\035\n\033ClearTableFromCacheRespons" +
-      "e*\337\004\n\014MutationCode\022\030\n\024TABLE_ALREADY_EXIS" +
-      "TS\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_" +
-      "FOUND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CO" +
-      "NCURRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_I" +
-      "N_REGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNA" +
-      "LLOWED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS" +
-      "\020\010\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t\022\033\n\027FUNCTI" +
-      "ON_ALREADY_EXISTS\020\n\022\026\n\022FUNCTION_NOT_FOUN",
-      "D\020\013\022\030\n\024NEWER_FUNCTION_FOUND\020\014\022\032\n\026FUNCTIO" +
-      "N_NOT_IN_REGION\020\r\022\031\n\025SCHEMA_ALREADY_EXIS" +
-      "TS\020\016\022\026\n\022NEWER_SCHEMA_FOUND\020\017\022\024\n\020SCHEMA_N" +
-      "OT_FOUND\020\020\022\030\n\024SCHEMA_NOT_IN_REGION\020\021\022\032\n\026" +
-      "TABLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UNALLOWED_SC" +
-      "HEMA_MUTATION\020\023\022%\n!AUTO_PARTITION_SEQUEN" +
-      "CE_NOT_FOUND\020\024\022#\n\037CANNOT_COERCE_AUTO_PAR" +
-      "TITION_ID\020\0252\345\006\n\017MetaDataService\022/\n\010getTa" +
-      "ble\022\020.GetTableRequest\032\021.MetaDataResponse" +
-      "\0227\n\014getFunctions\022\024.GetFunctionsRequest\032\021",
-      ".MetaDataResponse\0221\n\tgetSchema\022\021.GetSche" +
-      "maRequest\032\021.MetaDataResponse\0225\n\013createTa" +
-      "ble\022\023.CreateTableRequest\032\021.MetaDataRespo" +
-      "nse\022;\n\016createFunction\022\026.CreateFunctionRe" +
-      "quest\032\021.MetaDataResponse\0227\n\014createSchema" +
-      "\022\024.CreateSchemaRequest\032\021.MetaDataRespons" +
-      "e\0221\n\tdropTable\022\021.DropTableRequest\032\021.Meta" +
-      "DataResponse\0223\n\ndropSchema\022\022.DropSchemaR" +
-      "equest\032\021.MetaDataResponse\0227\n\014dropFunctio" +
-      "n\022\024.DropFunctionRequest\032\021.MetaDataRespon",
-      "se\0221\n\taddColumn\022\021.AddColumnRequest\032\021.Met" +
-      "aDataResponse\0223\n\ndropColumn\022\022.DropColumn" +
-      "Request\032\021.MetaDataResponse\022?\n\020updateInde" +
-      "xState\022\030.UpdateIndexStateRequest\032\021.MetaD" +
-      "ataResponse\0225\n\nclearCache\022\022.ClearCacheRe" +
-      "quest\032\023.ClearCacheResponse\0225\n\ngetVersion" +
-      "\022\022.GetVersionRequest\032\023.GetVersionRespons" +
-      "e\022P\n\023clearTableFromCache\022\033.ClearTableFro" +
-      "mCacheRequest\032\034.ClearTableFromCacheRespo" +
-      "nseBB\n(org.apache.phoenix.coprocessor.ge",
-      "neratedB\016MetaDataProtosH\001\210\001\001\240\001\001"
+      "artitionNum\030\013 \001(\003\022\023\n\013viewIndexId\030\014 \001(\005\"\222" +
+      "\001\n\017GetTableRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\n" +
+      "schemaName\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\026\n\016t" +
+      "ableTimestamp\030\004 \002(\003\022\027\n\017clientTimestamp\030\005" +
+      " \002(\003\022\025\n\rclientVersion\030\006 \001(\005\"\212\001\n\023GetFunct" +
+      "ionsRequest\022\020\n\010tenantId\030\001 \002(\014\022\025\n\rfunctio" +
+      "nNames\030\002 \003(\014\022\032\n\022functionTimestamps\030\003 \003(\003",
+      "\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n\rclientVersi" +
+      "on\030\005 \001(\005\"V\n\020GetSchemaRequest\022\022\n\nschemaNa" +
+      "me\030\001 \002(\t\022\027\n\017clientTimestamp\030\002 \002(\003\022\025\n\rcli" +
+      "entVersion\030\003 \002(\005\"d\n\022CreateTableRequest\022\036" +
+      "\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rclient" +
+      "Version\030\002 \001(\005\022\027\n\017allocateIndexId\030\003 \001(\010\"r" +
+      "\n\025CreateFunctionRequest\022\036\n\026tableMetadata" +
+      "Mutations\030\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007re" +
+      "place\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\"`\n\023Cr" +
+      "eateSchemaRequest\022\036\n\026tableMetadataMutati",
+      "ons\030\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientV" +
+      "ersion\030\003 \002(\005\"m\n\020DropTableRequest\022\036\n\026tabl" +
+      "eMetadataMutations\030\001 \003(\014\022\021\n\ttableType\030\002 " +
+      "\002(\t\022\017\n\007cascade\030\003 \001(\010\022\025\n\rclientVersion\030\004 " +
+      "\001(\005\"_\n\021DropSchemaRequest\022\037\n\027schemaMetada" +
+      "taMutations\030\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n" +
+      "\rclientVersion\030\003 \002(\005\"I\n\020AddColumnRequest" +
+      "\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rclie" +
+      "ntVersion\030\002 \001(\005\"J\n\021DropColumnRequest\022\036\n\026" +
+      "tableMetadataMutations\030\001 \003(\014\022\025\n\rclientVe",
+      "rsion\030\002 \001(\005\"^\n\023DropFunctionRequest\022\036\n\026ta" +
+      "bleMetadataMutations\030\001 \003(\014\022\020\n\010ifExists\030\002" +
+      " \001(\010\022\025\n\rclientVersion\030\003 \001(\005\"P\n\027UpdateInd" +
+      "exStateRequest\022\036\n\026tableMetadataMutations" +
+      "\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"*\n\021ClearCa" +
+      "cheRequest\022\025\n\rclientVersion\030\001 \001(\005\"*\n\022Cle" +
+      "arCacheResponse\022\024\n\014unfreedBytes\030\001 \001(\003\"*\n" +
+      "\021GetVersionRequest\022\025\n\rclientVersion\030\001 \001(" +
+      "\005\"%\n\022GetVersionResponse\022\017\n\007version\030\001 \002(\003" +
+      "\"\205\001\n\032ClearTableFromCacheRequest\022\020\n\010tenan",
+      "tId\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttableNa" +
+      "me\030\003 \002(\014\022\027\n\017clientTimestamp\030\004 \002(\003\022\025\n\rcli" +
+      "entVersion\030\005 \001(\005\"\035\n\033ClearTableFromCacheR" +
+      "esponse*\365\004\n\014MutationCode\022\030\n\024TABLE_ALREAD" +
+      "Y_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUM" +
+      "N_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_TABLE_MUTATION\020\007\022\021\n\rNO_PK_C" +
+      "OLUMNS\020\010\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t\022\033\n\027",
+      "FUNCTION_ALREADY_EXISTS\020\n\022\026\n\022FUNCTION_NO" +
+      "T_FOUND\020\013\022\030\n\024NEWER_FUNCTION_FOUND\020\014\022\032\n\026F" +
+      "UNCTION_NOT_IN_REGION\020\r\022\031\n\025SCHEMA_ALREAD" +
+      "Y_EXISTS\020\016\022\026\n\022NEWER_SCHEMA_FOUND\020\017\022\024\n\020SC" +
+      "HEMA_NOT_FOUND\020\020\022\030\n\024SCHEMA_NOT_IN_REGION" +
+      "\020\021\022\032\n\026TABLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UNALLO" +
+      "WED_SCHEMA_MUTATION\020\023\022%\n!AUTO_PARTITION_" +
+      "SEQUENCE_NOT_FOUND\020\024\022#\n\037CANNOT_COERCE_AU" +
+      "TO_PARTITION_ID\020\025\022\024\n\020TOO_MANY_INDEXES\020\0262" +
+      "\345\006\n\017MetaDataService\022/\n\010getTable\022\020.GetTab",
+      "leRequest\032\021.MetaDataResponse\0227\n\014getFunct" +
+      "ions\022\024.GetFunctionsRequest\032\021.MetaDataRes" +
+      "ponse\0221\n\tgetSchema\022\021.GetSchemaRequest\032\021." +
+      "MetaDataResponse\0225\n\013createTable\022\023.Create" +
+      "TableRequest\032\021.MetaDataResponse\022;\n\016creat" +
+      "eFunction\022\026.CreateFunctionRequest\032\021.Meta" +
+      "DataResponse\0227\n\014createSchema\022\024.CreateSch" +
+      "emaRequest\032\021.MetaDataResponse\0221\n\tdropTab" +
+      "le\022\021.DropTableRequest\032\021.MetaDataResponse" +
+      "\0223\n\ndropSchema\022\022.DropSchemaRequest\032\021.Met",
+      "aDataResponse\0227\n\014dropFunction\022\024.DropFunc" +
+      "tionRequest\032\021.MetaDataResponse\0221\n\taddCol" +
+      "umn\022\021.AddColumnRequest\032\021.MetaDataRespons" +
+      "e\0223\n\ndropColumn\022\022.DropColumnRequest\032\021.Me" +
+      "taDataResponse\022?\n\020updateIndexState\022\030.Upd" +
+      "ateIndexStateRequest\032\021.MetaDataResponse\022" +
+      "5\n\nclearCache\022\022.ClearCacheRequest\032\023.Clea" +
+      "rCacheResponse\0225\n\ngetVersion\022\022.GetVersio" +
+      "nRequest\032\023.GetVersionResponse\022P\n\023clearTa" +
+      "bleFromCache\022\033.ClearTableFromCacheReques",
+      "t\032\034.ClearTableFromCacheResponseBB\n(org.a" +
+      "pache.phoenix.coprocessor.generatedB\016Met" +
+      "aDataProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -16974,7 +17165,7 @@ public final class MetaDataProtos {
           internal_static_MetaDataResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_MetaDataResponse_descriptor,
-              new java.lang.String[] { "ReturnCode", "MutationTime", "Table", "TablesToDelete", "ColumnName", "FamilyName", "FunctionName", "Function", "SharedTablesToDelete", "Schema", "AutoPartitionNum", });
+              new java.lang.String[] { "ReturnCode", "MutationTime", "Table", "TablesToDelete", "ColumnName", "FamilyName", "FunctionName", "Function", "SharedTablesToDelete", "Schema", "AutoPartitionNum", "ViewIndexId", });
           internal_static_GetTableRequest_descriptor =
             getDescriptor().getMessageTypes().get(2);
           internal_static_GetTableRequest_fieldAccessorTable = new
@@ -16998,7 +17189,7 @@ public final class MetaDataProtos {
           internal_static_CreateTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CreateTableRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", });
+              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", "AllocateIndexId", });
           internal_static_CreateFunctionRequest_descriptor =
             getDescriptor().getMessageTypes().get(6);
           internal_static_CreateFunctionRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index 6f8b528..bc2c93b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -79,7 +79,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
 
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] tableName, PTableType tableType,
             Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped) throws SQLException;
+            boolean isNamespaceMapped, boolean allocateIndexId) throws SQLException;
     public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException;
     public MetaDataMutationResult dropFunction(List<Mutation> tableMetadata, boolean ifExists) throws SQLException;
     public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/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 f5f85e4..354a932 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
@@ -1373,9 +1373,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public MetaDataMutationResult createTable(final List<Mutation> tableMetaData, byte[] physicalTableName,
+    public MetaDataMutationResult createTable(final List<Mutation> tableMetaData, final byte[] physicalTableName,
             PTableType tableType, Map<String, Object> tableProps,
-            final List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, boolean isNamespaceMapped)
+            final List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, boolean isNamespaceMapped, final boolean allocateIndexId)
                     throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
         Mutation m = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetaData);
@@ -1446,6 +1446,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     builder.addTableMetadataMutations(mp.toByteString());
                 }
                 builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                        if (allocateIndexId) {
+                            builder.setAllocateIndexId(allocateIndexId);
+                        }
                 CreateTableRequest build = builder.build();
                 instance.createTable(controller, build, rpcCallback);
                 if(controller.getFailedOn() != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 337e43c..8c9f3ff 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -250,12 +250,16 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     @Override
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] physicalName, PTableType tableType,
             Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped) throws SQLException {
+            boolean isNamespaceMapped, boolean allocateIndexId) throws SQLException {
         if (splits != null) {
             byte[] tableName = getTableName(tableMetaData, physicalName);
             tableSplits.put(Bytes.toString(tableName), generateRegionLocations(tableName, splits));
         }
-        return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null);
+        if (!allocateIndexId) {
+            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null);
+        } else {
+            return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null, Short.MIN_VALUE);
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 81517e3..b00e92b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -120,9 +120,9 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     @Override
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] physicalName, PTableType tableType,
             Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped) throws SQLException {
+            boolean isNamespaceMapped, boolean allocateIndexId) throws SQLException {
         return getDelegate().createTable(tableMetaData, physicalName, tableType, tableProps, families, splits,
-                isNamespaceMapped);
+                isNamespaceMapped, allocateIndexId);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/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 73f1501..1e635d8 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
@@ -80,7 +80,6 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
@@ -96,7 +95,6 @@ import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.Date;
-import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
@@ -251,7 +249,6 @@ public class MetaDataClient {
                     DISABLE_WAL + "," +
                     MULTI_TENANT + "," +
                     VIEW_TYPE + "," +
-                    VIEW_INDEX_ID + "," +
                     INDEX_TYPE + "," +
                     STORE_NULLS + "," +
                     BASE_COLUMN_COUNT + "," +
@@ -260,7 +257,7 @@ public class MetaDataClient {
                     IS_NAMESPACE_MAPPED + "," +
                     AUTO_PARTITION_SEQ +  "," +
                     APPEND_ONLY_SCHEMA +
-                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
@@ -937,7 +934,7 @@ public class MetaDataClient {
                         true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
-        table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null, tableProps, commonFamilyProps);
+        table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, false, null, null, tableProps, commonFamilyProps);
 
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
@@ -1255,7 +1252,6 @@ public class MetaDataClient {
         TableRef tableRef = null;
         PTable table = null;
         int numRetries = 0;
-        Short indexId = null;
         boolean allocateIndexId = false;
         boolean isLocalIndex = statement.getIndexType() == IndexType.LOCAL;
         int hbaseVersion = connection.getQueryServices().getLowestClusterHBaseVersion();
@@ -1412,44 +1408,14 @@ public class MetaDataClient {
                     }
                 }
 
-                // Don't re-allocate indexId on ConcurrentTableMutationException,
-                // as there's no need to burn another sequence value.
-                if (allocateIndexId && indexId == null) {
-                    Long scn = connection.getSCN();
-                    PName tenantId = connection.getTenantId();
-                    String tenantIdStr = tenantId == null ? null : connection.getTenantId().getString();
-                    PName physicalName = dataTable.getPhysicalName();
-                    int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
-                    SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
-                            nSequenceSaltBuckets, dataTable.isNamespaceMapped());
-                    // if scn is set create at scn-1, so we can see the sequence or else use latest timestamp (so that latest server time is used)
-                    long sequenceTimestamp = scn!=null ? scn-1 : HConstants.LATEST_TIMESTAMP;
-                    createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                            true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
-                            sequenceTimestamp);
-                    long[] seqValues = new long[1];
-                    SQLException[] sqlExceptions = new SQLException[1];
-                    long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
-                    connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
-                            Math.max(timestamp, dataTable.getTimeStamp()), seqValues, sqlExceptions);
-                    if (sqlExceptions[0] != null) {
-                        throw sqlExceptions[0];
-                    }
-                    long seqValue = seqValues[0];
-                    if (seqValue > Short.MAX_VALUE) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.TOO_MANY_INDEXES)
-                        .setSchemaName(SchemaUtil.getSchemaNameFromFullName(physicalName.getString())).setTableName(SchemaUtil.getTableNameFromFullName(physicalName.getString())).build().buildException();
-                    }
-                    indexId = (short) seqValue;
-                }
                 // Set DEFAULT_COLUMN_FAMILY_NAME of index to match data table
                 // We need this in the props so that the correct column family is created
-                if (dataTable.getDefaultFamilyName() != null && dataTable.getType() != PTableType.VIEW && indexId == null) {
+                if (dataTable.getDefaultFamilyName() != null && dataTable.getType() != PTableType.VIEW && !allocateIndexId) {
                     statement.getProps().put("", new Pair<String,Object>(DEFAULT_COLUMN_FAMILY_NAME,dataTable.getDefaultFamilyName().getString()));
                 }
                 PrimaryKeyConstraint pk = FACTORY.primaryKey(null, allPkColumns);
                 CreateTableStatement tableStatement = FACTORY.createTable(indexTableName, statement.getProps(), columnDefs, pk, statement.getSplitNodes(), PTableType.INDEX, statement.ifNotExists(), null, null, statement.getBindCount());
-                table = createTableInternal(tableStatement, splits, dataTable, null, null, null, null, indexId, statement.getIndexType(), asyncCreatedDate, tableProps, commonFamilyProps);
+                table = createTableInternal(tableStatement, splits, dataTable, null, null, null, null, allocateIndexId, statement.getIndexType(), asyncCreatedDate, tableProps, commonFamilyProps);
                 break;
             } catch (ConcurrentTableMutationException e) { // Can happen if parent data table changes while above is in progress
                 if (numRetries<5) {
@@ -1645,7 +1611,7 @@ public class MetaDataClient {
 
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
-            final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId,
+            final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, boolean allocateIndexId,
             IndexType indexType, Date asyncCreatedDate,
             Map<String,Object> tableProps,
             Map<String,Object> commonFamilyProps) throws SQLException {
@@ -1762,7 +1728,7 @@ public class MetaDataClient {
             }
 
             // Can't set any of these on views or shared indexes on views
-            if (tableType != PTableType.VIEW && indexId == null) {
+            if (tableType != PTableType.VIEW && !allocateIndexId) {
                 saltBucketNum = (Integer) TableProperty.SALT_BUCKETS.getValue(tableProps);
                 if (saltBucketNum != null) {
                     if (saltBucketNum < 0 || saltBucketNum > SaltingUtil.MAX_BUCKET_NUM) {
@@ -1852,7 +1818,7 @@ public class MetaDataClient {
                 }
             }
 
-            boolean sharedTable = statement.getTableType() == PTableType.VIEW || indexId != null;
+            boolean sharedTable = statement.getTableType() == PTableType.VIEW || allocateIndexId;
             if (transactional) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
@@ -2151,7 +2117,7 @@ public class MetaDataClient {
                         Collections.<PTable>emptyList(), isImmutableRows,
                         Collections.<PName>emptyList(), defaultFamilyName == null ? null :
                                 PNameFactory.newName(defaultFamilyName), null,
-                        Boolean.TRUE.equals(disableWAL), false, false, null, indexId, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
+                        Boolean.TRUE.equals(disableWAL), false, false, null, null, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
                 connection.addTable(table, MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             }
 
@@ -2244,31 +2210,26 @@ public class MetaDataClient {
             } else {
                 tableUpsert.setByte(16, viewType.getSerializedValue());
             }
-            if (indexId == null) {
-                tableUpsert.setNull(17, Types.SMALLINT);
-            } else {
-                tableUpsert.setShort(17, indexId);
-            }
             if (indexType == null) {
-                tableUpsert.setNull(18, Types.TINYINT);
+                tableUpsert.setNull(17, Types.TINYINT);
             } else {
-                tableUpsert.setByte(18, indexType.getSerializedValue());
+                tableUpsert.setByte(17, indexType.getSerializedValue());
             }
-            tableUpsert.setBoolean(19, storeNulls);
+            tableUpsert.setBoolean(18, storeNulls);
             if (parent != null && tableType == PTableType.VIEW) {
-                tableUpsert.setInt(20, parent.getColumns().size());
+                tableUpsert.setInt(19, parent.getColumns().size());
             } else {
-                tableUpsert.setInt(20, BASE_TABLE_BASE_COLUMN_COUNT);
+                tableUpsert.setInt(19, BASE_TABLE_BASE_COLUMN_COUNT);
             }
-            tableUpsert.setBoolean(21, transactional);
-            tableUpsert.setLong(22, updateCacheFrequency);
-            tableUpsert.setBoolean(23, isNamespaceMapped);
+            tableUpsert.setBoolean(20, transactional);
+            tableUpsert.setLong(21, updateCacheFrequency);
+            tableUpsert.setBoolean(22, isNamespaceMapped);
             if (autoPartitionSeq == null) {
-                tableUpsert.setNull(24, Types.VARCHAR);
+                tableUpsert.setNull(23, Types.VARCHAR);
             } else {
-                tableUpsert.setString(24, autoPartitionSeq);
+                tableUpsert.setString(23, autoPartitionSeq);
             }
-            tableUpsert.setBoolean(25, isAppendOnlySchema);
+            tableUpsert.setBoolean(24, isAppendOnlySchema);
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2296,8 +2257,8 @@ public class MetaDataClient {
             }
             MetaDataMutationResult result = connection.getQueryServices().createTable(
                     tableMetaData,
-                    viewType == ViewType.MAPPED || indexId != null ? physicalNames.get(0).getBytes() : null,
-                    tableType, tableProps, familyPropList, splits, isNamespaceMapped);
+                    viewType == ViewType.MAPPED || allocateIndexId ? physicalNames.get(0).getBytes() : null,
+                    tableType, tableProps, familyPropList, splits, isNamespaceMapped, allocateIndexId);
             MutationCode code = result.getMutationCode();
             switch(code) {
             case TABLE_ALREADY_EXISTS:
@@ -2328,6 +2289,11 @@ public class MetaDataClient {
             case CANNOT_COERCE_AUTO_PARTITION_ID:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_COERCE_AUTO_PARTITION_ID)
                 .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            case TOO_MANY_INDEXES:
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.TOO_MANY_INDEXES)
+                        .setSchemaName(SchemaUtil.getSchemaNameFromFullName(parent.getPhysicalName().getString()))
+                        .setTableName(SchemaUtil.getTableNameFromFullName(parent.getPhysicalName().getString())).build()
+                        .buildException();
             default:
                 // If the parent table of the view has the auto partition sequence name attribute,
                 // set the view statement and relevant partition column attributes correctly
@@ -2362,7 +2328,7 @@ public class MetaDataClient {
                         PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns.values(),
                         parent == null ? null : parent.getSchemaName(), parent == null ? null : parent.getTableName(), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        indexId, indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
+                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java
index 27c30fc..d249a66 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/TenantSpecificViewIndexCompileTest.java
@@ -194,7 +194,7 @@ public class TenantSpecificViewIndexCompileTest extends BaseConnectionlessQueryT
         conn.createStatement().execute("CREATE INDEX i1 ON v(v2)");
         
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT v2 FROM v WHERE v2 > 'a' and k2 = 'a' ORDER BY v2,k2");
-        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-32766,'me','a'] - [-32766,'me',*]\n" + 
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-32768,'me','a'] - [-32768,'me',*]\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY",
                 QueryUtil.getExplainPlan(rs));
         
@@ -227,7 +227,7 @@ public class TenantSpecificViewIndexCompileTest extends BaseConnectionlessQueryT
         
         // Confirm that a read-only view on an updatable view still optimizes out the read-only parts of the updatable view
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT v2 FROM v2 WHERE v3 > 'a' and k2 = 'a' ORDER BY v3,k2");
-        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-32767,'me','a'] - [-32767,'me',*]",
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-32768,'me','a'] - [-32768,'me',*]",
                 QueryUtil.getExplainPlan(rs));
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 075828e..bdcd520 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -1715,14 +1715,22 @@ public abstract class BaseTest {
             conn.close();
         }
     }
-
-    protected static void verifySequence(String tenantID, String sequenceName, String sequenceSchemaName, boolean exists) throws SQLException {
+    protected static void verifySequenceNotExists(String tenantID, String sequenceName, String sequenceSchemaName) throws SQLException {
+        verifySequence(tenantID, sequenceName, sequenceSchemaName, false, 0);
+    }
+    
+    protected static void verifySequenceValue(String tenantID, String sequenceName, String sequenceSchemaName, long value) throws SQLException {
+        verifySequence(tenantID, sequenceName, sequenceSchemaName, true, value);
+    }
+    
+    private  static void verifySequence(String tenantID, String sequenceName, String sequenceSchemaName, boolean exists, long value) throws SQLException {
 
         PhoenixConnection phxConn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class);
         String ddl = "SELECT "
                 + PhoenixDatabaseMetaData.TENANT_ID + ","
                 + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
-                + PhoenixDatabaseMetaData.SEQUENCE_NAME
+                + PhoenixDatabaseMetaData.SEQUENCE_NAME + ","
+                + PhoenixDatabaseMetaData.CURRENT_VALUE
                 + " FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE
                 + " WHERE ";
 
@@ -1734,6 +1742,7 @@ public abstract class BaseTest {
 
         if(exists) {
             assertTrue(rs.next());
+            assertEquals(value, rs.getLong(4));
         } else {
             assertFalse(rs.next());
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/655fe2be/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index 70eee3b..2ba2b4c 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -49,6 +49,7 @@ enum MutationCode {
   UNALLOWED_SCHEMA_MUTATION = 19;
   AUTO_PARTITION_SEQUENCE_NOT_FOUND = 20;
   CANNOT_COERCE_AUTO_PARTITION_ID = 21;
+  TOO_MANY_INDEXES = 22;
 };
 
 message SharedTableState {
@@ -72,6 +73,7 @@ message MetaDataResponse {
   repeated SharedTableState sharedTablesToDelete = 9;
   optional PSchema schema = 10;
   optional int64 autoPartitionNum = 11;
+  optional int32 viewIndexId = 12;
 }
 
 message GetTableRequest {
@@ -101,6 +103,7 @@ message GetSchemaRequest {
 message CreateTableRequest {
   repeated bytes tableMetadataMutations = 1;
   optional int32 clientVersion = 2;
+  optional bool allocateIndexId = 3;
 }
 
 // each byte array represents a MutationProto instance