You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2015/08/31 19:28:57 UTC

[2/2] phoenix git commit: PHOENIX-1812 Only sync table metadata when necessary

PHOENIX-1812 Only sync table metadata when necessary


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

Branch: refs/heads/txn
Commit: 92ee51a0d4f25ab6e2f353a7cfeea36b36df3ebd
Parents: 81e52e8
Author: Thomas D'Silva <tw...@gmail.com>
Authored: Tue Jun 2 15:39:44 2015 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Mon Aug 31 10:28:49 2015 -0700

----------------------------------------------------------------------
 phoenix-core/pom.xml                            |   8 +-
 .../apache/phoenix/end2end/AlterTableIT.java    |  10 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java |   4 +-
 .../phoenix/end2end/index/IndexMetadataIT.java  |   4 +-
 .../phoenix/end2end/index/LocalIndexIT.java     |   6 +-
 .../phoenix/end2end/index/SaltedIndexIT.java    |   2 +-
 .../end2end/index/TxImmutableIndexIT.java       |   1 +
 .../phoenix/transactions/TransactionIT.java     |  52 ++----
 .../transactions/TxPointInTimeQueryIT.java      |  33 ----
 .../phoenix/transactions/UpdateCacheIT.java     | 122 +++++++++++++
 .../transactions/UpdateCacheWithScnIT.java      |  41 +++++
 .../phoenix/compile/CreateTableCompiler.java    |   4 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   4 +-
 .../apache/phoenix/compile/FromCompiler.java    |  15 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |   2 +-
 .../phoenix/compile/ProjectionCompiler.java     |   6 +-
 .../apache/phoenix/compile/QueryCompiler.java   |   2 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   6 +-
 .../apache/phoenix/execute/MutationState.java   |  11 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  52 ++++--
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   8 +-
 .../query/ConnectionQueryServicesImpl.java      |  31 ++--
 .../query/ConnectionlessQueryServicesImpl.java  |  21 ++-
 .../query/DelegateConnectionQueryServices.java  |  17 +-
 .../apache/phoenix/query/MetaDataMutated.java   |   7 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 179 ++++++++++++-------
 .../org/apache/phoenix/schema/PMetaData.java    |   4 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    | 107 +++++------
 .../org/apache/phoenix/schema/PTableRef.java    |  65 +++++++
 .../org/apache/phoenix/schema/TableRef.java     |   2 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |   4 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |   7 +-
 .../apache/phoenix/util/TransactionUtil.java    |  40 ++++-
 .../phoenix/compile/QueryCompilerTest.java      |   4 +-
 .../phoenix/compile/ViewCompilerTest.java       |  29 ++-
 .../phoenix/filter/SkipScanBigFilterTest.java   |   5 +-
 .../phoenix/index/IndexMaintainerTest.java      |   4 +-
 .../query/BaseConnectionlessQueryTest.java      |   2 +-
 .../java/org/apache/phoenix/query/BaseTest.java |   4 +-
 .../query/ParallelIteratorsSplitTest.java       |   2 +-
 .../phoenix/schema/PMetaDataImplTest.java       |   4 +-
 .../apache/phoenix/schema/RowKeySchemaTest.java |   2 +-
 .../phoenix/schema/RowKeyValueAccessorTest.java |   2 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |  25 +++
 phoenix-flume/pom.xml                           |   7 +
 phoenix-pig/pom.xml                             |   7 +
 pom.xml                                         |   1 +
 48 files changed, 653 insertions(+), 324 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 775411c..c205e87 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -224,24 +224,24 @@
     <dependency>
       <groupId>co.cask.tephra</groupId>
       <artifactId>tephra-api</artifactId>
-      <version>0.6.2-SNAPSHOT</version>
+      <version>${tephra.version}</version>
     </dependency>
     <dependency>
       <groupId>co.cask.tephra</groupId>
       <artifactId>tephra-core</artifactId>
-      <version>0.6.2-SNAPSHOT</version>
+      <version>${tephra.version}</version>
     </dependency>
     <dependency>
       <groupId>co.cask.tephra</groupId>
       <artifactId>tephra-core</artifactId>
       <type>test-jar</type>
-      <version>0.6.2-SNAPSHOT</version>
+      <version>${tephra.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>co.cask.tephra</groupId>
       <artifactId>tephra-hbase-compat-0.98</artifactId>
-      <version>0.6.2-SNAPSHOT</version>
+      <version>${tephra.version}</version>
     </dependency>
   
     <!-- Make sure we have all the antlr dependencies -->

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index 46f51cf..3004bd6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -718,7 +718,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
 
     private void asssertIsWALDisabled(Connection conn, String fullTableName, boolean expectedValue) throws SQLException {
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        assertEquals(expectedValue, pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isWALDisabled());
+        assertEquals(expectedValue, pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isWALDisabled());
     }
 
     @Test
@@ -1317,7 +1317,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
 
     private static void assertImmutableRows(Connection conn, String fullTableName, boolean expectedValue) throws SQLException {
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        assertEquals(expectedValue, pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isImmutableRows());
+        assertEquals(expectedValue, pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isImmutableRows());
     }
 
     @Test
@@ -1977,7 +1977,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
             ddl = "ALTER TABLE TESTCHANGEPHOENIXPROPS SET MULTI_TENANT = true";
             conn.createStatement().execute(ddl);
             // check metadata cache is updated with MULTI_TENANT = true
-            PTable t = conn.unwrap(PhoenixConnection.class).getMetaDataCache().getTable(new PTableKey(null, "TESTCHANGEPHOENIXPROPS"));
+            PTable t = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "TESTCHANGEPHOENIXPROPS"));
             assertTrue(t.isMultiTenant());
             
             // check table metadata updated server side
@@ -2017,7 +2017,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         String ddl = "CREATE TABLE TEST_TRANSACTIONAL_TABLE (k varchar primary key) transactional=true";
         conn.createStatement().execute(ddl);
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        PTable table = pconn.getMetaDataCache().getTable(new PTableKey(null, "TEST_TRANSACTIONAL_TABLE"));
+        PTable table = pconn.getTable(new PTableKey(null, "TEST_TRANSACTIONAL_TABLE"));
         HTableInterface htable = pconn.getQueryServices().getTable(Bytes.toBytes("TEST_TRANSACTIONAL_TABLE"));
         assertTrue(table.isTransactional());
         assertTrue(htable.getTableDescriptor().getCoprocessors().contains(TransactionProcessor.class.getName()));
@@ -2036,7 +2036,7 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         // Should be ok, as HBase metadata should match existing metadata.
         ddl = "CREATE TABLE IF NOT EXISTS TEST_TRANSACTIONAL_TABLE (k varchar primary key)"; 
         conn.createStatement().execute(ddl);
-        table = pconn.getMetaDataCache().getTable(new PTableKey(null, "TEST_TRANSACTIONAL_TABLE"));
+        table = pconn.getTable(new PTableKey(null, "TEST_TRANSACTIONAL_TABLE"));
         htable = pconn.getQueryServices().getTable(Bytes.toBytes("TEST_TRANSACTIONAL_TABLE"));
         assertTrue(table.isTransactional());
         assertTrue(htable.getTableDescriptor().getCoprocessors().contains(TransactionProcessor.class.getName()));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index 9eb9a57..ddcc0e8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -122,7 +122,7 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
             conn.close();
         }
     }
-    
+   
     @Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         testIndexWithNullableFixedWithCols(false);
@@ -189,7 +189,7 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
     
     private void assertImmutableRows(Connection conn, String fullTableName, boolean expectedValue) throws SQLException {
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        assertEquals(expectedValue, pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isImmutableRows());
+        assertEquals(expectedValue, pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isImmutableRows());
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index d6ced3c..e72ad0c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -100,7 +100,7 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
         String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
         conn.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName).next(); // client side cache will update
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), fullTableName)).getIndexMaintainers(ptr, pconn);
+        pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).getIndexMaintainers(ptr, pconn);
         assertTrue(ptr.getLength() > 0);
     }
     
@@ -109,7 +109,7 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
         String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
         conn.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName).next(); // client side cache will update
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), fullTableName)).getIndexMaintainers(ptr, pconn);
+        pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).getIndexMaintainers(ptr, pconn);
         assertTrue(ptr.getLength() == 0);
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/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 a7b7655..37d98a9 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
@@ -114,7 +114,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         Connection conn1 = DriverManager.getConnection(getUrl());
         conn1.createStatement().execute("CREATE LOCAL INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1)");
         conn1.createStatement().executeQuery("SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME).next();
-        PTable localIndex = conn1.unwrap(PhoenixConnection.class).getMetaDataCache().getTable(new PTableKey(null,TestUtil.DEFAULT_INDEX_TABLE_NAME));
+        PTable localIndex = conn1.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,TestUtil.DEFAULT_INDEX_TABLE_NAME));
         assertEquals(IndexType.LOCAL, localIndex.getIndexType());
         assertNotNull(localIndex.getViewIndexId());
     }
@@ -130,7 +130,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         } catch (SQLException e) { }
         try {
             conn2.createStatement().executeQuery("SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME).next();
-            conn2.unwrap(PhoenixConnection.class).getMetaDataCache().getTable(new PTableKey(null,TestUtil.DEFAULT_INDEX_TABLE_NAME));
+            conn2.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,TestUtil.DEFAULT_INDEX_TABLE_NAME));
             fail("Local index should not be created.");
         } catch (TableNotFoundException e) { }
     }
@@ -146,7 +146,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         } catch (SQLException e) { }
         try {
             conn2.createStatement().executeQuery("SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME).next();
-            conn2.unwrap(PhoenixConnection.class).getMetaDataCache().getTable(new PTableKey(null,TestUtil.DEFAULT_INDEX_TABLE_NAME));
+            conn2.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,TestUtil.DEFAULT_INDEX_TABLE_NAME));
             fail("Local index should not be created.");
         } catch (TableNotFoundException e) { }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
index 4cc80a4..b5c8477 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
@@ -68,7 +68,7 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
             conn.createStatement().execute("ALTER TABLE " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " SET IMMUTABLE_ROWS=true");
             conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME).next();
             PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-            assertTrue(pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), TestUtil.DEFAULT_DATA_TABLE_FULL_NAME)).isImmutableRows());
+            assertTrue(pconn.getTable(new PTableKey(pconn.getTenantId(), TestUtil.DEFAULT_DATA_TABLE_FULL_NAME)).isImmutableRows());
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/TxImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/TxImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/TxImmutableIndexIT.java
index d37b3a6..e9d685f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/TxImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/TxImmutableIndexIT.java
@@ -117,4 +117,5 @@ public class TxImmutableIndexIT extends ImmutableIndexIT {
             conn.close();
         }
     }
+	
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java
index 49e9ecb..933e8e9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/transactions/TransactionIT.java
@@ -16,9 +16,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.math.BigDecimal;
 import java.sql.Connection;
-import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -55,29 +53,7 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-	
-	private void setRowKeyColumns(PreparedStatement stmt, int i) throws SQLException {
-        // insert row
-        stmt.setString(1, "varchar" + String.valueOf(i));
-        stmt.setString(2, "char" + String.valueOf(i));
-        stmt.setInt(3, i);
-        stmt.setLong(4, i);
-        stmt.setBigDecimal(5, new BigDecimal(i*0.5d));
-        Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * TestUtil.NUM_MILLIS_IN_DAY);
-        stmt.setDate(6, date);
-    }
-	
-	private void validateRowKeyColumns(ResultSet rs, int i) throws SQLException {
-		assertTrue(rs.next());
-		assertEquals(rs.getString(1), "varchar" + String.valueOf(i));
-		assertEquals(rs.getString(2), "char" + String.valueOf(i));
-		assertEquals(rs.getInt(3), i);
-		assertEquals(rs.getInt(4), i);
-		assertEquals(rs.getBigDecimal(5), new BigDecimal(i*0.5d));
-		Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * TestUtil.NUM_MILLIS_IN_DAY);
-		assertEquals(rs.getDate(6), date);
-	}
-	
+		
 	@Test
 	public void testReadOwnWrites() throws Exception {
 		String selectSql = "SELECT * FROM "+FULL_TABLE_NAME;
@@ -90,23 +66,23 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
 	        String upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
 	        PreparedStatement stmt = conn.prepareStatement(upsert);
 			// upsert two rows
-			setRowKeyColumns(stmt, 1);
+			TestUtil.setRowKeyColumns(stmt, 1);
 			stmt.execute();
-			setRowKeyColumns(stmt, 2);
+			TestUtil.setRowKeyColumns(stmt, 2);
 			stmt.execute();
 	        
 	        // verify rows can be read even though commit has not been called
 			rs = conn.createStatement().executeQuery(selectSql);
-			validateRowKeyColumns(rs, 1);
-	        validateRowKeyColumns(rs, 2);
+			TestUtil.validateRowKeyColumns(rs, 1);
+			TestUtil.validateRowKeyColumns(rs, 2);
 	        assertFalse(rs.next());
 	        
 	        conn.commit();
 	        
 	        // verify rows can be read after commit
 	        rs = conn.createStatement().executeQuery(selectSql);
-	        validateRowKeyColumns(rs, 1);
-	        validateRowKeyColumns(rs, 2);
+	        TestUtil.validateRowKeyColumns(rs, 1);
+	        TestUtil.validateRowKeyColumns(rs, 2);
 	        assertFalse(rs.next());
 		}
         finally {
@@ -127,11 +103,11 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             String upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn1.prepareStatement(upsert);
             // upsert two rows
-            setRowKeyColumns(stmt, 1);
+            TestUtil.setRowKeyColumns(stmt, 1);
             stmt.execute();
             conn1.commit();
             
-            setRowKeyColumns(stmt, 2);
+            TestUtil.setRowKeyColumns(stmt, 2);
             stmt.execute();
             
             // verify rows can be read even though commit has not been called
@@ -196,12 +172,12 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
 			// upsert row using conn1
 			String upsertSql = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
 			PreparedStatement stmt = conn1.prepareStatement(upsertSql);
-			setRowKeyColumns(stmt, 1);
+			TestUtil.setRowKeyColumns(stmt, 1);
 			stmt.setInt(7, 10);
 	        stmt.execute();
 	        // upsert row using conn2
  			stmt = conn2.prepareStatement(upsertSql);
- 			setRowKeyColumns(stmt, 1);
+ 			TestUtil.setRowKeyColumns(stmt, 1);
 			stmt.setInt(7, 11);
 	        stmt.execute();
  	        
@@ -229,18 +205,18 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
 			String selectSql = "SELECT * FROM "+FULL_TABLE_NAME;
 			conn1.setAutoCommit(false);
 			ResultSet rs = conn1.createStatement().executeQuery(selectSql);
-			boolean immutableRows = conn1.unwrap(PhoenixConnection.class).getMetaDataCache().getTable(new PTableKey(null, FULL_TABLE_NAME)).isImmutableRows();
+			boolean immutableRows = conn1.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, FULL_TABLE_NAME)).isImmutableRows();
 	     	assertFalse(rs.next());
 			// upsert row using conn1
 			String upsertSql = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
 			PreparedStatement stmt = conn1.prepareStatement(upsertSql);
-			setRowKeyColumns(stmt, 1);
+			TestUtil.setRowKeyColumns(stmt, 1);
 			stmt.setInt(7, 10);
 	        stmt.execute();
 	        // upsert row using conn2
 	        upsertSql = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, b.int_col2) VALUES(?, ?, ?, ?, ?, ?, ?)";
  			stmt = conn2.prepareStatement(upsertSql);
- 			setRowKeyColumns(stmt, 1);
+ 			TestUtil.setRowKeyColumns(stmt, 1);
 			stmt.setInt(7, 11);
  	        stmt.execute();
  	        

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/transactions/TxPointInTimeQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/transactions/TxPointInTimeQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/transactions/TxPointInTimeQueryIT.java
index 0f16696..e532c0b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/transactions/TxPointInTimeQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/transactions/TxPointInTimeQueryIT.java
@@ -70,37 +70,4 @@ public class TxPointInTimeQueryIT extends BaseClientManagedTimeIT {
 		}
 	}
 
-//	@Test
-//	public void testDropColumn() throws Exception {
-//		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-//		props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
-//		Connection conn = DriverManager.getConnection(getUrl(), props);
-//		try {
-//			conn.createStatement()
-//					.execute(
-//							"CREATE TABLE t (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR) TRANSACTIONAL=true");
-//
-//			props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
-//			conn = DriverManager.getConnection(getUrl(), props);
-//			// drop a column
-//			conn.createStatement().execute("ALTER TABLE t DROP COLUMN v1");
-//
-//			props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 11));
-//			conn = DriverManager.getConnection(getUrl(), props);
-//			String selectQuery = "SELECT v1 FROM t";
-//			try {
-//				conn.createStatement().executeQuery(selectQuery);
-//				fail();
-//			} catch (ColumnNotFoundException e) {
-//			}
-//
-//			props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
-//			conn = DriverManager.getConnection(getUrl(), props);
-//			ResultSet rs = conn.createStatement().executeQuery(selectQuery);
-//			assertFalse(rs.next());
-//		} finally {
-//			conn.close();
-//		}
-//	}
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/transactions/UpdateCacheIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/transactions/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/transactions/UpdateCacheIT.java
new file mode 100644
index 0000000..68f154b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/transactions/UpdateCacheIT.java
@@ -0,0 +1,122 @@
+package org.apache.phoenix.transactions;
+
+import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
+import static org.apache.phoenix.util.TestUtil.MUTABLE_INDEX_DATA_TABLE;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.TRANSACTIONAL_DATA_TABLE;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+/**
+ * Verifies the number of rpcs calls from {@link MetaDataClient} updateCache() 
+ * for transactional and non-transactional tables.
+ */
+public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
+
+    @Before
+    public void setUp() throws SQLException {
+        ensureTableCreated(getUrl(), TRANSACTIONAL_DATA_TABLE);
+        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE);
+    }
+
+	@Test
+	public void testUpdateCacheForTxnTable() throws Exception {
+		helpTestUpdateCache(true, null);
+	}
+	
+	@Test
+	public void testUpdateCacheForNonTxnTable() throws Exception {
+		helpTestUpdateCache(false, null);
+	}
+	
+	public static void helpTestUpdateCache(boolean isTransactional, Long scn) throws Exception {
+		String tableName = isTransactional ? TRANSACTIONAL_DATA_TABLE : MUTABLE_INDEX_DATA_TABLE;
+		String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + tableName;
+		String selectSql = "SELECT * FROM "+fullTableName;
+		// use a spyed ConnectionQueryServices so we can verify calls to getTable
+		ConnectionQueryServices connectionQueryServices = Mockito.spy(driver.getConnectionQueryServices(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)));
+		Properties props = new Properties();
+		props.putAll(PhoenixEmbeddedDriver.DEFFAULT_PROPS.asMap());
+		if (scn!=null) {
+			props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(scn));
+		}
+		Connection conn = connectionQueryServices.connect(getUrl(), props);
+		try {
+			conn.setAutoCommit(false);
+	        String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+	        PreparedStatement stmt = conn.prepareStatement(upsert);
+			// upsert three rows
+	        TestUtil.setRowKeyColumns(stmt, 1);
+			stmt.execute();
+			TestUtil.setRowKeyColumns(stmt, 2);
+			stmt.execute();
+			TestUtil.setRowKeyColumns(stmt, 3);
+			stmt.execute();
+			conn.commit();
+			// for non txn tables verify only one rpc to fetch table metadata, 
+			// for txn tables the table will already be present in the cache because MetaDataClient.createTableInternal starts a txn 
+//			if (!isTransactional) {
+				verify(connectionQueryServices).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(INDEX_DATA_SCHEMA)), eq(PVarchar.INSTANCE.toBytes(tableName)), anyLong(), anyLong());
+				reset(connectionQueryServices);
+//			}
+			
+			if (scn!=null) {
+				// advance scn so that we can see the data we just upserted
+				props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(scn+2));
+				conn = connectionQueryServices.connect(getUrl(), props);
+			}
+			
+			ResultSet rs = conn.createStatement().executeQuery(selectSql);
+			TestUtil.validateRowKeyColumns(rs, 1);
+			TestUtil.validateRowKeyColumns(rs, 2);
+			TestUtil.validateRowKeyColumns(rs, 3);
+	        assertFalse(rs.next());
+	        
+	        rs = conn.createStatement().executeQuery(selectSql);
+	        TestUtil.validateRowKeyColumns(rs, 1);
+	        TestUtil.validateRowKeyColumns(rs, 2);
+	        TestUtil.validateRowKeyColumns(rs, 3);
+	        assertFalse(rs.next());
+	        
+	        rs = conn.createStatement().executeQuery(selectSql);
+	        TestUtil.validateRowKeyColumns(rs, 1);
+	        TestUtil.validateRowKeyColumns(rs, 2);
+	        TestUtil.validateRowKeyColumns(rs, 3);
+	        assertFalse(rs.next());
+	        
+	        // for non-transactional tables without a scn : verify one rpc to getTable occurs *per* query
+	        // for non-transactional tables with a scn : verify *only* one rpc occurs
+			// for transactional tables : verify *only* one rpc occurs
+	        int numRpcs = isTransactional || scn!=null ? 1 : 3; 
+	        verify(connectionQueryServices, times(numRpcs)).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(INDEX_DATA_SCHEMA)), eq(PVarchar.INSTANCE.toBytes(tableName)), anyLong(), anyLong());
+		}
+        finally {
+        	conn.close();
+        }
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/it/java/org/apache/phoenix/transactions/UpdateCacheWithScnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/transactions/UpdateCacheWithScnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/transactions/UpdateCacheWithScnIT.java
new file mode 100644
index 0000000..6300cff
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/transactions/UpdateCacheWithScnIT.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.transactions;
+
+import static org.apache.phoenix.util.TestUtil.MUTABLE_INDEX_DATA_TABLE;
+
+import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
+import org.junit.Before;
+import org.junit.Test;
+
+public class UpdateCacheWithScnIT extends BaseClientManagedTimeIT {
+	
+	protected long ts;
+
+	@Before
+	public void initTable() throws Exception {
+		ts = nextTimestamp();
+		ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE, ts);
+	}
+	
+	@Test
+	public void testUpdateCacheWithScn() throws Exception {
+		UpdateCacheIT.helpTestUpdateCache(false, ts+2);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index edee788..95a0b40 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -124,8 +124,8 @@ public class CreateTableCompiler {
                             // on our connection.
                             new DelegateConnectionQueryServices(connection.getQueryServices()) {
                                 @Override
-                                public PMetaData addTable(PTable table) throws SQLException {
-                                    return connection.addTable(table);
+                                public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
+                                    return connection.addTable(table, resolvedTime);
                                 }
                             },
                             connection, tableRef.getTimeStamp());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 408c622..328e05b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -349,7 +349,7 @@ public class DeleteCompiler {
                     // optimizer updated the cache if it found indexes that were out of date.
                     // If the index was marked as disabled, it should not be in the list
                     // of immutable indexes.
-                    table = connection.getMetaDataCache().getTable(new PTableKey(table.getTenantId(), table.getName().getString()));
+                    table = connection.getTable(new PTableKey(table.getTenantId(), table.getName().getString()));
                     tableRefToBe.setTable(table);
                     immutableIndex = getNonDisabledImmutableIndexes(tableRefToBe);
                 }
@@ -564,7 +564,7 @@ public class DeleteCompiler {
                         // Repeated from PhoenixStatement.executeQuery which this call bypasses.
                         // Send mutations to hbase, so they are visible to subsequent reads.
                         // Use original plan for data table so that data and immutable indexes will be sent.
-                        boolean isTransactional = connection.getMutationState().startTransaction(resolver.getTables().iterator());
+                        boolean isTransactional = connection.getMutationState().sendMutations(resolver.getTables().iterator());
                         if (isTransactional) {
                             // Use real query plan  so that we have the right context object.
                             plan.getContext().setTransaction(connection.getMutationState().getTransaction());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index e1f64f3..db98e25 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -77,6 +77,7 @@ import org.apache.phoenix.util.Closeables;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.LogUtil;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TransactionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -196,6 +197,12 @@ public class FromCompiler {
         PTable t = PTableImpl.makePTable(table, projectedColumns);
         return new SingleTableColumnResolver(connection, new TableRef(tableRef.getTableAlias(), t, tableRef.getLowerBoundTimeStamp(), tableRef.hasDynamicCols()));
     }
+    
+    public static ColumnResolver getResolver(PhoenixConnection connection, TableRef tableRef)
+            throws SQLException {
+        SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, tableRef);
+        return visitor;
+    }
 
     public static ColumnResolver getResolver(TableRef tableRef)
             throws SQLException {
@@ -340,18 +347,18 @@ public class FromCompiler {
             PTable theTable = null;
             if (updateCacheImmediately || connection.getAutoCommit()) {
                 MetaDataMutationResult result = client.updateCache(schemaName, tableName);
-                timeStamp = result.getMutationTime();
+                timeStamp = TransactionUtil.getTableTimestamp(connection, result);
                 theTable = result.getTable();
                 if (theTable == null) {
                     throw new TableNotFoundException(schemaName, tableName, timeStamp);
                 }
             } else {
                 try {
-                    theTable = connection.getMetaDataCache().getTable(new PTableKey(tenantId, fullTableName));
+                    theTable = connection.getTable(new PTableKey(tenantId, fullTableName));
                 } catch (TableNotFoundException e1) {
                     if (tenantId != null) { // Check with null tenantId next
                         try {
-                            theTable = connection.getMetaDataCache().getTable(new PTableKey(null, fullTableName));
+                            theTable = connection.getTable(new PTableKey(null, fullTableName));
                         } catch (TableNotFoundException e2) {
                         }
                     }
@@ -360,7 +367,7 @@ public class FromCompiler {
                 if (theTable == null) {
                     MetaDataMutationResult result = client.updateCache(schemaName, tableName);
                     if (result.wasUpdated()) {
-                        timeStamp = result.getMutationTime();
+                    	timeStamp = TransactionUtil.getTableTimestamp(connection, result);
                         theTable = result.getTable();
                     }
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
index 72a59c7..2933676 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
@@ -167,7 +167,7 @@ public class PostDDLCompiler {
                         // Not sure what the fix should be. We don't need conflict detection nor filtering of invalid transactions
                         // in this case, so maybe this is ok.
                         if (tableRef.getTable().isTransactional()) {
-                            ts = TransactionUtil.translateMillis(ts);
+                            ts = TransactionUtil.convertToNanoseconds(ts);
                         }
                         ScanUtil.setTimeRange(scan, ts);
                         if (emptyCF != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
index e84ca2a..c056337 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
@@ -183,11 +183,11 @@ public class ProjectionCompiler {
         String tableName = index.getParentName().getString();
         PTable dataTable = null;
         try {
-        	dataTable = conn.getMetaDataCache().getTable(new PTableKey(tenantId, tableName));
+        	dataTable = conn.getTable(new PTableKey(tenantId, tableName));
         } catch (TableNotFoundException e) {
             if (tenantId != null) { 
             	// Check with null tenantId 
-            	dataTable = conn.getMetaDataCache().getTable(new PTableKey(null, tableName));
+            	dataTable = conn.getTable(new PTableKey(null, tableName));
             }
             else {
             	throw e;
@@ -278,7 +278,7 @@ public class ProjectionCompiler {
         PTable index = tableRef.getTable();
         PhoenixConnection conn = context.getConnection();
         String tableName = index.getParentName().getString();
-        PTable table = conn.getMetaDataCache().getTable(new PTableKey(conn.getTenantId(), tableName));
+        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), tableName));
         PColumnFamily pfamily = table.getColumnFamily(cfName);
         for (PColumn column : pfamily.getColumns()) {
             String indexColName = IndexUtil.getIndexColumnName(column);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index 2276f4e..33ca4f4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -490,7 +490,7 @@ public class QueryCompiler {
         // Don't pass groupBy when building where clause expression, because we do not want to wrap these
         // expressions as group by key expressions since they're pre, not post filtered.
         if (innerPlan == null && !tableRef.equals(resolver.getTables().get(0))) {
-            context.setResolver(FromCompiler.getResolverForQuery(select, this.statement.getConnection()));
+        	context.setResolver(FromCompiler.getResolver(context.getConnection(), tableRef));
         }
         Set<SubqueryParseNode> subqueries = Sets.<SubqueryParseNode> newHashSet();
         Expression where = WhereCompiler.compile(context, select, viewWhere, subqueries);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 8e38ffc..b344c8e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -678,7 +678,7 @@ public class UpsertCompiler {
                     // Repeated from PhoenixStatement.executeQuery which this call bypasses.
                     // Send mutations to hbase, so they are visible to subsequent reads.
                     // Use original plan for data table so that data and immutable indexes will be sent.
-                    boolean isTransactional = connection.getMutationState().startTransaction(originalQueryPlan.getContext().getResolver().getTables().iterator());
+                    boolean isTransactional = connection.getMutationState().sendMutations(originalQueryPlan.getContext().getResolver().getTables().iterator());
                     if (isTransactional) {
                         // Use real query plan  so that we have the right context object.
                         queryPlan.getContext().setTransaction(connection.getMutationState().getTransaction());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/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 1a412c6..5d24f7e 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
@@ -1411,10 +1411,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     if (columnToDelete.isViewReferenced()) { // Disallow deletion of column referenced in WHERE clause of view
                                         return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), table, columnToDelete);
                                     }
-                                    // Look for columnToDelete in any indexes. If found as PK
-                                    // column, get lock and drop the index. If found as covered
-                                    // column, delete from index (do this client side?).
-                                    // In either case, invalidate index if the column is in it
+                                    // Look for columnToDelete in any indexes. If found as PK column, get lock and drop the index and then invalidate it
+                                    // Covered columns are deleted from the index by the client
                                     PhoenixConnection connection = QueryUtil.getConnection(env.getConfiguration()).unwrap(PhoenixConnection.class);
                                     for (PTable index : table.getIndexes()) {
                                         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 9695f2d..e051c4f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -323,17 +323,18 @@ public class MutationState implements SQLCloseable {
      * Get the unsorted list of HBase mutations for the tables with uncommitted data.
      * @return list of HBase mutations for uncommitted data.
      */
-    public Iterator<Pair<byte[],List<Mutation>>> toMutations() {
-        return toMutations(false);
+    public Iterator<Pair<byte[],List<Mutation>>> toMutations(Long timestamp) {
+        return toMutations(false, timestamp);
     }
     
-    public Iterator<Pair<byte[],List<Mutation>>> toMutations(final boolean includeMutableIndexes) {
+    public Iterator<Pair<byte[],List<Mutation>>> toMutations(final boolean includeMutableIndexes, final Long tableTimestamp) {
         final Iterator<Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>>> iterator = this.mutations.entrySet().iterator();
         if (!iterator.hasNext()) {
             return Iterators.emptyIterator();
         }
         Long scn = connection.getSCN();
-        final long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
+        final long timestamp = (tableTimestamp!=null && tableTimestamp!=QueryConstants.UNSET_TIMESTAMP) ? tableTimestamp : (scn == null ? HConstants.LATEST_TIMESTAMP : scn);
+//        final long timestamp = (scn == null ? HConstants.LATEST_TIMESTAMP : scn);
         return new Iterator<Pair<byte[],List<Mutation>>>() {
             private Map.Entry<TableRef, Map<ImmutableBytesPtr,Map<PColumn,byte[]>>> current = iterator.next();
             private Iterator<Pair<byte[],List<Mutation>>> innerIterator = init();
@@ -748,7 +749,7 @@ public class MutationState implements SQLCloseable {
      * @return true if at least partially transactional and false otherwise.
      * @throws SQLException
      */
-    public boolean startTransaction(Iterator<TableRef> tableRefs) throws SQLException {
+    public boolean sendMutations(Iterator<TableRef> tableRefs) throws SQLException {
         Iterator<TableRef> filteredTableRefs = Iterators.filter(tableRefs, new Predicate<TableRef>(){
             @Override
             public boolean apply(TableRef tableRef) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index d513362..7742376 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -73,9 +73,13 @@ import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PMetaData.Pruner;
+import org.apache.phoenix.schema.PMetaDataImpl;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.PTableRef;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.types.PArrayDataType;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDate;
@@ -151,13 +155,13 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
     }
 
     public PhoenixConnection(PhoenixConnection connection) throws SQLException {
-        this(connection.getQueryServices(), connection.getURL(), connection.getClientInfo(), connection.getMetaDataCache(), connection.getMutationState().getTransaction());
+        this(connection.getQueryServices(), connection.getURL(), connection.getClientInfo(), connection.metaData, connection.getMutationState().getTransaction());
         this.isAutoCommit = connection.isAutoCommit;
         this.sampler = connection.sampler;
     }
     
     public PhoenixConnection(ConnectionQueryServices services, PhoenixConnection connection, long scn) throws SQLException {
-        this(services, connection.getURL(), newPropsWithSCN(scn,connection.getClientInfo()), connection.getMetaDataCache(), connection.getMutationState().getTransaction());
+        this(services, connection.getURL(), newPropsWithSCN(scn,connection.getClientInfo()), connection.metaData, connection.getMutationState().getTransaction());
         this.isAutoCommit = connection.isAutoCommit;
         this.sampler = connection.sampler;
     }
@@ -166,6 +170,10 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
         this(services, url, info, metaData, null);
     }
     
+    public PhoenixConnection(PhoenixConnection connection, ConnectionQueryServices services, Properties info) throws SQLException {
+        this(services, connection.url, info, connection.metaData, null);
+    }
+    
     public PhoenixConnection(ConnectionQueryServices services, String url, Properties info, PMetaData metaData, Transaction txn) throws SQLException {
         this.url = url;
         // Copy so client cannot change
@@ -364,8 +372,12 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
         return mutateBatchSize;
     }
     
-    public PMetaData getMetaDataCache() {
-        return metaData;
+    public PTable getTable(PTableKey key) throws TableNotFoundException {
+    	return metaData.getTableRef(key).getTable();
+    }
+    
+    public PTableRef getTableRef(PTableKey key) throws TableNotFoundException {
+    	return metaData.getTableRef(key);
     }
 
     public MutationState getMutationState() {
@@ -746,25 +758,29 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
         // TODO Auto-generated method stub
         return 0;
     }
-
+    
     @Override
-    public PMetaData addTable(PTable table) throws SQLException {
-        // TODO: since a connection is only used by one thread at a time,
-        // we could modify this metadata in place since it's not shared.
-        if (scn == null || scn > table.getTimeStamp()) {
-            metaData = metaData.addTable(table);
-        }
+    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
+        metaData = metaData.addTable(table, resolvedTime);
         //Cascade through to connectionQueryServices too
-        getQueryServices().addTable(table);
+        getQueryServices().addTable(table, resolvedTime);
+        return metaData;
+    }
+    
+    @Override
+    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
+    	metaData = metaData.updateResolvedTimestamp(table, resolvedTime);
+    	//Cascade through to connectionQueryServices too
+        getQueryServices().updateResolvedTimestamp(table, resolvedTime);
         return metaData;
     }
 
     @Override
-    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls)
+    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, long resolvedTime)
             throws SQLException {
-        metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls);
+        metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, resolvedTime);
         //Cascade through to connectionQueryServices too
-        getQueryServices().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls);
+        getQueryServices().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, resolvedTime);
         return metaData;
     }
 
@@ -778,10 +794,10 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
 
     @Override
     public PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
-            long tableSeqNum) throws SQLException {
-        metaData = metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum);
+            long tableSeqNum, long resolvedTime) throws SQLException {
+        metaData = metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
         //Cascade through to connectionQueryServices too
-        getQueryServices().removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum);
+        getQueryServices().removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
         return metaData;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index eca0f35..76015bd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -241,7 +241,7 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
                         // TODO: for joins, we need to iterate through all tables, but we need the original table,
                         // not the projected table, so plan.getContext().getResolver().getTables() won't work.
                         Iterator<TableRef> tableRefs = plan.getTableRefs().iterator();
-                        boolean isTransactional = connection.getMutationState().startTransaction(tableRefs);
+                        boolean isTransactional = connection.getMutationState().sendMutations(tableRefs);
                         plan = connection.getQueryServices().getOptimizer().optimize(PhoenixStatement.this, plan);
                         if (isTransactional) {
                             // After optimize so that we have the right context object
@@ -286,14 +286,14 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         }
     }
     
-    private boolean startTransaction(StatementPlan plan) throws SQLException {
+    private void startTransaction(StatementPlan plan) throws SQLException {
+    	if (connection.getMutationState().getTransaction()!=null)
+    		return;
         for (TableRef ref : plan.getContext().getResolver().getTables()) {
             if (ref.getTable().isTransactional()) {
                 connection.getMutationState().startTransaction();
-                return true;
             }
         }
-        return false;
     }
     
     protected int executeMutation(final CompilableStatement stmt) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/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 1655a57..13a0559 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
@@ -479,18 +479,27 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public PMetaData addTable(PTable table) throws SQLException {
+    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
             try {
                 throwConnectionClosedIfNullMetaData();
                 // If existing table isn't older than new table, don't replace
                 // If a client opens a connection at an earlier timestamp, this can happen
-                PTable existingTable = latestMetaData.getTable(new PTableKey(table.getTenantId(), table.getName().getString()));
+                PTable existingTable = latestMetaData.getTableRef(new PTableKey(table.getTenantId(), table.getName().getString())).getTable();
                 if (existingTable.getTimeStamp() >= table.getTimeStamp()) {
                     return latestMetaData;
                 }
             } catch (TableNotFoundException e) {}
-            latestMetaData = latestMetaData.addTable(table);
+            latestMetaData = latestMetaData.addTable(table, resolvedTime);
+            latestMetaDataLock.notifyAll();
+            return latestMetaData;
+        }
+    }
+    
+    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
+    	synchronized (latestMetaDataLock) {
+            throwConnectionClosedIfNullMetaData();
+            latestMetaData = latestMetaData.updateResolvedTimestamp(table, resolvedTime);
             latestMetaDataLock.notifyAll();
             return latestMetaData;
         }
@@ -512,7 +521,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             while (true) {
                 try {
                     try {
-                        table = metaData.getTable(new PTableKey(tenantId, tableName));
+                        table = metaData.getTableRef(new PTableKey(tenantId, tableName)).getTable();
                         /* If the table is at the prior sequence number, then we're good to go.
                          * We know if we've got this far, that the server validated the mutations,
                          * so we'd just need to wait until the other connection that mutated the same
@@ -553,12 +562,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
      }
 
     @Override
-    public PMetaData addColumn(final PName tenantId, final String tableName, final List<PColumn> columns, final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows, final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls) throws SQLException {
+    public PMetaData addColumn(final PName tenantId, final String tableName, final List<PColumn> columns, final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows, final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls, final long resolvedTime) throws SQLException {
         return metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public PMetaData mutate(PMetaData metaData) throws SQLException {
                 try {
-                    return metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls);
+                    return metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, resolvedTime);
                 } catch (TableNotFoundException e) {
                     // The DROP TABLE may have been processed first, so just ignore.
                     return metaData;
@@ -578,12 +587,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public PMetaData removeColumn(final PName tenantId, final String tableName, final List<PColumn> columnsToRemove, final long tableTimeStamp, final long tableSeqNum) throws SQLException {
+    public PMetaData removeColumn(final PName tenantId, final String tableName, final List<PColumn> columnsToRemove, final long tableTimeStamp, final long tableSeqNum, final long resolvedTime) throws SQLException {
         return metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public PMetaData mutate(PMetaData metaData) throws SQLException {
                 try {
-                    return metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum);
+                    return metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
                 } catch (TableNotFoundException e) {
                     // The DROP TABLE may have been processed first, so just ignore.
                     return metaData;
@@ -1131,7 +1140,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             synchronized (latestMetaDataLock) {
                 throwConnectionClosedIfNullMetaData();
-                table = latestMetaData.getTable(new PTableKey(PName.EMPTY_NAME, parentTableName));
+                table = latestMetaData.getTableRef(new PTableKey(PName.EMPTY_NAME, parentTableName)).getTable();
                 latestMetaDataLock.notifyAll();
             }
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be the case
@@ -1474,7 +1483,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (metadata == null) {
                 throwConnectionClosedException();
             }
-            table = metadata.getTable(new PTableKey(tenantId, name));
+            table = metadata.getTableRef(new PTableKey(tenantId, name)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be the case
                 throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
             }
@@ -1858,7 +1867,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         Properties props = new Properties(oldMetaConnection.getClientInfo());
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
         // Cannot go through DriverManager or you end up in an infinite loop because it'll call init again
-        PhoenixConnection metaConnection = new PhoenixConnection(this, oldMetaConnection.getURL(), props, oldMetaConnection.getMetaDataCache());
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
         SQLException sqlE = null;
         try {
             metaConnection.createStatement().executeUpdate("ALTER TABLE " + tableName + " ADD IF NOT EXISTS " + columns );

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/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 82b0e1a..0e639c7 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
@@ -162,14 +162,19 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData addTable(PTable table) throws SQLException {
-        return metaData = metaData.addTable(table);
+    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
+        return metaData = metaData.addTable(table, resolvedTime);
+    }
+    
+    @Override
+    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
+        return metaData = metaData.updateResolvedTimestamp(table, resolvedTimestamp);
     }
 
     @Override
     public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
-            long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls) throws SQLException {
-        return metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls);
+            long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, long resolvedTime) throws SQLException {
+        return metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, resolvedTime);
     }
 
     @Override
@@ -180,8 +185,8 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
 
     @Override
     public PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
-            long tableSeqNum) throws SQLException {
-        return metaData = metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum);
+            long tableSeqNum, long resolvedTime) throws SQLException {
+        return metaData = metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
     }
 
     
@@ -196,7 +201,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         // to get anything from the server (since we don't have a connection)
         try {
             String fullTableName = SchemaUtil.getTableName(schemaBytes, tableBytes);
-            PTable table = metaData.getTable(new PTableKey(tenantId, fullTableName));
+            PTable table = metaData.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, table, true);
         } catch (TableNotFoundException e) {
             return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null);
@@ -361,7 +366,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         String indexName = Bytes.toString(rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]);
         String indexTableName = SchemaUtil.getTableName(schemaName, indexName);
         PName tenantId = tenantIdBytes.length == 0 ? null : PNameFactory.newName(tenantIdBytes);
-        PTable index = metaData.getTable(new PTableKey(tenantId, indexTableName));
+        PTable index = metaData.getTableRef(new PTableKey(tenantId, indexTableName)).getTable();
         index = PTableImpl.makePTable(index,newState == PIndexState.USABLE ? PIndexState.ACTIVE : newState == PIndexState.UNUSABLE ? PIndexState.INACTIVE : newState);
         return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, index);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/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 cf593e2..460a879 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
@@ -74,14 +74,19 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData addTable(PTable table) throws SQLException {
-        return getDelegate().addTable(table);
+    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
+        return getDelegate().addTable(table, resolvedTime);
+    }
+    
+    @Override
+    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
+        return getDelegate().updateResolvedTimestamp(table, resolvedTimestamp);
     }
 
     @Override
     public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
-            long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls) throws SQLException {
-        return getDelegate().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls);
+            long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, long resolvedTime) throws SQLException {
+        return getDelegate().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, resolvedTime);
     }
 
     @Override
@@ -92,8 +97,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
 
     @Override
     public PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
-            long tableSeqNum) throws SQLException {
-        return getDelegate().removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum);
+            long tableSeqNum, long resolvedTime) throws SQLException {
+        return getDelegate().removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92ee51a0/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
index ae37ac6..c78d0b6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
@@ -34,8 +34,9 @@ import org.apache.phoenix.schema.PTable;
  * @since 0.1
  */
 public interface MetaDataMutated {
-    PMetaData addTable(PTable table) throws SQLException;
+    PMetaData addTable(PTable table, long resolvedTime) throws SQLException;
+    PMetaData updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException;
     PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException;
-    PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls) throws SQLException;
-    PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum) throws SQLException;
+    PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, long resolvedTime) throws SQLException;
+    PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum, long resolvedTime) throws SQLException;
 }