You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2017/02/14 23:42:36 UTC

[47/50] [abbrv] phoenix git commit: PHOENIX-1598 Column encoding to save space and improve performance

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropertiesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropertiesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropertiesIT.java
new file mode 100644
index 0000000..52cfe9c
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropertiesIT.java
@@ -0,0 +1,189 @@
+/*
+ * 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.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Test;
+
+public class ImmutableTablePropertiesIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testImmutableKeyword() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            // create table with immutable keyword
+            String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) STORE_NULLS=true";
+            stmt.execute(ddl);
+            
+            // create table without immutable keyword
+            ddl = "CREATE TABLE  " + mutableDataTableFullName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) STORE_NULLS=true";
+            stmt.execute(ddl);
+            
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable immutableTable = phxConn.getTable(new PTableKey(null, immutableDataTableFullName));
+            assertTrue("IMMUTABLE_ROWS should be set to true", immutableTable.isImmutableRows());
+            PTable mutableTable = phxConn.getTable(new PTableKey(null, mutableDataTableFullName));
+            assertFalse("IMMUTABLE_ROWS should be set to false", mutableTable.isImmutableRows());
+        } 
+    }
+    
+    @Test
+    public void testImmutableProperty() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            // create table with immutable table property set to true
+            String ddl = "CREATE TABLE  " + immutableDataTableFullName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=true";
+            stmt.execute(ddl);
+            
+            // create table with immutable table property set to false
+            ddl = "CREATE TABLE  " + mutableDataTableFullName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))  IMMUTABLE_ROWS=false";
+            stmt.execute(ddl);
+            
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable immutableTable = phxConn.getTable(new PTableKey(null, immutableDataTableFullName));
+            assertTrue("IMMUTABLE_ROWS should be set to true", immutableTable.isImmutableRows());
+            PTable mutableTable = phxConn.getTable(new PTableKey(null, mutableDataTableFullName));
+            assertFalse("IMMUTABLE_ROWS should be set to false", mutableTable.isImmutableRows());
+        } 
+    }
+    
+    @Test
+    public void testImmutableKeywordAndProperty() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            try {
+                // create immutable table with immutable table property set to true 
+                String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName +
+                        "  (a_string varchar not null, col1 integer" +
+                        "  CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=true";
+                stmt.execute(ddl);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID.getErrorCode(), e.getErrorCode());
+            }
+            
+            try {
+                // create immutable table with immutable table property set to false
+                String ddl = "CREATE IMMUTABLE TABLE  " + mutableDataTableFullName +
+                        "  (a_string varchar not null, col1 integer" +
+                        "  CONSTRAINT pk PRIMARY KEY (a_string))  IMMUTABLE_ROWS=false";
+                stmt.execute(ddl);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID.getErrorCode(), e.getErrorCode());
+            }
+            
+        } 
+    }
+    
+    @Test
+    public void testImmutableTableWithStorageSchemeAndColumnEncodingProps() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            try {
+                // create immutable table with immutable table property set to true 
+                String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName +
+                        "  (a_string varchar not null, col1 integer" +
+                        "  CONSTRAINT pk PRIMARY KEY (a_string)) COLUMN_ENCODED_BYTES=0, IMMUTABLE_STORAGE_SCHEME="
+                        + PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
+                stmt.execute(ddl);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_AND_COLUMN_QUALIFIER_BYTES.getErrorCode(), e.getErrorCode());
+            }
+        } 
+    }
+    
+    @Test
+    public void testAlterImmutableStorageSchemeProp() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName1 = SchemaUtil.getTableName("", generateUniqueName());
+        String immutableDataTableFullName2 = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            // create an immutable table with  ONE_CELL_PER_COLUMN storage scheme
+            String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName1 +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) COLUMN_ENCODED_BYTES=0, IMMUTABLE_STORAGE_SCHEME="
+                    + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
+            stmt.execute(ddl);
+            // create an immutable table with  SINGLE_CELL_ARRAY_WITH_OFFSETS storage scheme
+            ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName2 +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) COLUMN_ENCODED_BYTES=4, IMMUTABLE_STORAGE_SCHEME="
+                    + PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
+            stmt.execute(ddl);
+            
+            // changing the storage scheme from/to ONCE_CELL_PER_COLUMN should fail
+            try {
+                stmt.execute("ALTER TABLE " + immutableDataTableFullName1 + " SET IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE.getErrorCode(), e.getErrorCode());
+            }
+            try {
+                stmt.execute("ALTER TABLE " + immutableDataTableFullName2 + " SET IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE.getErrorCode(), e.getErrorCode());
+            }
+        } 
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableQueryIT.java
new file mode 100644
index 0000000..da36288
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableQueryIT.java
@@ -0,0 +1,424 @@
+/*
+ * 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.end2end;
+
+import static org.apache.phoenix.util.TestUtil.A_VALUE;
+import static org.apache.phoenix.util.TestUtil.B_VALUE;
+import static org.apache.phoenix.util.TestUtil.C_VALUE;
+import static org.apache.phoenix.util.TestUtil.E_VALUE;
+import static org.apache.phoenix.util.TestUtil.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW3;
+import static org.apache.phoenix.util.TestUtil.ROW4;
+import static org.apache.phoenix.util.TestUtil.ROW5;
+import static org.apache.phoenix.util.TestUtil.ROW6;
+import static org.apache.phoenix.util.TestUtil.ROW7;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class MutableQueryIT extends BaseQueryIT {
+    
+    @Parameters(name="indexDDL={0},mutable={1},columnEncoded={2}")
+    @Shadower(classBeingShadowed = BaseQueryIT.class)
+    public static Collection<Object> data() {
+        List<Object> testCases = Lists.newArrayList();
+        for (String indexDDL : INDEX_DDLS) {
+            for (boolean columnEncoded : new boolean[]{false,true}) {
+                testCases.add(new Object[] { indexDDL, true, columnEncoded });
+            }
+        }
+        return testCases;
+    }
+    
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseQueryIT.class)
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
+        props.put(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, Boolean.TRUE.toString());
+        BaseQueryIT.doSetup(props);
+    }
+
+    public MutableQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
+    }
+    
+    @Test
+    public void testSumOverNullIntegerColumn() throws Exception {
+        String query = "SELECT sum(a_integer) FROM " + tableName + " a";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (organization_id,entity_id,a_integer) VALUES('" + getOrganizationId() + "','" + ROW3 + "',NULL)");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, tableName);
+        conn1.close();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 50));
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(42, rs.getInt(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 70));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (organization_id,entity_id,a_integer) SELECT organization_id, entity_id, CAST(null AS integer) FROM " + tableName);
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 90));
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(0, rs.getInt(1));
+            assertTrue(rs.wasNull());
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    private void testNoStringValue(String value) throws Exception {
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 10);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(
+                "upsert into " + tableName + " VALUES (?, ?, ?)"); // without specifying columns
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW5);
+        stmt.setString(3, value);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+        
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, tableName);
+        conn1.close();
+        
+        String query = "SELECT a_string, b_string FROM " + tableName + " WHERE organization_id=? and a_integer = 5";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(null, rs.getString(1));
+            assertTrue(rs.wasNull());
+            assertEquals(C_VALUE, rs.getString("B_string"));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testNullStringValue() throws Exception {
+        testNoStringValue(null);
+    }
+    
+    @Test
+    public void testEmptyStringValue() throws Exception {
+        testNoStringValue("");
+    }
+    
+    @Test
+    public void testUnfoundSingleColumnCaseStatement() throws Exception {
+        String query = "SELECT entity_id, b_string FROM " + tableName + " WHERE organization_id=? and CASE WHEN a_integer = 0 or a_integer != 0 THEN 1 ELSE 0 END = 0";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        // Set ROW5.A_INTEGER to null so that we have one row
+        // where the else clause of the CASE statement will
+        // fire.
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        String upsertStmt =
+            "upsert into " + tableName +
+            " (" +
+            "    ENTITY_ID, " +
+            "    ORGANIZATION_ID, " +
+            "    A_INTEGER) " +
+            "VALUES ('" + ROW5 + "','" + tenantId + "', null)";
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+        
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        ResultSet rs = statement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(ROW5, rs.getString(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testGroupByCondition() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement statement = conn.prepareStatement("SELECT count(*) FROM " + tableName + " WHERE organization_id=? GROUP BY a_integer=6");
+        statement.setString(1, tenantId);
+        ResultSet rs = statement.executeQuery();
+        assertValueEqualsResultSet(rs, Arrays.<Object>asList(1L,8L));
+        try {
+            statement = conn.prepareStatement("SELECT count(*),a_integer=6 FROM " + tableName + " WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null) GROUP BY a_integer=6");
+            statement.setString(1, tenantId);
+            rs = statement.executeQuery();
+            List<List<Object>> expectedResults = Lists.newArrayList(
+                    Arrays.<Object>asList(1L,false),
+                    Arrays.<Object>asList(1L,true));
+            assertValuesEqualsResultSet(rs, expectedResults);
+        } finally {
+            conn.close();
+        }
+
+        
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            statement = conn.prepareStatement("UPSERT into " + tableName + " (organization_id,entity_id,a_integer) values(?,?,null)");
+            statement.setString(1, tenantId);
+            statement.setString(2, ROW3);
+            statement.executeUpdate();
+            conn.commit();
+        } finally {
+            conn.close();
+        }
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 60));
+        conn = DriverManager.getConnection(getUrl(), props);
+        statement = conn.prepareStatement("SELECT count(*) FROM " + tableName + " WHERE organization_id=? GROUP BY a_integer=6");
+        statement.setString(1, tenantId);
+        rs = statement.executeQuery();
+        assertValueEqualsResultSet(rs, Arrays.<Object>asList(1L,1L,7L));
+        statement = conn.prepareStatement("SELECT a_integer, entity_id FROM " + tableName + " WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null)");
+        statement.setString(1, tenantId);
+        rs = statement.executeQuery();
+        List<List<Object>> expectedResults = Lists.newArrayList(
+                Arrays.<Object>asList(null,ROW3),
+                Arrays.<Object>asList(5,ROW5),
+                Arrays.<Object>asList(6,ROW6));
+        assertValuesEqualsResultSet(rs, expectedResults);
+        try {
+            statement = conn.prepareStatement("SELECT count(*),a_integer=6 FROM " + tableName + " WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null) GROUP BY a_integer=6");
+            statement.setString(1, tenantId);
+            rs = statement.executeQuery();
+            expectedResults = Lists.newArrayList(
+                    Arrays.<Object>asList(1L,null),
+                    Arrays.<Object>asList(1L,false),
+                    Arrays.<Object>asList(1L,true));
+            assertValuesEqualsResultSet(rs, expectedResults);
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testPointInTimeDeleteUngroupedAggregation() throws Exception {
+        String updateStmt = 
+            "upsert into " + tableName +
+            " (" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_STRING) " +
+            "VALUES (?, ?, ?)";
+        
+        // Override value that was set at creation time
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+
+        // Remove column value at ts + 1 (i.e. equivalent to setting the value to null)
+        Connection conn = DriverManager.getConnection(url, props);
+        PreparedStatement stmt = conn.prepareStatement(updateStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW7);
+        stmt.setString(3, null);
+        stmt.execute();
+        
+        // Delete row 
+        stmt = conn.prepareStatement("delete from " + tableName + " where organization_id=? and entity_id=?");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW5);
+        stmt.execute();
+        conn.commit();
+        conn.close();
+        
+        // Delete row at timestamp 3. This should not be seen by the query executing
+        // Remove column value at ts + 1 (i.e. equivalent to setting the value to null)
+        Connection futureConn = DriverManager.getConnection(getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3), props);
+        stmt = futureConn.prepareStatement("delete from " + tableName + " where organization_id=? and entity_id=?");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW6);
+        stmt.execute();
+        futureConn.commit();
+        futureConn.close();
+
+        String query = "SELECT count(1) FROM " + tableName + " WHERE organization_id=? and a_string = ?";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        statement.setString(2, B_VALUE);
+        ResultSet rs = statement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(2, rs.getLong(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
+    @Test
+    public void testPointInTimeScan() throws Exception {
+        // Override value that was set at creation time
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 10);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        String upsertStmt =
+            "upsert into " + tableName +
+            " (" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_INTEGER) " +
+            "VALUES (?, ?, ?)";
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW4);
+        stmt.setInt(3, 5);
+        stmt.execute(); // should commit too
+        
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 15);
+        Connection conn1 = DriverManager.getConnection(url, props);
+        analyzeTable(conn1, tableName);
+        conn1.close();
+        upsertConn.close();
+
+        // Override value again, but should be ignored since it's past the SCN
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 30);
+        upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW4);
+        stmt.setInt(3, 9);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+        
+        String query = "SELECT organization_id, a_string AS a FROM " + tableName + " WHERE organization_id=? and a_integer = 5";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        ResultSet rs = statement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(tenantId, rs.getString(1));
+        assertEquals(A_VALUE, rs.getString("a"));
+        assertTrue(rs.next());
+        assertEquals(tenantId, rs.getString(1));
+        assertEquals(B_VALUE, rs.getString(2));
+        assertFalse(rs.next());
+        conn.close();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testPointInTimeLimitedScan() throws Exception {
+        // Override value that was set at creation time
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        String upsertStmt =
+            "upsert into " + tableName +
+            " (" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_INTEGER) " +
+            "VALUES (?, ?, ?)";
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW1);
+        stmt.setInt(3, 6);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+
+        // Override value again, but should be ignored since it's past the SCN
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3);
+        upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW1);
+        stmt.setInt(3, 0);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+        
+        String query = "SELECT a_integer,b_string FROM " + tableName + " WHERE organization_id=? and a_integer <= 5 limit 2";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        ResultSet rs = statement.executeQuery();
+        List<List<Object>> expectedResultsA = Lists.newArrayList(
+                Arrays.<Object>asList(2, C_VALUE),
+                Arrays.<Object>asList( 3, E_VALUE));
+        List<List<Object>> expectedResultsB = Lists.newArrayList(
+                Arrays.<Object>asList( 5, C_VALUE),
+                Arrays.<Object>asList(4, B_VALUE));
+        // Since we're not ordering and we may be using a descending index, we don't
+        // know which rows we'll get back.
+        assertOneOfValuesEqualsResultSet(rs, expectedResultsA,expectedResultsB);
+       conn.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
index 3d0fa2c..965071f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
@@ -151,7 +151,7 @@ public class NativeHBaseTypesIT extends BaseClientManagedTimeIT {
         }
         // Create Phoenix table after HBase table was created through the native APIs
         // The timestamp of the table creation must be later than the timestamp of the data
-        ensureTableCreated(getUrl(),HBASE_NATIVE,HBASE_NATIVE,null, ts+1);
+        ensureTableCreated(getUrl(),HBASE_NATIVE,HBASE_NATIVE,null, ts+1, null);
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
index 76627be..036d278 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
@@ -52,8 +52,8 @@ import com.google.common.primitives.Floats;
 @RunWith(Parameterized.class)
 public class NotQueryIT extends BaseQueryIT {
 
-    public NotQueryIT(String indexDDL) {
-        super(indexDDL);
+    public NotQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Parameters(name="NotQueryIT_{index}") // name is used by failsafe as file name in reports
@@ -63,7 +63,7 @@ public class NotQueryIT extends BaseQueryIT {
     
     @Test
     public void testNotInList() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and entity_id NOT IN (?,?,?,?,?,?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and entity_id NOT IN (?,?,?,?,?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -91,7 +91,7 @@ public class NotQueryIT extends BaseQueryIT {
     
     @Test
     public void testNotInListOfFloat() throws Exception {
-        String query = "SELECT a_float FROM aTable WHERE organization_id=? and a_float NOT IN (?,?,?,?,?,?)";
+        String query = "SELECT a_float FROM " + tableName + " WHERE organization_id=? and a_float NOT IN (?,?,?,?,?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -119,7 +119,7 @@ public class NotQueryIT extends BaseQueryIT {
     
     @Test
     public void testNotInListOfDouble() throws Exception {
-        String query = "SELECT a_double FROM aTable WHERE organization_id=? and a_double NOT IN (?,?,?,?,?,?)";
+        String query = "SELECT a_double FROM " + tableName + " WHERE organization_id=? and a_double NOT IN (?,?,?,?,?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -148,7 +148,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEquals() throws Exception {
         String query = "SELECT entity_id -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_integer != 1 and a_integer <= 2";
+        "FROM " + tableName + " WHERE organization_id=? and a_integer != 1 and a_integer <= 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -167,7 +167,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByTinyInt() throws Exception {
         String query = "SELECT a_byte -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_byte != 1 and a_byte <= 2";
+        "FROM " + tableName + " WHERE organization_id=? and a_byte != 1 and a_byte <= 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -186,7 +186,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsBySmallInt() throws Exception {
         String query = "SELECT a_short -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_short != 128 and a_short !=0 and a_short <= 129";
+        "FROM " + tableName + " WHERE organization_id=? and a_short != 128 and a_short !=0 and a_short <= 129";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -205,7 +205,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByFloat() throws Exception {
         String query = "SELECT a_float -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_float != CAST(0.01 AS FLOAT) and a_float <= CAST(0.02 AS FLOAT)";
+        "FROM " + tableName + " WHERE organization_id=? and a_float != CAST(0.01 AS FLOAT) and a_float <= CAST(0.02 AS FLOAT)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -224,7 +224,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByUnsignedFloat() throws Exception {
         String query = "SELECT a_unsigned_float -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_unsigned_float != 0.01 and a_unsigned_float <= 0.02";
+        "FROM " + tableName + " WHERE organization_id=? and a_unsigned_float != 0.01 and a_unsigned_float <= 0.02";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -243,7 +243,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByDouble() throws Exception {
         String query = "SELECT a_double -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_double != CAST(0.0001 AS DOUBLE) and a_double <= CAST(0.0002 AS DOUBLE)";
+        "FROM " + tableName + " WHERE organization_id=? and a_double != CAST(0.0001 AS DOUBLE) and a_double <= CAST(0.0002 AS DOUBLE)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -262,7 +262,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByUnsignedDouble() throws Exception {
         String query = "SELECT a_unsigned_double -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_unsigned_double != 0.0001 and a_unsigned_double <= 0.0002";
+        "FROM " + tableName + " WHERE organization_id=? and a_unsigned_double != 0.0001 and a_unsigned_double <= 0.0002";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -280,8 +280,8 @@ public class NotQueryIT extends BaseQueryIT {
 
     @Test
     public void testNotEquals2() throws Exception {
-        String query = "SELECT entity_id FROM // one more comment  \n" +
-        "aTable WHERE organization_id=? and not a_integer = 1 and a_integer <= 2";
+        String query = "SELECT entity_id FROM // one more comment  \n" + tableName + 
+        " WHERE organization_id=? and not a_integer = 1 and a_integer <= 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
index cbdd6e1..ebbeeb4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
@@ -288,7 +288,6 @@ public class OrderByIT extends ParallelStatsDisabledIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
-
         try {
             String tableName1 = generateUniqueName();
             String ddl = "CREATE TABLE " + tableName1 +
@@ -376,7 +375,6 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             assertEquals("a",rs.getString(1));  
             assertEquals(40,rs.getInt(2));
             assertFalse(rs.next()); 
-        } catch (SQLException e) {
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelClientManagedTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelClientManagedTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelClientManagedTimeIT.java
new file mode 100644
index 0000000..1270ede
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelClientManagedTimeIT.java
@@ -0,0 +1,39 @@
+/*
+ * 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.end2end;
+
+import org.junit.After;
+import org.junit.AfterClass;
+
+/**
+ * Base class for tests whose methods control time stamps at which SQL statements are executed You must create unique
+ * names using {@link #generateUniqueName()} for each table and sequence used to prevent collisions. Because of
+ * uniqueness of table names and sequences, classes extending this class can execute in parallel (on the same JVM).
+ */
+public abstract class ParallelClientManagedTimeIT extends BaseClientManagedTimeIT {
+//    @Override
+//    @After
+//    public void cleanUpAfterTest() throws Exception {
+//        // Don't do anything as tests use unique table names and sequences
+//    }
+
+    @AfterClass
+    public static void doTeardown() throws Exception {
+        // Don't do anything as tests use unique table names and sequences
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
index af5a905..408bf18 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
@@ -511,9 +511,9 @@ public class PercentileIT extends ParallelStatsDisabledIT {
             Date date, Long ts) throws Exception {
         String tableName = generateUniqueName();
         if (ts == null) {
-            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits);
+            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits, null);
         } else {
-            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits, ts - 2);
+            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits, ts - 2, null);
         }
 
         Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
index cddca04..91e9370 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
@@ -61,11 +61,11 @@ public class PhoenixRuntimeIT extends ParallelStatsDisabledIT {
         scan.setFilter(filter);
         ResultScanner scanner = htable.getScanner(scan);
         Result result = null;
-        ResultTuple tuple = new ResultTuple();
+        ResultTuple tuple;
         Set<String> actualTenantIds = Sets.newHashSetWithExpectedSize(tenantIds.length);
         Set<String> expectedTenantIds = new HashSet<>(Arrays.asList(tenantIds));
         while ((result = scanner.next()) != null) {
-            tuple.setResult(result);
+            tuple = new ResultTuple(result);
             e.evaluate(tuple, ptr);
             String tenantId = (String)PVarchar.INSTANCE.toObject(ptr);
             actualTenantIds.add(tenantId == null ? "" : tenantId);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
index 73eb2a3..bb1eabe 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
@@ -17,18 +17,13 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.A_VALUE;
-import static org.apache.phoenix.util.TestUtil.B_VALUE;
-import static org.apache.phoenix.util.TestUtil.ROW4;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
-import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.util.Map;
 import java.util.Properties;
@@ -44,8 +39,8 @@ import com.google.common.collect.Maps;
 
 public class PointInTimeQueryIT extends BaseQueryIT {
 
-    public PointInTimeQueryIT(String indexDDL) {
-        super(indexDDL);
+    public PointInTimeQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
 
     @BeforeClass
@@ -57,61 +52,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
     }
     
     @Test
-    public void testPointInTimeScan() throws Exception {
-        // Override value that was set at creation time
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 10);
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        String upsertStmt =
-            "upsert into " +
-            "ATABLE(" +
-            "    ORGANIZATION_ID, " +
-            "    ENTITY_ID, " +
-            "    A_INTEGER) " +
-            "VALUES (?, ?, ?)";
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW4);
-        stmt.setInt(3, 5);
-        stmt.execute(); // should commit too
-        
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 15);
-        Connection conn1 = DriverManager.getConnection(url, props);
-        analyzeTable(conn1, "ATABLE");
-        conn1.close();
-        upsertConn.close();
-
-        // Override value again, but should be ignored since it's past the SCN
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 30);
-        upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        stmt = upsertConn.prepareStatement(upsertStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW4);
-        stmt.setInt(3, 9);
-        stmt.execute(); // should commit too
-        upsertConn.close();
-        
-        String query = "SELECT organization_id, a_string AS a FROM atable WHERE organization_id=? and a_integer = 5";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement statement = conn.prepareStatement(query);
-        statement.setString(1, tenantId);
-        ResultSet rs = statement.executeQuery();
-        assertTrue(rs.next());
-        assertEquals(tenantId, rs.getString(1));
-        assertEquals(A_VALUE, rs.getString("a"));
-        assertTrue(rs.next());
-        assertEquals(tenantId, rs.getString(1));
-        assertEquals(B_VALUE, rs.getString(2));
-        assertFalse(rs.next());
-        conn.close();
-    }
-
-    @Test
     public void testPointInTimeSequence() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn;
@@ -122,7 +62,7 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         conn.createStatement().execute("CREATE SEQUENCE s");
         
         try {
-            conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+            conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
             fail();
         } catch (SequenceNotFoundException e) {
             conn.close();
@@ -130,14 +70,14 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+10));
         conn = DriverManager.getConnection(getUrl(), props);
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         conn.close();
         
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+7));
         conn = DriverManager.getConnection(getUrl(), props);
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(2, rs.getInt(1));
         conn.close();
@@ -145,7 +85,7 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+15));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute("DROP SEQUENCE s");
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(3, rs.getInt(1));
         conn.close();
@@ -153,7 +93,7 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+20));
         conn = DriverManager.getConnection(getUrl(), props);
         try {
-            rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+            rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
             fail();
         } catch (SequenceNotFoundException e) {
             conn.close();            
@@ -163,14 +103,14 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         conn.close();
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+25));
         conn = DriverManager.getConnection(getUrl(), props);
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         conn.close();
 
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+6));
         conn = DriverManager.getConnection(getUrl(), props);
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(4, rs.getInt(1));
         conn.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
index 17c854a..87b7af6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
@@ -89,7 +89,7 @@ public class ProductMetricsIT extends BaseClientManagedTimeIT {
     }
     
     private static void initTable(byte[][] splits, long ts) throws Exception {
-        ensureTableCreated(getUrl(), PRODUCT_METRICS_NAME, PRODUCT_METRICS_NAME,splits, ts-2);
+        ensureTableCreated(getUrl(), PRODUCT_METRICS_NAME, PRODUCT_METRICS_NAME,splits, ts-2, null);
     }
 
     private static void assertNoRows(Connection conn) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index ec9f32f..6b3778a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -655,7 +655,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testCreateDropTable() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         ensureTableCreated(getUrl(), BTABLE_NAME, BTABLE_NAME, ts-2);
         ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts-2);
@@ -977,7 +977,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddKVColumnToExistingFamily() throws Throwable {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1008,7 +1008,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddKVColumnToNewFamily() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
@@ -1034,7 +1034,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddPKColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1067,7 +1067,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropKVColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1107,7 +1107,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropPKColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1125,7 +1125,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropAllKVCols() throws Exception {
         ResultSet rs;
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), MDTEST_NAME, MDTEST_NAME, null, ts);
+        ensureTableCreated(getUrl(), MDTEST_NAME, MDTEST_NAME, null, ts, null);
         
         Properties props = new Properties();
         
@@ -1171,7 +1171,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     @Test
     public void testNewerTableDisallowed() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, null, ts);
+        ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, null, ts, null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
index d3bbe23..1084f14 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
@@ -62,15 +62,15 @@ import org.junit.Test;
  */
 public class QueryIT extends BaseQueryIT {
     
-    public QueryIT(String indexDDL) {
-        super(indexDDL);
+    public QueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Test
     public void testIntFilter() throws Exception {
         String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName +
+            " (" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
             "    A_INTEGER) " +
@@ -89,10 +89,10 @@ public class QueryIT extends BaseQueryIT {
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 6);
         props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         upsertConn = DriverManager.getConnection(url, props);
-        analyzeTable(upsertConn, "ATABLE");
+        analyzeTable(upsertConn, tableName);
         upsertConn.close();
 
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer >= ?";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_integer >= ?";
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
         PreparedStatement statement = conn.prepareStatement(query);
@@ -101,19 +101,19 @@ public class QueryIT extends BaseQueryIT {
         ResultSet rs = statement.executeQuery();
         assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW7, ROW8, ROW9));
 
-        query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer < 2";
+        query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_integer < 2";
         statement = conn.prepareStatement(query);
         statement.setString(1, tenantId);
         rs = statement.executeQuery();
         assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW1, ROW4));
 
-        query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer <= 2";
+        query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_integer <= 2";
         statement = conn.prepareStatement(query);
         statement.setString(1, tenantId);
         rs = statement.executeQuery();
         assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW1, ROW2, ROW4));
 
-        query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer >=9";
+        query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_integer >=9";
         statement = conn.prepareStatement(query);
         statement.setString(1, tenantId);
         rs = statement.executeQuery();
@@ -124,13 +124,8 @@ public class QueryIT extends BaseQueryIT {
     }
     
     @Test
-    public void testEmptyStringValue() throws Exception {
-        testNoStringValue("");
-    }
-
-    @Test
     public void testToDateOnString() throws Exception { // TODO: test more conversion combinations
-        String query = "SELECT a_string FROM aTable WHERE organization_id=? and a_integer = 5";
+        String query = "SELECT a_string FROM " + tableName + " WHERE organization_id=? and a_integer = 5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -151,7 +146,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testColumnOnBothSides() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_string = b_string";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_string = b_string";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -167,50 +162,9 @@ public class QueryIT extends BaseQueryIT {
         }
     }
 
-    private void testNoStringValue(String value) throws Exception {
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 10);
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        PreparedStatement stmt = upsertConn.prepareStatement(
-                "upsert into ATABLE VALUES (?, ?, ?)"); // without specifying columns
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW5);
-        stmt.setString(3, value);
-        stmt.execute(); // should commit too
-        upsertConn.close();
-        
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
-        Connection conn1 = DriverManager.getConnection(getUrl(), props);
-        analyzeTable(conn1, "ATABLE");
-        conn1.close();
-        
-        String query = "SELECT a_string, b_string FROM aTable WHERE organization_id=? and a_integer = 5";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(null, rs.getString(1));
-            assertTrue(rs.wasNull());
-            assertEquals(C_VALUE, rs.getString("B_string"));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testNullStringValue() throws Exception {
-        testNoStringValue(null);
-    }
-    
     @Test
     public void testDateInList() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE a_date IN (?,?) AND a_integer < 4";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE a_date IN (?,?) AND a_integer < 4";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -230,8 +184,8 @@ public class QueryIT extends BaseQueryIT {
     @Test
     public void testTimestamp() throws Exception {
         String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName +
+            " (" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
             "    A_TIMESTAMP) " +
@@ -251,12 +205,12 @@ public class QueryIT extends BaseQueryIT {
         
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 15);       
         Connection conn1 = DriverManager.getConnection(url, props);
-        analyzeTable(conn1, "ATABLE");
+        analyzeTable(conn1, tableName);
         conn1.close();
         
         updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName +
+            " (" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
             "    A_TIMESTAMP," +
@@ -276,7 +230,7 @@ public class QueryIT extends BaseQueryIT {
         assertTrue(compare(CompareOp.GREATER, new ImmutableBytesWritable(ts2), new ImmutableBytesWritable(ts1)));
         assertFalse(compare(CompareOp.GREATER, new ImmutableBytesWritable(ts1), new ImmutableBytesWritable(ts1)));
 
-        String query = "SELECT entity_id, a_timestamp, a_time FROM aTable WHERE organization_id=? and a_timestamp > ?";
+        String query = "SELECT entity_id, a_timestamp, a_time FROM " + tableName + " WHERE organization_id=? and a_timestamp > ?";
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -296,7 +250,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testSimpleInListStatement() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_integer IN (2,4)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND a_integer IN (2,4)";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -312,7 +266,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testPartiallyQualifiedRVCInList() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE (a_integer,a_string) IN ((2,'a'),(5,'b'))";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE (a_integer,a_string) IN ((2,'a'),(5,'b'))";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -327,7 +281,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testFullyQualifiedRVCInList() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE (a_integer,a_string, organization_id,entity_id) IN ((2,'a',:1,:2),(5,'b',:1,:3))";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE (a_integer,a_string, organization_id,entity_id) IN ((2,'a',:1,:2),(5,'b',:1,:3))";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -345,7 +299,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testOneInListStatement() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND b_string IN (?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND b_string IN (?)";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -369,7 +323,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testMixedTypeInListStatement() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_long IN (5, ?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND x_long IN (5, ?)";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -391,7 +345,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testIsNull() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE X_DECIMAL is null";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE X_DECIMAL is null";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -418,7 +372,7 @@ public class QueryIT extends BaseQueryIT {
 
     @Test
     public void testIsNotNull() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE X_DECIMAL is not null";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE X_DECIMAL is not null";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -442,11 +396,11 @@ public class QueryIT extends BaseQueryIT {
         int counter=0;
         String[] answers = new String[]{"00D300000000XHP5bar","a5bar","15bar","5bar","5bar"};
         String[] queries = new String[] { 
-        		"SELECT  organization_id || 5 || 'bar' FROM atable limit 1",
-        		"SELECT a_string || 5 || 'bar' FROM atable  order by a_string  limit 1",
-        		"SELECT a_integer||5||'bar' FROM atable order by a_integer  limit 1",
-        		"SELECT x_decimal||5||'bar' FROM atable limit 1",
-        		"SELECT x_long||5||'bar' FROM atable limit 1"
+        		"SELECT  organization_id || 5 || 'bar' FROM " + tableName + " limit 1",
+        		"SELECT a_string || 5 || 'bar' FROM " + tableName + "  order by a_string  limit 1",
+        		"SELECT a_integer||5||'bar' FROM " + tableName + " order by a_integer  limit 1",
+        		"SELECT x_decimal||5||'bar' FROM " + tableName + " limit 1",
+        		"SELECT x_long||5||'bar' FROM " + tableName + " limit 1"
         };
 
         for (String query : queries) {
@@ -468,7 +422,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testRowKeySingleIn() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and entity_id IN (?,?,?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and entity_id IN (?,?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -494,7 +448,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testRowKeyMultiIn() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and entity_id IN (?,?,?) and a_string IN (?,?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and entity_id IN (?,?,?) and a_string IN (?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -519,7 +473,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testColumnAliasMapping() throws Exception {
-        String query = "SELECT a.a_string, aTable.b_string FROM aTable a WHERE ?=organization_id and 5=a_integer ORDER BY a_string, b_string";
+        String query = "SELECT a.a_string, " + tableName + ".b_string FROM " + tableName + " a WHERE ?=organization_id and 5=a_integer ORDER BY a_string, b_string";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b49fc0d1/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
index 1573533..8b320fa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
@@ -42,7 +42,7 @@ public class ReadIsolationLevelIT extends BaseClientManagedTimeIT {
 
     protected static void initTableValues(long ts, byte[][] splits) throws Exception {
         String tenantId = getOrganizationId();
-        ensureTableCreated(getUrl(),ATABLE_NAME, ATABLE_NAME, splits, ts-2);
+        ensureTableCreated(getUrl(),ATABLE_NAME, ATABLE_NAME, splits, ts-2, null);
 
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));