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

[1/7] PHOENIX-982 Avoid spinning up and tearing down mini cluster in tests (SamarthJain)

Repository: incubator-phoenix
Updated Branches:
  refs/heads/master 0ca6c1e8d -> 91b7922c1


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
index acfa5e6..1d82362 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
@@ -19,6 +19,8 @@
  */
 package org.apache.phoenix.pig;
 
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+import static org.apache.phoenix.util.TestUtil.LOCALHOST;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -28,16 +30,11 @@ import java.sql.ResultSet;
 import java.sql.Statement;
 import java.util.Collection;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.phoenix.jdbc.PhoenixDriver;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ConfigUtil;
-import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecJob.JOB_STATUS;
-import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.builtin.mock.Storage;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
@@ -47,39 +44,29 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
 
-public class PhoenixHBaseStorerIT {
+@Category(HBaseManagedTimeTest.class)
+public class PhoenixHBaseStorerIT extends BaseHBaseManagedTimeIT {
 
     private static TupleFactory tupleFactory;
-    private static HBaseTestingUtility hbaseTestUtil;
-    private static String zkQuorum;
     private static Connection conn;
     private static PigServer pigServer;
-    private static Configuration conf;
-
+    private static String zkQuorum;
+    
     @BeforeClass
     public static void setUpBeforeClass() throws Exception {
-        hbaseTestUtil = new HBaseTestingUtility();
-        conf = hbaseTestUtil.getConfiguration();
-        ConfigUtil.setReplicationConfigIfAbsent(conf);
-        conf.setInt(QueryServices.MASTER_INFO_PORT_ATTRIB, -1);
-        conf.setInt(QueryServices.REGIONSERVER_INFO_PORT_ATTRIB, -1);
-        hbaseTestUtil.startMiniCluster();
-
-        Class.forName(PhoenixDriver.class.getName());
-        zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
-        conn = DriverManager.getConnection(PhoenixRuntime.JDBC_PROTOCOL +
-                 PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum);
+        conn = DriverManager.getConnection(getUrl());
+        zkQuorum = LOCALHOST + JDBC_PROTOCOL_SEPARATOR + getZKClientPort(getTestClusterConfig());
         // Pig variables
         tupleFactory = TupleFactory.getInstance();
     }
     
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.LOCAL,
-                ConfigurationUtil.toProperties(conf));
+        pigServer = new PigServer(ExecType.LOCAL, getTestClusterConfig());
     }
     
     @After
@@ -90,7 +77,6 @@ public class PhoenixHBaseStorerIT {
     @AfterClass
     public static void tearDownAfterClass() throws Exception {
         conn.close();
-        hbaseTestUtil.shutdownMiniCluster();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java
index e867a4d..fc0267f 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java
@@ -30,10 +30,13 @@ import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class PhoenixPigConfigurationIT extends BaseHBaseManagedTimeIT {
     private static final String zkQuorum = TestUtil.LOCALHOST + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
     

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c31a643..98cb0c5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -114,8 +114,8 @@
     <maven.rat.version>0.8</maven.rat.version>
 	
     <!--  Plugin options -->
-    <numForkedIT>3</numForkedIT>
-    <numForkedUT>3</numForkedUT>
+    <numForkedIT>4</numForkedIT>
+    <numForkedUT>4</numForkedUT>
     
     <!-- Set default encoding so multi-byte tests work correctly on the Mac -->
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
@@ -243,20 +243,54 @@
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-failsafe-plugin</artifactId>
           <version>${maven-failsafe-plugin.version}</version>
-          <configuration>
-            <encoding>UTF-8</encoding>
-            <forkCount>${numForkedIT}</forkCount>
-            <reuseForks>true</reuseForks>
-            <argLine>-enableassertions -Xmx3000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom</argLine>
-            <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
-            <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
-          </configuration>
           <executions>
             <execution>
-              <goals>
-                <goal>integration-test</goal>
-                <goal>verify</goal>
-              </goals>
+               <id>ClientManagedTimeTests</id>
+                 <configuration>
+                 <encoding>UTF-8</encoding>
+                 <forkCount>${numForkedIT}</forkCount>
+                 <reuseForks>true</reuseForks>
+                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom</argLine>
+                 <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
+                 <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
+                 <groups>org.apache.phoenix.end2end.ClientManagedTimeTest</groups>
+               </configuration>
+               <goals>
+                 <goal>integration-test</goal>
+                 <goal>verify</goal>
+               </goals>
+            </execution>
+            <execution>
+               <id>HBaseManagedTimeTests</id>
+                 <configuration>
+                 <encoding>UTF-8</encoding>
+                 <forkCount>${numForkedIT}</forkCount>
+                 <reuseForks>true</reuseForks>
+                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom</argLine>
+                 <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
+                 <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
+                 <groups>org.apache.phoenix.end2end.HBaseManagedTimeTest</groups>
+               </configuration>
+               <goals>
+                 <goal>integration-test</goal>
+                 <goal>verify</goal>
+               </goals>
+            </execution>
+	        <execution>
+               <id>NeedTheirOwnClusterTests</id>
+                 <configuration>
+                 <encoding>UTF-8</encoding>
+                 <forkCount>${numForkedIT}</forkCount>
+                 <reuseForks>true</reuseForks>
+                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom</argLine>
+                 <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
+                 <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
+                 <groups>org.apache.phoenix.end2end.NeedsOwnMiniClusterTest</groups>
+               </configuration>
+               <goals>
+                 <goal>integration-test</goal>
+                 <goal>verify</goal>
+               </goals>
             </execution>
           </executions>
         </plugin>


[4/7] PHOENIX-982 Avoid spinning up and tearing down mini cluster in tests (SamarthJain)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 fc1c4d5..867eb2a 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
@@ -22,7 +22,6 @@ 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.MILLIS_IN_DAY;
 import static org.apache.phoenix.util.TestUtil.ROW1;
 import static org.apache.phoenix.util.TestUtil.ROW2;
 import static org.apache.phoenix.util.TestUtil.ROW3;
@@ -39,20 +38,16 @@ import static org.junit.Assert.assertNotEquals;
 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;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
-import java.sql.Statement;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -76,6 +71,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
@@ -83,8 +79,6 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Doubles;
-import com.google.common.primitives.Floats;
 
 
 
@@ -95,13 +89,16 @@ import com.google.common.primitives.Floats;
  * 
  * @since 0.1
  */
+
+@Category(ClientManagedTimeTest.class)
 @RunWith(Parameterized.class)
 public class QueryIT extends BaseClientManagedTimeIT {
-    private static final String tenantId = getOrganizationId();
-    private static final String ATABLE_INDEX_NAME = "ATABLE_IDX";
-    private static final long BATCH_SIZE = 3;
+    protected static final String tenantId = getOrganizationId();
+    protected static final String ATABLE_INDEX_NAME = "ATABLE_IDX";
+    protected static final long BATCH_SIZE = 3;
     
     @BeforeClass
+    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
         int targetQueryConcurrency = 2;
         int maxQueryConcurrency = 3;
@@ -114,10 +111,10 @@ public class QueryIT extends BaseClientManagedTimeIT {
         props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB, Long.toString(BATCH_SIZE));
         
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
     
-    private long ts;
+    protected long ts;
     private Date date;
     private String indexDDL;
     
@@ -150,7 +147,7 @@ public class QueryIT extends BaseClientManagedTimeIT {
         return testCases;
     }
     
-    private void assertValueEqualsResultSet(ResultSet rs, List<Object> expectedResults) throws SQLException {
+    protected void assertValueEqualsResultSet(ResultSet rs, List<Object> expectedResults) throws SQLException {
         List<List<Object>> nestedExpectedResults = Lists.newArrayListWithExpectedSize(expectedResults.size());
         for (Object expectedResult : expectedResults) {
             nestedExpectedResults.add(Arrays.asList(expectedResult));
@@ -163,7 +160,7 @@ public class QueryIT extends BaseClientManagedTimeIT {
      * have an order by and we're going through indexes, but we assert that each expected result occurs once as
      * expected (in any order).
      */
-    private void assertValuesEqualsResultSet(ResultSet rs, List<List<Object>> expectedResults) throws SQLException {
+    protected void assertValuesEqualsResultSet(ResultSet rs, List<List<Object>> expectedResults) throws SQLException {
         int expectedCount = expectedResults.size();
         int count = 0;
         List<List<Object>> actualResults = Lists.newArrayList();
@@ -183,7 +180,7 @@ public class QueryIT extends BaseClientManagedTimeIT {
         assertEquals(count, expectedCount);
     }
     
-    private void assertOneOfValuesEqualsResultSet(ResultSet rs, List<List<Object>>... expectedResultsArray) throws SQLException {
+    protected void assertOneOfValuesEqualsResultSet(ResultSet rs, List<List<Object>>... expectedResultsArray) throws SQLException {
         List<List<Object>> results = Lists.newArrayList();
         while (rs.next()) {
             List<Object> result = Lists.newArrayList();
@@ -268,9 +265,10 @@ public class QueryIT extends BaseClientManagedTimeIT {
         testNoStringValue("");
     }
 
+    
     @Test
-    public void testScan() throws Exception {
-        String query = "SELECT a_string, /* comment ok? */ b_string FROM aTable WHERE ?=organization_id and 5=a_integer";
+    public void testGroupByPlusOne() throws Exception {
+        String query = "SELECT a_integer+1 FROM aTable WHERE organization_id=? and a_integer = 5 GROUP BY a_integer+1";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -279,17 +277,18 @@ public class QueryIT extends BaseClientManagedTimeIT {
             statement.setString(1, tenantId);
             ResultSet rs = statement.executeQuery();
             assertTrue (rs.next());
-            assertEquals(rs.getString(1), B_VALUE);
-            assertEquals(rs.getString("B_string"), C_VALUE);
+            assertEquals(6, rs.getInt(1));
             assertFalse(rs.next());
         } finally {
             conn.close();
         }
     }
     
+    
+
     @Test
-    public void testScanByByteValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_byte FROM aTable WHERE ?=organization_id and 1=a_byte";
+    public void testToDateOnString() throws Exception { // TODO: test more conversion combinations
+        String query = "SELECT a_string FROM aTable WHERE organization_id=? and a_integer = 5";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -298,18 +297,18 @@ public class QueryIT extends BaseClientManagedTimeIT {
             statement.setString(1, tenantId);
             ResultSet rs = statement.executeQuery();
             assertTrue (rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getString("B_string"), B_VALUE);
-            assertEquals(rs.getByte(3), 1);
-            assertFalse(rs.next());
+            rs.getDate(1);
+            fail();
+        } catch (ConstraintViolationException e) { // Expected
         } finally {
             conn.close();
         }
     }
+
     
     @Test
-    public void testScanByShortValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_short FROM aTable WHERE ?=organization_id and 128=a_short";
+    public void testColumnOnBothSides() throws Exception {
+        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_string = b_string";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -318,72 +317,186 @@ public class QueryIT extends BaseClientManagedTimeIT {
             statement.setString(1, tenantId);
             ResultSet rs = statement.executeQuery();
             assertTrue (rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getString("B_string"), B_VALUE);
-            assertEquals(rs.getShort("a_short"), 128);
+            assertEquals(rs.getString(1), ROW7);
             assertFalse(rs.next());
         } finally {
             conn.close();
         }
     }
-    
-    @Test
-    public void testScanByFloatValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_float FROM aTable WHERE ?=organization_id and ?=a_float";
+
+    public void testNoStringValue(String value) throws Exception {
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1);
         Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        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();
+        
+        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 + 2));
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             PreparedStatement statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
-            statement.setFloat(2, 0.01f);
             ResultSet rs = statement.executeQuery();
             assertTrue (rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getString("B_string"), B_VALUE);
-            assertTrue(Floats.compare(rs.getFloat(3), 0.01f) == 0);
+            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 testScanByUnsignedFloatValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_unsigned_float FROM aTable WHERE ?=organization_id and ?=a_unsigned_float";
+    public void testPointInTimeScan() 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 = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        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
+        upsertConn.close();
+
+        // Override value again, but should be ignored since it's past the SCN
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3); // Run query at timestamp 5
+        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 + 2));
         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 = new Properties(TEST_PROPERTIES);
+        Connection conn;
+        ResultSet rs;
+
+        props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+5));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.createStatement().execute("CREATE SEQUENCE s");
+        
         try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setFloat(2, 0.01f);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getString("B_string"), B_VALUE);
-            assertTrue(Floats.compare(rs.getFloat(3), 0.01f) == 0);
-            assertFalse(rs.next());
-        } finally {
+            conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+            fail();
+        } catch (SequenceNotFoundException e) {
             conn.close();
         }
+        
+        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");
+        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");
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+        conn.close();
+        
+        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");
+        assertTrue(rs.next());
+        assertEquals(3, rs.getInt(1));
+        conn.close();
+
+        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");
+            fail();
+        } catch (SequenceNotFoundException e) {
+            conn.close();            
+        }
+        
+        conn.createStatement().execute("CREATE SEQUENCE s");
+        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");
+        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");
+        assertTrue(rs.next());
+        assertEquals(4, rs.getInt(1));
+        conn.close();
     }
     
+
+    
+
+    private static boolean compare(CompareOp op, ImmutableBytesWritable lhsOutPtr, ImmutableBytesWritable rhsOutPtr) {
+        int compareResult = Bytes.compareTo(lhsOutPtr.get(), lhsOutPtr.getOffset(), lhsOutPtr.getLength(), rhsOutPtr.get(), rhsOutPtr.getOffset(), rhsOutPtr.getLength());
+        return ByteUtil.compare(op, compareResult);
+    }
+    
+    
     @Test
-    public void testScanByDoubleValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_double FROM aTable WHERE ?=organization_id and ?=a_double";
+    public void testDateInList() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE WHERE a_date IN (?,?) AND a_integer < 4";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = DriverManager.getConnection(url, props);
         try {
             PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setDouble(2, 0.0001);
+            statement.setDate(1, new Date(0));
+            statement.setDate(2, date);
             ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getString("B_string"), B_VALUE);
-            assertTrue(Doubles.compare(rs.getDouble(3), 0.0001) == 0);
+            assertTrue(rs.next());
+            assertEquals(ROW1, rs.getString(1));
             assertFalse(rs.next());
         } finally {
             conn.close();
@@ -391,20 +504,61 @@ public class QueryIT extends BaseClientManagedTimeIT {
     }
     
     @Test
-    public void testScanByUnsigned_DoubleValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_unsigned_double FROM aTable WHERE ?=organization_id and ?=a_unsigned_double";
+    public void testTimestamp() throws Exception {
+        String updateStmt = 
+            "upsert into " +
+            "ATABLE(" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_TIMESTAMP) " +
+            "VALUES (?, ?, ?)";
+        // Override value that was set at creation time
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1);
         Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        PreparedStatement stmt = upsertConn.prepareStatement(updateStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW4);
+        Timestamp tsValue1 = new Timestamp(5000);
+        byte[] ts1 = PDataType.TIMESTAMP.toBytes(tsValue1);
+        stmt.setTimestamp(3, tsValue1);
+        stmt.execute();
+
+        updateStmt = 
+            "upsert into " +
+            "ATABLE(" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_TIMESTAMP," +
+            "    A_TIME) " +
+            "VALUES (?, ?, ?, ?)";
+        stmt = upsertConn.prepareStatement(updateStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW5);
+        Timestamp tsValue2 = new Timestamp(5000);
+        tsValue2.setNanos(200);
+        byte[] ts2 = PDataType.TIMESTAMP.toBytes(tsValue2);
+        stmt.setTimestamp(3, tsValue2);
+        stmt.setTime(4, new Time(tsValue2.getTime()));
+        stmt.execute();
+        upsertConn.close();
+        
+        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 > ?";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 3)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             PreparedStatement statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
-            statement.setDouble(2, 0.0001);
+            statement.setTimestamp(2, new Timestamp(5000));
             ResultSet rs = statement.executeQuery();
             assertTrue (rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getString("B_string"), B_VALUE);
-            assertTrue(Doubles.compare(rs.getDouble(3), 0.0001) == 0);
+            assertEquals(rs.getString(1), ROW5);
+            assertEquals(rs.getTimestamp("A_TIMESTAMP"), tsValue2);
+            assertEquals(rs.getTime("A_TIME"), new Time(tsValue2.getTime()));
             assertFalse(rs.next());
         } finally {
             conn.close();
@@ -412,2041 +566,71 @@ public class QueryIT extends BaseClientManagedTimeIT {
     }
     
     @Test
-    public void testAllScan() throws Exception {
-        String query = "SELECT ALL a_string, b_string FROM aTable WHERE ?=organization_id and 5=a_integer";
+    public void testSimpleInListStatement() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE 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 = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = DriverManager.getConnection(url, props);
         try {
             PreparedStatement statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), B_VALUE);
-            assertEquals(rs.getString("B_string"), C_VALUE);
-            assertFalse(rs.next());
+            assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW2, ROW4));
         } finally {
             conn.close();
         }
     }
     
     @Test
-    public void testDistinctScan() throws Exception {
-        String query = "SELECT DISTINCT a_string FROM aTable WHERE organization_id=?";
+    public void testPartiallyQualifiedRVCInList() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE 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 = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = DriverManager.getConnection(url, props);
         try {
             PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
             ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), B_VALUE);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), C_VALUE);
-            assertFalse(rs.next());
+            assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW2, ROW5));
         } finally {
             conn.close();
         }
     }
-
+    
     @Test
-    public void testDistinctLimitScan() throws Exception {
-        String query = "SELECT DISTINCT a_string FROM aTable WHERE organization_id=? LIMIT 1";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testInListSkipScan() throws Exception {
-        String query = "SELECT entity_id, b_string FROM aTable WHERE organization_id=? and entity_id IN (?,?)";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setString(2, ROW2);
-            statement.setString(3, ROW4);
-            ResultSet rs = statement.executeQuery();
-            Set<String> expectedvals = new HashSet<String>();
-            expectedvals.add(ROW2+"_"+C_VALUE);
-            expectedvals.add(ROW4+"_"+B_VALUE);
-            Set<String> vals = new HashSet<String>();
-            assertTrue (rs.next());
-            vals.add(rs.getString(1) + "_" + rs.getString(2));
-            assertTrue (rs.next());
-            vals.add(rs.getString(1) + "_" + rs.getString(2));
-            assertFalse(rs.next());
-            assertEquals(expectedvals, vals);
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testNotInList() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and entity_id NOT IN (?,?,?,?,?,?)";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setString(2, ROW2);
-            statement.setString(3, ROW4);
-            statement.setString(4, ROW1);
-            statement.setString(5, ROW5);
-            statement.setString(6, ROW7);
-            statement.setString(7, ROW8);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(ROW3, rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW6, rs.getString(1));
-            assertTrue (rs.next());
-            assertEquals(ROW9, rs.getString(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testNotInListOfFloat() throws Exception {
-        String query = "SELECT a_float FROM aTable WHERE organization_id=? and a_float NOT IN (?,?,?,?,?,?)";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setFloat(2, 0.01f);
-            statement.setFloat(3, 0.02f);
-            statement.setFloat(4, 0.03f);
-            statement.setFloat(5, 0.04f);
-            statement.setFloat(6, 0.05f);
-            statement.setFloat(7, 0.06f);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertTrue(Doubles.compare(rs.getDouble(1), 0.07f)==0);
-            assertTrue (rs.next());
-            assertTrue(Doubles.compare(rs.getDouble(1), 0.08f)==0);
-            assertTrue (rs.next());
-            assertTrue(Doubles.compare(rs.getDouble(1), 0.09f)==0);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testNotInListOfDouble() throws Exception {
-        String query = "SELECT a_double FROM aTable WHERE organization_id=? and a_double NOT IN (?,?,?,?,?,?)";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setDouble(2, 0.0001);
-            statement.setDouble(3, 0.0002);
-            statement.setDouble(4, 0.0003);
-            statement.setDouble(5, 0.0004);
-            statement.setDouble(6, 0.0005);
-            statement.setDouble(7, 0.0006);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertTrue(Doubles.compare(rs.getDouble(1), 0.0007)==0);
-            assertTrue (rs.next());
-            assertTrue(Doubles.compare(rs.getDouble(1), 0.0008)==0);
-            assertTrue (rs.next());
-            assertTrue(Doubles.compare(rs.getDouble(1), 0.0009)==0);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testGroupByPlusOne() throws Exception {
-        String query = "SELECT a_integer+1 FROM aTable WHERE organization_id=? and a_integer = 5 GROUP BY a_integer+1";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(6, rs.getInt(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testGroupByCondition() throws Exception {
-        Properties props = new Properties(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 aTable 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 aTable 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 aTable(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 aTable 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 aTable 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 aTable 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 testNoWhereScan() throws Exception {
-        String query = "SELECT y_integer FROM aTable";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            ResultSet rs = statement.executeQuery();
-            for (int i =0; i < 8; i++) {
-                assertTrue (rs.next());
-                assertEquals(0, rs.getInt(1));
-                assertTrue(rs.wasNull());
-            }
-            assertTrue (rs.next());
-            assertEquals(300, rs.getInt(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @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";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            rs.getDate(1);
-            fail();
-        } catch (ConstraintViolationException e) { // Expected
-        } finally {
-            conn.close();
-        }
-    }
-
-    @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";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW2);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @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";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getByte(1), 2);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @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";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getShort(1), 129);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testNotEqualsByFloat() throws Exception {
-        String query = "SELECT a_float -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_float != 0.01d and a_float <= 0.02d";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertTrue(Floats.compare(rs.getFloat(1), 0.02f) == 0);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @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.01d and a_unsigned_float <= 0.02d";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertTrue(Floats.compare(rs.getFloat(1), 0.02f) == 0);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testNotEqualsByDouble() throws Exception {
-        String query = "SELECT a_double -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_double != 0.0001d and a_double <= 0.0002d";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertTrue(Doubles.compare(rs.getDouble(1), 0.0002) == 0);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @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.0001d and a_unsigned_double <= 0.0002d";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertTrue(Doubles.compare(rs.getDouble(1), 0.0002) == 0);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @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";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW2);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testColumnOnBothSides() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_string = b_string";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW7);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    public void testNoStringValue(String value) throws Exception {
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1);
-        Properties props = new Properties(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();
-        
-        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 + 2));
-        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 testPointInTimeScan() 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 = new Properties(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
-        upsertConn.close();
-
-        // Override value again, but should be ignored since it's past the SCN
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3); // Run query at timestamp 5
-        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 + 2));
-        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 = new Properties(TEST_PROPERTIES);
-        Connection conn;
-        ResultSet rs;
-
-        props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+5));
-        conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE SEQUENCE s");
-        
-        try {
-            conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
-            fail();
-        } catch (SequenceNotFoundException e) {
-            conn.close();
-        }
-        
-        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");
-        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");
-        assertTrue(rs.next());
-        assertEquals(2, rs.getInt(1));
-        conn.close();
-        
-        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");
-        assertTrue(rs.next());
-        assertEquals(3, rs.getInt(1));
-        conn.close();
-
-        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");
-            fail();
-        } catch (SequenceNotFoundException e) {
-            conn.close();            
-        }
-        
-        conn.createStatement().execute("CREATE SEQUENCE s");
-        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");
-        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");
-        assertTrue(rs.next());
-        assertEquals(4, rs.getInt(1));
-        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 = new Properties(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, 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 atable 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();
-    }
-
-    @Test
-    public void testUpperLowerBoundRangeScan() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) > '00A' and substr(entity_id,1,3) < '00C'";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW5);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW6);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW7);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW8);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testUpperBoundRangeScan() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) >= '00B' ";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW5);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW6);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW7);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW8);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW9);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testLowerBoundRangeScan() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) < '00B' ";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW1);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW2);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW3);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW4);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testUnboundRangeScan1() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id <= ?";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW1);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW2);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW3);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW4);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW5);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW6);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW7);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW8);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW9);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testUnboundRangeScan2() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id >= ?";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW1);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW2);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW3);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW4);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW5);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW6);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW7);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW8);
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW9);
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    // FIXME: this is flapping with an phoenix.memory.InsufficientMemoryException
-    // in the GroupedAggregateRegionObserver. We can work around it by increasing
-    // the amount of available memory in QueryServicesTestImpl, but we shouldn't
-    // have to. I think something may no be being closed to reclaim the memory.
-    @Test
-    public void testGroupedAggregation() throws Exception {
-        // Tests that you don't get an ambiguous column exception when using the same alias as the column name
-        String query = "SELECT a_string as a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getLong(2), 4L);
-            assertEquals(rs.getString(3), "foo");
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), B_VALUE);
-            assertEquals(rs.getLong(2), 4L);
-            assertEquals(rs.getString(3), "foo");
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), C_VALUE);
-            assertEquals(rs.getLong(2), 1L);
-            assertEquals(rs.getString(3), "foo");
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testDistinctGroupedAggregation() throws Exception {
-        String query = "SELECT DISTINCT a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string, b_string ORDER BY a_string, count(1)";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getLong(2), 1L);
-            assertEquals(rs.getString(3), "foo");
-            
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getLong(2), 2L);
-            assertEquals(rs.getString(3), "foo");
-            
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), B_VALUE);
-            assertEquals(rs.getLong(2), 1L);
-            assertEquals(rs.getString(3), "foo");
-            
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), B_VALUE);
-            assertEquals(rs.getLong(2), 2L);
-            assertEquals(rs.getString(3), "foo");
-            
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), C_VALUE);
-            assertEquals(rs.getLong(2), 1L);
-            assertEquals(rs.getString(3), "foo");
-            
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testDistinctLimitedGroupedAggregation() throws Exception {
-        String query = "SELECT /*+ NO_INDEX */ DISTINCT a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string, b_string ORDER BY count(1) desc,a_string LIMIT 2";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-
-            /*
-            List<List<Object>> expectedResultsA = Lists.newArrayList(
-                    Arrays.<Object>asList(A_VALUE, 2L, "foo"),
-                    Arrays.<Object>asList(B_VALUE, 2L, "foo"));
-            List<List<Object>> expectedResultsB = Lists.newArrayList(expectedResultsA);
-            Collections.reverse(expectedResultsB);
-            // 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);
-            */
-
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(rs.getLong(2), 2L);
-            assertEquals(rs.getString(3), "foo");
-            
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), B_VALUE);
-            assertEquals(rs.getLong(2), 2L);
-            assertEquals(rs.getString(3), "foo");
-            
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testDistinctUngroupedAggregation() throws Exception {
-        String query = "SELECT DISTINCT count(1), 'foo' FROM atable WHERE organization_id=?";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            
-            assertTrue(rs.next());
-            assertEquals(9L, rs.getLong(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testGroupedLimitedAggregation() throws Exception {
-        String query = "SELECT a_string, count(1) FROM atable WHERE organization_id=? GROUP BY a_string LIMIT 2";
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), A_VALUE);
-            assertEquals(4L, rs.getLong(2));
-            assertTrue(rs.next());
-            assertEquals(rs.getString(1), B_VALUE);
-            assertEquals(4L, rs.getLong(2));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testPointInTimeGroupedAggregation() throws Exception {
-        String updateStmt = 
-            "upsert into " +
-            "ATABLE VALUES ('" + tenantId + "','" + ROW5 + "','" + C_VALUE +"')";
-        // Override value that was set at creation time
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        Statement stmt = upsertConn.createStatement();
-        stmt.execute(updateStmt); // 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); // Run query at timestamp 5
-        upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        updateStmt = 
-            "upsert into " +
-            "ATABLE VALUES (?, ?, ?)";
-        // Insert all rows at ts
-        PreparedStatement pstmt = upsertConn.prepareStatement(updateStmt);
-        pstmt.setString(1, tenantId);
-        pstmt.setString(2, ROW5);
-        pstmt.setString(3, E_VALUE);
-        pstmt.execute(); // should commit too
-        upsertConn.close();
-        
-        String query = "SELECT a_string, count(1) FROM atable WHERE organization_id='" + tenantId + "' GROUP BY a_string";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        Statement statement = conn.createStatement();
-        ResultSet rs = statement.executeQuery(query);
-        assertTrue(rs.next());
-        assertEquals(A_VALUE, rs.getString(1));
-        assertEquals(4, rs.getInt(2));
-        assertTrue(rs.next());
-        assertEquals(B_VALUE, rs.getString(1));
-        assertEquals(3, rs.getLong(2));
-        assertTrue(rs.next());
-        assertEquals(C_VALUE, rs.getString(1));
-        assertEquals(2, rs.getInt(2));
-        assertFalse(rs.next());
-        conn.close();
-    }
-
-    @Test
-    public void testUngroupedAggregation() throws Exception {
-        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ?";
-        String url = getUrl();
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5)); // Execute query at ts + 5
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setString(2, B_VALUE);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(4, rs.getLong(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-        // Run again to catch unintentianal deletion of rows during an ungrouped aggregation (W-1455633)
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 6)); // Execute at ts + 6
-        conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setString(2, B_VALUE);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(4, rs.getLong(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testUngroupedAggregationNoWhere() throws Exception {
-        String query = "SELECT count(*) FROM atable";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(9, rs.getLong(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testPointInTimeUngroupedAggregation() 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 = new Properties(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
-            "    ORGANIZATION_ID, " +
-            "    ENTITY_ID, " +
-            "    A_STRING) " +
-            "VALUES (?, ?, ?)";
-        // Insert all rows at ts
-        PreparedStatement stmt = upsertConn.prepareStatement(updateStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW5);
-        stmt.setString(3, null);
-        stmt.execute();
-        stmt.setString(3, C_VALUE);
-        stmt.execute();
-        stmt.setString(2, ROW7);
-        stmt.setString(3, E_VALUE);
-        stmt.execute();
-        upsertConn.commit();
-        upsertConn.close();
-
-        // Override value again, but should be ignored since it's past the SCN
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3); // Run query at timestamp 5
-        upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        stmt = upsertConn.prepareStatement(updateStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW6);
-        stmt.setString(3, E_VALUE);
-        stmt.execute();
-        upsertConn.close();
-        
-        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ?";
-        // Specify CurrentSCN on URL with extra stuff afterwards (which should be ignored)
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 2) + ";foo=bar"; // Run query at timestamp 2 
-        Connection conn = DriverManager.getConnection(url, 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 testPointInTimeUngroupedLimitedAggregation() throws Exception {
-        String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
-            "    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 = new Properties(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        PreparedStatement stmt = upsertConn.prepareStatement(updateStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW6);
-        stmt.setString(3, C_VALUE);
-        stmt.execute();
-        stmt.setString(3, E_VALUE);
-        stmt.execute();
-        stmt.setString(3, B_VALUE);
-        stmt.execute();
-        stmt.setString(3, B_VALUE);
-        stmt.execute();
-        upsertConn.close();
-
-        // Override value again, but should be ignored since it's past the SCN
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3); // Run query at timestamp 5
-        upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        stmt = upsertConn.prepareStatement(updateStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW6);
-        stmt.setString(3, E_VALUE);
-        stmt.execute();
-        upsertConn.close();
-
-        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ? LIMIT 3";
-        // Specify CurrentSCN on URL with extra stuff afterwards (which should be ignored)
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 2) + ";foo=bar"; // Run query at timestamp 2 
-        Connection conn = DriverManager.getConnection(url, props);
-        PreparedStatement statement = conn.prepareStatement(query);
-        statement.setString(1, tenantId);
-        statement.setString(2, B_VALUE);
-        ResultSet rs = statement.executeQuery();
-        assertTrue(rs.next());
-        assertEquals(4, rs.getLong(1)); // LIMIT applied at end, so all rows would be counted
-        assertFalse(rs.next());
-        conn.close();
-    }
-
-    @Test
-    public void testPointInTimeDeleteUngroupedAggregation() throws Exception {
-        String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
-            "    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 = new Properties(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 atable 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 atable 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 atable 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();
-    }
-
-
-    private static boolean compare(CompareOp op, ImmutableBytesWritable lhsOutPtr, ImmutableBytesWritable rhsOutPtr) {
-        int compareResult = Bytes.compareTo(lhsOutPtr.get(), lhsOutPtr.getOffset(), lhsOutPtr.getLength(), rhsOutPtr.get(), rhsOutPtr.getOffset(), rhsOutPtr.getLength());
-        return ByteUtil.compare(op, compareResult);
-    }
-    
-    @Test
-    public void testDateAdd() throws Exception {
-        String query = "SELECT entity_id, b_string FROM ATABLE WHERE a_date + 0.5d < ?";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setDate(1, new Date(System.currentTimeMillis() + MILLIS_IN_DAY));
-            ResultSet rs = statement.executeQuery();
-            @SuppressWarnings("unchecked")
-            List<List<Object>> expectedResults = Lists.newArrayList(
-                    Arrays.<Object>asList(ROW1, B_VALUE),
-                    Arrays.<Object>asList( ROW4, B_VALUE), 
-                    Arrays.<Object>asList(ROW7, B_VALUE));
-            assertValuesEqualsResultSet(rs, expectedResults);
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testDateInList() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE a_date IN (?,?) AND a_integer < 4";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setDate(1, new Date(0));
-            statement.setDate(2, date);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(ROW1, rs.getString(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testDateSubtract() throws Exception {
-        String query = "SELECT entity_id, b_string FROM ATABLE WHERE a_date - 0.5d > ?";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setDate(1, new Date(System.currentTimeMillis() + MILLIS_IN_DAY));
-            ResultSet rs = statement.executeQuery();
-            @SuppressWarnings("unchecked")
-            List<List<Object>> expectedResults = Lists.newArrayList(
-                    Arrays.<Object>asList(ROW3, E_VALUE),
-                    Arrays.<Object>asList( ROW6, E_VALUE), 
-                    Arrays.<Object>asList(ROW9, E_VALUE));
-            assertValuesEqualsResultSet(rs, expectedResults);
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testTimestamp() throws Exception {
-        String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
-            "    ORGANIZATION_ID, " +
-            "    ENTITY_ID, " +
-            "    A_TIMESTAMP) " +
-            "VALUES (?, ?, ?)";
-        // Override value that was set at creation time
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1);
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        PreparedStatement stmt = upsertConn.prepareStatement(updateStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW4);
-        Timestamp tsValue1 = new Timestamp(5000);
-        byte[] ts1 = PDataType.TIMESTAMP.toBytes(tsValue1);
-        stmt.setTimestamp(3, tsValue1);
-        stmt.execute();
-
-        updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
-            "    ORGANIZATION_ID, " +
-            "    ENTITY_ID, " +
-            "    A_TIMESTAMP," +
-            "    A_TIME) " +
-            "VALUES (?, ?, ?, ?)";
-        stmt = upsertConn.prepareStatement(updateStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW5);
-        Timestamp tsValue2 = new Timestamp(5000);
-        tsValue2.setNanos(200);
-        byte[] ts2 = PDataType.TIMESTAMP.toBytes(tsValue2);
-        stmt.setTimestamp(3, tsValue2);
-        stmt.setTime(4, new Time(tsValue2.getTime()));
-        stmt.execute();
-        upsertConn.close();
-        
-        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 > ?";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 3)); // Execute at timestamp 2
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            statement.setTimestamp(2, new Timestamp(5000));
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(rs.getString(1), ROW5);
-            assertEquals(rs.getTimestamp("A_TIMESTAMP"), tsValue2);
-            assertEquals(rs.getTime("A_TIME"), new Time(tsValue2.getTime()));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testCoerceTinyIntToSmallInt() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_byte >= a_short";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(ROW9, rs.getString(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testCoerceIntegerToLong() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_long >= x_integer";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(ROW7, rs.getString(1));
-            assertTrue(rs.next());
-            assertEquals(ROW9, rs.getString(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testCoerceLongToDecimal1() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_decimal > x_integer";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(ROW9, rs.getString(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testCoerceLongToDecimal2() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_integer <= x_decimal";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue(rs.next());
-            assertEquals(ROW9, rs.getString(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-    
-    @Test
-    public void testSimpleCaseStatement() throws Exception {
-        String query = "SELECT CASE a_integer WHEN 1 THEN 'a' WHEN 2 THEN 'b' WHEN 3 THEN 'c' ELSE 'd' END, entity_id AS a FROM ATABLE WHERE organization_id=? AND a_integer < 6";
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
-        Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            @SuppressWarnings("unchecked")
-            List<List<Object>> expectedResu

<TRUNCATED>

[2/7] PHOENIX-982 Avoid spinning up and tearing down mini cluster in tests (SamarthJain)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java
index 0fe696d..1d0dbfd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.hbase.index;
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -33,126 +34,129 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.VersionInfo;
-import org.apache.phoenix.util.ConfigUtil;
-import org.junit.Test;
-
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.hbase.index.covered.example.ColumnGroup;
 import org.apache.phoenix.hbase.index.covered.example.CoveredColumn;
 import org.apache.phoenix.hbase.index.covered.example.CoveredColumnIndexSpecifierBuilder;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Test that we correctly fail for versions of HBase that don't support current properties
  */
+@Category(NeedsOwnMiniClusterTest.class)
 public class FailForUnsupportedHBaseVersionsIT {
-  private static final Log LOG = LogFactory.getLog(FailForUnsupportedHBaseVersionsIT.class);
-
-  /**
-   * We don't support WAL Compression for HBase &lt; 0.94.9, so we shouldn't even allow the server
-   * to start if both indexing and WAL Compression are enabled for the wrong versions.
-   */
-  @Test
-  public void testDoesNotSupportCompressedWAL() {
-    Configuration conf = HBaseConfiguration.create();
-    IndexTestingUtils.setupConfig(conf);
-    // get the current version
-    String version = VersionInfo.getVersion();
-    
-    // ensure WAL Compression not enabled
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
-    
-    //we support all versions without WAL Compression
-    String supported = Indexer.validateVersion(version, conf);
-    assertNull(
-      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! All versions should"
-          + " support writing without a compressed WAL. Message: "+supported, supported);
-
-    // enable WAL Compression
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
-
-    // set the version to something we know isn't supported
-    version = "0.94.4";
-    supported = Indexer.validateVersion(version, conf);
-    assertNotNull("WAL Compression was enabled, but incorrectly marked version as supported",
-      supported);
-    
-    //make sure the first version of 0.94 that supports Indexing + WAL Compression works
-    version = "0.94.9";
-    supported = Indexer.validateVersion(version, conf);
-    assertNull(
-      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! Message: "+supported, supported);
-    
-    //make sure we support snapshot builds too
-    version = "0.94.9-SNAPSHOT";
-    supported = Indexer.validateVersion(version, conf);
-    assertNull(
-      "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! Message: "+supported, supported);
-  }
-
-  /**
-   * Test that we correctly abort a RegionServer when we run tests with an unsupported HBase
-   * version. The 'completeness' of this test requires that we run the test with both a version of
-   * HBase that wouldn't be supported with WAL Compression. Currently, this is the default version
-   * (0.94.4) so just running 'mvn test' will run the full test. However, this test will not fail
-   * when running against a version of HBase with WALCompression enabled. Therefore, to fully test
-   * this functionality, we need to run the test against both a supported and an unsupported version
-   * of HBase (as long as we want to support an version of HBase that doesn't support custom WAL
-   * Codecs).
-   * @throws Exception on failure
-   */
-  @Test(timeout = 300000 /* 5 mins */)
-  public void testDoesNotStartRegionServerForUnsupportedCompressionAndVersion() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    IndexTestingUtils.setupConfig(conf);
-    // enable WAL Compression
-    conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
-
-    // check the version to see if it isn't supported
-    String version = VersionInfo.getVersion();
-    boolean supported = false;
-    if (Indexer.validateVersion(version, conf) == null) {
-      supported = true;
+    private static final Log LOG = LogFactory.getLog(FailForUnsupportedHBaseVersionsIT.class);
+
+    /**
+     * We don't support WAL Compression for HBase &lt; 0.94.9, so we shouldn't even allow the server
+     * to start if both indexing and WAL Compression are enabled for the wrong versions.
+     */
+    @Test
+    public void testDoesNotSupportCompressedWAL() {
+        Configuration conf = HBaseConfiguration.create();
+        IndexTestingUtils.setupConfig(conf);
+        // get the current version
+        String version = VersionInfo.getVersion();
+
+        // ensure WAL Compression not enabled
+        conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
+
+        //we support all versions without WAL Compression
+        String supported = Indexer.validateVersion(version, conf);
+        assertNull(
+                "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! All versions should"
+                        + " support writing without a compressed WAL. Message: "+supported, supported);
+
+        // enable WAL Compression
+        conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+
+        // set the version to something we know isn't supported
+        version = "0.94.4";
+        supported = Indexer.validateVersion(version, conf);
+        assertNotNull("WAL Compression was enabled, but incorrectly marked version as supported",
+                supported);
+
+        //make sure the first version of 0.94 that supports Indexing + WAL Compression works
+        version = "0.94.9";
+        supported = Indexer.validateVersion(version, conf);
+        assertNull(
+                "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! Message: "+supported, supported);
+
+        //make sure we support snapshot builds too
+        version = "0.94.9-SNAPSHOT";
+        supported = Indexer.validateVersion(version, conf);
+        assertNull(
+                "WAL Compression wasn't enabled, but version "+version+" of HBase wasn't supported! Message: "+supported, supported);
     }
 
-    // start the minicluster
-    HBaseTestingUtility util = new HBaseTestingUtility(conf);
-    // set replication required parameter
-    ConfigUtil.setReplicationConfigIfAbsent(conf);
-    util.startMiniCluster();
-
-    // setup the primary table
-    HTableDescriptor desc = new HTableDescriptor(
-        "testDoesNotStartRegionServerForUnsupportedCompressionAndVersion");
-    byte[] family = Bytes.toBytes("f");
-    desc.addFamily(new HColumnDescriptor(family));
-
-    // enable indexing to a non-existant index table
-    String indexTableName = "INDEX_TABLE";
-    ColumnGroup fam1 = new ColumnGroup(indexTableName);
-    fam1.add(new CoveredColumn(family, CoveredColumn.ALL_QUALIFIERS));
-    CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
-    builder.addIndexGroup(fam1);
-    builder.build(desc);
-
-    // get a reference to the regionserver, so we can ensure it aborts
-    HRegionServer server = util.getMiniHBaseCluster().getRegionServer(0);
-
-    // create the primary table
-    HBaseAdmin admin = util.getHBaseAdmin();
-    if (supported) {
-      admin.createTable(desc);
-      assertFalse("Hosting regeion server failed, even the HBase version (" + version
-          + ") supports WAL Compression.", server.isAborted());
-    } else {
-      admin.createTableAsync(desc, null);
-
-      // wait for the regionserver to abort - if this doesn't occur in the timeout, assume its
-      // broken.
-      while (!server.isAborted()) {
-        LOG.debug("Waiting on regionserver to abort..");
-      }
+    /**
+     * Test that we correctly abort a RegionServer when we run tests with an unsupported HBase
+     * version. The 'completeness' of this test requires that we run the test with both a version of
+     * HBase that wouldn't be supported with WAL Compression. Currently, this is the default version
+     * (0.94.4) so just running 'mvn test' will run the full test. However, this test will not fail
+     * when running against a version of HBase with WALCompression enabled. Therefore, to fully test
+     * this functionality, we need to run the test against both a supported and an unsupported version
+     * of HBase (as long as we want to support an version of HBase that doesn't support custom WAL
+     * Codecs).
+     * @throws Exception on failure
+     */
+    @Test(timeout = 300000 /* 5 mins */)
+    public void testDoesNotStartRegionServerForUnsupportedCompressionAndVersion() throws Exception {
+        Configuration conf = HBaseConfiguration.create();
+        setUpConfigForMiniCluster(conf);
+        IndexTestingUtils.setupConfig(conf);
+        // enable WAL Compression
+        conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
+
+        // check the version to see if it isn't supported
+        String version = VersionInfo.getVersion();
+        boolean supported = false;
+        if (Indexer.validateVersion(version, conf) == null) {
+            supported = true;
+        }
+
+        // start the minicluster
+        HBaseTestingUtility util = new HBaseTestingUtility(conf);
+        util.startMiniCluster();
+
+        try {
+            // setup the primary table
+            HTableDescriptor desc = new HTableDescriptor(
+                    "testDoesNotStartRegionServerForUnsupportedCompressionAndVersion");
+            byte[] family = Bytes.toBytes("f");
+            desc.addFamily(new HColumnDescriptor(family));
+
+            // enable indexing to a non-existant index table
+            String indexTableName = "INDEX_TABLE";
+            ColumnGroup fam1 = new ColumnGroup(indexTableName);
+            fam1.add(new CoveredColumn(family, CoveredColumn.ALL_QUALIFIERS));
+            CoveredColumnIndexSpecifierBuilder builder = new CoveredColumnIndexSpecifierBuilder();
+            builder.addIndexGroup(fam1);
+            builder.build(desc);
+
+            // get a reference to the regionserver, so we can ensure it aborts
+            HRegionServer server = util.getMiniHBaseCluster().getRegionServer(0);
+
+            // create the primary table
+            HBaseAdmin admin = util.getHBaseAdmin();
+            if (supported) {
+                admin.createTable(desc);
+                assertFalse("Hosting regeion server failed, even the HBase version (" + version
+                        + ") supports WAL Compression.", server.isAborted());
+            } else {
+                admin.createTableAsync(desc, null);
+
+                // wait for the regionserver to abort - if this doesn't occur in the timeout, assume its
+                // broken.
+                while (!server.isAborted()) {
+                    LOG.debug("Waiting on regionserver to abort..");
+                }
+            }
+
+        } finally {
+            // cleanup
+            util.shutdownMiniCluster();
+        }
     }
-
-    // cleanup
-    util.shutdownMiniCluster();
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
index 26515ec..fc134a3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.hbase.index.covered;
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -43,23 +44,24 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.phoenix.util.ConfigUtil;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.hbase.index.IndexTestingUtils;
 import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.hbase.index.TableName;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.scanner.Scanner;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * End-to-End test of just the {@link CoveredColumnsIndexBuilder}, but with a simple
  * {@link IndexCodec} and BatchCache implementation.
  */
+@Category(NeedsOwnMiniClusterTest.class)
 public class EndToEndCoveredColumnsIndexBuilderIT {
 
   public class TestState {
@@ -96,12 +98,11 @@ public class EndToEndCoveredColumnsIndexBuilderIT {
   @BeforeClass
   public static void setupCluster() throws Exception {
     Configuration conf = UTIL.getConfiguration();
+    setUpConfigForMiniCluster(conf);
     IndexTestingUtils.setupConfig(conf);
     // disable version checking, so we can test against whatever version of HBase happens to be
     // installed (right now, its generally going to be SNAPSHOT versions).
     conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-    // set replication required parameter
-    ConfigUtil.setReplicationConfigIfAbsent(conf);
     UTIL.startMiniCluster();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndToEndCoveredIndexingIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndToEndCoveredIndexingIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndToEndCoveredIndexingIT.java
index 0fb06f5..5196031 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndToEndCoveredIndexingIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndToEndCoveredIndexingIT.java
@@ -18,6 +18,8 @@
 package org.apache.phoenix.hbase.index.covered.example;
 
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -41,22 +43,23 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.phoenix.util.ConfigUtil;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
+import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.TableName;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
-
-import org.apache.phoenix.hbase.index.IndexTestingUtils;
-import org.apache.phoenix.hbase.index.Indexer;
-import org.apache.phoenix.hbase.index.TableName;
+import org.junit.experimental.categories.Category;
 
 /**
  * Test Covered Column indexing in an 'end-to-end' manner on a minicluster. This covers cases where
  * we manage custom timestamped updates that arrive in and out of order as well as just using the
  * generically timestamped updates.
  */
+@Category(NeedsOwnMiniClusterTest.class)
 public class EndToEndCoveredIndexingIT {
   private static final Log LOG = LogFactory.getLog(EndToEndCoveredIndexingIT.class);
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@@ -101,12 +104,11 @@ public class EndToEndCoveredIndexingIT {
   @BeforeClass
   public static void setupCluster() throws Exception {
     Configuration conf = UTIL.getConfiguration();
+    setUpConfigForMiniCluster(conf);
     IndexTestingUtils.setupConfig(conf);
     // disable version checking, so we can test against whatever version of HBase happens to be
     // installed (right now, its generally going to be SNAPSHOT versions).
     conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-    // set replication required parameter
-    ConfigUtil.setReplicationConfigIfAbsent(conf);
     UTIL.startMiniCluster();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndtoEndIndexingWithCompressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndtoEndIndexingWithCompressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndtoEndIndexingWithCompressionIT.java
index 4fc5c16..38aeb88 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndtoEndIndexingWithCompressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/EndtoEndIndexingWithCompressionIT.java
@@ -17,25 +17,30 @@
  */
 package org.apache.phoenix.hbase.index.covered.example;
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
-import org.apache.phoenix.util.ConfigUtil;
-import org.junit.BeforeClass;
-
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.hbase.index.IndexTestingUtils;
 import org.apache.phoenix.hbase.index.Indexer;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
 
 /**
  * Test secondary indexing from an end-to-end perspective (client to server to index table).
  */
+
+@Category(NeedsOwnMiniClusterTest.class)
 public class EndtoEndIndexingWithCompressionIT extends EndToEndCoveredIndexingIT {
 
   @BeforeClass
   public static void setupCluster() throws Exception {
     //add our codec and enable WAL compression
     Configuration conf = UTIL.getConfiguration();
+    setUpConfigForMiniCluster(conf);
     IndexTestingUtils.setupConfig(conf);
     // disable version checking, so we can test against whatever version of HBase happens to be
     // installed (right now, its generally going to be SNAPSHOT versions).
@@ -43,8 +48,6 @@ public class EndtoEndIndexingWithCompressionIT extends EndToEndCoveredIndexingIT
     conf.set(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY,
     IndexedWALEditCodec.class.getName());
     conf.setBoolean(HConstants.ENABLE_WAL_COMPRESSION, true);
-    // set replication required parameter
-    ConfigUtil.setReplicationConfigIfAbsent(conf);
     //start the mini-cluster
     UTIL.startMiniCluster();
   }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/FailWithoutRetriesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/FailWithoutRetriesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/FailWithoutRetriesIT.java
index 7310709..6a79e31 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/FailWithoutRetriesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/example/FailWithoutRetriesIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.hbase.index.covered.example;
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -34,24 +35,26 @@ import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.hbase.index.IndexTestingUtils;
+import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.hbase.index.TableName;
 import org.apache.phoenix.hbase.index.covered.IndexUpdate;
 import org.apache.phoenix.hbase.index.covered.TableState;
 import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 import org.apache.phoenix.index.BaseIndexCodec;
-import org.apache.phoenix.util.ConfigUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * If {@link DoNotRetryIOException} is not subclassed correctly (with the {@link String}
  * constructor), {@link MultiResponse#readFields(java.io.DataInput)} will not correctly deserialize
  * the exception, and just return <tt>null</tt> to the client, which then just goes and retries.
  */
+@Category(NeedsOwnMiniClusterTest.class)
 public class FailWithoutRetriesIT {
 
   private static final Log LOG = LogFactory.getLog(FailWithoutRetriesIT.class);
@@ -84,10 +87,9 @@ public class FailWithoutRetriesIT {
   public static void setupCluster() throws Exception {
     // setup and verify the config
     Configuration conf = UTIL.getConfiguration();
+    setUpConfigForMiniCluster(conf);
     IndexTestingUtils.setupConfig(conf);
     IndexManagementUtil.ensureMutableIndexingCorrectlyConfigured(conf);
-    // set replication required parameter
-    ConfigUtil.setReplicationConfigIfAbsent(conf);
     // start the cluster
     UTIL.startMiniCluster();
   }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
index 81ef78f..744b5d6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolIT.java
@@ -17,29 +17,32 @@
  */
 package org.apache.phoenix.mapreduce;
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.PrintWriter;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.jdbc.PhoenixDriver;
-import org.apache.phoenix.util.ConfigUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-import java.io.PrintWriter;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.Statement;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
+@Category(NeedsOwnMiniClusterTest.class)
 public class CsvBulkLoadToolIT {
 
     // We use HBaseTestUtil because we need to start up a MapReduce cluster as well
@@ -51,7 +54,7 @@ public class CsvBulkLoadToolIT {
     public static void setUp() throws Exception {
         hbaseTestUtil = new HBaseTestingUtility();
         Configuration conf = hbaseTestUtil.getConfiguration();
-        ConfigUtil.setReplicationConfigIfAbsent(conf);
+        setUpConfigForMiniCluster(conf);
         hbaseTestUtil.startMiniCluster();
         hbaseTestUtil.startMiniMapReduceCluster();
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
index 7834d0e..3df23a6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
@@ -174,12 +174,15 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
                     SQLCloseables.closeAll(connectionQueryServices);
                 } finally {
                     // We know there's a services object if any connections were made
-                    services.getExecutor().shutdownNow();
+                    services.close();
                 }
             } finally {
+                //even if something wrong happened while closing services above, we still
+                //want to set it to null. Otherwise, we will end up having a possibly non-working
+                //services instance. 
+                services = null;
                 connectionQueryServices.clear();
             }
         }
-        services = null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java
index 1e8fec7..10e562b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/BaseQueryServicesImpl.java
@@ -69,6 +69,7 @@ public abstract class BaseQueryServicesImpl implements QueryServices {
 
     @Override
     public void close() {
+        executor.shutdown();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
index bdf4144..3c8a6df 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.hbase.index.write;
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -55,6 +56,7 @@ import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.hbase.index.IndexTestingUtils;
 import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.hbase.index.TableName;
@@ -70,6 +72,7 @@ import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Multimap;
 
@@ -78,6 +81,7 @@ import com.google.common.collect.Multimap;
  * region was present on the same server, we have to make a best effort to not kill the server for
  * not succeeding on index writes while the index region is coming up.
  */
+@Category(NeedsOwnMiniClusterTest.class)
 public class TestWALRecoveryCaching {
 
   private static final Log LOG = LogFactory.getLog(TestWALRecoveryCaching.class);
@@ -145,6 +149,7 @@ public class TestWALRecoveryCaching {
   public void testWaitsOnIndexRegionToReload() throws Exception {
     HBaseTestingUtility util = new HBaseTestingUtility();
     Configuration conf = util.getConfiguration();
+    setUpConfigForMiniCluster(conf);
 
     // setup other useful stats
     IndexTestingUtils.setupConfig(conf);

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixTestDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixTestDriver.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixTestDriver.java
index afc7143..7d0755f 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixTestDriver.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixTestDriver.java
@@ -20,6 +20,8 @@ package org.apache.phoenix.jdbc;
 import java.sql.SQLException;
 import java.util.Properties;
 
+import javax.annotation.concurrent.ThreadSafe;
+
 import org.apache.phoenix.end2end.ConnectionQueryServicesTestImpl;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.ConnectionlessQueryServicesImpl;
@@ -37,26 +39,27 @@ import org.apache.phoenix.util.ReadOnlyProps;
  * 
  * @since 0.1
  */
+@ThreadSafe
 public class PhoenixTestDriver extends PhoenixEmbeddedDriver {
-    private ConnectionQueryServices queryServices;
+    
+    private ConnectionQueryServices connectionQueryServices;
     private final ReadOnlyProps overrideProps;
-    private QueryServices services;
+    private final QueryServices queryServices;
 
     public PhoenixTestDriver() {
         this.overrideProps = ReadOnlyProps.EMPTY_PROPS;
+        queryServices = new QueryServicesTestImpl();
     }
 
     // For tests to override the default configuration
-    public PhoenixTestDriver(ReadOnlyProps overrideProps) {
-        this.overrideProps = overrideProps;
+    public PhoenixTestDriver(ReadOnlyProps props) {
+        overrideProps = props;
+        queryServices = new QueryServicesTestImpl(overrideProps);
     }
 
     @Override
-    public synchronized QueryServices getQueryServices() {
-        if (services == null) {
-            services = new QueryServicesTestImpl(overrideProps);
-        }
-        return services;
+    public QueryServices getQueryServices() {
+        return queryServices;
     }
 
     @Override
@@ -66,27 +69,22 @@ public class PhoenixTestDriver extends PhoenixEmbeddedDriver {
     }
 
     @Override // public for testing
-    public ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException {
-        if (queryServices != null) {
-            return queryServices;
-        }
-        QueryServices services = getQueryServices();
+    public synchronized ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException {
+        if (connectionQueryServices != null) { return connectionQueryServices; }
         ConnectionInfo connInfo = ConnectionInfo.create(url);
         if (connInfo.isConnectionless()) {
-            queryServices =  new ConnectionlessQueryServicesImpl(services);
+            connectionQueryServices = new ConnectionlessQueryServicesImpl(queryServices);
         } else {
-            queryServices =  new ConnectionQueryServicesTestImpl(services, connInfo);
+            connectionQueryServices = new ConnectionQueryServicesTestImpl(queryServices, connInfo);
         }
-        queryServices.init(url, info);
-        return queryServices;
+        connectionQueryServices.init(url, info);
+        return connectionQueryServices;
     }
     
     @Override
-    public void close() throws SQLException {
-        try {
-            queryServices.close();
-        } finally {
-            queryServices = null;
-        }
+    public synchronized void close() throws SQLException {
+        connectionQueryServices.close();
+        queryServices.close();
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
index e8a3716..6fa9c7f 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
@@ -29,6 +29,7 @@ import static org.apache.phoenix.util.TestUtil.MULTI_CF_NAME;
 import static org.apache.phoenix.util.TestUtil.PHOENIX_CONNECTIONLESS_JDBC_URL;
 import static org.apache.phoenix.util.TestUtil.PTSDB_NAME;
 import static org.apache.phoenix.util.TestUtil.TABLE_WITH_ARRAY;
+import static org.junit.Assert.assertTrue;
 
 import java.sql.DriverManager;
 import java.util.Properties;
@@ -36,11 +37,14 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.phoenix.jdbc.PhoenixTestDriver;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 
@@ -66,7 +70,28 @@ public class BaseConnectionlessQueryTest extends BaseTest {
     protected static String getUrl(String tenantId) {
         return getUrl() + ';' + TENANT_ID_ATTRIB + '=' + tenantId;
     }
-
+    
+    protected static PhoenixTestDriver driver;
+    
+    private static void startServer(String url) throws Exception {
+        //assertNull(driver);
+        // only load the test driver if we are testing locally - for integration tests, we want to
+        // test on a wider scale
+        if (PhoenixEmbeddedDriver.isTestUrl(url)) {
+            driver = initDriver(ReadOnlyProps.EMPTY_PROPS);
+            assertTrue(DriverManager.getDriver(url) == driver);
+            driver.connect(url, TestUtil.TEST_PROPERTIES);
+        }
+    }
+    
+    protected static synchronized PhoenixTestDriver initDriver(ReadOnlyProps props) throws Exception {
+        if (driver == null) {
+            driver = new PhoenixTestDriver(props);
+            DriverManager.registerDriver(driver);
+        }
+        return driver;
+    }
+    
     @BeforeClass
     public static void doSetup() throws Exception {
         startServer(getUrl());

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 6a76178..e105fbb 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -17,11 +17,29 @@
  */
 package org.apache.phoenix.query;
 
+import static org.apache.phoenix.util.PhoenixRuntime.CURRENT_SCN_ATTRIB;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
 import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
+import static org.apache.phoenix.util.TestUtil.A_VALUE;
 import static org.apache.phoenix.util.TestUtil.BTABLE_NAME;
+import static org.apache.phoenix.util.TestUtil.B_VALUE;
 import static org.apache.phoenix.util.TestUtil.CUSTOM_ENTITY_DATA_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.C_VALUE;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID1;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID2;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID3;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID4;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID5;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID6;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID7;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID8;
+import static org.apache.phoenix.util.TestUtil.ENTITYHISTID9;
 import static org.apache.phoenix.util.TestUtil.ENTITY_HISTORY_SALTED_TABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.ENTITY_HISTORY_TABLE_NAME;
+import static org.apache.phoenix.util.TestUtil.E_VALUE;
 import static org.apache.phoenix.util.TestUtil.FUNKY_NAME;
 import static org.apache.phoenix.util.TestUtil.GROUPBYTEST_NAME;
 import static org.apache.phoenix.util.TestUtil.HBASE_DYNAMIC_COLUMNS;
@@ -33,42 +51,104 @@ import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.KEYONLY_NAME;
+import static org.apache.phoenix.util.TestUtil.LOCALHOST;
 import static org.apache.phoenix.util.TestUtil.MDTEST_NAME;
+import static org.apache.phoenix.util.TestUtil.MILLIS_IN_DAY;
 import static org.apache.phoenix.util.TestUtil.MULTI_CF_NAME;
 import static org.apache.phoenix.util.TestUtil.MUTABLE_INDEX_DATA_TABLE;
+import static org.apache.phoenix.util.TestUtil.PARENTID1;
+import static org.apache.phoenix.util.TestUtil.PARENTID2;
+import static org.apache.phoenix.util.TestUtil.PARENTID3;
+import static org.apache.phoenix.util.TestUtil.PARENTID4;
+import static org.apache.phoenix.util.TestUtil.PARENTID5;
+import static org.apache.phoenix.util.TestUtil.PARENTID6;
+import static org.apache.phoenix.util.TestUtil.PARENTID7;
+import static org.apache.phoenix.util.TestUtil.PARENTID8;
+import static org.apache.phoenix.util.TestUtil.PARENTID9;
 import static org.apache.phoenix.util.TestUtil.PRODUCT_METRICS_NAME;
 import static org.apache.phoenix.util.TestUtil.PTSDB2_NAME;
 import static org.apache.phoenix.util.TestUtil.PTSDB3_NAME;
 import static org.apache.phoenix.util.TestUtil.PTSDB_NAME;
+import static org.apache.phoenix.util.TestUtil.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+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.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
 import static org.apache.phoenix.util.TestUtil.STABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.TABLE_WITH_ARRAY;
 import static org.apache.phoenix.util.TestUtil.TABLE_WITH_SALTING;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
 
+import java.math.BigDecimal;
+import java.sql.Array;
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
+import javax.annotation.Nonnull;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
+import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.ConfigUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.AfterClass;
+import org.junit.Assert;
 
 import com.google.common.collect.ImmutableMap;
 
+/**
+ * 
+ * Base class that contains all the methods needed by
+ * client-time and hbase-time managed tests.
+ * 
+ * For tests needing connectivity to a cluster, please use
+ * {@link BaseHBaseManagedTimeIT} or {@link BaseClientManagedTimeIT}. 
+ * 
+ * In the rare case when a test can't share the same mini cluster as the 
+ * ones used by {@link BaseHBaseManagedTimeIT} or {@link BaseClientManagedTimeIT}
+ * one could extend this class and spin up your own mini cluster. Please 
+ * make sure to shutdown the mini cluster in a method annotated by @AfterClass.  
+ *
+ */
 public abstract class BaseTest {
     private static final Map<String,String> tableDDLMap;
+    private static Logger logger = Logger.getLogger("BaseTest.class");
+    
     static {
         ImmutableMap.Builder<String,String> builder = ImmutableMap.builder();
         builder.put(ENTITY_HISTORY_TABLE_NAME,"create table " + ENTITY_HISTORY_TABLE_NAME +
@@ -350,79 +430,160 @@ public abstract class BaseTest {
                 "    loc_id char(5))");
         tableDDLMap = builder.build();
     }
-
+    
     private static final String ORG_ID = "00D300000000XHP";
-
-    protected static String getOrganizationId() {
-        return ORG_ID;
+    private static final int NUM_SLAVES_BASE = 1;
+    
+    protected static String getZKClientPort(Configuration conf) {
+        return conf.get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
     }
-
-    private static long timestamp;
-
-    public static long nextTimestamp() {
-        timestamp += 100;
-        return timestamp;
+    
+    /**
+     * Set up the test hbase cluster. 
+     * @return url to be used by clients to connect to the cluster.
+     */
+    protected static String setUpTestCluster(@Nonnull Configuration conf) {
+        boolean isDistributedCluster = isDistributedClusterModeEnabled(conf);
+        if (!isDistributedCluster) {
+            return initMiniCluster(conf);
+       } else {
+            return initClusterDistributedMode(conf);
+        }
     }
-
-    protected static PhoenixTestDriver driver;
-    private static int driverRefCount = 0;
-
-    protected static synchronized PhoenixTestDriver initDriver(ReadOnlyProps props) throws Exception {
-        if (driver == null) {
-            if (driverRefCount == 0) {
-                BaseTest.driver = new PhoenixTestDriver(props);
-                DriverManager.registerDriver(driver);
-                driverRefCount++;
-            }
+    
+    private static boolean isDistributedClusterModeEnabled(Configuration conf) {
+        boolean isDistributedCluster = false;
+        //check if the distributed mode was specified as a system property.
+        isDistributedCluster = Boolean.parseBoolean(System.getProperty(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER, "false"));
+        if (!isDistributedCluster) {
+           //fall back on hbase-default.xml or hbase-site.xml to check for distributed mode              
+           isDistributedCluster = conf.getBoolean(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER, false);
         }
-        return BaseTest.driver;
+        return isDistributedCluster;
     }
+    
+    /**
+     * Initialize the mini cluster using phoenix-test specific configuration.
+     * @return url to be used by clients to connect to the mini cluster.
+     */
+    private static String initMiniCluster(Configuration conf) {
+        setUpConfigForMiniCluster(conf);
+        final HBaseTestingUtility utility = new HBaseTestingUtility(conf);
+        try {
+            utility.startMiniCluster();
+            String clientPort = utility.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
 
-    // We need to deregister an already existing driver in order
-    // to register a new one. We need to create a new one so that
-    // we register the new one with the new Configuration instance.
-    // Otherwise, we get connection errors because the prior one
-    // is no longer associated with the miniCluster.
-    protected static synchronized boolean destroyDriver() {
-        if (driver != null) {
-            driverRefCount--;
-            if (driverRefCount == 0) {
-                try {
+            // add shutdown hook to kill the mini cluster
+            Runtime.getRuntime().addShutdownHook(new Thread() {
+                @Override
+                public void run() {
                     try {
-                        driver.close();
-                        return true;
-                    } finally {
-                        try {
-                            DriverManager.deregisterDriver(driver);
-                        } finally {
-                            driver = null;
-                        }
+                        utility.shutdownMiniCluster();
+                    } catch (Exception e) {
+                        logger.log(Level.WARNING, "Exception caught when shutting down mini cluster: " + e.getMessage());
                     }
-                } catch (SQLException e) {
                 }
-            }
+            });
+            return JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR + clientPort
+                    + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
+        } catch (Throwable t) {
+            throw new RuntimeException(t);
         }
-        return false;
+    }
+    
+    /**
+     * Initialize the cluster in distributed mode
+     * @return url to be used by clients to connect to the mini cluster.
+     */
+    private static String initClusterDistributedMode(Configuration conf) {
+        setTestConfigForDistribuedCluster(conf);
+        try {
+            IntegrationTestingUtility util =  new IntegrationTestingUtility(conf);
+            util.initializeCluster(NUM_SLAVES_BASE);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        return JDBC_PROTOCOL + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
+    }
+
+    private static void setTestConfigForDistribuedCluster(Configuration conf) {
+        setDefaultTestConfig(conf);
+    }
+    
+    private static void setDefaultTestConfig(Configuration conf) {
+        ConfigUtil.setReplicationConfigIfAbsent(conf);
+        QueryServicesOptions options = QueryServicesTestImpl.getDefaultTestServicesOptions();
+        for (Entry<String,String> entry : options.getProps()) {
+            conf.set(entry.getKey(), entry.getValue());
+        }
+        //no point doing sanity checks when running tests.
+        conf.setBoolean("hbase.table.sanity.checks", false);
+    }
+    
+    public static Configuration setUpConfigForMiniCluster(Configuration conf) {
+        assertNotNull(conf);
+        setDefaultTestConfig(conf);
+        /*
+         * The default configuration of mini cluster ends up spawning a lot of threads
+         * that are not really needed by phoenix for test purposes. Limiting these threads
+         * helps us in running several mini clusters at the same time without hitting 
+         * the threads limit imposed by the OS. 
+         */
+        conf.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 5);
+        conf.setInt(HConstants.REGION_SERVER_META_HANDLER_COUNT, 2);
+        conf.setInt(HConstants.MASTER_HANDLER_COUNT, 2);
+        conf.setInt("dfs.namenode.handler.count", 1);
+        conf.setInt("dfs.namenode.service.handler.count", 1);
+        conf.setInt("dfs.datanode.handler.count", 1);
+        conf.setInt("hadoop.http.max.threads", 1);
+        conf.setInt("ipc.server.read.threadpool.size", 2);
+        conf.setInt("ipc.server.handler.threadpool.size", 2);
+        conf.setInt("hbase.hconnection.threads.max", 2);
+        conf.setInt("hbase.hconnection.threads.core", 2);
+        conf.setInt("hbase.htable.threads.max", 2);
+        conf.setInt("hbase.regionserver.hlog.syncer.count", 2);
+        conf.setInt("hbase.hlog.asyncer.number", 2);
+        conf.setInt("hbase.assignment.zkevent.workers", 5);
+        conf.setInt("hbase.assignment.threads.max", 5);
+        return conf;
     }
 
-    protected static void startServer(String url, ReadOnlyProps props) throws Exception {
-        assertNull(BaseTest.driver);
-        // only load the test driver if we are testing locally - for integration tests, we want to
-        // test on a wider scale
-        if (PhoenixEmbeddedDriver.isTestUrl(url)) {
-            PhoenixTestDriver driver = initDriver(props);
-            assertTrue(DriverManager.getDriver(url) == driver);
-            driver.connect(url, TestUtil.TEST_PROPERTIES);
+    /**
+     * Create a {@link PhoenixTestDriver} and register it.
+     * @return an initialized and registered {@link PhoenixTestDriver} 
+     */
+    protected static PhoenixTestDriver initAndRegisterDriver(String url, ReadOnlyProps props) throws Exception {
+        PhoenixTestDriver driver = new PhoenixTestDriver(props);
+        DriverManager.registerDriver(driver);
+        Assert.assertTrue(DriverManager.getDriver(url) == driver);
+        driver.connect(url, TestUtil.TEST_PROPERTIES);
+        return driver;
+    }
+    
+    //Close and unregister the driver.
+    protected static boolean destroyDriver(PhoenixTestDriver driver) {
+        if (driver != null) {
+            try {
+                try {
+                    driver.close();
+                    return true;
+                } finally {
+                    DriverManager.deregisterDriver(driver);
+                }
+            } catch (Exception ignored) {}
         }
+        return false;
     }
     
-    protected static void startServer(String url) throws Exception {
-        startServer(url, ReadOnlyProps.EMPTY_PROPS);
+    protected static String getOrganizationId() {
+        return ORG_ID;
     }
 
-    @AfterClass
-    public static void stopServer() throws Exception {
-        assertTrue(destroyDriver());
+    private static long timestamp;
+
+    public static long nextTimestamp() {
+        timestamp += 100;
+        return timestamp;
     }
 
     protected static void ensureTableCreated(String url, String tableName) throws SQLException {
@@ -482,4 +643,673 @@ public abstract class BaseTest {
             conn.close();
         }
     }
+    
+    protected static byte[][] getDefaultSplits(String tenantId) {
+        return new byte[][] { 
+            Bytes.toBytes(tenantId + "00A"),
+            Bytes.toBytes(tenantId + "00B"),
+            Bytes.toBytes(tenantId + "00C"),
+            };
+    }
+    
+    protected static void deletePriorTables(long ts, String url) throws Exception {
+        deletePriorTables(ts, (String)null, url);
+    }
+    
+    protected static void deletePriorTables(long ts, String tenantId, String url) throws Exception {
+        Properties props = new Properties();
+        if (ts != HConstants.LATEST_TIMESTAMP) {
+            props.setProperty(CURRENT_SCN_ATTRIB, Long.toString(ts));
+        }
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            deletePriorTables(ts, conn, url);
+            deletePriorSequences(ts, conn);
+        }
+        finally {
+            conn.close();
+        }
+    }
+    
+    private static void deletePriorTables(long ts, Connection globalConn, String url) throws Exception {
+        DatabaseMetaData dbmd = globalConn.getMetaData();
+        PMetaData cache = globalConn.unwrap(PhoenixConnection.class).getMetaDataCache();
+        cache.getTables();
+        // Drop VIEWs first, as we don't allow a TABLE with views to be dropped
+        // Tables are sorted by TENANT_ID
+        List<String[]> tableTypesList = Arrays.asList(new String[] {PTableType.VIEW.toString()}, new String[] {PTableType.TABLE.toString()});
+        for (String[] tableTypes: tableTypesList) {
+            ResultSet rs = dbmd.getTables(null, null, null, tableTypes);
+            String lastTenantId = null;
+            Connection conn = globalConn;
+            while (rs.next()) {
+                String fullTableName = SchemaUtil.getEscapedTableName(
+                        rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM),
+                        rs.getString(PhoenixDatabaseMetaData.TABLE_NAME));
+                String ddl = "DROP " + rs.getString(PhoenixDatabaseMetaData.TABLE_TYPE) + " " + fullTableName;
+                String tenantId = rs.getString(1);
+                if (tenantId != null && !tenantId.equals(lastTenantId))  {
+                    if (lastTenantId != null) {
+                        conn.close();
+                    }
+                    // Open tenant-specific connection when we find a new one
+                    Properties props = new Properties(globalConn.getClientInfo());
+                    props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+                    conn = DriverManager.getConnection(url, props);
+                    lastTenantId = tenantId;
+                }
+                try {
+                    conn.createStatement().executeUpdate(ddl);
+                } catch (NewerTableAlreadyExistsException ex) {
+                    logger.info("Newer table " + fullTableName + " or its delete marker exists. Ignore current deletion");
+                } catch (TableNotFoundException ex) {
+                    logger.info("Table " + fullTableName + " is already deleted.");
+                }
+            }
+            if (lastTenantId != null) {
+                conn.close();
+            }
+        }
+    }
+    
+    private static void deletePriorSequences(long ts, Connection conn) throws Exception {
+        // TODO: drop tenant-specific sequences too
+        ResultSet rs = conn.createStatement().executeQuery("SELECT " 
+                + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + "," 
+                + PhoenixDatabaseMetaData.SEQUENCE_NAME 
+                + " FROM " + PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME);
+        while (rs.next()) {
+            try {
+                conn.createStatement().execute("DROP SEQUENCE " + SchemaUtil.getTableName(rs.getString(1), rs.getString(2)));
+            } catch (Exception e) {
+                //FIXME: see https://issues.apache.org/jira/browse/PHOENIX-973
+            }
+        }
+    }
+    
+    protected static void initSumDoubleValues(byte[][] splits, String url) throws Exception {
+        ensureTableCreated(url, "SumDoubleTest", splits);
+        Properties props = new Properties();
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            // Insert all rows at ts
+            PreparedStatement stmt = conn.prepareStatement(
+                    "upsert into " +
+                    "SumDoubleTest(" +
+                    "    id, " +
+                    "    d, " +
+                    "    f, " +
+                    "    ud, " +
+                    "    uf) " +
+                    "VALUES (?, ?, ?, ?, ?)");
+            stmt.setString(1, "1");
+            stmt.setDouble(2, 0.001);
+            stmt.setFloat(3, 0.01f);
+            stmt.setDouble(4, 0.001);
+            stmt.setFloat(5, 0.01f);
+            stmt.execute();
+                
+            stmt.setString(1, "2");
+            stmt.setDouble(2, 0.002);
+            stmt.setFloat(3, 0.02f);
+            stmt.setDouble(4, 0.002);
+            stmt.setFloat(5, 0.02f);
+            stmt.execute();
+                
+            stmt.setString(1, "3");
+            stmt.setDouble(2, 0.003);
+            stmt.setFloat(3, 0.03f);
+            stmt.setDouble(4, 0.003);
+            stmt.setFloat(5, 0.03f);
+            stmt.execute();
+                
+            stmt.setString(1, "4");
+            stmt.setDouble(2, 0.004);
+            stmt.setFloat(3, 0.04f);
+            stmt.setDouble(4, 0.004);
+            stmt.setFloat(5, 0.04f);
+            stmt.execute();
+                
+            stmt.setString(1, "5");
+            stmt.setDouble(2, 0.005);
+            stmt.setFloat(3, 0.05f);
+            stmt.setDouble(4, 0.005);
+            stmt.setFloat(5, 0.05f);
+            stmt.execute();
+                
+            conn.commit();
+        } finally {
+            conn.close();
+        }
+    }
+    
+    protected static void initATableValues(String tenantId, byte[][] splits, String url) throws Exception {
+        initATableValues(tenantId, splits, null, url);
+    }
+    
+    protected static void initATableValues(String tenantId, byte[][] splits, Date date, String url) throws Exception {
+        initATableValues(tenantId, splits, date, null, url);
+    }
+    
+    
+    
+    protected static void initATableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
+        if (ts == null) {
+            ensureTableCreated(url, ATABLE_NAME, splits);
+        } else {
+            ensureTableCreated(url, ATABLE_NAME, splits, ts-5);
+        }
+        
+        Properties props = new Properties();
+        if (ts != null) {
+            props.setProperty(CURRENT_SCN_ATTRIB, Long.toString(ts-3));
+        }
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            // Insert all rows at ts
+            PreparedStatement stmt = conn.prepareStatement(
+                    "upsert into " +
+                    "ATABLE(" +
+                    "    ORGANIZATION_ID, " +
+                    "    ENTITY_ID, " +
+                    "    A_STRING, " +
+                    "    B_STRING, " +
+                    "    A_INTEGER, " +
+                    "    A_DATE, " +
+                    "    X_DECIMAL, " +
+                    "    X_LONG, " +
+                    "    X_INTEGER," +
+                    "    Y_INTEGER," +
+                    "    A_BYTE," +
+                    "    A_SHORT," +
+                    "    A_FLOAT," +
+                    "    A_DOUBLE," +
+                    "    A_UNSIGNED_FLOAT," +
+                    "    A_UNSIGNED_DOUBLE)" +
+                    "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)");
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW1);
+            stmt.setString(3, A_VALUE);
+            stmt.setString(4, B_VALUE);
+            stmt.setInt(5, 1);
+            stmt.setDate(6, date);
+            stmt.setBigDecimal(7, null);
+            stmt.setNull(8, Types.BIGINT);
+            stmt.setNull(9, Types.INTEGER);
+            stmt.setNull(10, Types.INTEGER);
+            stmt.setByte(11, (byte)1);
+            stmt.setShort(12, (short) 128);
+            stmt.setFloat(13, 0.01f);
+            stmt.setDouble(14, 0.0001);
+            stmt.setFloat(15, 0.01f);
+            stmt.setDouble(16, 0.0001);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW2);
+            stmt.setString(3, A_VALUE);
+            stmt.setString(4, C_VALUE);
+            stmt.setInt(5, 2);
+            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 1));
+            stmt.setBigDecimal(7, null);
+            stmt.setNull(8, Types.BIGINT);
+            stmt.setNull(9, Types.INTEGER);
+            stmt.setNull(10, Types.INTEGER);
+            stmt.setByte(11, (byte)2);
+            stmt.setShort(12, (short) 129);
+            stmt.setFloat(13, 0.02f);
+            stmt.setDouble(14, 0.0002);
+            stmt.setFloat(15, 0.02f);
+            stmt.setDouble(16, 0.0002);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW3);
+            stmt.setString(3, A_VALUE);
+            stmt.setString(4, E_VALUE);
+            stmt.setInt(5, 3);
+            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 2));
+            stmt.setBigDecimal(7, null);
+            stmt.setNull(8, Types.BIGINT);
+            stmt.setNull(9, Types.INTEGER);
+            stmt.setNull(10, Types.INTEGER);
+            stmt.setByte(11, (byte)3);
+            stmt.setShort(12, (short) 130);
+            stmt.setFloat(13, 0.03f);
+            stmt.setDouble(14, 0.0003);
+            stmt.setFloat(15, 0.03f);
+            stmt.setDouble(16, 0.0003);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW4);
+            stmt.setString(3, A_VALUE);
+            stmt.setString(4, B_VALUE);
+            stmt.setInt(5, 4);
+            stmt.setDate(6, date == null ? null : date);
+            stmt.setBigDecimal(7, null);
+            stmt.setNull(8, Types.BIGINT);
+            stmt.setNull(9, Types.INTEGER);
+            stmt.setNull(10, Types.INTEGER);
+            stmt.setByte(11, (byte)4);
+            stmt.setShort(12, (short) 131);
+            stmt.setFloat(13, 0.04f);
+            stmt.setDouble(14, 0.0004);
+            stmt.setFloat(15, 0.04f);
+            stmt.setDouble(16, 0.0004);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW5);
+            stmt.setString(3, B_VALUE);
+            stmt.setString(4, C_VALUE);
+            stmt.setInt(5, 5);
+            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 1));
+            stmt.setBigDecimal(7, null);
+            stmt.setNull(8, Types.BIGINT);
+            stmt.setNull(9, Types.INTEGER);
+            stmt.setNull(10, Types.INTEGER);
+            stmt.setByte(11, (byte)5);
+            stmt.setShort(12, (short) 132);
+            stmt.setFloat(13, 0.05f);
+            stmt.setDouble(14, 0.0005);
+            stmt.setFloat(15, 0.05f);
+            stmt.setDouble(16, 0.0005);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW6);
+            stmt.setString(3, B_VALUE);
+            stmt.setString(4, E_VALUE);
+            stmt.setInt(5, 6);
+            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 2));
+            stmt.setBigDecimal(7, null);
+            stmt.setNull(8, Types.BIGINT);
+            stmt.setNull(9, Types.INTEGER);
+            stmt.setNull(10, Types.INTEGER);
+            stmt.setByte(11, (byte)6);
+            stmt.setShort(12, (short) 133);
+            stmt.setFloat(13, 0.06f);
+            stmt.setDouble(14, 0.0006);
+            stmt.setFloat(15, 0.06f);
+            stmt.setDouble(16, 0.0006);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW7);
+            stmt.setString(3, B_VALUE);
+            stmt.setString(4, B_VALUE);
+            stmt.setInt(5, 7);
+            stmt.setDate(6, date == null ? null : date);
+            stmt.setBigDecimal(7, BigDecimal.valueOf(0.1));
+            stmt.setLong(8, 5L);
+            stmt.setInt(9, 5);
+            stmt.setNull(10, Types.INTEGER);
+            stmt.setByte(11, (byte)7);
+            stmt.setShort(12, (short) 134);
+            stmt.setFloat(13, 0.07f);
+            stmt.setDouble(14, 0.0007);
+            stmt.setFloat(15, 0.07f);
+            stmt.setDouble(16, 0.0007);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW8);
+            stmt.setString(3, B_VALUE);
+            stmt.setString(4, C_VALUE);
+            stmt.setInt(5, 8);
+            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 1));
+            stmt.setBigDecimal(7, BigDecimal.valueOf(3.9));
+            long l = Integer.MIN_VALUE - 1L;
+            assert(l < Integer.MIN_VALUE);
+            stmt.setLong(8, l);
+            stmt.setInt(9, 4);
+            stmt.setNull(10, Types.INTEGER);
+            stmt.setByte(11, (byte)8);
+            stmt.setShort(12, (short) 135);
+            stmt.setFloat(13, 0.08f);
+            stmt.setDouble(14, 0.0008);
+            stmt.setFloat(15, 0.08f);
+            stmt.setDouble(16, 0.0008);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW9);
+            stmt.setString(3, C_VALUE);
+            stmt.setString(4, E_VALUE);
+            stmt.setInt(5, 9);
+            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 2));
+            stmt.setBigDecimal(7, BigDecimal.valueOf(3.3));
+            l = Integer.MAX_VALUE + 1L;
+            assert(l > Integer.MAX_VALUE);
+            stmt.setLong(8, l);
+            stmt.setInt(9, 3);
+            stmt.setInt(10, 300);
+            stmt.setByte(11, (byte)9);
+            stmt.setShort(12, (short) 0);
+            stmt.setFloat(13, 0.09f);
+            stmt.setDouble(14, 0.0009);
+            stmt.setFloat(15, 0.09f);
+            stmt.setDouble(16, 0.0009);
+            stmt.execute();
+                
+            conn.commit();
+        } finally {
+            conn.close();
+        }
+    }
+    
+    protected static void initTablesWithArrays(String tenantId, Date date, Long ts, boolean useNull, String url) throws Exception {
+        Properties props = new Properties();
+        if (ts != null) {
+            props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
+        }
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            // Insert all rows at ts
+            PreparedStatement stmt = conn.prepareStatement(
+                    "upsert into " +
+                            "TABLE_WITH_ARRAY(" +
+                            "    ORGANIZATION_ID, " +
+                            "    ENTITY_ID, " +
+                            "    a_string_array, " +
+                            "    B_STRING, " +
+                            "    A_INTEGER, " +
+                            "    A_DATE, " +
+                            "    X_DECIMAL, " +
+                            "    x_long_array, " +
+                            "    X_INTEGER," +
+                            "    a_byte_array," +
+                            "    A_SHORT," +
+                            "    A_FLOAT," +
+                            "    a_double_array," +
+                            "    A_UNSIGNED_FLOAT," +
+                            "    A_UNSIGNED_DOUBLE)" +
+                    "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)");
+            stmt.setString(1, tenantId);
+            stmt.setString(2, ROW1);
+            // Need to support primitive
+            String[] strArr =  new String[4];
+            strArr[0] = "ABC";
+            if (useNull) {
+                strArr[1] = null;
+            } else {
+                strArr[1] = "CEDF";
+            }
+            strArr[2] = "XYZWER";
+            strArr[3] = "AB";
+            Array array = conn.createArrayOf("VARCHAR", strArr);
+            stmt.setArray(3, array);
+            stmt.setString(4, B_VALUE);
+            stmt.setInt(5, 1);
+            stmt.setDate(6, date);
+            stmt.setBigDecimal(7, null);
+            // Need to support primitive
+            Long[] longArr =  new Long[2];
+            longArr[0] = 25l;
+            longArr[1] = 36l;
+            array = conn.createArrayOf("BIGINT", longArr);
+            stmt.setArray(8, array);
+            stmt.setNull(9, Types.INTEGER);
+            // Need to support primitive
+            Byte[] byteArr =  new Byte[2];
+            byteArr[0] = 25;
+            byteArr[1] = 36;
+            array = conn.createArrayOf("TINYINT", byteArr);
+            stmt.setArray(10, array);
+            stmt.setShort(11, (short) 128);
+            stmt.setFloat(12, 0.01f);
+            // Need to support primitive
+            Double[] doubleArr =  new Double[4];
+            doubleArr[0] = 25.343;
+            doubleArr[1] = 36.763;
+            doubleArr[2] = 37.56;
+            doubleArr[3] = 386.63;
+            array = conn.createArrayOf("DOUBLE", doubleArr);
+            stmt.setArray(13, array);
+            stmt.setFloat(14, 0.01f);
+            stmt.setDouble(15, 0.0001);
+            stmt.execute();
+
+            conn.commit();
+        } finally {
+            conn.close();
+        }
+    }
+    
+    protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, String url) throws Exception {
+        initEntityHistoryTableValues(tenantId, splits, null);
+    }
+    
+    protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, String url) throws Exception {
+        initEntityHistoryTableValues(tenantId, splits, date, null);
+    }
+    
+    protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
+        if (ts == null) {
+            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, splits);
+        } else {
+            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, splits, ts-2);
+        }
+        
+        Properties props = new Properties();
+        if (ts != null) {
+            props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
+        }
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            // Insert all rows at ts
+            PreparedStatement stmt = conn.prepareStatement(
+                    "upsert into " +
+                    ENTITY_HISTORY_TABLE_NAME+
+                    "(" +
+                    "    ORGANIZATION_ID, " +
+                    "    PARENT_ID, " +
+                    "    CREATED_DATE, " +
+                    "    ENTITY_HISTORY_ID, " +
+                    "    OLD_VALUE, " +
+                    "    NEW_VALUE) " +
+                    "VALUES (?, ?, ?, ?, ?, ?)");
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID1);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID1);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID2);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID2);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID3);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID3);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID4);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID4);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID5);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID5);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID6);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID6);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID7);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID7);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID8);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID8);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID9);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID9);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            conn.commit();
+        } finally {
+            conn.close();
+        }
+    }
+    
+    protected static void initSaltedEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
+        if (ts == null) {
+            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, splits);
+        } else {
+            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, splits, ts-2);
+        }
+        
+        Properties props = new Properties();
+        if (ts != null) {
+            props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
+        }
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            // Insert all rows at ts
+            PreparedStatement stmt = conn.prepareStatement(
+                    "upsert into " +
+                    ENTITY_HISTORY_SALTED_TABLE_NAME+
+                    "(" +
+                    "    ORGANIZATION_ID, " +
+                    "    PARENT_ID, " +
+                    "    CREATED_DATE, " +
+                    "    ENTITY_HISTORY_ID, " +
+                    "    OLD_VALUE, " +
+                    "    NEW_VALUE) " +
+                    "VALUES (?, ?, ?, ?, ?, ?)");
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID1);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID1);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID2);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID2);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+                
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID3);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID3);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID4);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID4);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID5);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID5);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID6);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID6);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID7);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID7);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID8);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID8);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            stmt.setString(1, tenantId);
+            stmt.setString(2, PARENTID9);
+            stmt.setDate(3, date);
+            stmt.setString(4, ENTITYHISTID9);
+            stmt.setString(5,  A_VALUE);
+            stmt.setString(6,  B_VALUE);
+            stmt.execute();
+            
+            conn.commit();
+        } finally {
+            conn.close();
+        }
+    }
+    
+    /**
+     * Disable and drop all the tables except SYSTEM.CATALOG and SYSTEM.SEQUENCE
+     */
+    protected static void disableAndDropNonSystemTables(PhoenixTestDriver driver) throws Exception {
+        HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin();
+        try {
+            HTableDescriptor[] tables = admin.listTables();
+            for (HTableDescriptor table : tables) {
+                boolean isCatalogTable = (Bytes.compareTo(table.getName(), PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES) == 0);
+                boolean isSequenceTable = (Bytes.compareTo(table.getName(), PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES) == 0);
+                if (!isCatalogTable && !isSequenceTable) {
+                    admin.disableTable(table.getName());
+                    admin.deleteTable(table.getName());
+                }
+            }    
+        } finally {
+            admin.close();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index 090b675..72e7aef 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -57,8 +57,8 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
         this(ReadOnlyProps.EMPTY_PROPS);
     }
     
-    public QueryServicesTestImpl(ReadOnlyProps overrideProps) {
-        super(withDefaults()
+    public static QueryServicesOptions getDefaultTestServicesOptions() {
+    	return withDefaults()
                 .setThreadPoolSize(DEFAULT_THREAD_POOL_SIZE)
                 .setQueueSize(DEFAULT_QUEUE_SIZE)
                 .setMaxMemoryPerc(DEFAULT_MAX_MEMORY_PERC)
@@ -78,8 +78,10 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .setWALEditCodec(DEFAULT_WAL_EDIT_CODEC)
                 .setDropMetaData(DEFAULT_DROP_METADATA)
                 .setMaxClientMetaDataCacheSize(DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE)
-                .setMaxServerMetaDataCacheSize(DEFAULT_MAX_SERVER_METADATA_CACHE_SIZE)
-                .setAll(overrideProps)
-        );
+                .setMaxServerMetaDataCacheSize(DEFAULT_MAX_SERVER_METADATA_CACHE_SIZE);
+    }
+    
+    public QueryServicesTestImpl(ReadOnlyProps overrideProps) {
+        super(getDefaultTestServicesOptions().setAll(overrideProps));
     }    
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
----------------------------------------------------------------------
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
index fce9a9d..a4b3655 100644
--- a/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
@@ -30,13 +30,15 @@ import org.apache.flume.lifecycle.LifecycleState;
 import org.apache.flume.sink.DefaultSinkFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
-import org.junit.Assert;
-import org.junit.Test;
-
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
 import org.apache.phoenix.flume.serializer.EventSerializers;
 import org.apache.phoenix.flume.sink.PhoenixSink;
 import org.apache.phoenix.util.TestUtil;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class PhoenixSinkIT extends BaseHBaseManagedTimeIT {
 
     private Context sinkContext;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
index 33349a0..7f4a6af 100644
--- a/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
@@ -40,15 +40,18 @@ import org.apache.flume.conf.Configurables;
 import org.apache.flume.event.EventBuilder;
 import org.apache.flume.lifecycle.LifecycleState;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
+import org.apache.phoenix.flume.serializer.EventSerializers;
+import org.apache.phoenix.flume.sink.PhoenixSink;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
-import org.apache.phoenix.flume.serializer.EventSerializers;
-import org.apache.phoenix.flume.sink.PhoenixSink;
 
+@Category(HBaseManagedTimeTest.class)
 public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
 
     private Context sinkContext;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
index 2bb47c8..28afb9a 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
@@ -19,6 +19,7 @@
  */
 package org.apache.phoenix.pig;
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -36,9 +37,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ConfigUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
@@ -56,6 +57,7 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.base.Preconditions;
 
@@ -63,6 +65,7 @@ import com.google.common.base.Preconditions;
  * 
  * Test class to run all the integration tests against a virtual map reduce cluster.
  */
+@Category(NeedsOwnMiniClusterTest.class)
 public class PhoenixHBaseLoaderIT {
     
     private static final Log LOG = LogFactory.getLog(PhoenixHBaseLoaderIT.class);
@@ -79,10 +82,8 @@ public class PhoenixHBaseLoaderIT {
     public static void setUpBeforeClass() throws Exception {
         hbaseTestUtil = new HBaseTestingUtility();
         conf = hbaseTestUtil.getConfiguration();
-        ConfigUtil.setReplicationConfigIfAbsent(conf);
+        setUpConfigForMiniCluster(conf);
         conf.set(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        conf.setInt(QueryServices.MASTER_INFO_PORT_ATTRIB, -1);
-        conf.setInt(QueryServices.REGIONSERVER_INFO_PORT_ATTRIB, -1);
         hbaseTestUtil.startMiniCluster();
 
         Class.forName(PhoenixDriver.class.getName());
@@ -439,4 +440,4 @@ public class PhoenixHBaseLoaderIT {
         conn.close();
         hbaseTestUtil.shutdownMiniCluster();
     }
-}
+}
\ No newline at end of file


[5/7] PHOENIX-982 Avoid spinning up and tearing down mini cluster in tests (SamarthJain)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
new file mode 100644
index 0000000..e905bad
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
@@ -0,0 +1,562 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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.ROW3;
+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.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+
+@Category(ClientManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class GroupByIT extends QueryIT {
+
+    public GroupByIT(String indexDDL) {
+        super(indexDDL);
+    }
+    
+    @Parameters(name="{0}")
+    public static Collection<Object> data() {
+        return QueryIT.data();
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testGroupByCondition() throws Exception {
+        Properties props = new Properties(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 aTable 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 aTable 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 aTable(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 aTable 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 aTable 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 aTable 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 testNoWhereScan() throws Exception {
+        String query = "SELECT y_integer FROM aTable";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            for (int i =0; i < 8; i++) {
+                assertTrue (rs.next());
+                assertEquals(0, rs.getInt(1));
+                assertTrue(rs.wasNull());
+            }
+            assertTrue (rs.next());
+            assertEquals(300, rs.getInt(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+ // FIXME: this is flapping with an phoenix.memory.InsufficientMemoryException
+    // in the GroupedAggregateRegionObserver. We can work around it by increasing
+    // the amount of available memory in QueryServicesTestImpl, but we shouldn't
+    // have to. I think something may no be being closed to reclaim the memory.
+    @Test
+    public void testGroupedAggregation() throws Exception {
+        // Tests that you don't get an ambiguous column exception when using the same alias as the column name
+        String query = "SELECT a_string as a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getLong(2), 4L);
+            assertEquals(rs.getString(3), "foo");
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), B_VALUE);
+            assertEquals(rs.getLong(2), 4L);
+            assertEquals(rs.getString(3), "foo");
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), C_VALUE);
+            assertEquals(rs.getLong(2), 1L);
+            assertEquals(rs.getString(3), "foo");
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testDistinctGroupedAggregation() throws Exception {
+        String query = "SELECT DISTINCT a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string, b_string ORDER BY a_string, count(1)";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getLong(2), 1L);
+            assertEquals(rs.getString(3), "foo");
+            
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getLong(2), 2L);
+            assertEquals(rs.getString(3), "foo");
+            
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), B_VALUE);
+            assertEquals(rs.getLong(2), 1L);
+            assertEquals(rs.getString(3), "foo");
+            
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), B_VALUE);
+            assertEquals(rs.getLong(2), 2L);
+            assertEquals(rs.getString(3), "foo");
+            
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), C_VALUE);
+            assertEquals(rs.getLong(2), 1L);
+            assertEquals(rs.getString(3), "foo");
+            
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testDistinctLimitedGroupedAggregation() throws Exception {
+        String query = "SELECT /*+ NO_INDEX */ DISTINCT a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string, b_string ORDER BY count(1) desc,a_string LIMIT 2";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+
+            /*
+            List<List<Object>> expectedResultsA = Lists.newArrayList(
+                    Arrays.<Object>asList(A_VALUE, 2L, "foo"),
+                    Arrays.<Object>asList(B_VALUE, 2L, "foo"));
+            List<List<Object>> expectedResultsB = Lists.newArrayList(expectedResultsA);
+            Collections.reverse(expectedResultsB);
+            // 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);
+            */
+
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getLong(2), 2L);
+            assertEquals(rs.getString(3), "foo");
+            
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), B_VALUE);
+            assertEquals(rs.getLong(2), 2L);
+            assertEquals(rs.getString(3), "foo");
+            
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testDistinctUngroupedAggregation() throws Exception {
+        String query = "SELECT DISTINCT count(1), 'foo' FROM atable WHERE organization_id=?";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            
+            assertTrue(rs.next());
+            assertEquals(9L, rs.getLong(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testGroupedLimitedAggregation() throws Exception {
+        String query = "SELECT a_string, count(1) FROM atable WHERE organization_id=? GROUP BY a_string LIMIT 2";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(4L, rs.getLong(2));
+            assertTrue(rs.next());
+            assertEquals(rs.getString(1), B_VALUE);
+            assertEquals(4L, rs.getLong(2));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testPointInTimeGroupedAggregation() throws Exception {
+        String updateStmt = 
+            "upsert into " +
+            "ATABLE VALUES ('" + tenantId + "','" + ROW5 + "','" + C_VALUE +"')";
+        // Override value that was set at creation time
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        Statement stmt = upsertConn.createStatement();
+        stmt.execute(updateStmt); // 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); // Run query at timestamp 5
+        upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        updateStmt = 
+            "upsert into " +
+            "ATABLE VALUES (?, ?, ?)";
+        // Insert all rows at ts
+        PreparedStatement pstmt = upsertConn.prepareStatement(updateStmt);
+        pstmt.setString(1, tenantId);
+        pstmt.setString(2, ROW5);
+        pstmt.setString(3, E_VALUE);
+        pstmt.execute(); // should commit too
+        upsertConn.close();
+        
+        String query = "SELECT a_string, count(1) FROM atable WHERE organization_id='" + tenantId + "' GROUP BY a_string";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Statement statement = conn.createStatement();
+        ResultSet rs = statement.executeQuery(query);
+        assertTrue(rs.next());
+        assertEquals(A_VALUE, rs.getString(1));
+        assertEquals(4, rs.getInt(2));
+        assertTrue(rs.next());
+        assertEquals(B_VALUE, rs.getString(1));
+        assertEquals(3, rs.getLong(2));
+        assertTrue(rs.next());
+        assertEquals(C_VALUE, rs.getString(1));
+        assertEquals(2, rs.getInt(2));
+        assertFalse(rs.next());
+        conn.close();
+    }
+
+    @Test
+    public void testUngroupedAggregation() throws Exception {
+        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ?";
+        String url = getUrl();
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5)); // Execute query at ts + 5
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setString(2, B_VALUE);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(4, rs.getLong(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+        // Run again to catch unintentianal deletion of rows during an ungrouped aggregation (W-1455633)
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 6)); // Execute at ts + 6
+        conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setString(2, B_VALUE);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(4, rs.getLong(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testUngroupedAggregationNoWhere() throws Exception {
+        String query = "SELECT count(*) FROM atable";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(9, rs.getLong(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testPointInTimeUngroupedAggregation() 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 = new Properties(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        String updateStmt = 
+            "upsert into " +
+            "ATABLE(" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_STRING) " +
+            "VALUES (?, ?, ?)";
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(updateStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW5);
+        stmt.setString(3, null);
+        stmt.execute();
+        stmt.setString(3, C_VALUE);
+        stmt.execute();
+        stmt.setString(2, ROW7);
+        stmt.setString(3, E_VALUE);
+        stmt.execute();
+        upsertConn.commit();
+        upsertConn.close();
+
+        // Override value again, but should be ignored since it's past the SCN
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3); // Run query at timestamp 5
+        upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        stmt = upsertConn.prepareStatement(updateStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW6);
+        stmt.setString(3, E_VALUE);
+        stmt.execute();
+        upsertConn.close();
+        
+        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ?";
+        // Specify CurrentSCN on URL with extra stuff afterwards (which should be ignored)
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 2) + ";foo=bar"; // Run query at timestamp 2 
+        Connection conn = DriverManager.getConnection(url, 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 testPointInTimeUngroupedLimitedAggregation() throws Exception {
+        String updateStmt = 
+            "upsert into " +
+            "ATABLE(" +
+            "    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 = new Properties(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        PreparedStatement stmt = upsertConn.prepareStatement(updateStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW6);
+        stmt.setString(3, C_VALUE);
+        stmt.execute();
+        stmt.setString(3, E_VALUE);
+        stmt.execute();
+        stmt.setString(3, B_VALUE);
+        stmt.execute();
+        stmt.setString(3, B_VALUE);
+        stmt.execute();
+        upsertConn.close();
+
+        // Override value again, but should be ignored since it's past the SCN
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3); // Run query at timestamp 5
+        upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        stmt = upsertConn.prepareStatement(updateStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW6);
+        stmt.setString(3, E_VALUE);
+        stmt.execute();
+        upsertConn.close();
+
+        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ? LIMIT 3";
+        // Specify CurrentSCN on URL with extra stuff afterwards (which should be ignored)
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 2) + ";foo=bar"; // Run query at timestamp 2 
+        Connection conn = DriverManager.getConnection(url, props);
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        statement.setString(2, B_VALUE);
+        ResultSet rs = statement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(4, rs.getLong(1)); // LIMIT applied at end, so all rows would be counted
+        assertFalse(rs.next());
+        conn.close();
+    }
+
+    @Test
+    public void testPointInTimeDeleteUngroupedAggregation() throws Exception {
+        String updateStmt = 
+            "upsert into " +
+            "ATABLE(" +
+            "    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 = new Properties(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 atable 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 atable 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 atable 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();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/HBaseManagedTimeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HBaseManagedTimeTest.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HBaseManagedTimeTest.java
new file mode 100644
index 0000000..31cc00b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HBaseManagedTimeTest.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Inherited;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * 
+ * Annotation to denote that the test lets HBase manage timestamp 
+ * itself.
+ * 
+ * Tests using a mini cluster need to be classified either 
+ * as {@link HBaseManagedTimeTest} or {@link ClientManagedTimeTest} 
+ * or {@link NeedsOwnMiniClusterTest} otherwise they won't be run 
+ * when one runs mvn verify or mvn install.
+ * 
+ * @since 4.1
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+@Inherited
+public @interface HBaseManagedTimeTest {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
index 6493a2e..26c9455 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
@@ -17,15 +17,15 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.JOIN_CUSTOMER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_CUSTOMER_TABLE_DISPLAY_NAME;
-import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_CUSTOMER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_DISPLAY_NAME;
-import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_SCHEMA;
-import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_DISPLAY_NAME;
+import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -56,13 +56,14 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-
+@Category(HBaseManagedTimeTest.class)
 @RunWith(Parameterized.class)
 public class HashJoinIT extends BaseHBaseManagedTimeIT {
     
@@ -75,7 +76,8 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
         this.plans = plans;
     }
     
-    @BeforeClass 
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
@@ -83,7 +85,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
         // Forces server cache to be used
         props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     @Before

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
index 3275999..5e04f8e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
@@ -19,22 +19,25 @@ package org.apache.phoenix.end2end;
 
 import java.util.Map;
 
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
 
+@Category(ClientManagedTimeTest.class)
 public class InMemoryOrderByIT extends OrderByIT {
 
     public InMemoryOrderByIT() {
     }
 
     @BeforeClass
+    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         props.put(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(1024*1024));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
index 9c60914..cb9516c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
@@ -29,8 +29,9 @@ import java.util.Properties;
 
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class IsNullIT extends BaseClientManagedTimeIT {
     @Test
     public void testIsNullInPk() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
index 799e111..b3ba37a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
@@ -29,11 +29,11 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class KeyOnlyIT extends BaseClientManagedTimeIT {
     @Test
     public void testKeyOnly() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/MD5FunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MD5FunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MD5FunctionIT.java
index ade0915..bec5b65 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MD5FunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MD5FunctionIT.java
@@ -28,7 +28,9 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class MD5FunctionIT extends BaseHBaseManagedTimeIT {
   
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
index 2d68523..40190fe 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
@@ -18,17 +18,22 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.math.BigDecimal;
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
     private static final String MULTI_CF = "MULTI_CF";
     

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 baf0ef3..6c9c36b 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
@@ -52,6 +52,7 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
 /**
@@ -63,6 +64,7 @@ import org.junit.Test;
  * 
  * @since 0.1
  */
+@Category(ClientManagedTimeTest.class)
 public class NativeHBaseTypesIT extends BaseClientManagedTimeIT {
     private static final byte[] HBASE_NATIVE_BYTES = SchemaUtil.getTableNameAsBytes(HBASE_NATIVE_SCHEMA_NAME, HBASE_NATIVE);
     private static final byte[] FAMILY_NAME = Bytes.toBytes(SchemaUtil.normalizeIdentifier("1"));

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/NeedsOwnMiniClusterTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NeedsOwnMiniClusterTest.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NeedsOwnMiniClusterTest.java
new file mode 100644
index 0000000..b9dc04a
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NeedsOwnMiniClusterTest.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * 
+ * Annotation to denote that the test needs to run its own 
+ * mini cluster that is separate from the clusters used by 
+ * tests annotated as {@link HBaseManagedTimeTest} or 
+ * {@link ClientManagedTimeTest}.
+ * 
+ * As much as possible, tests should be able to run in one of the
+ * mini clusters used by {@link HBaseManagedTimeTest} or 
+ * {@link ClientManagedTimeTest}. In the *rare* case it can't
+ * you would need to annotate the test as {@link NeedsOwnMiniClusterTest}
+ * otherwise the test won't be executed when you run mvn verify or mvn install.
+ * 
+ * @since 4.1
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+public @interface NeedsOwnMiniClusterTest {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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
new file mode 100644
index 0000000..e7ced92
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
@@ -0,0 +1,303 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+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.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.Collection;
+import java.util.Properties;
+
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.primitives.Doubles;
+import com.google.common.primitives.Floats;
+
+@Category(ClientManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class NotQueryIT extends QueryIT {
+
+    public NotQueryIT(String indexDDL) {
+        super(indexDDL);
+    }
+    
+    @Parameters(name="{0}")
+    public static Collection<Object> data() {
+        return QueryIT.data();
+    }
+    
+    @Test
+    public void testNotInList() throws Exception {
+        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and entity_id NOT IN (?,?,?,?,?,?)";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setString(2, ROW2);
+            statement.setString(3, ROW4);
+            statement.setString(4, ROW1);
+            statement.setString(5, ROW5);
+            statement.setString(6, ROW7);
+            statement.setString(7, ROW8);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW3, rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW6, rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testNotInListOfFloat() throws Exception {
+        String query = "SELECT a_float FROM aTable WHERE organization_id=? and a_float NOT IN (?,?,?,?,?,?)";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setFloat(2, 0.01f);
+            statement.setFloat(3, 0.02f);
+            statement.setFloat(4, 0.03f);
+            statement.setFloat(5, 0.04f);
+            statement.setFloat(6, 0.05f);
+            statement.setFloat(7, 0.06f);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertTrue(Doubles.compare(rs.getDouble(1), 0.07f)==0);
+            assertTrue (rs.next());
+            assertTrue(Doubles.compare(rs.getDouble(1), 0.08f)==0);
+            assertTrue (rs.next());
+            assertTrue(Doubles.compare(rs.getDouble(1), 0.09f)==0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testNotInListOfDouble() throws Exception {
+        String query = "SELECT a_double FROM aTable WHERE organization_id=? and a_double NOT IN (?,?,?,?,?,?)";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setDouble(2, 0.0001);
+            statement.setDouble(3, 0.0002);
+            statement.setDouble(4, 0.0003);
+            statement.setDouble(5, 0.0004);
+            statement.setDouble(6, 0.0005);
+            statement.setDouble(7, 0.0006);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertTrue(Doubles.compare(rs.getDouble(1), 0.0007)==0);
+            assertTrue (rs.next());
+            assertTrue(Doubles.compare(rs.getDouble(1), 0.0008)==0);
+            assertTrue (rs.next());
+            assertTrue(Doubles.compare(rs.getDouble(1), 0.0009)==0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @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";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW2);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @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";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getByte(1), 2);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @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";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getShort(1), 129);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testNotEqualsByFloat() throws Exception {
+        String query = "SELECT a_float -- and here comment\n" + 
+        "FROM aTable WHERE organization_id=? and a_float != 0.01d and a_float <= 0.02d";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertTrue(Floats.compare(rs.getFloat(1), 0.02f) == 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @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.01d and a_unsigned_float <= 0.02d";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertTrue(Floats.compare(rs.getFloat(1), 0.02f) == 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testNotEqualsByDouble() throws Exception {
+        String query = "SELECT a_double -- and here comment\n" + 
+        "FROM aTable WHERE organization_id=? and a_double != 0.0001d and a_double <= 0.0002d";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertTrue(Doubles.compare(rs.getDouble(1), 0.0002) == 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @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.0001d and a_unsigned_double <= 0.0002d";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertTrue(Doubles.compare(rs.getDouble(1), 0.0002) == 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @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";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW2);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 1d87f1e..56f81eb 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
@@ -37,11 +37,12 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.util.Properties;
 
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-import org.apache.phoenix.util.PhoenixRuntime;
-
+@Category(ClientManagedTimeTest.class)
 public class OrderByIT extends BaseClientManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 a9fc8b5..e14ac2c 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
@@ -17,18 +17,44 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
+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.INDEX_DATA_SCHEMA;
+import static org.apache.phoenix.util.TestUtil.INDEX_DATA_TABLE;
+import static org.apache.phoenix.util.TestUtil.MILLIS_IN_DAY;
+import static org.apache.phoenix.util.TestUtil.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+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.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.math.BigDecimal;
 import java.math.RoundingMode;
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
 import java.util.Properties;
 
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class PercentileIT extends BaseClientManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 2113038..05a91c3 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
@@ -40,10 +40,6 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Ordering;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryConstants;
@@ -53,7 +49,13 @@ import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Ordering;
 
+@Category(ClientManagedTimeTest.class)
 public class ProductMetricsIT extends BaseClientManagedTimeIT {
     private static Format format = DateUtil.getDateParser(DateUtil.DEFAULT_DATE_FORMAT);
     private static final String PRODUCT_METRICS_NAME = "PRODUCT_METRICS";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 5a38076..647e15a 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
@@ -76,8 +76,9 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java
index 08b1fe1..99060dd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java
@@ -17,20 +17,28 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+import static org.apache.phoenix.util.TestUtil.B_VALUE;
+import static org.apache.phoenix.util.TestUtil.C_VALUE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class QueryExecWithoutSCNIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testScanNoSCN() throws Exception {
         long ts = System.currentTimeMillis();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_string, b_string FROM aTable WHERE organization_id=? and a_integer = 5";
         Properties props = new Properties(); // Test with no CurrentSCN property set
         Connection conn = DriverManager.getConnection(getUrl(), props);


[3/7] PHOENIX-982 Avoid spinning up and tearing down mini cluster in tests (SamarthJain)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
index 34aded6..0f40d90 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
@@ -34,18 +34,21 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
 
-public class QueryPlanIT extends BaseConnectedQueryIT {
+@Category(HBaseManagedTimeTest.class)
+public class QueryPlanIT extends BaseHBaseManagedTimeIT {
     
     @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         // Override date format so we don't have a bunch of zeros
         props.put(QueryServices.DATE_FORMAT_ATTRIB, "yyyy-MM-dd");
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 0b1648a..fb0016b 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
@@ -17,17 +17,23 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
+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.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class ReadIsolationLevelIT extends BaseClientManagedTimeIT {
     private static final String ENTITY_ID1= "000000000000001";
     private static final String ENTITY_ID2= "000000000000002";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseFunctionIT.java
index f2f675f..7dc58ab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseFunctionIT.java
@@ -28,7 +28,9 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class ReverseFunctionIT extends BaseHBaseManagedTimeIT {
     private void initTable(Connection conn, String sortOrder, String s) throws Exception {
         String ddl = "CREATE TABLE REVERSE_TEST (pk VARCHAR NOT NULL PRIMARY KEY " + sortOrder + ", kv VARCHAR)";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/RoundFloorCeilFunctionsEnd2EndIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RoundFloorCeilFunctionsEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RoundFloorCeilFunctionsEnd2EndIT.java
index 413d6a8..dcb79ce 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RoundFloorCeilFunctionsEnd2EndIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RoundFloorCeilFunctionsEnd2EndIT.java
@@ -30,13 +30,13 @@ import java.sql.ResultSet;
 import java.sql.Time;
 import java.sql.Timestamp;
 
-import org.junit.BeforeClass;
-import org.junit.Test;
-
 import org.apache.phoenix.expression.function.CeilFunction;
 import org.apache.phoenix.expression.function.FloorFunction;
 import org.apache.phoenix.expression.function.RoundFunction;
 import org.apache.phoenix.util.DateUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.primitives.Doubles;
 import com.google.common.primitives.Floats;
@@ -48,6 +48,8 @@ import com.google.common.primitives.Floats;
  * 
  * @since 3.0.0
  */
+
+@Category(ClientManagedTimeTest.class)
 public class RoundFloorCeilFunctionsEnd2EndIT extends BaseClientManagedTimeIT {
     
     private static long millisPart = 660;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
index 7315ea0..29cf38b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
@@ -54,7 +54,9 @@ import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java
index a6fc562..ea59b85 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SaltedViewIT.java
@@ -18,7 +18,9 @@
 package org.apache.phoenix.end2end;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class SaltedViewIT extends BaseViewIT {
     
     /**

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java
new file mode 100644
index 0000000..3b3a94c
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java
@@ -0,0 +1,498 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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.ROW2;
+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.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Doubles;
+import com.google.common.primitives.Floats;
+
+@Category(ClientManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class ScanQueryIT extends QueryIT {
+    
+    @Parameters(name="{0}")
+    public static Collection<Object> data() {
+        return QueryIT.data();
+    }
+
+    public ScanQueryIT(String indexDDL) {
+        super(indexDDL);
+    }
+    
+    @Test
+    public void testScan() throws Exception {
+        String query = "SELECT a_string, /* comment ok? */ b_string FROM aTable WHERE ?=organization_id and 5=a_integer";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), B_VALUE);
+            assertEquals(rs.getString("B_string"), C_VALUE);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testScanByByteValue() throws Exception {
+        String query = "SELECT a_string, b_string, a_byte FROM aTable WHERE ?=organization_id and 1=a_byte";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getString("B_string"), B_VALUE);
+            assertEquals(rs.getByte(3), 1);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testScanByShortValue() throws Exception {
+        String query = "SELECT a_string, b_string, a_short FROM aTable WHERE ?=organization_id and 128=a_short";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getString("B_string"), B_VALUE);
+            assertEquals(rs.getShort("a_short"), 128);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testScanByFloatValue() throws Exception {
+        String query = "SELECT a_string, b_string, a_float FROM aTable WHERE ?=organization_id and ?=a_float";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setFloat(2, 0.01f);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getString("B_string"), B_VALUE);
+            assertTrue(Floats.compare(rs.getFloat(3), 0.01f) == 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testScanByUnsignedFloatValue() throws Exception {
+        String query = "SELECT a_string, b_string, a_unsigned_float FROM aTable WHERE ?=organization_id and ?=a_unsigned_float";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setFloat(2, 0.01f);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getString("B_string"), B_VALUE);
+            assertTrue(Floats.compare(rs.getFloat(3), 0.01f) == 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testScanByDoubleValue() throws Exception {
+        String query = "SELECT a_string, b_string, a_double FROM aTable WHERE ?=organization_id and ?=a_double";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setDouble(2, 0.0001);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getString("B_string"), B_VALUE);
+            assertTrue(Doubles.compare(rs.getDouble(3), 0.0001) == 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testScanByUnsigned_DoubleValue() throws Exception {
+        String query = "SELECT a_string, b_string, a_unsigned_double FROM aTable WHERE ?=organization_id and ?=a_unsigned_double";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setDouble(2, 0.0001);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertEquals(rs.getString("B_string"), B_VALUE);
+            assertTrue(Doubles.compare(rs.getDouble(3), 0.0001) == 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testAllScan() throws Exception {
+        String query = "SELECT ALL a_string, b_string FROM aTable WHERE ?=organization_id and 5=a_integer";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), B_VALUE);
+            assertEquals(rs.getString("B_string"), C_VALUE);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testDistinctScan() throws Exception {
+        String query = "SELECT DISTINCT a_string FROM aTable WHERE organization_id=?";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), B_VALUE);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), C_VALUE);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testDistinctLimitScan() throws Exception {
+        String query = "SELECT DISTINCT a_string FROM aTable WHERE organization_id=? LIMIT 1";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), A_VALUE);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testInListSkipScan() throws Exception {
+        String query = "SELECT entity_id, b_string FROM aTable WHERE organization_id=? and entity_id IN (?,?)";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setString(2, ROW2);
+            statement.setString(3, ROW4);
+            ResultSet rs = statement.executeQuery();
+            Set<String> expectedvals = new HashSet<String>();
+            expectedvals.add(ROW2+"_"+C_VALUE);
+            expectedvals.add(ROW4+"_"+B_VALUE);
+            Set<String> vals = new HashSet<String>();
+            assertTrue (rs.next());
+            vals.add(rs.getString(1) + "_" + rs.getString(2));
+            assertTrue (rs.next());
+            vals.add(rs.getString(1) + "_" + rs.getString(2));
+            assertFalse(rs.next());
+            assertEquals(expectedvals, vals);
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testUnboundRangeScan1() throws Exception {
+        String query = "SELECT entity_id FROM aTable WHERE organization_id <= ?";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW1);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW2);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW3);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW4);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW5);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW6);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW7);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW8);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW9);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testUnboundRangeScan2() throws Exception {
+        String query = "SELECT entity_id FROM aTable WHERE organization_id >= ?";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW1);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW2);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW3);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW4);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW5);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW6);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW7);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW8);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW9);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testUpperLowerBoundRangeScan() throws Exception {
+        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) > '00A' and substr(entity_id,1,3) < '00C'";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW5);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW6);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW7);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW8);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testUpperBoundRangeScan() throws Exception {
+        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) >= '00B' ";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW5);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW6);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW7);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW8);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW9);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testLowerBoundRangeScan() throws Exception {
+        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) < '00B' ";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW1);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW2);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW3);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW4);
+            assertFalse(rs.next());
+        } finally {
+            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 = new Properties(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, 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 atable 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/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
index 17f2308..02288ed 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
@@ -43,22 +43,25 @@ import org.apache.phoenix.util.TestUtil;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
 
+@Category(ClientManagedTimeTest.class)
 public class SequenceIT extends BaseClientManagedTimeIT {
     private static final long BATCH_SIZE = 3;
     
     private Connection conn;
     
-    @BeforeClass 
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
         
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         // Make a small batch size to test multiple calls to reserve sequences
         props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB, Long.toString(BATCH_SIZE));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
     
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ServerExceptionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ServerExceptionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ServerExceptionIT.java
index ab9bb28..f5b2aa6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ServerExceptionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ServerExceptionIT.java
@@ -29,8 +29,9 @@ import java.sql.SQLException;
 import java.util.Properties;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class ServerExceptionIT extends BaseHBaseManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/Shadower.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/Shadower.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/Shadower.java
new file mode 100644
index 0000000..fdecbd2
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/Shadower.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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 java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * 
+ * Annotation to denote that a method is shadowing/hiding a 
+ * public static method of one of the super classes in its 
+ * ancestor tree. 
+ *
+ * @since 4.1
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+@Documented
+public @interface Shadower {
+    Class classBeingShadowed();
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
index e3869d9..20ce768 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
@@ -57,6 +57,7 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
@@ -70,6 +71,7 @@ import com.google.common.collect.Maps;
  * Tests for {@link SkipRangeParallelIteratorRegionSplitter}.
  */
 @RunWith(Parameterized.class)
+@Category(ClientManagedTimeTest.class)
 public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManagedTimeIT {
 
     private static final String TABLE_NAME = "TEST_SKIP_RANGE_PARALLEL_ITERATOR";
@@ -319,6 +321,7 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
      }
     
     @BeforeClass
+    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
         int targetQueryConcurrency = 3;
         int maxQueryConcurrency = 5;
@@ -327,7 +330,7 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
         props.put(QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB, Integer.toString(targetQueryConcurrency));
         props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(Integer.MAX_VALUE));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
 
     private static List<KeyRange> getSplits(TableRef tableRef, final Scan scan, final List<HRegionLocation> regions,

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
index 8d9f81d..540197c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
@@ -33,7 +33,9 @@ import java.util.List;
 
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     
     private void initIntInTable(Connection conn, List<Integer> data) throws SQLException {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java
index 679e0bf..f833278 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderFIT.java
@@ -34,12 +34,15 @@ import java.util.Properties;
 import org.apache.commons.lang.ArrayUtils;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
 
 /**
  * @since 1.2
  */
+
+@Category(HBaseManagedTimeTest.class)
 public class SortOrderFIT extends BaseHBaseManagedTimeIT {
     
     private static final String TABLE = "DescColumnSortOrderTest";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
index adcd48c..825119b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
@@ -37,10 +37,12 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
 
-public class SpillableGroupByIT extends BaseConnectedQueryIT {
+@Category(HBaseManagedTimeTest.class)
+public class SpillableGroupByIT extends BaseHBaseManagedTimeIT {
 
     private static final int NUM_ROWS_INSERTED = 1000;
     
@@ -52,6 +54,7 @@ public class SpillableGroupByIT extends BaseConnectedQueryIT {
     private int id;
 
     @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
         // Set a very small cache size to force plenty of spilling
@@ -62,9 +65,7 @@ public class SpillableGroupByIT extends BaseConnectedQueryIT {
                 Integer.toString(1));
         // Large enough to not run out of memory, but small enough to spill
         props.put(QueryServices.MAX_MEMORY_SIZE_ATTRIB, Integer.toString(40000));
-
-        // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
 
     private long createTable() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
index ca743c4..859a66a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
@@ -19,20 +19,23 @@ package org.apache.phoenix.end2end;
 
 import java.util.Map;
 
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
 
+@Category(ClientManagedTimeTest.class)
 public class SpooledOrderByIT extends OrderByIT {
 
     @BeforeClass
+    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         props.put(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(100));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatementHintsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatementHintsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatementHintsIT.java
index efc52d2..ea46cce 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatementHintsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatementHintsIT.java
@@ -17,18 +17,26 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+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.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
 /**
  * End-to-End tests on various statement hints.
  */
+
+@Category(HBaseManagedTimeTest.class)
 public class StatementHintsIT extends BaseHBaseManagedTimeIT {
 
     private static void initTableValues() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java
index b643ce3..64ec1e6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java
@@ -35,6 +35,7 @@ import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.TimeKeeper;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
 /**
@@ -45,6 +46,8 @@ import org.junit.Test;
  * cleared between test runs.
  *
  */
+
+@Category(ClientManagedTimeTest.class)
 public class StatsManagerIT extends BaseParallelIteratorsRegionSplitterIT {
     
     private static class ManualTimeKeeper implements TimeKeeper {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java
index 7a192d9..c80fe4d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java
@@ -18,17 +18,22 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.math.BigDecimal;
 import java.math.RoundingMode;
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
-
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+@Category(ClientManagedTimeTest.class)
 public class StddevIT extends BaseClientManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index e77e314..d3e42b3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -53,7 +53,9 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
index 21d9126..e809935 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
@@ -32,7 +32,9 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index 8ade47d..1f8eb55 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -31,7 +31,9 @@ import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
 	
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexSaltedIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexSaltedIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexSaltedIT.java
index e72419a..8b35b51 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexSaltedIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexSaltedIT.java
@@ -18,7 +18,9 @@
 package org.apache.phoenix.end2end;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class TenantSpecificViewIndexSaltedIT extends BaseTenantSpecificViewIndexIT {
     private static final Integer SALT_BUCKETS = 3;
     

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java
index a5e3b53..641ddc4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java
@@ -36,10 +36,11 @@ import java.text.SimpleDateFormat;
 import java.util.Properties;
 import java.util.TimeZone;
 
+import org.apache.phoenix.expression.function.ToCharFunction;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Before;
 import org.junit.Test;
-
-import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.experimental.categories.Category;
 
 /**
  * Tests for the TO_CHAR built-in function.
@@ -48,6 +49,7 @@ import org.apache.phoenix.util.PhoenixRuntime;
  * 
  * @since 0.1
  */
+@Category(ClientManagedTimeTest.class)
 public class ToCharFunctionIT extends BaseClientManagedTimeIT {
     
     public static final String TO_CHAR_TABLE_NAME = "TO_CHAR_TABLE";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToNumberFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToNumberFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToNumberFunctionIT.java
index eb11305..0fbb23d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToNumberFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToNumberFunctionIT.java
@@ -33,13 +33,14 @@ import java.sql.Timestamp;
 import java.util.Locale;
 import java.util.Properties;
 
+import org.apache.phoenix.expression.function.ToNumberFunction;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
-import org.apache.phoenix.schema.PDataType;
-import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.experimental.categories.Category;
 
 /**
  * Tests for the TO_NUMBER built-in function.
@@ -48,6 +49,7 @@ import org.apache.phoenix.util.PhoenixRuntime;
  * 
  * @since 0.1
  */
+@Category(ClientManagedTimeTest.class)
 public class ToNumberFunctionIT extends BaseClientManagedTimeIT {
 
     // This test changes to locale to en_US, and saves the previous locale here

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
index f37e103..e9b7298 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
@@ -37,10 +37,11 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class TopNIT extends BaseClientManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
index b499c32..bed5146 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
@@ -39,7 +39,9 @@ import java.util.Properties;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.util.DateUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class TruncateFunctionIT extends BaseClientManagedTimeIT {
     private static Format format = DateUtil.getDateParser(DateUtil.DEFAULT_MS_DATE_FORMAT);
     private static final String DS1 = "1970-01-10 00:58:01.587";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
index db4b0ca..e1ce962 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
@@ -17,14 +17,20 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
 
     private static final long INTEGER_MIN_MINUS_ONE = (long)Integer.MIN_VALUE - 1;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
index feb3fbc..2f043df 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
@@ -32,7 +32,9 @@ import java.sql.ResultSet;
 import java.util.Properties;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class UpsertSelectAutoCommitIT extends BaseHBaseManagedTimeIT {
 
     public UpsertSelectAutoCommitIT() {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index 16e6070..fea4ce4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -47,8 +47,9 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class UpsertSelectIT extends BaseClientManagedTimeIT {
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
index 6389088..043bba9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
@@ -43,8 +43,9 @@ import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class UpsertValuesIT extends BaseClientManagedTimeIT {
     @Test
     public void testGroupByWithLimitOverRowKey() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
index 1dd0103..0077383 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
@@ -46,8 +46,10 @@ import org.apache.phoenix.schema.ConstraintViolationException;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
+@Category(ClientManagedTimeTest.class)
 public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     private static Format format = DateUtil.getDateParser(DateUtil.DEFAULT_DATE_FORMAT);
     private static final String DS1 = "1970-01-01 00:58:00";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index 2dedba1..b75bb30 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -30,7 +30,9 @@ import java.sql.SQLException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class ViewIT extends BaseViewIT {
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
index 6d1e995..88c88bd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
@@ -18,8 +18,11 @@
 package org.apache.phoenix.end2end.index;
 
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
 import org.apache.phoenix.util.SchemaUtil;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class BaseIndexIT extends BaseHBaseManagedTimeIT {
     public static final String SCHEMA_NAME = "";
     public static final String DATA_TABLE_NAME = "T";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
index 4495877..917dd2e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseMutableIndexIT.java
@@ -27,8 +27,11 @@ import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.Properties;
 
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
 import org.apache.phoenix.util.TestUtil;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class BaseMutableIndexIT extends BaseIndexIT {
     public BaseMutableIndexIT() {
     }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java
index 499b69e..7e94894 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java
@@ -32,25 +32,31 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
+import org.apache.phoenix.end2end.Shadower;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.phoenix.util.SchemaUtil;
 
+@Category(HBaseManagedTimeTest.class)
 public class DropViewIT extends BaseMutableIndexIT {
     private static final byte[] HBASE_NATIVE_BYTES = SchemaUtil.getTableNameAsBytes(HBASE_NATIVE_SCHEMA_NAME, HBASE_NATIVE);
     private static final byte[] FAMILY_NAME = Bytes.toBytes(SchemaUtil.normalizeIdentifier("1"));
     
-    @BeforeClass 
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         // Drop the HBase table metadata for this test
         props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 cd3a8fb..9128595 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
@@ -34,14 +34,16 @@ import java.sql.SQLException;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.util.QueryUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
     // Populate the test table with data.
     private static void populateTestTable() throws SQLException {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 d81140b..11c64f8 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
@@ -37,6 +37,7 @@ import java.util.Properties;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
@@ -48,8 +49,9 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
 
 	private enum Order {ASC, DESC};

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index 7bb9776..4a60f91 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -17,6 +17,11 @@
  */
 package org.apache.phoenix.end2end.index;
 
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
+import static org.apache.phoenix.util.TestUtil.LOCALHOST;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -24,98 +29,131 @@ 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.sql.SQLException;
-import java.util.Map;
 import java.util.Properties;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import com.google.common.collect.Maps;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
-
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 /**
  * 
- * Test for failure of region server to write to index table. Keep in its own
- * separate test, as otherwise it makes tests run after it fail.
- *
+ * Test for failure of region server to write to index table.
+ * For some reason dropping tables after running this test
+ * fails unless it runs its own mini cluster. 
+ * 
  * 
  * @since 2.1
  */
-public class MutableIndexFailureIT extends BaseMutableIndexIT {
-    @BeforeClass 
+@Category(NeedsOwnMiniClusterTest.class)
+public class MutableIndexFailureIT extends BaseTest {
+    private static String url;
+    private static PhoenixTestDriver driver;
+    private static HBaseTestingUtility util;
+
+    private static final String SCHEMA_NAME = "";
+    private static final String INDEX_TABLE_NAME = "I";
+    private static final String DATA_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, "T");
+    private static final String INDEX_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, "I");
+
+    @BeforeClass
     public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put("hbase.client.retries.number", Integer.toString(2));
-        props.put("hbase.client.pause", Integer.toString(5000));
-        // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        Configuration conf = HBaseConfiguration.create();
+        setUpConfigForMiniCluster(conf);
+        conf.setInt("hbase.client.retries.number", 2);
+        conf.setInt("hbase.client.pause", 5000);
+        util = new HBaseTestingUtility(conf);
+        util.startMiniCluster();
+        String clientPort = util.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
+        url = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR + clientPort
+                + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
+        driver = initAndRegisterDriver(url, ReadOnlyProps.EMPTY_PROPS);
     }
-    
-    
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        try {
+            destroyDriver(driver);
+        } finally {
+            util.shutdownMiniCluster();
+        }
+    }
+
     private static void destroyIndexTable() throws Exception {
         Properties props = new Properties(TEST_PROPERTIES);
-        ConnectionQueryServices services = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class).getQueryServices();
+        Connection conn = driver.connect(url, props);
+        ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
         HBaseAdmin admin = services.getAdmin();
         try {
             admin.disableTable(INDEX_TABLE_FULL_NAME);
             admin.deleteTable(INDEX_TABLE_FULL_NAME);
-        } catch (TableNotFoundException e) {
-        } finally {
+        } catch (TableNotFoundException e) {} finally {
+            conn.close();
             admin.close();
         }
     }
-    
+
     @Test
     public void testWriteFailureDisablesIndex() throws Exception {
         String query;
         ResultSet rs;
-        
+
         Properties props = new Properties(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = driver.connect(url, props);
         conn.setAutoCommit(false);
-        conn.createStatement().execute("CREATE TABLE " + DATA_TABLE_FULL_NAME + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+        conn.createStatement().execute(
+                "CREATE TABLE " + DATA_TABLE_FULL_NAME + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
         query = "SELECT * FROM " + DATA_TABLE_FULL_NAME;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
-        
-        conn.createStatement().execute("CREATE INDEX " + INDEX_TABLE_NAME + " ON " + DATA_TABLE_FULL_NAME + " (v1) INCLUDE (v2)");
+
+        conn.createStatement().execute(
+                "CREATE INDEX " + INDEX_TABLE_NAME + " ON " + DATA_TABLE_FULL_NAME + " (v1) INCLUDE (v2)");
         query = "SELECT * FROM " + INDEX_TABLE_FULL_NAME;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
 
         // Verify the metadata for index is correct.
-        rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(SCHEMA_NAME), INDEX_TABLE_NAME, new String[] {PTableType.INDEX.toString()});
+        rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(SCHEMA_NAME), INDEX_TABLE_NAME,
+                new String[] { PTableType.INDEX.toString() });
         assertTrue(rs.next());
         assertEquals(INDEX_TABLE_NAME, rs.getString(3));
         assertEquals(PIndexState.ACTIVE.toString(), rs.getString("INDEX_STATE"));
         assertFalse(rs.next());
 
         destroyIndexTable();
-        
+
         PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + DATA_TABLE_FULL_NAME + " VALUES(?,?,?)");
-        stmt.setString(1,"a");
+        stmt.setString(1, "a");
         stmt.setString(2, "x");
         stmt.setString(3, "1");
         stmt.execute();
         try {
             conn.commit();
             fail();
-        } catch (SQLException e) {
-        }
-        
+        } catch (SQLException e) {}
+
         // Verify the metadata for index is correct.
-        rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(SCHEMA_NAME), INDEX_TABLE_NAME, new String[] {PTableType.INDEX.toString()});
+        rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(SCHEMA_NAME), INDEX_TABLE_NAME,
+                new String[] { PTableType.INDEX.toString() });
         assertTrue(rs.next());
         assertEquals(INDEX_TABLE_NAME, rs.getString(3));
         assertEquals(PIndexState.DISABLE.toString(), rs.getString("INDEX_STATE"));

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index 64bd940..6e6a274 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@ -34,19 +34,25 @@ import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
+import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
 import com.google.common.primitives.Doubles;
 
+@Category(HBaseManagedTimeTest.class)
 public class MutableIndexIT extends BaseMutableIndexIT {
-    @BeforeClass 
+    
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
@@ -54,7 +60,7 @@ public class MutableIndexIT extends BaseMutableIndexIT {
         // Forces server cache to be used
         props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 91a32aa..a46b170 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
@@ -29,6 +29,9 @@ import java.sql.ResultSet;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
+import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTableKey;
@@ -36,15 +39,17 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class SaltedIndexIT extends BaseIndexIT {
     private static final int TABLE_SPLITS = 3;
     private static final int INDEX_SPLITS = 4;
     
-    @BeforeClass 
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
@@ -54,7 +59,7 @@ public class SaltedIndexIT extends BaseIndexIT {
         // Drop the HBase table metadata for this test
         props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     private static void makeImmutableAndDeleteData() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
index 798fd28..83e2a08 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
@@ -31,16 +31,18 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
+import org.apache.phoenix.end2end.ClientManagedTimeTest;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
 /**
  * Tests for table with transparent salting.
  */
+@Category(ClientManagedTimeTest.class)
 public class SaltedTableIT extends BaseClientManagedTimeIT {
 
     private static void initTableValues(byte[][] splits, long ts) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
index 386632a..d5263ee 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
@@ -30,9 +30,11 @@ import java.sql.ResultSet;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.HBaseManagedTimeTest;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class SaltedTableUpsertSelectIT extends BaseHBaseManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
index 9bb3ec0..06142ec 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
@@ -19,7 +19,9 @@
 package org.apache.phoenix.end2end.salted;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.*;
+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;
@@ -28,9 +30,11 @@ import java.sql.ResultSet;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
+import org.apache.phoenix.end2end.ClientManagedTimeTest;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class SaltedTableVarLengthRowKeyIT extends BaseClientManagedTimeIT {
 
     private static void initTableValues() throws Exception {


[7/7] git commit: PHOENIX-982 Avoid spinning up and tearing down mini cluster in tests (SamarthJain)

Posted by ja...@apache.org.
PHOENIX-982 Avoid spinning up and tearing down mini cluster in tests (SamarthJain)


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

Branch: refs/heads/master
Commit: 91b7922c1c806b0432854b5f10fc40d350db8da4
Parents: 0ca6c1e
Author: James Taylor <ja...@apache.org>
Authored: Fri May 16 12:51:04 2014 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Fri May 16 12:51:04 2014 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    |    4 +-
 .../phoenix/end2end/ArithmeticQueryIT.java      |   11 +-
 .../org/apache/phoenix/end2end/ArrayIT.java     |  108 +-
 .../apache/phoenix/end2end/AutoCommitIT.java    |   14 +-
 .../end2end/BaseClientManagedTimeIT.java        |   97 +-
 .../phoenix/end2end/BaseConnectedQueryIT.java   |  763 -----
 .../phoenix/end2end/BaseHBaseManagedTimeIT.java |   87 +-
 .../BaseParallelIteratorsRegionSplitterIT.java  |    6 +-
 .../end2end/BaseTenantSpecificTablesIT.java     |    3 +
 .../org/apache/phoenix/end2end/BaseViewIT.java  |    7 +-
 .../apache/phoenix/end2end/BinaryRowKeyIT.java  |   13 +-
 .../phoenix/end2end/CSVCommonsLoaderIT.java     |    2 +
 .../apache/phoenix/end2end/CaseStatementIT.java |  287 ++
 .../apache/phoenix/end2end/CastAndCoerceIT.java |  167 ++
 .../phoenix/end2end/ClientManagedTimeTest.java  |   44 +
 .../end2end/ClientTimeArithmeticQueryIT.java    |  599 ++++
 .../phoenix/end2end/CoalesceFunctionIT.java     |   18 +-
 .../end2end/ColumnProjectionOptimizationIT.java |   32 +-
 .../phoenix/end2end/CompareDecimalToLongIT.java |   15 +-
 .../ConnectionQueryServicesTestImpl.java        |   79 +-
 .../apache/phoenix/end2end/CreateTableIT.java   |    2 +
 .../phoenix/end2end/CustomEntityDataIT.java     |   22 +-
 ...efaultParallelIteratorsRegionSplitterIT.java |    3 +
 .../org/apache/phoenix/end2end/DeleteIT.java    |    2 +
 .../apache/phoenix/end2end/DerivedTableIT.java  |    2 +
 .../apache/phoenix/end2end/DistinctCountIT.java |   29 +-
 .../apache/phoenix/end2end/DynamicColumnIT.java |    2 +
 .../apache/phoenix/end2end/DynamicFamilyIT.java |    2 +
 .../apache/phoenix/end2end/DynamicUpsertIT.java |   20 +-
 .../phoenix/end2end/ExecuteStatementsIT.java    |    7 +-
 .../phoenix/end2end/ExtendedQueryExecIT.java    |   19 +-
 .../apache/phoenix/end2end/FunkyNamesIT.java    |   19 +-
 .../apache/phoenix/end2end/GroupByCaseIT.java   |    6 +-
 .../org/apache/phoenix/end2end/GroupByIT.java   |  562 ++++
 .../phoenix/end2end/HBaseManagedTimeTest.java   |   45 +
 .../org/apache/phoenix/end2end/HashJoinIT.java  |   16 +-
 .../phoenix/end2end/InMemoryOrderByIT.java      |    9 +-
 .../org/apache/phoenix/end2end/IsNullIT.java    |    3 +-
 .../org/apache/phoenix/end2end/KeyOnlyIT.java   |    6 +-
 .../apache/phoenix/end2end/MD5FunctionIT.java   |    2 +
 .../phoenix/end2end/MultiCfQueryExecIT.java     |   15 +-
 .../phoenix/end2end/NativeHBaseTypesIT.java     |    2 +
 .../end2end/NeedsOwnMiniClusterTest.java        |   46 +
 .../org/apache/phoenix/end2end/NotQueryIT.java  |  303 ++
 .../org/apache/phoenix/end2end/OrderByIT.java   |    5 +-
 .../apache/phoenix/end2end/PercentileIT.java    |   32 +-
 .../phoenix/end2end/ProductMetricsIT.java       |   10 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |    3 +-
 .../phoenix/end2end/QueryExecWithoutSCNIT.java  |   16 +-
 .../org/apache/phoenix/end2end/QueryIT.java     | 2600 ++----------------
 .../org/apache/phoenix/end2end/QueryPlanIT.java |    7 +-
 .../phoenix/end2end/ReadIsolationLevelIT.java   |   18 +-
 .../phoenix/end2end/ReverseFunctionIT.java      |    2 +
 .../RoundFloorCeilFunctionsEnd2EndIT.java       |    8 +-
 .../phoenix/end2end/RowValueConstructorIT.java  |    2 +
 .../apache/phoenix/end2end/SaltedViewIT.java    |    2 +
 .../org/apache/phoenix/end2end/ScanQueryIT.java |  498 ++++
 .../org/apache/phoenix/end2end/SequenceIT.java  |    7 +-
 .../phoenix/end2end/ServerExceptionIT.java      |    3 +-
 .../org/apache/phoenix/end2end/Shadower.java    |   41 +
 ...ipRangeParallelIteratorRegionSplitterIT.java |    5 +-
 .../apache/phoenix/end2end/SkipScanQueryIT.java |    2 +
 .../apache/phoenix/end2end/SortOrderFIT.java    |    3 +
 .../phoenix/end2end/SpillableGroupByIT.java     |    9 +-
 .../phoenix/end2end/SpooledOrderByIT.java       |    9 +-
 .../phoenix/end2end/StatementHintsIT.java       |   14 +-
 .../apache/phoenix/end2end/StatsManagerIT.java  |    3 +
 .../org/apache/phoenix/end2end/StddevIT.java    |   15 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |    2 +
 .../end2end/TenantSpecificTablesDMLIT.java      |    2 +
 .../end2end/TenantSpecificViewIndexIT.java      |    2 +
 .../TenantSpecificViewIndexSaltedIT.java        |    2 +
 .../phoenix/end2end/ToCharFunctionIT.java       |    6 +-
 .../phoenix/end2end/ToNumberFunctionIT.java     |    8 +-
 .../java/org/apache/phoenix/end2end/TopNIT.java |    5 +-
 .../phoenix/end2end/TruncateFunctionIT.java     |    2 +
 .../phoenix/end2end/UpsertBigValuesIT.java      |   12 +-
 .../end2end/UpsertSelectAutoCommitIT.java       |    2 +
 .../apache/phoenix/end2end/UpsertSelectIT.java  |    3 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |    3 +-
 .../phoenix/end2end/VariableLengthPKIT.java     |    2 +
 .../java/org/apache/phoenix/end2end/ViewIT.java |    2 +
 .../phoenix/end2end/index/BaseIndexIT.java      |    3 +
 .../end2end/index/BaseMutableIndexIT.java       |    3 +
 .../phoenix/end2end/index/DropViewIT.java       |   16 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java |    4 +-
 .../phoenix/end2end/index/IndexMetadataIT.java  |    4 +-
 .../end2end/index/MutableIndexFailureIT.java    |  108 +-
 .../phoenix/end2end/index/MutableIndexIT.java   |   12 +-
 .../phoenix/end2end/index/SaltedIndexIT.java    |   11 +-
 .../phoenix/end2end/salted/SaltedTableIT.java   |    6 +-
 .../salted/SaltedTableUpsertSelectIT.java       |    4 +-
 .../salted/SaltedTableVarLengthRowKeyIT.java    |    8 +-
 .../FailForUnsupportedHBaseVersionsIT.java      |  226 +-
 .../EndToEndCoveredColumnsIndexBuilderIT.java   |   19 +-
 .../example/EndToEndCoveredIndexingIT.java      |   16 +-
 .../EndtoEndIndexingWithCompressionIT.java      |   13 +-
 .../covered/example/FailWithoutRetriesIT.java   |   18 +-
 .../phoenix/mapreduce/CsvBulkLoadToolIT.java    |   29 +-
 .../org/apache/phoenix/jdbc/PhoenixDriver.java  |    7 +-
 .../phoenix/query/BaseQueryServicesImpl.java    |    1 +
 .../index/write/TestWALRecoveryCaching.java     |    5 +
 .../apache/phoenix/jdbc/PhoenixTestDriver.java  |   46 +-
 .../query/BaseConnectionlessQueryTest.java      |   27 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  950 ++++++-
 .../phoenix/query/QueryServicesTestImpl.java    |   12 +-
 .../org/apache/phoenix/flume/PhoenixSinkIT.java |    8 +-
 .../phoenix/flume/RegexEventSerializerIT.java   |    9 +-
 .../phoenix/pig/PhoenixHBaseLoaderIT.java       |   11 +-
 .../phoenix/pig/PhoenixHBaseStorerIT.java       |   38 +-
 .../phoenix/pig/PhoenixPigConfigurationIT.java  |    3 +
 pom.xml                                         |   62 +-
 112 files changed, 4844 insertions(+), 3702 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/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 70b3d9a..b822108 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
@@ -47,9 +47,9 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
-
+@Category(HBaseManagedTimeTest.class)
 public class AlterTableIT extends BaseHBaseManagedTimeIT {
     public static final String SCHEMA_NAME = "";
     public static final String DATA_TABLE_NAME = "T";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
index 84cdc2f..4874174 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
@@ -34,11 +34,12 @@ import java.util.Properties;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.primitives.Doubles;
 import com.google.common.primitives.Floats;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class ArithmeticQueryIT extends BaseHBaseManagedTimeIT {
 
     @Test
@@ -524,7 +525,7 @@ public class ArithmeticQueryIT extends BaseHBaseManagedTimeIT {
     }
     @Test
     public void testSumDouble() throws Exception {
-        initSumDoubleValues(null);
+        initSumDoubleValues(null, getUrl());
         String query = "SELECT SUM(d) FROM SumDoubleTest";
         Properties props = new Properties(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -541,7 +542,7 @@ public class ArithmeticQueryIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSumUnsignedDouble() throws Exception {
-        initSumDoubleValues(null);
+        initSumDoubleValues(null, getUrl());
         String query = "SELECT SUM(ud) FROM SumDoubleTest";
         Properties props = new Properties(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -558,7 +559,7 @@ public class ArithmeticQueryIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSumFloat() throws Exception {
-        initSumDoubleValues(null);
+        initSumDoubleValues(null, getUrl());
         String query = "SELECT SUM(f) FROM SumDoubleTest";
         Properties props = new Properties(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -575,7 +576,7 @@ public class ArithmeticQueryIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSumUnsignedFloat() throws Exception {
-        initSumDoubleValues(null);
+        initSumDoubleValues(null, getUrl());
         String query = "SELECT SUM(uf) FROM SumDoubleTest";
         Properties props = new Properties(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
index 2c44ec6..7b13447 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
@@ -43,9 +43,11 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.primitives.Floats;
 
+@Category(ClientManagedTimeTest.class)
 public class ArrayIT extends BaseClientManagedTimeIT {
 
 	private static final String SIMPLE_TABLE_WITH_ARRAY = "SIMPLE_TABLE_WITH_ARRAY";
@@ -54,9 +56,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testScanByArrayValue() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_double_array, /* comment ok? */ b_string, a_float FROM table_with_array WHERE ?=organization_id and ?=a_float";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -90,9 +92,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testScanWithArrayInWhereClause() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_double_array, /* comment ok? */ b_string, a_float FROM table_with_array WHERE ?=organization_id and ?=a_byte_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -130,9 +132,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testScanWithNonFixedWidthArrayInWhereClause() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_double_array, /* comment ok? */ b_string, a_float FROM table_with_array WHERE ?=organization_id and ?=a_string_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -172,9 +174,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testScanWithNonFixedWidthArrayInSelectClause() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_string_array FROM table_with_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -203,9 +205,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 			throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT ARRAY_ELEM(a_double_array,2) FROM table_with_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -231,9 +233,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testSelectSpecificIndexOfAnArray() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_double_array[3] FROM table_with_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -258,9 +260,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testCaseWithArray() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(),
+        createTableWithArray(getUrl(),
                 getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT CASE WHEN A_INTEGER = 1 THEN a_double_array ELSE null END [3] FROM table_with_array";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -285,7 +287,7 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testUpsertValuesWithArray() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+        createTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
         String query = "upsert into table_with_array(ORGANIZATION_ID,ENTITY_ID,a_double_array) values('" + tenantId
                 + "','00A123122312312',ARRAY[2.0d,345.8d])";
         Properties props = new Properties(TEST_PROPERTIES);
@@ -323,10 +325,10 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testUpsertSelectWithSelectAsSubQuery1() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+        createTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
         Connection conn = null;
         try {
-            createSimpleTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+            createSimpleTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
             initSimpleArrayTable(tenantId, null, ts, false);
             Properties props = new Properties(TEST_PROPERTIES);
             props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -366,8 +368,8 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testSelectWithArrayWithColumnRef() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        createTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT a_integer,ARRAY[1,2,a_integer] FROM table_with_array where organization_id =  '"
                 + tenantId + "'";
         Properties props = new Properties(TEST_PROPERTIES);
@@ -397,8 +399,8 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testSelectWithArrayWithColumnRefWithVarLengthArray() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        createTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT b_string,ARRAY['abc','defgh',b_string] FROM table_with_array where organization_id =  '"
                 + tenantId + "'";
         Properties props = new Properties(TEST_PROPERTIES);
@@ -428,8 +430,8 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testSelectWithArrayWithColumnRefWithVarLengthArrayWithNullValue() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        createTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT b_string,ARRAY['abc',null,'bcd',null,null,b_string] FROM table_with_array where organization_id =  '"
                 + tenantId + "'";
         Properties props = new Properties(TEST_PROPERTIES);
@@ -462,10 +464,10 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testUpsertSelectWithColumnRef() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+        createTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
         Connection conn = null;
         try {
-            createSimpleTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+            createSimpleTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
             initSimpleArrayTable(tenantId, null, ts, false);
             Properties props = new Properties(TEST_PROPERTIES);
             props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -505,7 +507,7 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testCharArraySpecificIndex() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createSimpleTableWithArray(BaseConnectedQueryIT.getUrl(), getDefaultSplits(tenantId), null, ts - 2);
+        createSimpleTableWithArray(getUrl(), getDefaultSplits(tenantId), null, ts - 2);
         initSimpleArrayTable(tenantId, null, ts, false);
         String query = "SELECT a_char_array[2] FROM SIMPLE_TABLE_WITH_ARRAY";
         Properties props = new Properties(TEST_PROPERTIES);
@@ -703,9 +705,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testSelectArrayUsingUpsertLikeSyntax() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_double_array FROM TABLE_WITH_ARRAY WHERE a_double_array = ARRAY [ 25.343d, 36.763d, 37.56d,386.63d]";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -733,9 +735,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testArrayIndexUsedInWhereClause() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		int a_index = 0;
 		String query = "SELECT a_double_array[2] FROM table_with_array where a_double_array["+a_index+"2]<?";
 		Properties props = new Properties(TEST_PROPERTIES);
@@ -765,9 +767,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testArrayIndexUsedInGroupByClause() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_double_array[2] FROM table_with_array  GROUP BY a_double_array[2]";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -794,9 +796,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testVariableLengthArrayWithNullValue() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, true);
+		initTablesWithArrays(tenantId, null, ts, true, getUrl());
 		String query = "SELECT a_string_array[2] FROM table_with_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -819,9 +821,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testSelectSpecificIndexOfAVariableArrayAlongWithAnotherColumn1() throws Exception {
 	    long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(),
+        createTableWithArray(getUrl(),
                 getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT a_string_array[3],A_INTEGER FROM table_with_array";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -847,9 +849,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testSelectSpecificIndexOfAVariableArrayAlongWithAnotherColumn2() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(),
+        createTableWithArray(getUrl(),
                 getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT A_INTEGER, a_string_array[3] FROM table_with_array";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -875,9 +877,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testSelectMultipleArrayColumns() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(),
+        createTableWithArray(getUrl(),
                 getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT  a_string_array[3], a_double_array[2] FROM table_with_array";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -905,9 +907,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testSelectSameArrayColumnMultipleTimesWithDifferentIndices() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(),
+        createTableWithArray(getUrl(),
                 getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT a_string_array[1], a_string_array[3] FROM table_with_array";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -934,9 +936,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testSelectSameArrayColumnMultipleTimesWithSameIndices() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(),
+        createTableWithArray(getUrl(),
                 getDefaultSplits(tenantId), null, ts - 2);
-        initTablesWithArrays(tenantId, null, ts, false);
+        initTablesWithArrays(tenantId, null, ts, false, getUrl());
         String query = "SELECT a_string_array[3], a_string_array[3] FROM table_with_array";
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -962,9 +964,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testSelectSpecificIndexOfAVariableArray() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_string_array[3] FROM table_with_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -988,9 +990,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testWithOutOfRangeIndex() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT a_double_array[100] FROM table_with_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -1016,9 +1018,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testArrayLengthFunctionForVariableLength() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT ARRAY_LENGTH(a_string_array) FROM table_with_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -1041,9 +1043,9 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 	public void testArrayLengthFunctionForFixedLength() throws Exception {
 		long ts = nextTimestamp();
 		String tenantId = getOrganizationId();
-		createTableWithArray(BaseConnectedQueryIT.getUrl(),
+		createTableWithArray(getUrl(),
 				getDefaultSplits(tenantId), null, ts - 2);
-		initTablesWithArrays(tenantId, null, ts, false);
+		initTablesWithArrays(tenantId, null, ts, false, getUrl());
 		String query = "SELECT ARRAY_LENGTH(a_double_array) FROM table_with_array";
 		Properties props = new Properties(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
@@ -1065,7 +1067,7 @@ public class ArrayIT extends BaseClientManagedTimeIT {
     public void testArraySizeRoundtrip() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        createTableWithArray(BaseConnectedQueryIT.getUrl(),
+        createTableWithArray(getUrl(),
                 getDefaultSplits(tenantId), null, ts - 2);
         Properties props = new Properties(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoCommitIT.java
index 0f4f642..b117ced 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoCommitIT.java
@@ -17,15 +17,21 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+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.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class AutoCommitIT extends BaseHBaseManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
index 61e2762..7590a31 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
@@ -17,25 +17,104 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Date;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
 
 /**
- * 
  * Base class for tests that manage their own time stamps
- * We need to separate these from tests that manged the time stamp
- * themselves, because we create/destroy the Phoenix tables
+ * We need to separate these from tests that rely on hbase to set
+ * timestamps, because we create/destroy the Phoenix tables
  * between tests and only allow a table time stamp to increase.
- * If we let HBase set the time stamps, then our client time stamps
- * will usually be smaller than these time stamps and the table
- * deletion/creation would fail.
+ * Without this separation table deletion/creation would fail.
  * 
+ * All tests extending this class use the mini cluster that is
+ * different from the mini cluster used by test classes extending 
+ * {@link BaseHBaseManagedTimeIT}.
  * 
  * @since 0.1
  */
-public abstract class BaseClientManagedTimeIT extends BaseConnectedQueryIT {
+@NotThreadSafe
+@Category(ClientManagedTimeTest.class)
+public abstract class BaseClientManagedTimeIT extends BaseTest {
+    private static String url;
+    protected static PhoenixTestDriver driver;
+    private static final Configuration config = HBaseConfiguration.create(); 
+    private static boolean clusterInitialized = false;
+    
+    protected final static String getUrl() {
+        return checkClusterInitialized();
+    }
+    
+    protected static Configuration getTestClusterConfig() {
+        // don't want callers to modify config.
+        return new Configuration(config);
+    }
+    
     @Before
-    public void doTestSetup() throws Exception {
+    public void cleanUpBeforeTest() throws Exception {
         long ts = nextTimestamp();
-        deletePriorTables(ts-1);    
+        deletePriorTables(ts - 1, getUrl());    
+    }
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        setUpTestDriver(getUrl(), ReadOnlyProps.EMPTY_PROPS);
+    }
+    
+    protected static void setUpTestDriver(String url, ReadOnlyProps props) throws Exception {
+        if (PhoenixEmbeddedDriver.isTestUrl(url)) {
+            checkClusterInitialized();
+            if (driver == null) {
+                driver = initAndRegisterDriver(url, props);
+            }
+        }
+    }
+
+    private static String checkClusterInitialized() {
+        if (!clusterInitialized) {
+            url = setUpTestCluster(config);
+            clusterInitialized = true;
+        }
+        return url;
+    }
+    
+    @AfterClass
+    public static void dropTables() throws Exception {
+        try {
+            disableAndDropNonSystemTables(driver);
+        } finally {
+            try {
+                assertTrue(destroyDriver(driver));
+            } finally {
+                driver = null;
+            }
+        }
+    }
+    
+    protected static void initATableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
+        BaseTest.initATableValues(tenantId, splits, date, ts, getUrl());
+    }
+    
+    protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
+        BaseTest.initEntityHistoryTableValues(tenantId, splits, date, ts, getUrl());
+    }
+    
+    protected static void initSaltedEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
+        BaseTest.initSaltedEntityHistoryTableValues(tenantId, splits, date, ts, getUrl());
     }
+        
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseConnectedQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseConnectedQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseConnectedQueryIT.java
deleted file mode 100644
index e6bb65f..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseConnectedQueryIT.java
+++ /dev/null
@@ -1,763 +0,0 @@
-/*
- * 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.PhoenixRuntime.CURRENT_SCN_ATTRIB;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
-import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
-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.ENTITYHISTID1;
-import static org.apache.phoenix.util.TestUtil.ENTITYHISTID2;
-import static org.apache.phoenix.util.TestUtil.ENTITYHISTID3;
-import static org.apache.phoenix.util.TestUtil.ENTITYHISTID4;
-import static org.apache.phoenix.util.TestUtil.ENTITYHISTID5;
-import static org.apache.phoenix.util.TestUtil.ENTITYHISTID6;
-import static org.apache.phoenix.util.TestUtil.ENTITYHISTID7;
-import static org.apache.phoenix.util.TestUtil.ENTITYHISTID8;
-import static org.apache.phoenix.util.TestUtil.ENTITYHISTID9;
-import static org.apache.phoenix.util.TestUtil.ENTITY_HISTORY_SALTED_TABLE_NAME;
-import static org.apache.phoenix.util.TestUtil.ENTITY_HISTORY_TABLE_NAME;
-import static org.apache.phoenix.util.TestUtil.E_VALUE;
-import static org.apache.phoenix.util.TestUtil.MILLIS_IN_DAY;
-import static org.apache.phoenix.util.TestUtil.PARENTID1;
-import static org.apache.phoenix.util.TestUtil.PARENTID2;
-import static org.apache.phoenix.util.TestUtil.PARENTID3;
-import static org.apache.phoenix.util.TestUtil.PARENTID4;
-import static org.apache.phoenix.util.TestUtil.PARENTID5;
-import static org.apache.phoenix.util.TestUtil.PARENTID6;
-import static org.apache.phoenix.util.TestUtil.PARENTID7;
-import static org.apache.phoenix.util.TestUtil.PARENTID8;
-import static org.apache.phoenix.util.TestUtil.PARENTID9;
-import static org.apache.phoenix.util.TestUtil.ROW1;
-import static org.apache.phoenix.util.TestUtil.ROW2;
-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.ROW8;
-import static org.apache.phoenix.util.TestUtil.ROW9;
-
-import java.math.BigDecimal;
-import java.sql.Array;
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.Date;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.Types;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.IntegrationTestingUtility;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.query.BaseTest;
-import org.apache.phoenix.query.HBaseFactoryProvider;
-import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
-import org.apache.phoenix.schema.PTableType;
-import org.apache.phoenix.schema.TableNotFoundException;
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.SchemaUtil;
-import org.apache.phoenix.util.TestUtil;
-import org.junit.BeforeClass;
-import org.slf4j.LoggerFactory;
-
-/**
- * 
- * Base class for tests that need to be connected to an HBase server
- *
- * 
- * @since 0.1
- */
-public abstract class BaseConnectedQueryIT extends BaseTest {
-    private static final org.slf4j.Logger logger = LoggerFactory.getLogger(BaseConnectedQueryIT.class);
-    
-    protected static byte[][] getDefaultSplits(String tenantId) {
-        return new byte[][] { 
-            Bytes.toBytes(tenantId + "00A"),
-            Bytes.toBytes(tenantId + "00B"),
-            Bytes.toBytes(tenantId + "00C"),
-            };
-    }
-    
-    protected static String getUrl() {
-      Configuration conf = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
-      boolean isDistributedCluster = false;
-      isDistributedCluster =
-          Boolean.parseBoolean(System.getProperty(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER,
-            "false"));
-      if (!isDistributedCluster) {
-        isDistributedCluster =
-            conf.getBoolean(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER, false);
-      }
-      // reconstruct url when running against a live cluster
-      if (isDistributedCluster) {
-        // Get all info from hbase-site.xml
-        return JDBC_PROTOCOL + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;      
-      } else {
-        return TestUtil.PHOENIX_JDBC_URL;
-      }
-    }
-
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        startServer(getUrl());
-    }
-    
-    protected static void deletePriorTables(long ts) throws Exception {
-        deletePriorTables(ts, (String)null);
-    }
-    
-    protected static void deletePriorTables(long ts, String tenantId) throws Exception {
-        Properties props = new Properties();
-        if (ts != HConstants.LATEST_TIMESTAMP) {
-            props.setProperty(CURRENT_SCN_ATTRIB, Long.toString(ts));
-        }
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            deletePriorTables(ts, conn);
-            deletePriorSequences(ts, conn);
-        }
-        finally {
-            conn.close();
-        }
-    }
-    
-    private static void deletePriorTables(long ts, Connection globalConn) throws Exception {
-        DatabaseMetaData dbmd = globalConn.getMetaData();
-        // Drop VIEWs first, as we don't allow a TABLE with views to be dropped
-        // Tables are sorted by TENANT_ID
-        List<String[]> tableTypesList = Arrays.asList(new String[] {PTableType.VIEW.toString()}, new String[] {PTableType.TABLE.toString()});
-        for (String[] tableTypes: tableTypesList) {
-            ResultSet rs = dbmd.getTables(null, null, null, tableTypes);
-            String lastTenantId = null;
-            Connection conn = globalConn;
-            while (rs.next()) {
-                String fullTableName = SchemaUtil.getEscapedTableName(
-                        rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM),
-                        rs.getString(PhoenixDatabaseMetaData.TABLE_NAME));
-                String ddl = "DROP " + rs.getString(PhoenixDatabaseMetaData.TABLE_TYPE) + " " + fullTableName;
-                String tenantId = rs.getString(1);
-                if (tenantId != null && !tenantId.equals(lastTenantId))  {
-                    if (lastTenantId != null) {
-                        conn.close();
-                    }
-                    // Open tenant-specific connection when we find a new one
-                    Properties props = new Properties(globalConn.getClientInfo());
-                    props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
-                    conn = DriverManager.getConnection(getUrl(), props);
-                    lastTenantId = tenantId;
-                }
-                try {
-                    conn.createStatement().executeUpdate(ddl);
-                } catch (NewerTableAlreadyExistsException ex) {
-                    logger.info("Newer table " + fullTableName + " or its delete marker exists. Ignore current deletion");
-                } catch (TableNotFoundException ex) {
-                    logger.info("Table " + fullTableName + " is already deleted.");
-                }
-            }
-            if (lastTenantId != null) {
-                conn.close();
-            }
-        }
-    }
-    
-    private static void deletePriorSequences(long ts, Connection conn) throws Exception {
-        // TODO: drop tenant-specific sequences too
-        ResultSet rs = conn.createStatement().executeQuery("SELECT " 
-                + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + "," 
-                + PhoenixDatabaseMetaData.SEQUENCE_NAME 
-                + " FROM " + PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME);
-        while (rs.next()) {
-            conn.createStatement().execute("DROP SEQUENCE " + SchemaUtil.getTableName(rs.getString(1), rs.getString(2)));
-        }
-    }
-    
-    protected static void initSumDoubleValues(byte[][] splits) throws Exception {
-        ensureTableCreated(getUrl(), "SumDoubleTest", splits);
-        Properties props = new Properties();
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            // Insert all rows at ts
-            PreparedStatement stmt = conn.prepareStatement(
-                    "upsert into " +
-                    "SumDoubleTest(" +
-                    "    id, " +
-                    "    d, " +
-                    "    f, " +
-                    "    ud, " +
-                    "    uf) " +
-                    "VALUES (?, ?, ?, ?, ?)");
-            stmt.setString(1, "1");
-            stmt.setDouble(2, 0.001);
-            stmt.setFloat(3, 0.01f);
-            stmt.setDouble(4, 0.001);
-            stmt.setFloat(5, 0.01f);
-            stmt.execute();
-                
-            stmt.setString(1, "2");
-            stmt.setDouble(2, 0.002);
-            stmt.setFloat(3, 0.02f);
-            stmt.setDouble(4, 0.002);
-            stmt.setFloat(5, 0.02f);
-            stmt.execute();
-                
-            stmt.setString(1, "3");
-            stmt.setDouble(2, 0.003);
-            stmt.setFloat(3, 0.03f);
-            stmt.setDouble(4, 0.003);
-            stmt.setFloat(5, 0.03f);
-            stmt.execute();
-                
-            stmt.setString(1, "4");
-            stmt.setDouble(2, 0.004);
-            stmt.setFloat(3, 0.04f);
-            stmt.setDouble(4, 0.004);
-            stmt.setFloat(5, 0.04f);
-            stmt.execute();
-                
-            stmt.setString(1, "5");
-            stmt.setDouble(2, 0.005);
-            stmt.setFloat(3, 0.05f);
-            stmt.setDouble(4, 0.005);
-            stmt.setFloat(5, 0.05f);
-            stmt.execute();
-                
-            conn.commit();
-        } finally {
-            conn.close();
-        }
-    }
-    
-    protected static void initATableValues(String tenantId, byte[][] splits) throws Exception {
-        initATableValues(tenantId, splits, null);
-    }
-    
-    protected static void initATableValues(String tenantId, byte[][] splits, Date date) throws Exception {
-        initATableValues(tenantId, splits, date, null);
-    }
-    
-    protected static void initTablesWithArrays(String tenantId, Date date, Long ts, boolean useNull) throws Exception {
-    	 Properties props = new Properties();
-         if (ts != null) {
-             props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
-         }
-         Connection conn = DriverManager.getConnection(getUrl(), props);
-         try {
-             // Insert all rows at ts
-             PreparedStatement stmt = conn.prepareStatement(
-                     "upsert into " +
-                     "TABLE_WITH_ARRAY(" +
-                     "    ORGANIZATION_ID, " +
-                     "    ENTITY_ID, " +
-                     "    a_string_array, " +
-                     "    B_STRING, " +
-                     "    A_INTEGER, " +
-                     "    A_DATE, " +
-                     "    X_DECIMAL, " +
-                     "    x_long_array, " +
-                     "    X_INTEGER," +
-                     "    a_byte_array," +
-                     "    A_SHORT," +
-                     "    A_FLOAT," +
-                     "    a_double_array," +
-                     "    A_UNSIGNED_FLOAT," +
-                     "    A_UNSIGNED_DOUBLE)" +
-                     "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)");
-             stmt.setString(1, tenantId);
-             stmt.setString(2, ROW1);
-             // Need to support primitive
-             String[] strArr =  new String[4];
-             strArr[0] = "ABC";
-			if (useNull) {
-				strArr[1] = null;
-			} else {
-				strArr[1] = "CEDF";
-			}
-             strArr[2] = "XYZWER";
-             strArr[3] = "AB";
-             Array array = conn.createArrayOf("VARCHAR", strArr);
-             stmt.setArray(3, array);
-             stmt.setString(4, B_VALUE);
-             stmt.setInt(5, 1);
-             stmt.setDate(6, date);
-             stmt.setBigDecimal(7, null);
-             // Need to support primitive
-             Long[] longArr =  new Long[2];
-             longArr[0] = 25l;
-             longArr[1] = 36l;
-             array = conn.createArrayOf("BIGINT", longArr);
-             stmt.setArray(8, array);
-             stmt.setNull(9, Types.INTEGER);
-             // Need to support primitive
-             Byte[] byteArr =  new Byte[2];
-             byteArr[0] = 25;
-             byteArr[1] = 36;
-             array = conn.createArrayOf("TINYINT", byteArr);
-             stmt.setArray(10, array);
-             stmt.setShort(11, (short) 128);
-             stmt.setFloat(12, 0.01f);
-             // Need to support primitive
-             Double[] doubleArr =  new Double[4];
-             doubleArr[0] = 25.343;
-             doubleArr[1] = 36.763;
-             doubleArr[2] = 37.56;
-             doubleArr[3] = 386.63;
-             array = conn.createArrayOf("DOUBLE", doubleArr);
-             stmt.setArray(13, array);
-             stmt.setFloat(14, 0.01f);
-             stmt.setDouble(15, 0.0001);
-             stmt.execute();
-                 
-             conn.commit();
-         } finally {
-             conn.close();
-         }
-    }
-    
-    protected static void initATableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
-        if (ts == null) {
-            ensureTableCreated(getUrl(), ATABLE_NAME, splits);
-        } else {
-            ensureTableCreated(getUrl(), ATABLE_NAME, splits, ts-5);
-        }
-        
-        Properties props = new Properties();
-        if (ts != null) {
-            props.setProperty(CURRENT_SCN_ATTRIB, Long.toString(ts-3));
-        }
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            // Insert all rows at ts
-            PreparedStatement stmt = conn.prepareStatement(
-                    "upsert into " +
-                    "ATABLE(" +
-                    "    ORGANIZATION_ID, " +
-                    "    ENTITY_ID, " +
-                    "    A_STRING, " +
-                    "    B_STRING, " +
-                    "    A_INTEGER, " +
-                    "    A_DATE, " +
-                    "    X_DECIMAL, " +
-                    "    X_LONG, " +
-                    "    X_INTEGER," +
-                    "    Y_INTEGER," +
-                    "    A_BYTE," +
-                    "    A_SHORT," +
-                    "    A_FLOAT," +
-                    "    A_DOUBLE," +
-                    "    A_UNSIGNED_FLOAT," +
-                    "    A_UNSIGNED_DOUBLE)" +
-                    "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)");
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW1);
-            stmt.setString(3, A_VALUE);
-            stmt.setString(4, B_VALUE);
-            stmt.setInt(5, 1);
-            stmt.setDate(6, date);
-            stmt.setBigDecimal(7, null);
-            stmt.setNull(8, Types.BIGINT);
-            stmt.setNull(9, Types.INTEGER);
-            stmt.setNull(10, Types.INTEGER);
-            stmt.setByte(11, (byte)1);
-            stmt.setShort(12, (short) 128);
-            stmt.setFloat(13, 0.01f);
-            stmt.setDouble(14, 0.0001);
-            stmt.setFloat(15, 0.01f);
-            stmt.setDouble(16, 0.0001);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW2);
-            stmt.setString(3, A_VALUE);
-            stmt.setString(4, C_VALUE);
-            stmt.setInt(5, 2);
-            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 1));
-            stmt.setBigDecimal(7, null);
-            stmt.setNull(8, Types.BIGINT);
-            stmt.setNull(9, Types.INTEGER);
-            stmt.setNull(10, Types.INTEGER);
-            stmt.setByte(11, (byte)2);
-            stmt.setShort(12, (short) 129);
-            stmt.setFloat(13, 0.02f);
-            stmt.setDouble(14, 0.0002);
-            stmt.setFloat(15, 0.02f);
-            stmt.setDouble(16, 0.0002);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW3);
-            stmt.setString(3, A_VALUE);
-            stmt.setString(4, E_VALUE);
-            stmt.setInt(5, 3);
-            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 2));
-            stmt.setBigDecimal(7, null);
-            stmt.setNull(8, Types.BIGINT);
-            stmt.setNull(9, Types.INTEGER);
-            stmt.setNull(10, Types.INTEGER);
-            stmt.setByte(11, (byte)3);
-            stmt.setShort(12, (short) 130);
-            stmt.setFloat(13, 0.03f);
-            stmt.setDouble(14, 0.0003);
-            stmt.setFloat(15, 0.03f);
-            stmt.setDouble(16, 0.0003);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW4);
-            stmt.setString(3, A_VALUE);
-            stmt.setString(4, B_VALUE);
-            stmt.setInt(5, 4);
-            stmt.setDate(6, date == null ? null : date);
-            stmt.setBigDecimal(7, null);
-            stmt.setNull(8, Types.BIGINT);
-            stmt.setNull(9, Types.INTEGER);
-            stmt.setNull(10, Types.INTEGER);
-            stmt.setByte(11, (byte)4);
-            stmt.setShort(12, (short) 131);
-            stmt.setFloat(13, 0.04f);
-            stmt.setDouble(14, 0.0004);
-            stmt.setFloat(15, 0.04f);
-            stmt.setDouble(16, 0.0004);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW5);
-            stmt.setString(3, B_VALUE);
-            stmt.setString(4, C_VALUE);
-            stmt.setInt(5, 5);
-            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 1));
-            stmt.setBigDecimal(7, null);
-            stmt.setNull(8, Types.BIGINT);
-            stmt.setNull(9, Types.INTEGER);
-            stmt.setNull(10, Types.INTEGER);
-            stmt.setByte(11, (byte)5);
-            stmt.setShort(12, (short) 132);
-            stmt.setFloat(13, 0.05f);
-            stmt.setDouble(14, 0.0005);
-            stmt.setFloat(15, 0.05f);
-            stmt.setDouble(16, 0.0005);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW6);
-            stmt.setString(3, B_VALUE);
-            stmt.setString(4, E_VALUE);
-            stmt.setInt(5, 6);
-            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 2));
-            stmt.setBigDecimal(7, null);
-            stmt.setNull(8, Types.BIGINT);
-            stmt.setNull(9, Types.INTEGER);
-            stmt.setNull(10, Types.INTEGER);
-            stmt.setByte(11, (byte)6);
-            stmt.setShort(12, (short) 133);
-            stmt.setFloat(13, 0.06f);
-            stmt.setDouble(14, 0.0006);
-            stmt.setFloat(15, 0.06f);
-            stmt.setDouble(16, 0.0006);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW7);
-            stmt.setString(3, B_VALUE);
-            stmt.setString(4, B_VALUE);
-            stmt.setInt(5, 7);
-            stmt.setDate(6, date == null ? null : date);
-            stmt.setBigDecimal(7, BigDecimal.valueOf(0.1));
-            stmt.setLong(8, 5L);
-            stmt.setInt(9, 5);
-            stmt.setNull(10, Types.INTEGER);
-            stmt.setByte(11, (byte)7);
-            stmt.setShort(12, (short) 134);
-            stmt.setFloat(13, 0.07f);
-            stmt.setDouble(14, 0.0007);
-            stmt.setFloat(15, 0.07f);
-            stmt.setDouble(16, 0.0007);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW8);
-            stmt.setString(3, B_VALUE);
-            stmt.setString(4, C_VALUE);
-            stmt.setInt(5, 8);
-            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 1));
-            stmt.setBigDecimal(7, BigDecimal.valueOf(3.9));
-            long l = Integer.MIN_VALUE - 1L;
-            assert(l < Integer.MIN_VALUE);
-            stmt.setLong(8, l);
-            stmt.setInt(9, 4);
-            stmt.setNull(10, Types.INTEGER);
-            stmt.setByte(11, (byte)8);
-            stmt.setShort(12, (short) 135);
-            stmt.setFloat(13, 0.08f);
-            stmt.setDouble(14, 0.0008);
-            stmt.setFloat(15, 0.08f);
-            stmt.setDouble(16, 0.0008);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, ROW9);
-            stmt.setString(3, C_VALUE);
-            stmt.setString(4, E_VALUE);
-            stmt.setInt(5, 9);
-            stmt.setDate(6, date == null ? null : new Date(date.getTime() + MILLIS_IN_DAY * 2));
-            stmt.setBigDecimal(7, BigDecimal.valueOf(3.3));
-            l = Integer.MAX_VALUE + 1L;
-            assert(l > Integer.MAX_VALUE);
-            stmt.setLong(8, l);
-            stmt.setInt(9, 3);
-            stmt.setInt(10, 300);
-            stmt.setByte(11, (byte)9);
-            stmt.setShort(12, (short) 0);
-            stmt.setFloat(13, 0.09f);
-            stmt.setDouble(14, 0.0009);
-            stmt.setFloat(15, 0.09f);
-            stmt.setDouble(16, 0.0009);
-            stmt.execute();
-                
-            conn.commit();
-        } finally {
-            conn.close();
-        }
-    }
-    protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits) throws Exception {
-        initEntityHistoryTableValues(tenantId, splits, null);
-    }
-    
-    protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date) throws Exception {
-        initEntityHistoryTableValues(tenantId, splits, date, null);
-    }
-    
-    protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
-        if (ts == null) {
-            ensureTableCreated(getUrl(), ENTITY_HISTORY_TABLE_NAME, splits);
-        } else {
-            ensureTableCreated(getUrl(), ENTITY_HISTORY_TABLE_NAME, splits, ts-2);
-        }
-        
-        Properties props = new Properties();
-        if (ts != null) {
-            props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
-        }
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            // Insert all rows at ts
-            PreparedStatement stmt = conn.prepareStatement(
-                    "upsert into " +
-                    ENTITY_HISTORY_TABLE_NAME+
-                    "(" +
-                    "    ORGANIZATION_ID, " +
-                    "    PARENT_ID, " +
-                    "    CREATED_DATE, " +
-                    "    ENTITY_HISTORY_ID, " +
-                    "    OLD_VALUE, " +
-                    "    NEW_VALUE) " +
-                    "VALUES (?, ?, ?, ?, ?, ?)");
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID1);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID1);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID2);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID2);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID3);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID3);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID4);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID4);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID5);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID5);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID6);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID6);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID7);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID7);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID8);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID8);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID9);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID9);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            conn.commit();
-        } finally {
-            conn.close();
-        }
-    }
-    
-    protected static void initSaltedEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
-        if (ts == null) {
-            ensureTableCreated(getUrl(), ENTITY_HISTORY_SALTED_TABLE_NAME, splits);
-        } else {
-            ensureTableCreated(getUrl(), ENTITY_HISTORY_SALTED_TABLE_NAME, splits, ts-2);
-        }
-        
-        Properties props = new Properties();
-        if (ts != null) {
-            props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
-        }
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            // Insert all rows at ts
-            PreparedStatement stmt = conn.prepareStatement(
-                    "upsert into " +
-                    ENTITY_HISTORY_SALTED_TABLE_NAME+
-                    "(" +
-                    "    ORGANIZATION_ID, " +
-                    "    PARENT_ID, " +
-                    "    CREATED_DATE, " +
-                    "    ENTITY_HISTORY_ID, " +
-                    "    OLD_VALUE, " +
-                    "    NEW_VALUE) " +
-                    "VALUES (?, ?, ?, ?, ?, ?)");
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID1);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID1);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID2);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID2);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-                
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID3);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID3);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID4);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID4);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID5);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID5);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID6);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID6);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID7);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID7);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID8);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID8);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            stmt.setString(1, tenantId);
-            stmt.setString(2, PARENTID9);
-            stmt.setDate(3, date);
-            stmt.setString(4, ENTITYHISTID9);
-            stmt.setString(5,  A_VALUE);
-            stmt.setString(6,  B_VALUE);
-            stmt.execute();
-            
-            conn.commit();
-        } finally {
-            conn.close();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
index c00b74c..ae83585 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
@@ -17,25 +17,90 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertTrue;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
 
 /**
- * 
- * Base class for tests that have HBase manage the time stamps.
- * We need to separate these from tests that manged the time stamp
- * themselves, because we create/destroy the Phoenix tables
+ * Base class for tests that let HBase set timestamps.
+ * We need to separate these from tests that rely on clients 
+ * to set timestamps, because we create/destroy the Phoenix tables
  * between tests and only allow a table time stamp to increase.
- * If we let HBase set the time stamps, then our client time stamps
- * will usually be smaller than these time stamps and the table
- * deletion/creation would fail.
- *
+ * Without this separation table deletion/creation would fail.
+ * 
+ * All tests extending this class use the mini cluster that is
+ * different from the mini cluster used by test classes extending 
+ * {@link BaseClientManagedTimeIT}.
  * 
  * @since 0.1
  */
-public abstract class BaseHBaseManagedTimeIT extends BaseConnectedQueryIT {
+@NotThreadSafe
+@Category(HBaseManagedTimeTest.class)
+public abstract class BaseHBaseManagedTimeIT extends BaseTest {
+    private static String url;
+    protected static PhoenixTestDriver driver;
+    private static final Configuration config = HBaseConfiguration.create(); 
+    private static boolean clusterInitialized = false;
+    
+    protected final static String getUrl() {
+        return checkClusterInitialized();
+    }
+    
+    protected static Configuration getTestClusterConfig() {
+        // don't want callers to modify config.
+        return new Configuration(config);
+    }
+    
     @Before
-    public void doTestSetup() throws Exception {
-        deletePriorTables(HConstants.LATEST_TIMESTAMP);
+    public void cleanUpBeforeTest() throws Exception {
+        deletePriorTables(HConstants.LATEST_TIMESTAMP, getUrl());    
+    }
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        setUpTestDriver(getUrl(), ReadOnlyProps.EMPTY_PROPS);
+    }
+    
+    protected static void setUpTestDriver(String url, ReadOnlyProps props) throws Exception {
+        if (PhoenixEmbeddedDriver.isTestUrl(url)) {
+            checkClusterInitialized();
+            if (driver == null) {
+                driver = initAndRegisterDriver(url, props);
+            }
+        }
+    }
+
+    private static String checkClusterInitialized() {
+        if (!clusterInitialized) {
+            url = setUpTestCluster(config);
+            clusterInitialized = true;
+        }
+        return url;
+    }
+    
+    @AfterClass
+    public static void dropTables() throws Exception {
+        try {
+            disableAndDropNonSystemTables(driver);
+        } finally {
+            try {
+                assertTrue(destroyDriver(driver));
+            } finally {
+                driver = null;
+            }
+        }
     }
+        
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
index c664fc8..0bc3324 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
@@ -34,10 +34,11 @@ import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
 
-
+@Category(ClientManagedTimeTest.class)
 public class BaseParallelIteratorsRegionSplitterIT extends BaseClientManagedTimeIT {
 
     protected static final byte[] KMIN  = new byte[] {'!'};
@@ -54,6 +55,7 @@ public class BaseParallelIteratorsRegionSplitterIT extends BaseClientManagedTime
     protected static final byte[] KMAX2  = new byte[] {'z'};
     
     @BeforeClass
+    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
         int targetQueryConcurrency = 3;
         int maxQueryConcurrency = 5;
@@ -62,7 +64,7 @@ public class BaseParallelIteratorsRegionSplitterIT extends BaseClientManagedTime
         props.put(QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB, Integer.toString(targetQueryConcurrency));
         props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(Integer.MAX_VALUE));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     protected void initTableValues(long ts) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
index db3118b..85d65e2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import java.sql.SQLException;
 
 import org.junit.Before;
+import org.junit.experimental.categories.Category;
 
 /**
  * Describe your class here.
@@ -29,6 +30,8 @@ import org.junit.Before;
  * 
  * @since 2.2
  */
+
+@Category(ClientManagedTimeTest.class)
 public abstract class BaseTenantSpecificTablesIT extends BaseClientManagedTimeIT {
     protected static final String TENANT_ID = "ZZTop";
     protected static final String TENANT_TYPE_ID = "abc";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
index 03e8a43..b125ab2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
@@ -31,18 +31,21 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Maps;
 
+@Category(HBaseManagedTimeTest.class)
 public class BaseViewIT extends BaseHBaseManagedTimeIT {
 
-    @BeforeClass 
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
         props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(1));
         // Must update config before starting server
-        startServer(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
 
     protected void testUpdatableViewWithIndex(Integer saltBuckets) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java
index a1cc0a8..d55cfd0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java
@@ -18,13 +18,22 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.Properties;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class BinaryRowKeyIT extends BaseHBaseManagedTimeIT {
 
     private static void initTableValues() throws SQLException {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
index 28142bc..1704e8f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
@@ -40,7 +40,9 @@ import org.apache.phoenix.util.CSVCommonsLoader;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
 
     private static final String DATATYPE_TABLE = "DATATYPE";


[6/7] PHOENIX-982 Avoid spinning up and tearing down mini cluster in tests (SamarthJain)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java
new file mode 100644
index 0000000..7babfb9
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java
@@ -0,0 +1,287 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+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.ROW7;
+import static org.apache.phoenix.util.TestUtil.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+
+@Category(ClientManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class CaseStatementIT extends QueryIT {
+
+    public CaseStatementIT(String indexDDL) {
+        super(indexDDL);
+    }
+    
+    @Parameters(name="{0}")
+    public static Collection<Object> data() {
+        return QueryIT.data();
+    }    
+    
+    @Test
+    public void testSimpleCaseStatement() throws Exception {
+        String query = "SELECT CASE a_integer WHEN 1 THEN 'a' WHEN 2 THEN 'b' WHEN 3 THEN 'c' ELSE 'd' END, entity_id AS a FROM ATABLE WHERE organization_id=? AND a_integer < 6";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            @SuppressWarnings("unchecked")
+            List<List<Object>> expectedResults = Lists.newArrayList(
+                Arrays.<Object>asList("a",ROW1),
+                Arrays.<Object>asList( "b",ROW2), 
+                Arrays.<Object>asList("c",ROW3),
+                Arrays.<Object>asList("d",ROW4),
+                Arrays.<Object>asList("d",ROW5));
+            assertValuesEqualsResultSet(rs, expectedResults);
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testMultiCondCaseStatement() throws Exception {
+        String query = "SELECT CASE WHEN a_integer <= 2 THEN 1.5 WHEN a_integer = 3 THEN 2 WHEN a_integer <= 6 THEN 4.5 ELSE 5 END AS a FROM ATABLE WHERE organization_id=?";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(1.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(1.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(2), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(4.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(4.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(4.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(5), rs.getBigDecimal(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testPartialEvalCaseStatement() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? and CASE WHEN 1234 = a_integer THEN 1 WHEN x_integer = 5 THEN 2 ELSE 3 END = 2";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(ROW7, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testFoundIndexOnPartialEvalCaseStatement() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? and CASE WHEN a_integer = 1234 THEN 1 WHEN x_integer = 3 THEN y_integer ELSE 3 END = 300";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    // TODO: we need some tests that have multiple versions of key values
+    @Test
+    public void testUnfoundMultiColumnCaseStatement() throws Exception {
+        String query = "SELECT entity_id, b_string FROM ATABLE WHERE organization_id=? and CASE WHEN a_integer = 1234 THEN 1 WHEN a_date < ? THEN y_integer WHEN x_integer = 4 THEN 4 ELSE 3 END = 4";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            statement.setDate(2, new Date(System.currentTimeMillis()));
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(ROW8, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testUnfoundSingleColumnCaseStatement() throws Exception {
+        String query = "SELECT entity_id, b_string FROM ATABLE 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 = new Properties(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 " +
+            "ATABLE(" +
+            "    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();
+    }
+    
+    @Test
+    public void testNonNullMultiCondCaseStatement() throws Exception {
+        String query = "SELECT CASE WHEN entity_id = '000000000000000' THEN 1 WHEN entity_id = '000000000000001' THEN 2 ELSE 3 END FROM ATABLE WHERE organization_id=?";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            ResultSetMetaData rsm = rs.getMetaData();
+            assertEquals(ResultSetMetaData.columnNoNulls,rsm.isNullable(1));
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testNullMultiCondCaseStatement() throws Exception {
+        String query = "SELECT CASE WHEN entity_id = '000000000000000' THEN 1 WHEN entity_id = '000000000000001' THEN 2 END FROM ATABLE WHERE organization_id=?";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            ResultSetMetaData rsm = rs.getMetaData();
+            assertEquals(ResultSetMetaData.columnNullable,rsm.isNullable(1));
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testNullabilityMultiCondCaseStatement() throws Exception {
+        String query = "SELECT CASE WHEN a_integer <= 2 THEN ? WHEN a_integer = 3 THEN ? WHEN a_integer <= ? THEN ? ELSE 5 END AS a FROM ATABLE WHERE organization_id=?";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setBigDecimal(1,BigDecimal.valueOf(1.5));
+            statement.setInt(2,2);
+            statement.setInt(3,6);
+            statement.setBigDecimal(4,BigDecimal.valueOf(4.5));
+            statement.setString(5, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(1.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(1.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(2), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(4.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(4.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(4.5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(5), rs.getBigDecimal(1));
+            assertTrue(rs.next());
+            assertEquals(BigDecimal.valueOf(5), rs.getBigDecimal(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java
new file mode 100644
index 0000000..a1ad306
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java
@@ -0,0 +1,167 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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.ROW7;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Collection;
+import java.util.Properties;
+
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@Category(ClientManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class CastAndCoerceIT extends QueryIT {
+
+    public CastAndCoerceIT(String indexDDL) {
+        super(indexDDL);
+    }
+    
+    @Parameters(name="{0}")
+    public static Collection<Object> data() {
+        return QueryIT.data();
+    }
+    
+    @Test
+    public void testCastOperatorInSelect() throws Exception {
+        String query = "SELECT CAST(a_integer AS decimal)/2 FROM aTable WHERE ?=organization_id and 5=a_integer";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(BigDecimal.valueOf(2.5), rs.getBigDecimal(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testCastOperatorInWhere() throws Exception {
+        String query = "SELECT a_integer FROM aTable WHERE ?=organization_id and 2.5 = CAST(a_integer AS DECIMAL)/2 ";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(5, rs.getInt(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testCoerceIntegerToLong() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_long >= x_integer";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(ROW7, rs.getString(1));
+            assertTrue(rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testCoerceLongToDecimal1() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_decimal > x_integer";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testCoerceLongToDecimal2() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_integer <= x_decimal";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testCoerceTinyIntToSmallInt() throws Exception {
+        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_byte >= a_short";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientManagedTimeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientManagedTimeTest.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientManagedTimeTest.java
new file mode 100644
index 0000000..2139ac7
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientManagedTimeTest.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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 java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * 
+ * Annotation to denote that the in the test client manages timestamps 
+ * itself and not HBase.
+ * 
+ * Tests using a mini cluster need to be classified either 
+ * as {@link HBaseManagedTimeTest} or {@link ClientManagedTimeTest} 
+ * or {@link NeedsOwnMiniClusterTest} otherwise they won't be run 
+ * when one runs mvn verify or mvn install.
+ *
+ * @since 4.1
+ */
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+public @interface ClientManagedTimeTest {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java
new file mode 100644
index 0000000..4b7e66e
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java
@@ -0,0 +1,599 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * 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 maynot 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 applicablelaw 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.B_VALUE;
+import static org.apache.phoenix.util.TestUtil.E_VALUE;
+import static org.apache.phoenix.util.TestUtil.MILLIS_IN_DAY;
+import static org.apache.phoenix.util.TestUtil.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+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.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+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.Properties;
+
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+
+@Category(ClientManagedTimeTest.class)
+@RunWith(Parameterized.class)
+public class ClientTimeArithmeticQueryIT extends QueryIT {
+
+    public ClientTimeArithmeticQueryIT(String indexDDL) {
+        super(indexDDL);
+    }
+    
+    @Parameters(name="{0}")
+    public static Collection<Object> data() {
+        return QueryIT.data();
+    }
+    
+    @Test
+    public void testDateAdd() throws Exception {
+        String query = "SELECT entity_id, b_string FROM ATABLE WHERE a_date + 0.5d < ?";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setDate(1, new Date(System.currentTimeMillis() + MILLIS_IN_DAY));
+            ResultSet rs = statement.executeQuery();
+            @SuppressWarnings("unchecked")
+            List<List<Object>> expectedResults = Lists.newArrayList(
+                    Arrays.<Object>asList(ROW1, B_VALUE),
+                    Arrays.<Object>asList( ROW4, B_VALUE), 
+                    Arrays.<Object>asList(ROW7, B_VALUE));
+            assertValuesEqualsResultSet(rs, expectedResults);
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testDecimalAddExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER + X_DECIMAL > 11";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW8, rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testDoubleAddExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where a_double + a_float > 0.08";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW8, rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testUnsignedDoubleAddExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where a_unsigned_double + a_unsigned_float > 0.08";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW8, rs.getString(1));
+            assertTrue (rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+            value="RV_RETURN_VALUE_IGNORED",
+            justification="Test code.")
+    @Test
+    public void testValidArithmetic() throws Exception {
+        String[] queries = new String[] { 
+                "SELECT entity_id,organization_id FROM atable where (A_DATE - A_DATE) * 5 < 0",
+                "SELECT entity_id,organization_id FROM atable where 1 + A_DATE  < A_DATE",
+                "SELECT entity_id,organization_id FROM atable where A_DATE - 1 < A_DATE",
+                "SELECT entity_id,organization_id FROM atable where A_INTEGER - 45 < 0",
+                "SELECT entity_id,organization_id FROM atable where X_DECIMAL / 45 < 0", };
+
+        for (String query : queries) {
+            Properties props = new Properties();
+            props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+            Connection conn = DriverManager.getConnection(getUrl(), props);
+            try {
+                PreparedStatement statement = conn.prepareStatement(query);
+                statement.executeQuery();
+            }
+            finally {
+                conn.close();
+            }
+        }
+    }
+    
+    @Test
+    public void testIntSubtractionExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER - 4  <= 0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW1, ROW2, ROW3, ROW4));
+        } finally {
+            conn.close();
+        }
+    }
+    @Test
+    public void testDecimalSubtraction1Expression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER - 3.5  <= 0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW1, ROW2, ROW3));
+        } finally {
+            conn.close();
+        }
+    }
+    @Test
+    public void testDecimalSubtraction2Expression() throws Exception {// check if decimal part makes a difference
+        String query = "SELECT entity_id FROM aTable where X_DECIMAL - 3.5  > 0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW8);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    @Test
+    public void testLongSubtractionExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where X_LONG - 1  < 0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW8);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    @Test
+    public void testDoubleSubtractionExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where a_double - 0.0002d  < 0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW1);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    @Test
+    public void testSmallIntSubtractionExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where a_short - 129  = 0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW2);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testTernarySubtractionExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where  X_INTEGER - X_LONG - 10  < 0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW7);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW9);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    @Test
+    public void testSelectWithSubtractionExpression() throws Exception {
+        String query = "SELECT entity_id, x_integer - 4 FROM aTable where  x_integer - 4 = 0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW8);
+            assertEquals(rs.getInt(2), 0);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    @Test
+    public void testConstantSubtractionExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER = 5 - 1 - 2";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW2);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testIntDivideExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER / 3 > 2";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testDoubleDivideExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where a_double / 3.0d = 0.0003";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testSmallIntDivideExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where a_short / 135 = 1";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW8, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testIntToDecimalDivideExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER / 3.0 > 2";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW7, ROW8, ROW9));
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testConstantDivideExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER = 9 / 3 / 3";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW1);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    @Test
+    public void testSelectWithDivideExpression() throws Exception {
+        String query = "SELECT entity_id, a_integer/3 FROM aTable where  a_integer = 9";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW9, rs.getString(1));
+            assertEquals(3, rs.getInt(2));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testNegateExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER - 4 = -1";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW3, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testIntMultiplyExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER * 2 = 16";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW8, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testDoubleMultiplyExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_DOUBLE * 2.0d = 0.0002";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW1, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testLongMultiplyExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where X_LONG * 2 * 2 = 20";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW7, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testIntToDecimalMultiplyExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER * 1.5 > 9";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW7, ROW8, ROW9));
+        } finally {
+            conn.close();
+        }
+    }
+    
+
+    @Test
+    public void testDecimalMultiplyExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where X_DECIMAL * A_INTEGER > 29.5";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW8, ROW9));
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testIntAddExpression() throws Exception {
+        String query = "SELECT entity_id FROM aTable where A_INTEGER + 2 = 4";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(ROW2, rs.getString(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testCoalesceFunction() throws Exception {
+        String query = "SELECT entity_id FROM aTable WHERE a_integer > 0 and coalesce(X_DECIMAL,0.0) = 0.0";
+        Properties props = new Properties(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10)); // Execute at timestamp 2
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO aTable(organization_id,entity_id,x_decimal) values(?,?,?)");
+        stmt.setString(1, getOrganizationId());
+        stmt.setString(2, ROW1);
+        stmt.setBigDecimal(3, BigDecimal.valueOf(1.0));
+        stmt.execute();
+        stmt.setString(2, ROW3);
+        stmt.setBigDecimal(3, BigDecimal.valueOf(2.0));
+        stmt.execute();
+        stmt.setString(2, ROW4);
+        stmt.setBigDecimal(3, BigDecimal.valueOf(3.0));
+        stmt.execute();
+        stmt.setString(2, ROW5);
+        stmt.setBigDecimal(3, BigDecimal.valueOf(0.0));
+        stmt.execute();
+        stmt.setString(2, ROW6);
+        stmt.setBigDecimal(3, BigDecimal.valueOf(4.0));
+        stmt.execute();
+        conn.commit();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20)); // Execute at timestamp 2
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW2);
+            assertTrue (rs.next());
+            assertEquals(rs.getString(1), ROW5);
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testDateSubtract() throws Exception {
+        String query = "SELECT entity_id, b_string FROM ATABLE WHERE a_date - 0.5d > ?";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setDate(1, new Date(System.currentTimeMillis() + MILLIS_IN_DAY));
+            ResultSet rs = statement.executeQuery();
+            @SuppressWarnings("unchecked")
+            List<List<Object>> expectedResults = Lists.newArrayList(
+                    Arrays.<Object>asList(ROW3, E_VALUE),
+                    Arrays.<Object>asList( ROW6, E_VALUE), 
+                    Arrays.<Object>asList(ROW9, E_VALUE));
+            assertValuesEqualsResultSet(rs, expectedResults);
+        } finally {
+            conn.close();
+        }
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
index b4289fa..48808ab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
@@ -17,17 +17,23 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+import static org.apache.phoenix.util.TestUtil.ROW6;
+import static org.apache.phoenix.util.TestUtil.ROW7;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class CoalesceFunctionIT extends BaseClientManagedTimeIT {
     @Test
     public void testCoalesce() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
index 41ac3a0..6fe7aec 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
@@ -17,22 +17,46 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+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.MDTEST_NAME;
+import static org.apache.phoenix.util.TestUtil.MDTEST_SCHEMA_NAME;
+import static org.apache.phoenix.util.TestUtil.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+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.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class ColumnProjectionOptimizationIT extends BaseClientManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CompareDecimalToLongIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CompareDecimalToLongIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CompareDecimalToLongIT.java
index cd4da2c..d67dbb9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CompareDecimalToLongIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CompareDecimalToLongIT.java
@@ -17,16 +17,21 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class CompareDecimalToLongIT extends BaseClientManagedTimeIT {
     protected static void initTableValues(byte[][] splits, long ts) throws Exception {
         ensureTableCreated(getUrl(),"LongInKeyTest",splits, ts-2);

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
index 5fb8e55..2f18188 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
@@ -20,11 +20,6 @@ package org.apache.phoenix.end2end;
 import java.sql.SQLException;
 import java.util.Properties;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.IntegrationTestingUtility;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.query.ConnectionQueryServicesImpl;
 import org.apache.phoenix.query.QueryServices;
@@ -40,60 +35,13 @@ import org.apache.phoenix.query.QueryServices;
 public class ConnectionQueryServicesTestImpl extends ConnectionQueryServicesImpl {
     protected int NUM_SLAVES_BASE = 1; // number of slaves for the cluster
     
-    private HBaseTestingUtility util;
-
     public ConnectionQueryServicesTestImpl(QueryServices services, ConnectionInfo info) throws SQLException {
         super(services, info);
     }
-
-    private Configuration setupServer(Configuration config) throws Exception {
-        // The HBaseTestingUtility has some kind of memory leak in HBase 0.94.15+ on the Mac
-        // so the pom will use 0.94.14 until this gets fixed.
-        if(isDistributedCluster(config)){
-            IntegrationTestingUtility util =  new IntegrationTestingUtility(config);
-            util.initializeCluster(this.NUM_SLAVES_BASE);
-            this.util = util;
-            // remove all hbase tables
-            HBaseAdmin admin = util.getHBaseAdmin();
-            HTableDescriptor[] tables = admin.listTables();
-            for(HTableDescriptor table : tables){
-                util.deleteTable(table.getName());
-            }
-        } else {
-            util = new HBaseTestingUtility(config);
-            util.startMiniCluster();
-        }
-        return util.getConfiguration();
-    }
-    
-    public boolean isDistributedCluster(Configuration conf) {
-        boolean isDistributedCluster = false;
-        isDistributedCluster = Boolean.parseBoolean(System.getProperty(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER, "false"));
-        if (!isDistributedCluster) {
-          isDistributedCluster = conf.getBoolean(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER, false);
-        }
-        return isDistributedCluster;
-    }
-    
-    private void teardownServer() throws Exception {
-        if(isDistributedCluster(util.getConfiguration())){
-            // remove all hbase tables
-            HBaseAdmin admin = util.getHBaseAdmin();
-            HTableDescriptor[] tables = admin.listTables();
-            for(HTableDescriptor table : tables){
-                util.deleteTable(table.getName());
-            }
-        } else {
-            util.shutdownMiniCluster();
-        }
-    }
     
     @Override
     public void init(String url, Properties props) throws SQLException {
         try {
-            // during test, we don't do hbase.table.sanity.checks
-            config.setBoolean("hbase.table.sanity.checks", false);
-            setupServer(config);
             super.init(url, props);
             /**
              * Clear the server-side meta data cache on initialization. Otherwise, if we
@@ -110,32 +58,13 @@ public class ConnectionQueryServicesTestImpl extends ConnectionQueryServicesImpl
         }
     }
 
-        @Override
+    @Override
     public void close() throws SQLException {
-        SQLException sqlE = null;
         try {
-            try {
-                // Attempt to fix apparent memory leak...
-                clearCache();
-            } finally {
-                super.close();
-            }
-        } catch (SQLException e)  {
-            sqlE = e;
+            // Attempt to fix apparent memory leak...
+            clearCache();
         } finally {
-            try {
-                teardownServer();
-            } catch (Exception e) {
-                if (sqlE == null) {
-                    sqlE = new SQLException(e);
-                } else {
-                    sqlE.setNextException(new SQLException(e));
-                }
-            } finally {
-                if (sqlE != null) {
-                    throw sqlE;
-                }
-            }
+            super.close();
         }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
index 2edb417..96b4a8e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
@@ -37,7 +37,9 @@ import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class CreateTableIT extends BaseClientManagedTimeIT {
     
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java
index ee49962..7a0a2fb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java
@@ -17,19 +17,29 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+import static org.apache.phoenix.util.TestUtil.CUSTOM_ENTITY_DATA_FULL_NAME;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+import static org.apache.phoenix.util.TestUtil.ROW5;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.math.BigDecimal;
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
-
+@Category(ClientManagedTimeTest.class)
 public class CustomEntityDataIT extends BaseClientManagedTimeIT {
     
     protected static void initTableValues(String tenantId, byte[][] splits, long ts) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java
index 7bdc0e2..152b955 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java
@@ -43,6 +43,7 @@ import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
 /**
@@ -51,6 +52,8 @@ import org.junit.Test;
  * 
  * @since 0.1
  */
+
+@Category(ClientManagedTimeTest.class)
 public class DefaultParallelIteratorsRegionSplitterIT extends BaseParallelIteratorsRegionSplitterIT {
     
     private static List<KeyRange> getSplits(Connection conn, long ts, final Scan scan)

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index e606cf1..03d5ccc 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -34,7 +34,9 @@ import java.util.List;
 
 import org.apache.phoenix.util.QueryUtil;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(HBaseManagedTimeTest.class)
 public class DeleteIT extends BaseHBaseManagedTimeIT {
     private static final int NUMBER_OF_ROWS = 20;
     private static final int NTH_ROW_NULL = 5;

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
index 3906ec8..d2b0c8a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
@@ -49,12 +49,14 @@ import java.util.Properties;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Lists;
 
+@Category(ClientManagedTimeTest.class)
 @RunWith(Parameterized.class)
 public class DerivedTableIT extends BaseClientManagedTimeIT {
     private static final String tenantId = getOrganizationId();

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
index 41b99b6..cdbed0e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
@@ -17,16 +17,39 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
+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.MILLIS_IN_DAY;
+import static org.apache.phoenix.util.TestUtil.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW2;
+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.ROW8;
+import static org.apache.phoenix.util.TestUtil.ROW9;
+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.math.BigDecimal;
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Types;
 import java.util.Properties;
 
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
+@Category(ClientManagedTimeTest.class)
 public class DistinctCountIT extends BaseClientManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
index 4f7e153..1114b9f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
@@ -45,6 +45,7 @@ import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Basic tests for Phoenix dynamic upserting
@@ -53,6 +54,7 @@ import org.junit.Test;
  * @since 1.3
  */
 
+@Category(ClientManagedTimeTest.class)
 public class DynamicColumnIT extends BaseClientManagedTimeIT {
     private static final byte[] HBASE_DYNAMIC_COLUMNS_BYTES = SchemaUtil.getTableNameAsBytes(null, HBASE_DYNAMIC_COLUMNS);
     private static final byte[] FAMILY_NAME = Bytes.toBytes(SchemaUtil.normalizeIdentifier("A"));

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java
index 89d2b6c..0b45968 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java
@@ -48,6 +48,7 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Basic tests for Phoenix dynamic family querying "cf.*"
@@ -59,6 +60,7 @@ import org.junit.Test;
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
         value="RV_RETURN_VALUE_IGNORED", 
         justification="Designed to ignore.")
+@Category(HBaseManagedTimeTest.class)
 public class DynamicFamilyIT extends BaseHBaseManagedTimeIT {
     private static final String WEB_STATS = "WEB_STATS";
     private static final String WEB_STATS_SCHEMA_NAME = "";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicUpsertIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicUpsertIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicUpsertIT.java
index 3245ccd..62c3551 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicUpsertIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicUpsertIT.java
@@ -18,17 +18,24 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.*;
-
-import java.sql.*;
+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.sql.SQLException;
 import java.util.Properties;
 
-import org.junit.BeforeClass;
-import org.junit.Test;
-
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Basic tests for Phoenix dynamic upserting
@@ -37,6 +44,7 @@ import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
  * @since 1.3
  */
 
+@Category(ClientManagedTimeTest.class)
 public class DynamicUpsertIT extends BaseClientManagedTimeIT {
 
     private static final String TABLE = "DynamicUpserts";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java
index f9d2e58..0611c33 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java
@@ -46,14 +46,15 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(HBaseManagedTimeTest.class)
 public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testExecuteStatements() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId));
+        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
         String statements = 
             "create table if not exists " + ATABLE_NAME + // Shouldn't error out b/c of if not exists clause
             "   (organization_id char(15) not null, \n" + 
@@ -197,7 +198,7 @@ public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
             
             // test upsert statement with padding
             String tenantId = getOrganizationId();
-            initATableValues(tenantId, getDefaultSplits(tenantId), null, nextTimestamp()-1);
+            initATableValues(tenantId, getDefaultSplits(tenantId), null, nextTimestamp()-1, getUrl());
             
             upsert = "UPSERT INTO " + tableName + "(a_id, a_string, b_string) " +
                     "SELECT A_INTEGER, A_STRING, B_STRING FROM ATABLE WHERE a_string = ?";

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
index 5694969..9ea55d4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
@@ -18,14 +18,22 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.*;
-
-import java.sql.*;
+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.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 
 
@@ -34,6 +42,7 @@ import org.apache.phoenix.util.PhoenixRuntime;
  * Extended tests for Phoenix JDBC implementation
  * 
  */
+@Category(ClientManagedTimeTest.class)
 public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
index 8b1a052..f188fd5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
@@ -17,18 +17,25 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.*;
-import static org.junit.Assert.*;
+import static org.apache.phoenix.util.TestUtil.FUNKY_NAME;
+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.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class FunkyNamesIT extends BaseClientManagedTimeIT {
 
     protected static void initTableValues(byte[][] splits, long ts) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/91b7922c/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
index 8459516..417729b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
@@ -32,11 +32,11 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Properties;
 
-import org.junit.Test;
-
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
+@Category(ClientManagedTimeTest.class)
 public class GroupByCaseIT extends BaseClientManagedTimeIT {
 
     private static String GROUPBY1 = "select " +