You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2014/10/07 11:23:45 UTC

[2/3] git commit: Atomically increment table timestamp and fix unit tests

Atomically increment table timestamp and fix unit tests

Conflicts:
	phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
	phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
	phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
	phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java


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

Branch: refs/heads/4.0
Commit: 450400f47c73e3ea80e6c4635ed044be8d7722b6
Parents: 4281a9c
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Oct 6 19:47:29 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Oct 7 01:43:25 2014 -0700

----------------------------------------------------------------------
 .../end2end/BaseTenantSpecificTablesIT.java     |   2 +-
 .../end2end/TenantSpecificTablesDMLIT.java      |  18 +-
 .../coprocessor/MetaDataEndpointImpl.java       | 193 +++++-----
 .../coprocessor/generated/MetaDataProtos.java   | 351 ++++++++++---------
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java      |  17 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  18 +-
 phoenix-protocol/src/main/MetaDataService.proto |   8 +-
 10 files changed, 320 insertions(+), 295 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
index b8fa035..fe08ac5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
@@ -84,7 +84,7 @@ public abstract class BaseTenantSpecificTablesIT extends BaseClientManagedTimeIT
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Must update config before starting server
-        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20l));
+        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
         PHOENIX_JDBC_TENANT_SPECIFIC_URL = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + TENANT_ID;
         PHOENIX_JDBC_TENANT_SPECIFIC_URL2 = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + TENANT_ID2;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
index b01c0ab..21f80f9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
@@ -53,6 +53,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.close();
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             analyzeTable(conn, TENANT_TABLE_NAME);
+            conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             ResultSet rs = conn.createStatement().executeQuery("select tenant_col from " + TENANT_TABLE_NAME + " where id = 1");
             assertTrue("Expected 1 row in result set", rs.next());
             assertEquals("Cheap Sunglasses", rs.getString(1));
@@ -75,6 +76,8 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn1.commit();
             conn1 = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             analyzeTable(conn1, TENANT_TABLE_NAME);
+            conn1 = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
+            
             conn2.setAutoCommit(true);
             conn2.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " values ('them','" + TENANT_TYPE_ID + "',1,'Long Hair')");
             conn2.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " values ('us','" + TENANT_TYPE_ID + "',2,'Black Hat')");
@@ -88,7 +91,8 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             assertFalse("Expected 1 row in result set", rs.next());
             conn2 = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2);
             analyzeTable(conn2, TENANT_TABLE_NAME);
-            rs = conn2.createStatement().executeQuery("select count(*) from " + TENANT_TABLE_NAME );
+            conn2 = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2);
+
             rs = conn2.createStatement().executeQuery("select * from " + TENANT_TABLE_NAME + " where id = 2");
             assertTrue("Expected 1 row in result set", rs.next());
             assertEquals(2, rs.getInt(3));
@@ -143,6 +147,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             assertTrue("Expected 1 row in result set", rs.next());
             assertEquals(2, rs.getInt(3));
             assertEquals("Viva Las Vegas", rs.getString(4));
+            
             List<KeyRange> splits = getAllSplits(conn1, TENANT_TABLE_NAME);
             assertEquals(3, splits.size());
         }
@@ -179,6 +184,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             analyzeTable(conn, TENANT_TABLE_NAME);
+            conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             ResultSet rs = conn.createStatement().executeQuery("select tenant_col from " + TENANT_TABLE_NAME + " join foo on k=id");
             assertTrue("Expected 1 row in result set", rs.next());
             assertEquals("Cheap Sunglasses", rs.getString(1));
@@ -250,6 +256,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             
             conn = nextConnection(getUrl());
             analyzeTable(conn, PARENT_TABLE_NAME);
+            conn = nextConnection(getUrl());
             rs = conn.createStatement().executeQuery("select count(*) from " + PARENT_TABLE_NAME);
             rs.next();
             assertEquals(2, rs.getInt(1));
@@ -311,6 +318,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             analyzeTable(conn, PARENT_TABLE_NAME);
+            conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             conn.createStatement().execute("delete from " + TENANT_TABLE_NAME);
             conn.commit();
             conn.close();
@@ -346,6 +354,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             
             conn = nextConnection(getUrl());
             analyzeTable(conn, PARENT_TABLE_NAME_NO_TENANT_TYPE_ID);
+            conn = nextConnection(getUrl());
             ResultSet rs = conn.createStatement().executeQuery("select count(*) from " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID);
             rs.next();
             assertEquals(3, rs.getInt(1));
@@ -371,8 +380,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
-            conn.setAutoCommit(true);
             analyzeTable(conn, TENANT_TABLE_NAME);
+            conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
+            conn.setAutoCommit(true);
             int count = conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, user) select id+100, user from " + TENANT_TABLE_NAME);
             assertEquals("Expected 1 row to have been inserted", 1, count);
             conn.close();
@@ -406,8 +416,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
-            conn.setAutoCommit(true);
             analyzeTable(conn, TENANT_TABLE_NAME);
+            conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
+            conn.setAutoCommit(true);
             int count = conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, user) select id+100, user from ANOTHER_TENANT_TABLE where id=2");
             assertEquals("Expected 1 row to have been inserted", 1, count);
             conn.close();
@@ -455,6 +466,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.close();
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             analyzeTable(conn, PARENT_TABLE_NAME);
+            conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             rs = conn.createStatement().executeQuery("select user from " + PARENT_TABLE_NAME);
             assertTrue(rs.next());
             assertEquals(rs.getString(1),"Billy Gibbons");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/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 2e6f0b4..09ca2dd 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
@@ -92,8 +92,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest;
@@ -102,6 +100,8 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropTableRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetTableRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
@@ -144,7 +144,6 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.cache.Cache;
 import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
@@ -347,64 +346,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    private PTable doGetTable(byte[] key, long clientTimeStamp) throws IOException, SQLException {
-        ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
-        Cache<ImmutableBytesPtr, PTable> metaDataCache =
-                GlobalCache.getInstance(this.env).getMetaDataCache();
-        PTable table = metaDataCache.getIfPresent(cacheKey);
-        // We only cache the latest, so we'll end up building the table with every call if the
-        // client connection has specified an SCN.
-        // TODO: If we indicate to the client that we're returning an older version, but there's a
-        // newer version available, the client
-        // can safely not call this, since we only allow modifications to the latest.
-        if (table != null && table.getTimeStamp() < clientTimeStamp) {
-            // Table on client is up-to-date with table on server, so just return
-            if (isTableDeleted(table)) {
-                return null;
-            }
-            return table;
-        }
-        // Ask Lars about the expense of this call - if we don't take the lock, we still won't get
-        // partial results
-        // get the co-processor environment
-        // TODO: check that key is within region.getStartKey() and region.getEndKey()
-        // and return special code to force client to lookup region from meta.
-        HRegion region = env.getRegion();
-        /*
-         * Lock directly on key, though it may be an index table. This will just prevent a table
-         * from getting rebuilt too often.
-         */
-        RowLock rowLock = region.getRowLock(key);
-        if (rowLock == null) {
-            throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
-        }
-        try {
-            // Try cache again in case we were waiting on a lock
-            table = metaDataCache.getIfPresent(cacheKey);
-            // We only cache the latest, so we'll end up building the table with every call if the
-            // client connection has specified an SCN.
-            // TODO: If we indicate to the client that we're returning an older version, but there's
-            // a newer version available, the client
-            // can safely not call this, since we only allow modifications to the latest.
-            if (table != null && table.getTimeStamp() < clientTimeStamp) {
-                // Table on client is up-to-date with table on server, so just return
-                if (isTableDeleted(table)) {
-                    return null;
-                }
-                return table;
-            }
-            // Query for the latest table first, since it's not cached
-            table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP);
-            if (table != null && table.getTimeStamp() < clientTimeStamp) {
-                return table;
-            }
-            // Otherwise, query for an older version of the table - it won't be cached
-            return buildTable(key, cacheKey, region, clientTimeStamp);
-        } finally {
-            rowLock.release();
-        }
-    }
-
     private PTable buildTable(byte[] key, ImmutableBytesPtr cacheKey, HRegion region,
             long clientTimeStamp) throws IOException, SQLException {
         Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
@@ -1322,6 +1263,97 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
     
+    private PTable incrementTableTimestamp(byte[] key, long clientTimeStamp) throws IOException, SQLException {
+        HRegion region = env.getRegion();
+        RowLock lid = region.getRowLock(key);
+        if (lid == null) {
+            throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
+        }
+        try {
+            PTable table = doGetTable(key, clientTimeStamp, lid);
+            if (table != null) {
+                long tableTimeStamp = table.getTimeStamp() + 1;
+                List<Mutation> mutations = Lists.newArrayListWithExpectedSize(1);
+                Put p = new Put(key);
+                p.add(TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, tableTimeStamp, ByteUtil.EMPTY_BYTE_ARRAY);
+                mutations.add(p);
+                region.mutateRowsWithLocks(mutations, Collections.<byte[]> emptySet());
+                
+                Cache<ImmutableBytesPtr, PTable> metaDataCache = GlobalCache.getInstance(env).getMetaDataCache();
+                ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
+                metaDataCache.invalidate(cacheKey);
+            }
+            return table;
+        } finally {
+            lid.release();
+        }
+    }
+    
+    private PTable doGetTable(byte[] key, long clientTimeStamp) throws IOException, SQLException {
+        return doGetTable(key, clientTimeStamp, null);
+    }
+    
+    private PTable doGetTable(byte[] key, long clientTimeStamp, RowLock rowLock) throws IOException, SQLException {
+        ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
+        Cache<ImmutableBytesPtr, PTable> metaDataCache =
+                GlobalCache.getInstance(this.env).getMetaDataCache();
+        PTable table = metaDataCache.getIfPresent(cacheKey);
+        // We only cache the latest, so we'll end up building the table with every call if the
+        // client connection has specified an SCN.
+        // TODO: If we indicate to the client that we're returning an older version, but there's a
+        // newer version available, the client
+        // can safely not call this, since we only allow modifications to the latest.
+        if (table != null && table.getTimeStamp() < clientTimeStamp) {
+            // Table on client is up-to-date with table on server, so just return
+            if (isTableDeleted(table)) {
+                return null;
+            }
+            return table;
+        }
+        // Ask Lars about the expense of this call - if we don't take the lock, we still won't get
+        // partial results
+        // get the co-processor environment
+        // TODO: check that key is within region.getStartKey() and region.getEndKey()
+        // and return special code to force client to lookup region from meta.
+        HRegion region = env.getRegion();
+        /*
+         * Lock directly on key, though it may be an index table. This will just prevent a table
+         * from getting rebuilt too often.
+         */
+        final boolean wasLocked = (rowLock != null);
+        if (!wasLocked) {
+            rowLock = region.getRowLock(key);
+            if (rowLock == null) {
+                throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
+            }
+        }
+        try {
+            // Try cache again in case we were waiting on a lock
+            table = metaDataCache.getIfPresent(cacheKey);
+            // We only cache the latest, so we'll end up building the table with every call if the
+            // client connection has specified an SCN.
+            // TODO: If we indicate to the client that we're returning an older version, but there's
+            // a newer version available, the client
+            // can safely not call this, since we only allow modifications to the latest.
+            if (table != null && table.getTimeStamp() < clientTimeStamp) {
+                // Table on client is up-to-date with table on server, so just return
+                if (isTableDeleted(table)) {
+                    return null;
+                }
+                return table;
+            }
+            // Query for the latest table first, since it's not cached
+            table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP);
+            if (table != null && table.getTimeStamp() < clientTimeStamp) {
+                return table;
+            }
+            // Otherwise, query for an older version of the table - it won't be cached
+            return buildTable(key, cacheKey, region, clientTimeStamp);
+        } finally {
+            if (!wasLocked) rowLock.release();
+        }
+    }
+
     @Override
     public void dropColumn(RpcController controller, DropColumnRequest request,
             RpcCallback<MetaDataResponse> done) {
@@ -1672,36 +1704,19 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
     
     @Override
-    public void clearCacheForTable(RpcController controller, ClearCacheForTableRequest request,
-            RpcCallback<ClearCacheForTableResponse> done) {
-        ByteString tenantId = request.getTenantId();
-        ByteString schemaName = request.getSchemaName();
-        ByteString tableName = request.getTableName();
-        byte[] tableKey = SchemaUtil.getTableKey(tenantId.toByteArray(), schemaName.toByteArray(),
-                tableName.toByteArray());
-        ImmutableBytesPtr key = new ImmutableBytesPtr(tableKey);
+    public void incrementTableTimeStamp(RpcController controller, IncrementTableTimeStampRequest request,
+            RpcCallback<IncrementTableTimeStampResponse> done) {
+        byte[] schemaName = request.getSchemaName().toByteArray();
+        byte[] tableName = request.getTableName().toByteArray();
         try {
-            PTable table = doGetTable(tableKey, request.getClientTimestamp());
-            if (table != null) {
-                Cache<ImmutableBytesPtr, PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-                // Add +1 to the ts
-                // TODO : refactor doGetTable() to do this - optionally increment the timestamp
-                // TODO : clear metadata if it is spread across multiple region servers
-                long ts = table.getTimeStamp() + 1;
-                // Here we could add an empty puti
-                HRegion region = env.getRegion();
-                List<Mutation> mutations = new ArrayList<Mutation>();
-                Put p = new Put(tableKey);
-                p.add(TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ts, ByteUtil.EMPTY_BYTE_ARRAY);
-                mutations.add(p);
-                region.mutateRowsWithLocks(mutations, Collections.<byte[]> emptySet());
-                metaDataCache.invalidate(key);
-            }
+            byte[] tenantId = request.getTenantId().toByteArray();
+            long clientTimeStamp = request.getClientTimestamp();
+            byte[] tableKey = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+            incrementTableTimestamp(tableKey, clientTimeStamp);
         } catch (Throwable t) {
-            // We could still invalidate it
-            logger.error("clearCacheForTable failed to update the latest ts ", t);
-            ProtobufUtil.setControllerException(controller, ServerUtil.createIOException(
-                    SchemaUtil.getTableName(schemaName.toString(), tableName.toString()), t));
+            logger.error("incrementTableTimeStamp failed", t);
+            ProtobufUtil.setControllerException(controller,
+                ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/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 5ac9cb7..8d153b2 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
@@ -6316,7 +6316,7 @@ public final class MetaDataProtos {
     // @@protoc_insertion_point(class_scope:GetVersionResponse)
   }
 
-  public interface ClearCacheForTableRequestOrBuilder
+  public interface IncrementTableTimeStampRequestOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
     // required bytes tenantId = 1;
@@ -6360,24 +6360,24 @@ public final class MetaDataProtos {
     long getClientTimestamp();
   }
   /**
-   * Protobuf type {@code ClearCacheForTableRequest}
+   * Protobuf type {@code IncrementTableTimeStampRequest}
    */
-  public static final class ClearCacheForTableRequest extends
+  public static final class IncrementTableTimeStampRequest extends
       com.google.protobuf.GeneratedMessage
-      implements ClearCacheForTableRequestOrBuilder {
-    // Use ClearCacheForTableRequest.newBuilder() to construct.
-    private ClearCacheForTableRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements IncrementTableTimeStampRequestOrBuilder {
+    // Use IncrementTableTimeStampRequest.newBuilder() to construct.
+    private IncrementTableTimeStampRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private ClearCacheForTableRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private IncrementTableTimeStampRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final ClearCacheForTableRequest defaultInstance;
-    public static ClearCacheForTableRequest getDefaultInstance() {
+    private static final IncrementTableTimeStampRequest defaultInstance;
+    public static IncrementTableTimeStampRequest getDefaultInstance() {
       return defaultInstance;
     }
 
-    public ClearCacheForTableRequest getDefaultInstanceForType() {
+    public IncrementTableTimeStampRequest getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -6387,7 +6387,7 @@ public final class MetaDataProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private ClearCacheForTableRequest(
+    private IncrementTableTimeStampRequest(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -6444,28 +6444,28 @@ public final class MetaDataProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_descriptor;
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampRequest_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_fieldAccessorTable
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.Builder.class);
+              org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<ClearCacheForTableRequest> PARSER =
-        new com.google.protobuf.AbstractParser<ClearCacheForTableRequest>() {
-      public ClearCacheForTableRequest parsePartialFrom(
+    public static com.google.protobuf.Parser<IncrementTableTimeStampRequest> PARSER =
+        new com.google.protobuf.AbstractParser<IncrementTableTimeStampRequest>() {
+      public IncrementTableTimeStampRequest parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new ClearCacheForTableRequest(input, extensionRegistry);
+        return new IncrementTableTimeStampRequest(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<ClearCacheForTableRequest> getParserForType() {
+    public com.google.protobuf.Parser<IncrementTableTimeStampRequest> getParserForType() {
       return PARSER;
     }
 
@@ -6622,10 +6622,10 @@ public final class MetaDataProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest)) {
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest)) {
         return super.equals(obj);
       }
-      org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest) obj;
+      org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest) obj;
 
       boolean result = true;
       result = result && (hasTenantId() == other.hasTenantId());
@@ -6682,53 +6682,53 @@ public final class MetaDataProtos {
       return hash;
     }
 
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(byte[] data)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(java.io.InputStream input)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseDelimitedFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -6737,7 +6737,7 @@ public final class MetaDataProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest prototype) {
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -6749,24 +6749,24 @@ public final class MetaDataProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code ClearCacheForTableRequest}
+     * Protobuf type {@code IncrementTableTimeStampRequest}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequestOrBuilder {
+       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequestOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_descriptor;
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampRequest_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_fieldAccessorTable
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.Builder.class);
+                org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.Builder.class);
       }
 
-      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.newBuilder()
+      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -6803,23 +6803,23 @@ public final class MetaDataProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_descriptor;
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampRequest_descriptor;
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest getDefaultInstanceForType() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.getDefaultInstance();
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.getDefaultInstance();
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest build() {
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest result = buildPartial();
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest build() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest buildPartial() {
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest(this);
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest buildPartial() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -6844,16 +6844,16 @@ public final class MetaDataProtos {
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest) {
-          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest)other);
+        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest other) {
-        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.getDefaultInstance()) return this;
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest other) {
+        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.getDefaultInstance()) return this;
         if (other.hasTenantId()) {
           setTenantId(other.getTenantId());
         }
@@ -6894,11 +6894,11 @@ public final class MetaDataProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parsedMessage = null;
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -7050,39 +7050,39 @@ public final class MetaDataProtos {
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:ClearCacheForTableRequest)
+      // @@protoc_insertion_point(builder_scope:IncrementTableTimeStampRequest)
     }
 
     static {
-      defaultInstance = new ClearCacheForTableRequest(true);
+      defaultInstance = new IncrementTableTimeStampRequest(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:ClearCacheForTableRequest)
+    // @@protoc_insertion_point(class_scope:IncrementTableTimeStampRequest)
   }
 
-  public interface ClearCacheForTableResponseOrBuilder
+  public interface IncrementTableTimeStampResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
   }
   /**
-   * Protobuf type {@code ClearCacheForTableResponse}
+   * Protobuf type {@code IncrementTableTimeStampResponse}
    */
-  public static final class ClearCacheForTableResponse extends
+  public static final class IncrementTableTimeStampResponse extends
       com.google.protobuf.GeneratedMessage
-      implements ClearCacheForTableResponseOrBuilder {
-    // Use ClearCacheForTableResponse.newBuilder() to construct.
-    private ClearCacheForTableResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements IncrementTableTimeStampResponseOrBuilder {
+    // Use IncrementTableTimeStampResponse.newBuilder() to construct.
+    private IncrementTableTimeStampResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private ClearCacheForTableResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private IncrementTableTimeStampResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final ClearCacheForTableResponse defaultInstance;
-    public static ClearCacheForTableResponse getDefaultInstance() {
+    private static final IncrementTableTimeStampResponse defaultInstance;
+    public static IncrementTableTimeStampResponse getDefaultInstance() {
       return defaultInstance;
     }
 
-    public ClearCacheForTableResponse getDefaultInstanceForType() {
+    public IncrementTableTimeStampResponse getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -7092,7 +7092,7 @@ public final class MetaDataProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private ClearCacheForTableResponse(
+    private IncrementTableTimeStampResponse(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -7128,28 +7128,28 @@ public final class MetaDataProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_descriptor;
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampResponse_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_fieldAccessorTable
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.Builder.class);
+              org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<ClearCacheForTableResponse> PARSER =
-        new com.google.protobuf.AbstractParser<ClearCacheForTableResponse>() {
-      public ClearCacheForTableResponse parsePartialFrom(
+    public static com.google.protobuf.Parser<IncrementTableTimeStampResponse> PARSER =
+        new com.google.protobuf.AbstractParser<IncrementTableTimeStampResponse>() {
+      public IncrementTableTimeStampResponse parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new ClearCacheForTableResponse(input, extensionRegistry);
+        return new IncrementTableTimeStampResponse(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<ClearCacheForTableResponse> getParserForType() {
+    public com.google.protobuf.Parser<IncrementTableTimeStampResponse> getParserForType() {
       return PARSER;
     }
 
@@ -7193,10 +7193,10 @@ public final class MetaDataProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse)) {
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse)) {
         return super.equals(obj);
       }
-      org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse) obj;
+      org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse) obj;
 
       boolean result = true;
       result = result &&
@@ -7217,53 +7217,53 @@ public final class MetaDataProtos {
       return hash;
     }
 
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(byte[] data)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(java.io.InputStream input)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseDelimitedFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -7272,7 +7272,7 @@ public final class MetaDataProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse prototype) {
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -7284,24 +7284,24 @@ public final class MetaDataProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code ClearCacheForTableResponse}
+     * Protobuf type {@code IncrementTableTimeStampResponse}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponseOrBuilder {
+       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponseOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_descriptor;
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampResponse_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_fieldAccessorTable
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.Builder.class);
+                org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.Builder.class);
       }
 
-      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.newBuilder()
+      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -7330,38 +7330,38 @@ public final class MetaDataProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_descriptor;
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_IncrementTableTimeStampResponse_descriptor;
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse getDefaultInstanceForType() {
-        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance();
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.getDefaultInstance();
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse build() {
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse result = buildPartial();
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse build() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse buildPartial() {
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse(this);
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse buildPartial() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse(this);
         onBuilt();
         return result;
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse) {
-          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse)other);
+        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse other) {
-        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance()) return this;
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse other) {
+        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.getDefaultInstance()) return this;
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -7374,11 +7374,11 @@ public final class MetaDataProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parsedMessage = null;
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -7388,15 +7388,15 @@ public final class MetaDataProtos {
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:ClearCacheForTableResponse)
+      // @@protoc_insertion_point(builder_scope:IncrementTableTimeStampResponse)
     }
 
     static {
-      defaultInstance = new ClearCacheForTableResponse(true);
+      defaultInstance = new IncrementTableTimeStampResponse(true);
       defaultInstance.initFields();
     }
 
-    // @@protoc_insertion_point(class_scope:ClearCacheForTableResponse)
+    // @@protoc_insertion_point(class_scope:IncrementTableTimeStampResponse)
   }
 
   /**
@@ -7472,12 +7472,12 @@ public final class MetaDataProtos {
           com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse> done);
 
       /**
-       * <code>rpc clearCacheForTable(.ClearCacheForTableRequest) returns (.ClearCacheForTableResponse);</code>
+       * <code>rpc incrementTableTimeStamp(.IncrementTableTimeStampRequest) returns (.IncrementTableTimeStampResponse);</code>
        */
-      public abstract void clearCacheForTable(
+      public abstract void incrementTableTimeStamp(
           com.google.protobuf.RpcController controller,
-          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request,
-          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse> done);
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest request,
+          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse> done);
 
     }
 
@@ -7549,11 +7549,11 @@ public final class MetaDataProtos {
         }
 
         @java.lang.Override
-        public  void clearCacheForTable(
+        public  void incrementTableTimeStamp(
             com.google.protobuf.RpcController controller,
-            org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request,
-            com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse> done) {
-          impl.clearCacheForTable(controller, request, done);
+            org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest request,
+            com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse> done) {
+          impl.incrementTableTimeStamp(controller, request, done);
         }
 
       };
@@ -7595,7 +7595,7 @@ public final class MetaDataProtos {
             case 7:
               return impl.getVersion(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest)request);
             case 8:
-              return impl.clearCacheForTable(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest)request);
+              return impl.incrementTableTimeStamp(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -7627,7 +7627,7 @@ public final class MetaDataProtos {
             case 7:
               return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest.getDefaultInstance();
             case 8:
-              return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.getDefaultInstance();
+              return org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -7659,7 +7659,7 @@ public final class MetaDataProtos {
             case 7:
               return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse.getDefaultInstance();
             case 8:
-              return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance();
+              return org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -7733,12 +7733,12 @@ public final class MetaDataProtos {
         com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse> done);
 
     /**
-     * <code>rpc clearCacheForTable(.ClearCacheForTableRequest) returns (.ClearCacheForTableResponse);</code>
+     * <code>rpc incrementTableTimeStamp(.IncrementTableTimeStampRequest) returns (.IncrementTableTimeStampResponse);</code>
      */
-    public abstract void clearCacheForTable(
+    public abstract void incrementTableTimeStamp(
         com.google.protobuf.RpcController controller,
-        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request,
-        com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse> done);
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest request,
+        com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse> done);
 
     public static final
         com.google.protobuf.Descriptors.ServiceDescriptor
@@ -7803,8 +7803,8 @@ public final class MetaDataProtos {
               done));
           return;
         case 8:
-          this.clearCacheForTable(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest)request,
-            com.google.protobuf.RpcUtil.<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse>specializeCallback(
+          this.incrementTableTimeStamp(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse>specializeCallback(
               done));
           return;
         default:
@@ -7838,7 +7838,7 @@ public final class MetaDataProtos {
         case 7:
           return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest.getDefaultInstance();
         case 8:
-          return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.getDefaultInstance();
+          return org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -7870,7 +7870,7 @@ public final class MetaDataProtos {
         case 7:
           return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse.getDefaultInstance();
         case 8:
-          return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance();
+          return org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -8012,19 +8012,19 @@ public final class MetaDataProtos {
             org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse.getDefaultInstance()));
       }
 
-      public  void clearCacheForTable(
+      public  void incrementTableTimeStamp(
           com.google.protobuf.RpcController controller,
-          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request,
-          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse> done) {
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest request,
+          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse> done) {
         channel.callMethod(
           getDescriptor().getMethods().get(8),
           controller,
           request,
-          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance(),
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.getDefaultInstance(),
           com.google.protobuf.RpcUtil.generalizeCallback(
             done,
-            org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.class,
-            org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance()));
+            org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.class,
+            org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.getDefaultInstance()));
       }
     }
 
@@ -8074,9 +8074,9 @@ public final class MetaDataProtos {
           org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest request)
           throws com.google.protobuf.ServiceException;
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse clearCacheForTable(
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse incrementTableTimeStamp(
           com.google.protobuf.RpcController controller,
-          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request)
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest request)
           throws com.google.protobuf.ServiceException;
     }
 
@@ -8183,15 +8183,15 @@ public final class MetaDataProtos {
       }
 
 
-      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse clearCacheForTable(
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse incrementTableTimeStamp(
           com.google.protobuf.RpcController controller,
-          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request)
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest request)
           throws com.google.protobuf.ServiceException {
-        return (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse) channel.callBlockingMethod(
+        return (org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse) channel.callBlockingMethod(
           getDescriptor().getMethods().get(8),
           controller,
           request,
-          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance());
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse.getDefaultInstance());
       }
 
     }
@@ -8255,15 +8255,15 @@ public final class MetaDataProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_GetVersionResponse_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ClearCacheForTableRequest_descriptor;
+    internal_static_IncrementTableTimeStampRequest_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ClearCacheForTableRequest_fieldAccessorTable;
+      internal_static_IncrementTableTimeStampRequest_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ClearCacheForTableResponse_descriptor;
+    internal_static_IncrementTableTimeStampResponse_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ClearCacheForTableResponse_fieldAccessorTable;
+      internal_static_IncrementTableTimeStampResponse_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -8291,32 +8291,33 @@ public final class MetaDataProtos {
       "st\022\036\n\026tableMetadataMutations\030\001 \003(\014\"\023\n\021Cl" +
       "earCacheRequest\"\024\n\022ClearCacheResponse\"\023\n" +
       "\021GetVersionRequest\"%\n\022GetVersionResponse" +
-      "\022\017\n\007version\030\001 \002(\003\"m\n\031ClearCacheForTableR" +
-      "equest\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\017clientTimesta" +
-      "mp\030\004 \002(\003\"\034\n\032ClearCacheForTableResponse*\212" +
-      "\002\n\014MutationCode\022\030\n\024TABLE_ALREADY_EXISTS\020" +
-      "\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FOU" +
-      "ND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CONCU" +
-      "RRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_R" +
-      "EGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNALLO" +
-      "WED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010\022" +
-      "\032\n\026PARENT_TABLE_NOT_FOUND\020\t2\222\004\n\017MetaData" +
-      "Service\022/\n\010getTable\022\020.GetTableRequest\032\021.",
-      "MetaDataResponse\0225\n\013createTable\022\023.Create" +
-      "TableRequest\032\021.MetaDataResponse\0221\n\tdropT" +
-      "able\022\021.DropTableRequest\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\022M\n\022clearCacheForTable\022\032.ClearCacheForT" +
-      "ableRequest\032\033.ClearCacheForTableResponse" +
-      "BB\n(org.apache.phoenix.coprocessor.gener" +
-      "atedB\016MetaDataProtosH\001\210\001\001\240\001\001"
+      "\022\017\n\007version\030\001 \002(\003\"r\n\036IncrementTableTimeS" +
+      "tampRequest\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\027\n\017clientTi" +
+      "mestamp\030\004 \002(\003\"!\n\037IncrementTableTimeStamp" +
+      "Response*\212\002\n\014MutationCode\022\030\n\024TABLE_ALREA" +
+      "DY_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLU" +
+      "MN_NOT_FOUND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020" +
+      "\003\022\035\n\031CONCURRENT_TABLE_MUTATION\020\004\022\027\n\023TABL" +
+      "E_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_" +
+      "COLUMNS\020\010\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t2\241\004" +
+      "\n\017MetaDataService\022/\n\010getTable\022\020.GetTable",
+      "Request\032\021.MetaDataResponse\0225\n\013createTabl" +
+      "e\022\023.CreateTableRequest\032\021.MetaDataRespons" +
+      "e\0221\n\tdropTable\022\021.DropTableRequest\032\021.Meta" +
+      "DataResponse\0221\n\taddColumn\022\021.AddColumnReq" +
+      "uest\032\021.MetaDataResponse\0223\n\ndropColumn\022\022." +
+      "DropColumnRequest\032\021.MetaDataResponse\022?\n\020" +
+      "updateIndexState\022\030.UpdateIndexStateReque" +
+      "st\032\021.MetaDataResponse\0225\n\nclearCache\022\022.Cl" +
+      "earCacheRequest\032\023.ClearCacheResponse\0225\n\n" +
+      "getVersion\022\022.GetVersionRequest\032\023.GetVers",
+      "ionResponse\022\\\n\027incrementTableTimeStamp\022\037" +
+      ".IncrementTableTimeStampRequest\032 .Increm" +
+      "entTableTimeStampResponseBB\n(org.apache." +
+      "phoenix.coprocessor.generatedB\016MetaDataP" +
+      "rotosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8389,17 +8390,17 @@ public final class MetaDataProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetVersionResponse_descriptor,
               new java.lang.String[] { "Version", });
-          internal_static_ClearCacheForTableRequest_descriptor =
+          internal_static_IncrementTableTimeStampRequest_descriptor =
             getDescriptor().getMessageTypes().get(11);
-          internal_static_ClearCacheForTableRequest_fieldAccessorTable = new
+          internal_static_IncrementTableTimeStampRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_ClearCacheForTableRequest_descriptor,
+              internal_static_IncrementTableTimeStampRequest_descriptor,
               new java.lang.String[] { "TenantId", "SchemaName", "TableName", "ClientTimestamp", });
-          internal_static_ClearCacheForTableResponse_descriptor =
+          internal_static_IncrementTableTimeStampResponse_descriptor =
             getDescriptor().getMessageTypes().get(12);
-          internal_static_ClearCacheForTableResponse_fieldAccessorTable = new
+          internal_static_IncrementTableTimeStampResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_ClearCacheForTableResponse_descriptor,
+              internal_static_IncrementTableTimeStampResponse_descriptor,
               new java.lang.String[] { });
           return null;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/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 15c8ebe..e156555 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
@@ -103,5 +103,5 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public boolean supportsFeature(Feature feature);
     
     public String getUserName();
-    public void clearCacheForTable(final byte[] tenantId, final byte[] schemaName, final byte[] tableName, long clientTS) throws SQLException;
+    public void incrementTableTimeStamp(final byte[] tenantId, final byte[] schemaName, final byte[] tableName, long clientTS) throws SQLException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/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 fd60eb8..9f30236 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
@@ -76,8 +76,6 @@ import org.apache.phoenix.coprocessor.ServerCachingEndpointImpl;
 import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest;
@@ -86,6 +84,8 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.DropTableRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetTableRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.IncrementTableTimeStampResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
@@ -1844,7 +1844,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
     
     @Override
-    public void clearCacheForTable(final byte[] tenantId, final byte[] schemaName, final byte[] tableName,
+    public void incrementTableTimeStamp(final byte[] tenantId, final byte[] schemaName, final byte[] tableName,
             final long clientTS) throws SQLException {
         // clear the meta data cache for the table here
         try {
@@ -1852,17 +1852,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             HTableInterface htable = this.getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             try {
                 htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
-                        new Batch.Call<MetaDataService, ClearCacheForTableResponse>() {
+                        new Batch.Call<MetaDataService, IncrementTableTimeStampResponse>() {
                             @Override
-                            public ClearCacheForTableResponse call(MetaDataService instance) throws IOException {
+                            public IncrementTableTimeStampResponse call(MetaDataService instance) throws IOException {
                                 ServerRpcController controller = new ServerRpcController();
-                                BlockingRpcCallback<ClearCacheForTableResponse> rpcCallback = new BlockingRpcCallback<ClearCacheForTableResponse>();
-                                ClearCacheForTableRequest.Builder builder = ClearCacheForTableRequest.newBuilder();
+                                BlockingRpcCallback<IncrementTableTimeStampResponse> rpcCallback = new BlockingRpcCallback<IncrementTableTimeStampResponse>();
+                                IncrementTableTimeStampRequest.Builder builder = IncrementTableTimeStampRequest.newBuilder();
                                 builder.setTenantId(HBaseZeroCopyByteString.wrap(tenantId));
                                 builder.setTableName(HBaseZeroCopyByteString.wrap(tableName));
                                 builder.setSchemaName(HBaseZeroCopyByteString.wrap(schemaName));
                                 builder.setClientTimestamp(clientTS);
-                                instance.clearCacheForTable(controller, builder.build(), rpcCallback);
+                                instance.incrementTableTimeStamp(controller, builder.build(), rpcCallback);
                                 if (controller.getFailedOn() != null) { throw controller.getFailedOn(); }
                                 return rpcCallback.get();
                             }
@@ -2055,5 +2055,4 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void throwConnectionClosedException() {
         throw new IllegalStateException("Connection to the cluster is closed");
     }
-    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/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 055bc79..37e46b7 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
@@ -190,7 +190,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
     
     @Override
-    public void clearCacheForTable(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
+    public void incrementTableTimeStamp(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
             throws SQLException {}
     // TODO: share this with ConnectionQueryServicesImpl
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/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 8bd2c61..86523fd 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
@@ -228,8 +228,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
     
     @Override
-    public void clearCacheForTable(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
+    public void incrementTableTimeStamp(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
             throws SQLException {
-        getDelegate().clearCacheForTable(tenantId, schemaName, tableName, clientTS);
+        getDelegate().incrementTableTimeStamp(tenantId, schemaName, tableName, clientTS);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/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 c4bfebb..18ea16d 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
@@ -489,20 +489,17 @@ public class MetaDataClient {
         Long scn = connection.getSCN();
         // Always invalidate the cache
         long clientTS = connection.getSCN() == null ? HConstants.LATEST_TIMESTAMP : scn;
-        connection.getQueryServices().clearCacheForTable(tenantIdBytes,
-                Bytes.toBytes(SchemaUtil.getSchemaNameFromFullName(physicalName.getString())),
-                Bytes.toBytes(SchemaUtil.getTableNameFromFullName(physicalName.getString())), clientTS);
-        String query = "SELECT CURRENT_DATE(),"+ LAST_STATS_UPDATE_TIME + " FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME
+        String query = "SELECT CURRENT_DATE() - " + LAST_STATS_UPDATE_TIME + " FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME
                 + " WHERE " + PHYSICAL_NAME + "='" + physicalName.getString() + "' AND " + COLUMN_FAMILY
-                + " IS NULL AND " + REGION_NAME + " IS NULL";
+                + " IS NULL AND " + REGION_NAME + " IS NULL AND " + LAST_STATS_UPDATE_TIME + " IS NOT NULL";
         ResultSet rs = connection.createStatement().executeQuery(query);
         long msSinceLastUpdate = Long.MAX_VALUE;
-        if (rs.next() && rs.getDate(2) != null) {
-            msSinceLastUpdate = rs.getDate(1).getTime() - rs.getDate(2).getTime();
+        if (rs.next()) {
+            msSinceLastUpdate = rs.getLong(1);
         }
         if (msSinceLastUpdate >= msMinBetweenUpdates) {
             // Here create the select query.
-            String countQuery = "SELECT /*+ NO_CACHE */ count(*) FROM " + table.getName().getString();
+            String countQuery = "SELECT /*+ NO_CACHE NO_INDEX */ count(*) FROM " + table.getName().getString();
             PhoenixStatement statement = (PhoenixStatement) connection.createStatement();
             QueryPlan plan = statement.compileQuery(countQuery);
             Scan scan = plan.getContext().getScan();
@@ -517,8 +514,9 @@ public class MetaDataClient {
             tempPtr.set(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
             // A single Cell will be returned with the count(*) - we decode that here
             long rowCount = PDataType.LONG.getCodec().decodeLong(tempPtr, SortOrder.getDefault());
-            // We need to update the stats table
-            connection.getQueryServices().clearCacheForTable(tenantIdBytes,
+            // We need to update the stats table so that client will pull the new one with
+            // the updated stats.
+            connection.getQueryServices().incrementTableTimeStamp(tenantIdBytes,
                     Bytes.toBytes(SchemaUtil.getSchemaNameFromFullName(physicalName.getString())),
                     Bytes.toBytes(SchemaUtil.getTableNameFromFullName(physicalName.getString())), clientTS);
             return new  MutationState(0, connection, rowCount);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450400f4/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index 9174b4d..accd5f2 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -92,14 +92,14 @@ message GetVersionResponse {
   required int64 version = 1;
 }
 
-message ClearCacheForTableRequest {
+message IncrementTableTimeStampRequest {
   required bytes tenantId = 1;
   required bytes schemaName  = 2;
   required bytes tableName = 3;
   required int64 clientTimestamp = 4;
 }
 
-message ClearCacheForTableResponse {
+message IncrementTableTimeStampResponse {
 }
 
 service MetaDataService {
@@ -127,6 +127,6 @@ service MetaDataService {
    rpc getVersion(GetVersionRequest)
     returns (GetVersionResponse);
    
-   rpc clearCacheForTable(ClearCacheForTableRequest)
-    returns (ClearCacheForTableResponse);
+   rpc incrementTableTimeStamp(IncrementTableTimeStampRequest)
+    returns (IncrementTableTimeStampResponse);
 }