You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2015/12/23 23:50:46 UTC

phoenix git commit: PHOENIX-2525 Views on transactional tables return no rows

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 419ee637e -> 012db0a0b


PHOENIX-2525 Views on transactional tables return no rows


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 012db0a0b8d0068830a9316553c5ef2bf7bcb857
Parents: 419ee63
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Tue Dec 15 12:21:59 2015 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Wed Dec 23 14:49:52 2015 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/BaseViewIT.java  |  62 ++++++--
 .../apache/phoenix/end2end/SaltedViewIT.java    |   8 +-
 .../java/org/apache/phoenix/end2end/ViewIT.java | 145 +++++++++++--------
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   |   2 +-
 .../org/apache/phoenix/tx/TransactionIT.java    |  31 ++++
 .../phoenix/compile/CreateTableCompiler.java    |   4 +-
 .../phoenix/exception/SQLExceptionCode.java     |   2 +-
 .../apache/phoenix/execute/MutationState.java   |   2 +
 .../apache/phoenix/schema/MetaDataClient.java   |  13 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   6 +
 10 files changed, 189 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/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 3140077..c713df4 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
@@ -27,6 +27,8 @@ import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
@@ -34,21 +36,50 @@ import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Maps;
 
-
+@RunWith(Parameterized.class)
 public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
+	
+	protected String tableName;
+	protected String fullTableName;
+	protected String tableDDLOptions;
+	protected String tableSuffix;
+	protected boolean transactional;
 
     @BeforeClass
     public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Integer.toString(20));
         props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(1024));
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
+    public BaseViewIT( boolean transactional) {
+		StringBuilder optionBuilder = new StringBuilder();
+		this.transactional = transactional;
+		if (transactional) {
+			optionBuilder.append(" TRANSACTIONAL=true ");
+		}
+		this.tableDDLOptions = optionBuilder.toString();
+		tableSuffix = transactional ?  "_TXN" : "";
+		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + tableSuffix;
+        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+	}
+    
+    @Parameters(name="transactional = {0}")
+    public static Collection<Boolean> data() {
+        return Arrays.asList(new Boolean[] { false, true });
+    }
+    
     protected void testUpdatableViewWithIndex(Integer saltBuckets, boolean localIndex) throws Exception {
         testUpdatableView(saltBuckets);
         testUpdatableViewIndex(saltBuckets, localIndex);
@@ -56,17 +87,22 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
 
     protected void testUpdatableView(Integer saltBuckets) throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2, k3))" + (saltBuckets == null ? "" : (" SALT_BUCKETS="+saltBuckets));
+		if (saltBuckets!=null) {
+			if (tableDDLOptions.length()!=0)
+				tableDDLOptions+=",";
+			tableDDLOptions+=(" SALT_BUCKETS="+saltBuckets);
+		}
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2, k3))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v AS SELECT * FROM t WHERE k1 = 1";
+        ddl = "CREATE VIEW v AS SELECT * FROM " + fullTableName + " WHERE k1 = 1";
         conn.createStatement().execute(ddl);
         for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO t VALUES(" + (i % 4) + "," + (i+100) + "," + (i > 5 ? 2 : 1) + ")");
+            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + (i % 4) + "," + (i+100) + "," + (i > 5 ? 2 : 1) + ")");
         }
         conn.commit();
         ResultSet rs;
         
-        rs = conn.createStatement().executeQuery("SELECT count(*) FROM t");
+        rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName);
         assertTrue(rs.next());
         assertEquals(10, rs.getInt(1));
         rs = conn.createStatement().executeQuery("SELECT count(*) FROM v");
@@ -132,14 +168,14 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         String queryPlan = QueryUtil.getExplainPlan(rs);
         if (localIndex) {
-            assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets)  +"-WAY RANGE SCAN OVER _LOCAL_IDX_T [-32768,51]\n"
+            assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets)  +"-WAY RANGE SCAN OVER _LOCAL_IDX_" + tableName +" [-32768,51]\n"
                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                     + "CLIENT MERGE SORT",
                 queryPlan);
         } else {
             assertEquals(saltBuckets == null
-                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [" + Short.MIN_VALUE + ",51]"
-                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER _IDX_T [0," + Short.MIN_VALUE + ",51]\nCLIENT MERGE SORT",
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + tableName +" [" + Short.MIN_VALUE + ",51]"
+                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER _IDX_T" + (transactional ? "_TXN" : "") + " [0," + Short.MIN_VALUE + ",51]\nCLIENT MERGE SORT",
                             queryPlan);
         }
 
@@ -154,7 +190,7 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
         assertEquals(saltBuckets == null ? 1 : 3, splits.size());
         
         // analyze table should analyze all view data
-        analyzeTable(conn, "t");        
+        analyzeTable(conn, tableName);        
         splits = getAllSplits(conn, "i2");
         assertEquals(saltBuckets == null ? 6 : 8, splits.size());
 
@@ -168,14 +204,14 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         if (localIndex) {
-            assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets)  +"-WAY RANGE SCAN OVER _LOCAL_IDX_T [" + (Short.MIN_VALUE+1) + ",'foo']\n"
+            assertEquals("CLIENT PARALLEL "+ (saltBuckets == null ? 1 : saltBuckets)  +"-WAY RANGE SCAN OVER _LOCAL_IDX_" + tableName +" [" + (Short.MIN_VALUE+1) + ",'foo']\n"
                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                     + "CLIENT MERGE SORT",QueryUtil.getExplainPlan(rs));
         } else {
             assertEquals(saltBuckets == null
-                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [" + (Short.MIN_VALUE+1) + ",'foo']\n"
+                    ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + tableName +" [" + (Short.MIN_VALUE+1) + ",'foo']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY"
-                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER _IDX_T [0," + (Short.MIN_VALUE+1) + ",'foo']\n"
+                            : "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER _IDX_T" + (transactional ? "_TXN" : "") + " [0," + (Short.MIN_VALUE+1) + ",'foo']\n"
                                     + "    SERVER FILTER BY FIRST KEY ONLY\n"
                                     + "CLIENT MERGE SORT",
                             QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/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 19f0a4a..e7518f6 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
@@ -21,8 +21,12 @@ import org.junit.Test;
 
 
 public class SaltedViewIT extends BaseViewIT {
-    
-    /**
+
+	public SaltedViewIT(boolean transactional) {
+		super(transactional);
+	}
+
+	/**
      * Salted tests must be in their own test file to ensure that the underlying
      * table is dropped. Otherwise, the splits may not be performed.
      * TODO: we should throw in that case

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/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 123fd85..55f26cd 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
@@ -47,14 +47,18 @@ import org.junit.Test;
 
 
 public class ViewIT extends BaseViewIT {
-    
-    @Test
+	
+    public ViewIT(boolean transactional) {
+		super(transactional);
+	}
+
+	@Test
     public void testReadOnlyView() throws Exception {
         Connection earlierCon = DriverManager.getConnection(getUrl());
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+		String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) "+ tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t WHERE k > 5";
+        ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM " + tableName + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         try {
             conn.createStatement().execute("UPSERT INTO v VALUES(1)");
@@ -63,17 +67,24 @@ public class ViewIT extends BaseViewIT {
             
         }
         for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ")");
         }
         conn.commit();
         
-        analyzeTable(conn, "v");
+        analyzeTable(conn, "v", transactional);
         
         List<KeyRange> splits = getAllSplits(conn, "v");
         assertEquals(4, splits.size());
         
         int count = 0;
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM v");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM " + tableName);
+        while (rs.next()) {
+            assertEquals(count++, rs.getInt(1));
+        }
+        assertEquals(10, count);
+        
+        count = 0;
+        rs = conn.createStatement().executeQuery("SELECT k FROM v");
         while (rs.next()) {
             count++;
             assertEquals(count + 5, rs.getInt(1));
@@ -88,7 +99,6 @@ public class ViewIT extends BaseViewIT {
         assertEquals(4, count);
     }
 
-
     @Test
     public void testReadOnlyOnReadOnlyView() throws Exception {
         testReadOnlyView();
@@ -181,7 +191,7 @@ public class ViewIT extends BaseViewIT {
             
         }
         
-        conn.createStatement().execute("UPSERT INTO t(k1, k2,k3) VALUES(1, 122, 5)");
+        conn.createStatement().execute("UPSERT INTO " + fullTableName + "(k1, k2,k3) VALUES(1, 122, 5)");
         conn.commit();
         rs = conn.createStatement().executeQuery("SELECT k1, k2, k3 FROM v2 WHERE k2 >= 120");
         assertTrue(rs.next());
@@ -194,9 +204,9 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testDisallowDropOfReferencedColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v1(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM t WHERE v1 = 1.0";
+        ddl = "CREATE VIEW v1(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
         
         try {
@@ -229,9 +239,10 @@ public class ViewIT extends BaseViewIT {
     public void testReadOnlyViewWithCaseSensitiveTableNames() throws Exception {
         Connection earlierCon = DriverManager.getConnection(getUrl());
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE \"case_SENSITIVE_table\" (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        String caseSensitiveTableName = "\"case_SENSITIVE_table" + tableSuffix + "\"" ;
+        String ddl = "CREATE TABLE " + caseSensitiveTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW \"v\" (v2 VARCHAR) AS SELECT * FROM \"case_SENSITIVE_table\" WHERE k > 5";
+        ddl = "CREATE VIEW \"v\" (v2 VARCHAR) AS SELECT * FROM " + caseSensitiveTableName + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         try {
             conn.createStatement().execute("UPSERT INTO \"v\" VALUES(1)");
@@ -240,7 +251,7 @@ public class ViewIT extends BaseViewIT {
             
         }
         for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO \"case_SENSITIVE_table\" VALUES(" + i + ")");
+            conn.createStatement().execute("UPSERT INTO " + caseSensitiveTableName + " VALUES(" + i + ")");
         }
         conn.commit();
         
@@ -263,9 +274,9 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testReadOnlyViewWithCaseSensitiveColumnNames() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (\"k\" INTEGER NOT NULL PRIMARY KEY, \"v1\" INTEGER, \"a\".v2 VARCHAR)";
+        String ddl = "CREATE TABLE " + fullTableName + " (\"k\" INTEGER NOT NULL PRIMARY KEY, \"v1\" INTEGER, \"a\".v2 VARCHAR)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v (v VARCHAR) AS SELECT * FROM t WHERE \"k\" > 5 and \"v1\" > 1";
+        ddl = "CREATE VIEW v (v VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE \"k\" > 5 and \"v1\" > 1";
         conn.createStatement().execute(ddl);
         try {
             conn.createStatement().execute("UPSERT INTO v VALUES(1)");
@@ -274,7 +285,7 @@ public class ViewIT extends BaseViewIT {
             
         }
         for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ", " + (i+10) + ",'A')");
+            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ", " + (i+10) + ",'A')");
         }
         conn.commit();
         
@@ -290,9 +301,9 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testViewWithCurrentDate() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER, v2 DATE)";
+        String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER, v2 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v (v VARCHAR) AS SELECT * FROM t WHERE v2 > CURRENT_DATE()-5 AND v2 > DATE '2010-01-01'";
+        ddl = "CREATE VIEW v (v VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE v2 > CURRENT_DATE()-5 AND v2 > DATE '2010-01-01'";
         conn.createStatement().execute(ddl);
         try {
             conn.createStatement().execute("UPSERT INTO v VALUES(1)");
@@ -301,7 +312,7 @@ public class ViewIT extends BaseViewIT {
             
         }
         for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ", " + (i+10) + ",CURRENT_DATE()-" + i + ")");
+            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ", " + (i+10) + ",CURRENT_DATE()-" + i + ")");
         }
         conn.commit();
         
@@ -317,11 +328,12 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testViewAndTableInDifferentSchemas() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE s1.t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        String fullTableName = "s1.t"+tableSuffix;
+		String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW s2.v1 (v2 VARCHAR) AS SELECT * FROM s1.t WHERE k > 5";
+        ddl = "CREATE VIEW s2.v1 (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v2 (v2 VARCHAR) AS SELECT * FROM s1.t WHERE k > 5";
+        ddl = "CREATE VIEW v2 (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         ddl = "DROP VIEW v1";
         try {
@@ -337,11 +349,11 @@ public class ViewIT extends BaseViewIT {
             fail();
         } catch (TableNotFoundException ignore) {
         }
-        ddl = "DROP TABLE s1.t";
-        validateCannotDropTableWithChildViewsWithoutCascade(conn, "s1.t");
+        ddl = "DROP TABLE " + fullTableName;
+        validateCannotDropTableWithChildViewsWithoutCascade(conn, fullTableName);
         ddl = "DROP VIEW v2";
         conn.createStatement().execute(ddl);
-        ddl = "DROP TABLE s1.t";
+        ddl = "DROP TABLE " + fullTableName;
         conn.createStatement().execute(ddl);
     }
 
@@ -349,13 +361,13 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testDisallowDropOfColumnOnParentTable() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v1(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM tp WHERE v1 = 1.0";
+        ddl = "CREATE VIEW v1(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
         
         try {
-            conn.createStatement().execute("ALTER TABLE tp DROP COLUMN v1");
+            conn.createStatement().execute("ALTER TABLE " + fullTableName + " DROP COLUMN v1");
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
@@ -365,18 +377,19 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testViewAndTableAndDropCascade() throws Exception {
     	// Setup
+    	String fullTableName = "s2.t"+tableSuffix;
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE s2.t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        String ddl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW s2.v1 (v2 VARCHAR) AS SELECT * FROM s2.t WHERE k > 5";
+        ddl = "CREATE VIEW s2.v1 (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW s2.v2 (v2 VARCHAR) AS SELECT * FROM s2.t WHERE k > 10";
+        ddl = "CREATE VIEW s2.v2 (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 10";
         conn.createStatement().execute(ddl);
 
-        validateCannotDropTableWithChildViewsWithoutCascade(conn, "s2.t");
+        validateCannotDropTableWithChildViewsWithoutCascade(conn, fullTableName);
         
         // Execute DROP...CASCADE
-        conn.createStatement().execute("DROP TABLE s2.t CASCADE");
+        conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
         
         validateViewDoesNotExist(conn, "s2.v1");
         validateViewDoesNotExist(conn, "s2.v2");
@@ -387,24 +400,27 @@ public class ViewIT extends BaseViewIT {
         
     	// Setup - Tables and Views with Indexes
     	Connection conn = DriverManager.getConnection(getUrl());
-    	
-        String ddl = "CREATE TABLE s3.t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) IMMUTABLE_ROWS=true";
+    	String fullTableName = "s3.t"+tableSuffix;
+		if (tableDDLOptions.length()!=0)
+			tableDDLOptions+=",";
+		tableDDLOptions+="IMMUTABLE_ROWS=true";
+        String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE INDEX IDX1 ON s3.t (v1)";
+        ddl = "CREATE INDEX IDX1 ON " + fullTableName + " (v1)";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW s3.v1 (v2 VARCHAR) AS SELECT * FROM s3.t WHERE k > 5";
+        ddl = "CREATE VIEW s3.v1 (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         ddl = "CREATE INDEX IDX2 ON s3.v1 (v2)";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW s3.v2 (v2 VARCHAR) AS SELECT * FROM s3.t WHERE k > 10";
+        ddl = "CREATE VIEW s3.v2 (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 10";
         conn.createStatement().execute(ddl);
         ddl = "CREATE INDEX IDX3 ON s3.v2 (v2)";
         conn.createStatement().execute(ddl);
 
-        validateCannotDropTableWithChildViewsWithoutCascade(conn, "s3.t");
+        validateCannotDropTableWithChildViewsWithoutCascade(conn, fullTableName);
         
         // Execute DROP...CASCADE
-        conn.createStatement().execute("DROP TABLE s3.t CASCADE");
+        conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
         
         // Validate Views were deleted - Try and delete child views, should throw TableNotFoundException
         validateViewDoesNotExist(conn, "s3.v1");
@@ -438,16 +454,16 @@ public class ViewIT extends BaseViewIT {
     public void testViewUsesTableIndex() throws Exception {
         ResultSet rs;
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s1 VARCHAR, s2 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2, k3))";
+        String ddl = "CREATE TABLE " + fullTableName + "  (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s1 VARCHAR, s2 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2, k3))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        conn.createStatement().execute("CREATE INDEX i1 ON t(k3, k2) INCLUDE(s1, s2)");
-        conn.createStatement().execute("CREATE INDEX i2 ON t(k3, k2, s2)");
+        conn.createStatement().execute("CREATE INDEX i1 ON " + fullTableName + "(k3, k2) INCLUDE(s1, s2)");
+        conn.createStatement().execute("CREATE INDEX i2 ON " + fullTableName + "(k3, k2, s2)");
         
-        ddl = "CREATE VIEW v AS SELECT * FROM t WHERE s1 = 'foo'";
+        ddl = "CREATE VIEW v AS SELECT * FROM " + fullTableName + " WHERE s1 = 'foo'";
         conn.createStatement().execute(ddl);
         String[] s1Values = {"foo","bar"};
         for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO t VALUES(" + (i % 4) + "," + (i+100) + "," + (i > 5 ? 2 : 1) + ",'" + s1Values[i%2] + "','bas')");
+            conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(" + (i % 4) + "," + (i+100) + "," + (i > 5 ? 2 : 1) + ",'" + s1Values[i%2] + "','bas')");
         }
         conn.commit();
         
@@ -473,9 +489,9 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testCreateViewDefinesPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v1(v2 VARCHAR, k3 VARCHAR PRIMARY KEY) AS SELECT * FROM tp WHERE K1 = 1";
+        ddl = "CREATE VIEW v1(v2 VARCHAR, k3 VARCHAR PRIMARY KEY) AS SELECT * FROM " + fullTableName + " WHERE K1 = 1";
         conn.createStatement().execute(ddl);
 
         // assert PK metadata
@@ -483,12 +499,12 @@ public class ViewIT extends BaseViewIT {
         assertPKs(rs, new String[] {"K1", "K2", "K3"});
         
         // sanity check upserts into base table and view
-        conn.createStatement().executeUpdate("upsert into tp (k1, k2, v1) values (1, 1, 1)");
+        conn.createStatement().executeUpdate("upsert into " + fullTableName + " (k1, k2, v1) values (1, 1, 1)");
         conn.createStatement().executeUpdate("upsert into v1 (k1, k2, k3, v2) values (1, 1, 'abc', 'def')");
         conn.commit();
         
         // expect 2 rows in the base table
-        rs = conn.createStatement().executeQuery("select count(*) from tp");
+        rs = conn.createStatement().executeQuery("select count(*) from " + fullTableName);
         assertTrue(rs.next());
         assertEquals(2, rs.getInt(1));
         
@@ -501,9 +517,9 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testCreateViewDefinesPKConstraint() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v1(v2 VARCHAR, k3 VARCHAR, k4 INTEGER NOT NULL, CONSTRAINT PKVEW PRIMARY KEY (k3, k4)) AS SELECT * FROM tp WHERE K1 = 1";
+        ddl = "CREATE VIEW v1(v2 VARCHAR, k3 VARCHAR, k4 INTEGER NOT NULL, CONSTRAINT PKVEW PRIMARY KEY (k3, k4)) AS SELECT * FROM " + fullTableName + " WHERE K1 = 1";
         conn.createStatement().execute(ddl);
 
         // assert PK metadata
@@ -514,9 +530,10 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testViewAddsPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String fullTableName2 = fullTableName;
+		String ddl = "CREATE TABLE " + fullTableName2 + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v1  AS SELECT * FROM tp WHERE v1 = 1.0";
+        ddl = "CREATE VIEW v1  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
         ddl = "ALTER VIEW V1 ADD k3 VARCHAR PRIMARY KEY, k4 VARCHAR PRIMARY KEY, v2 INTEGER";
         conn.createStatement().execute(ddl);
@@ -529,9 +546,9 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testViewAddsPKColumnWhoseParentsLastPKIsVarLength() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 VARCHAR NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 VARCHAR NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v1  AS SELECT * FROM tp WHERE v1 = 1.0";
+        ddl = "CREATE VIEW v1  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
         ddl = "ALTER VIEW V1 ADD k3 VARCHAR PRIMARY KEY, k4 VARCHAR PRIMARY KEY, v2 INTEGER";
         try {
@@ -540,7 +557,7 @@ public class ViewIT extends BaseViewIT {
         } catch (SQLException e) {
             assertEquals(CANNOT_MODIFY_VIEW_PK.getErrorCode(), e.getErrorCode());
         }
-        ddl = "CREATE VIEW v2 (k3 VARCHAR PRIMARY KEY)  AS SELECT * FROM tp WHERE v1 = 1.0";
+        ddl = "CREATE VIEW v2 (k3 VARCHAR PRIMARY KEY)  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         try {
         	conn.createStatement().execute(ddl);
         } catch (SQLException e) {
@@ -551,9 +568,9 @@ public class ViewIT extends BaseViewIT {
     @Test(expected=ColumnAlreadyExistsException.class)
     public void testViewAddsClashingPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v1  AS SELECT * FROM tp WHERE v1 = 1.0";
+        ddl = "CREATE VIEW v1  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
         ddl = "ALTER VIEW V1 ADD k3 VARCHAR PRIMARY KEY, k2 VARCHAR PRIMARY KEY, v2 INTEGER";
         conn.createStatement().execute(ddl);
@@ -562,9 +579,9 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testViewAddsNotNullPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v1  AS SELECT * FROM tp WHERE v1 = 1.0";
+        ddl = "CREATE VIEW v1  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
         try {
             ddl = "ALTER VIEW V1 ADD k3 VARCHAR NOT NULL PRIMARY KEY"; 
@@ -578,11 +595,11 @@ public class ViewIT extends BaseViewIT {
     @Test
     public void testQueryViewStatementOptimization() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String sql = "CREATE TABLE tp (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String sql = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
         conn.createStatement().execute(sql);
-        sql = "CREATE VIEW v1  AS SELECT * FROM tp";
+        sql = "CREATE VIEW v1  AS SELECT * FROM " + fullTableName;
         conn.createStatement().execute(sql);
-        sql = "CREATE VIEW v2  AS SELECT * FROM tp WHERE k1 = 1.0";
+        sql = "CREATE VIEW v2  AS SELECT * FROM " + fullTableName + " WHERE k1 = 1.0";
         conn.createStatement().execute(sql);
         
         sql = "SELECT * FROM v1 order by k1, k2";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
index 0e6d7af..ac69359 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
@@ -171,7 +171,7 @@ public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
 	        // for non-transactional tables without a scn : verify one rpc to getTable occurs *per* query
             // for non-transactional tables with a scn : verify *only* one rpc occurs
             // for transactional tables : verify *only* one rpc occurs
-	        // for non-transactional, system tables : verify non rpc occurs
+	        // for non-transactional, system tables : verify no rpc occurs
             int numRpcs = isSystem ? 0 : (isTransactional || scn!=null ? 1 : 3); 
             verify(connectionQueryServices, times(numRpcs)).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)), anyLong(), anyLong());
 		}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index 0e358a4..0b3ba91 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -20,6 +20,8 @@ package org.apache.phoenix.tx;
 import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.TRANSACTIONAL_DATA_TABLE;
+import static org.apache.phoenix.util.TestUtil.analyzeTable;
+import static org.apache.phoenix.util.TestUtil.getAllSplits;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -48,11 +50,13 @@ import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -572,4 +576,31 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
         	}
         }
     }
+    
+    @Test
+    public void testReadOnlyView() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+		String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>4";
+        conn.createStatement().execute(ddl);
+        for (int i = 0; i < 10; i++) {
+            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+        }
+        conn.commit();
+        
+        int count = 0;
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM t");
+        while (rs.next()) {
+            assertEquals(count++, rs.getInt(1));
+        }
+        assertEquals(10, count);
+        
+        count = 0;
+        rs = conn.createStatement().executeQuery("SELECT k FROM v");
+        while (rs.next()) {
+            assertEquals(5+count++, rs.getInt(1));
+        }
+        assertEquals(5, count);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index d5b74fb..e032feb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -120,8 +120,8 @@ public class CreateTableCompiler {
                 }
                 if (viewTypeToBe != ViewType.MAPPED) {
                     Long scn = connection.getSCN();
-                    connectionToBe = scn != null ? connection :
-                        // If we haved no SCN on our connection, freeze the SCN at when
+                    connectionToBe = (scn != null || tableRef.getTable().isTransactional()) ? connection :
+                        // If we haved no SCN on our connection and the base table is not transactional, freeze the SCN at when
                         // the base table was resolved to prevent any race condition on
                         // the error checking we do for the base table. The only potential
                         // issue is if the base table lives on a different region server

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 39f2c14..ef135eb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -274,7 +274,7 @@ public enum SQLExceptionCode {
     CANNOT_START_TRANSACTION_WITH_SCN_SET(1073, "44A04", "Cannot start a transaction on a connection with SCN set"),
     TX_MAX_VERSIONS_MUST_BE_GREATER_THAN_ONE(1074, "44A05", "A transactional table must define VERSION of greater than one"),
     CANNOT_SPECIFY_SCN_FOR_TXN_TABLE(1075, "44A06", "Cannot use a connection with SCN set for a transactional table"),
-    NULL_TRANSACTION_CONTEXT(1076, "44A07", "No Tranasction Context available"),
+    NULL_TRANSACTION_CONTEXT(1076, "44A07", "No Transaction Context available"),
     TRANSACTION_FAILED(1077, "44A08", "Transaction Failure "),
     CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED(1078, "44A09", "Cannot create a transactional table if transactions are disabled"),
     CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED(1079, "44A10", "Cannot alter table to be transactional table if transactions are disabled"),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 8c2acb1..195dce3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -162,6 +162,8 @@ public class MutationState implements SQLCloseable {
 					.getQueryServices().getTransactionSystemClient();
 			this.txContext = new TransactionContext(txServiceClient);
 		} else {
+			// this code path is only used while running child scans, we can't pass the txContext to child scans
+			// as it is not thread safe, so we use the tx member variable
 			txAwares = Lists.newArrayList();
 			txContext = null;
 		}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 35d61ff..7191a7c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -938,7 +938,15 @@ public class MetaDataClient {
              * since it may not represent a "real" table in the case of the view indexes of a base table.
              */
             PostDDLCompiler compiler = new PostDDLCompiler(connection);
-            TableRef tableRef = new TableRef(null, logicalTable, clientTimeStamp, false);
+            //even if table is transactional, while calculating stats we scan the table non-transactionally to 
+            //view all the data belonging to the table
+            PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
+                @Override
+                public boolean isTransactional() {
+                    return false;
+                }
+            };
+            TableRef tableRef = new TableRef(null, nonTxnLogicalTable, clientTimeStamp, false);
             MutationPlan plan = compiler.compile(Collections.singletonList(tableRef), null, null, null, clientTimeStamp);
             Scan scan = plan.getContext().getScan();
             scan.setCacheBlocks(false);
@@ -1562,7 +1570,7 @@ public class MetaDataClient {
             long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
             boolean multiTenant = false;
             boolean storeNulls = false;
-            boolean transactional = false;
+            boolean transactional = (parent!= null) ? parent.isTransactional() : false;
             Integer saltBucketNum = null;
             String defaultFamilyName = null;
             boolean isImmutableRows = false;
@@ -1571,7 +1579,6 @@ public class MetaDataClient {
             boolean rowKeyOrderOptimizable = true;
             Long timestamp = null;
             if (parent != null && tableType == PTableType.INDEX) {
-            	transactional = parent.isTransactional();
                 timestamp = TransactionUtil.getTableTimestamp(connection, transactional);
                 storeNulls = parent.getStoreNulls();
                 if (tableType == PTableType.INDEX) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/012db0a0/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index e66f8ca..e1258e4 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -556,8 +556,14 @@ public class TestUtil {
     }
 
     public static void analyzeTable(Connection conn, String tableName) throws IOException, SQLException {
+    	analyzeTable(conn, tableName, false);
+    }
+    
+    public static void analyzeTable(Connection conn, String tableName, boolean transactional) throws IOException, SQLException {
         String query = "UPDATE STATISTICS " + tableName;
         conn.createStatement().execute(query);
+        // if the table is transactional burn a txn in order to make sure the next txn read pointer is close to wall clock time
+        TransactionUtil.getTableTimestamp(conn.unwrap(PhoenixConnection.class), transactional);
     }
     
     public static void analyzeTableIndex(Connection conn, String tableName) throws IOException, SQLException {