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/01 06:27:29 UTC

[14/19] phoenix git commit: PHOENIX-2441 Do not require transaction manager to be run

PHOENIX-2441 Do not require transaction manager to be run


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

Branch: refs/heads/4.x-HBase-1.0
Commit: d559ea70a56536bae04f19d2d0cff49c5446c6bc
Parents: 1151d21
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Tue Nov 24 16:34:06 2015 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Mon Nov 30 20:23:36 2015 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    | 26 ++++++++++++++++++++
 .../phoenix/end2end/index/ImmutableIndexIT.java | 15 +++++++++++
 .../apache/phoenix/end2end/index/IndexIT.java   | 15 +++++++++++
 .../end2end/index/MutableIndexFailureIT.java    |  7 +++++-
 .../phoenix/end2end/index/MutableIndexIT.java   | 14 +++++++++++
 .../end2end/index/txn/MutableRollbackIT.java    |  3 ++-
 .../phoenix/end2end/index/txn/RollbackIT.java   |  3 ++-
 .../end2end/index/txn/TxWriteFailureIT.java     |  5 ++--
 .../org/apache/phoenix/tx/TransactionIT.java    | 14 +++++++++++
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |  5 ++--
 .../phoenix/exception/SQLExceptionCode.java     |  2 ++
 .../query/ConnectionQueryServicesImpl.java      |  8 +++++-
 .../org/apache/phoenix/query/QueryServices.java |  3 ++-
 .../phoenix/query/QueryServicesOptions.java     |  2 ++
 .../apache/phoenix/schema/MetaDataClient.java   | 23 +++++++++++++++--
 15 files changed, 134 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/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 cdf7b5d..c86f8c5 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
@@ -36,6 +36,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
@@ -43,6 +44,8 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 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.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -52,6 +55,8 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -60,6 +65,8 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+
 /**
  *
  * A lot of tests in this class test HBase level properties. As a result,
@@ -2177,5 +2184,24 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
         }
     }
     
+	@Test
+	public void testCreatingTxnTableFailsIfTxnsDisabled() throws Exception {
+		try (Connection conn = DriverManager.getConnection(getUrl())) {
+			// creating a transactional table should fail if transactions are disabled
+			try {
+				conn.createStatement().execute("CREATE TABLE DEMO1(k INTEGER PRIMARY KEY, v VARCHAR) TRANSACTIONAL=true");
+			} catch (SQLException e) {
+				assertEquals(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED.getErrorCode(), e.getErrorCode());
+			}
+			// altering a table to be transactional  should fail if transactions are disabled
+			conn.createStatement().execute("CREATE TABLE DEMO2(k INTEGER PRIMARY KEY, v VARCHAR)");
+			try {
+				conn.createStatement().execute("ALTER TABLE DEMO2 SET TRANSACTIONAL=true");
+			} catch (SQLException e) {
+				assertEquals(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED.getErrorCode(), e.getErrorCode());
+			}
+		}
+	}
+    
 }
  
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/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 b03037c..0d329fe 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
@@ -29,19 +29,26 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
+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.Test;
 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 class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
@@ -66,6 +73,14 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 	}
 	
+	@BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+	
 	@Parameters(name="localIndex = {0} , transactional = {1}")
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index d2c6267..003e564 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -17,27 +17,34 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.PropertiesUtil;
 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.Test;
 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 class IndexIT extends BaseHBaseManagedTimeIT {
 	
@@ -65,6 +72,14 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 	}
 	
+	@BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+	
 	@Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/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 39c90be..7c44401 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
@@ -35,6 +35,7 @@ import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Timer;
 import java.util.TimerTask;
@@ -80,6 +81,8 @@ 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.Maps;
 /**
  * 
  * Test for failure of region server to write to index table.
@@ -131,7 +134,9 @@ public class MutableIndexFailureIT extends BaseTest {
         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);
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
+        driver = initAndRegisterDriver(url, new ReadOnlyProps(props.entrySet().iterator()));
         clusterInitialized = true;
         setupTxManager();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/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 8f5301c..add282e 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
@@ -31,20 +31,26 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.util.PropertiesUtil;
 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.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import com.google.common.collect.Maps;
 import com.google.common.primitives.Doubles;
 
 @RunWith(Parameterized.class)
@@ -69,6 +75,14 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
         this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 	}
+    
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
 	
 	@Parameters(name="localIndex = {0} , transactional = {1}")
     public static Collection<Boolean[]> data() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
index 58f6226..e675dda 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
@@ -73,7 +73,8 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
-        props.put(QueryServices.DEFAULT_TRANSACTIONAL_ATTRIB, Boolean.toString(true));
+        props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 	

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
index dbddcb1..3e1d5ab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
@@ -67,7 +67,8 @@ public class RollbackIT extends BaseHBaseManagedTimeIT {
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
-        props.put(QueryServices.DEFAULT_TRANSACTIONAL_ATTRIB, Boolean.toString(true));
+        props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 	

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
index 0e16d97..04bf1dc 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
@@ -100,9 +100,10 @@ public class TxWriteFailureIT extends BaseTest {
 				+ JDBC_PROTOCOL_SEPARATOR + clientPort
 				+ JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
 
-		Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+		Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
 		// Must update config before starting server
-		props.put(QueryServices.DEFAULT_TRANSACTIONAL_ATTRIB, Boolean.toString(true));
+		props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
+		props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
 		driver = initAndRegisterDriver(url, new ReadOnlyProps(props.entrySet().iterator()));
 		clusterInitialized = true;
 		setupTxManager();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/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 5e059f0..2eb6edf 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
@@ -33,6 +33,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -43,24 +44,29 @@ 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.end2end.BaseHBaseManagedTimeIT;
+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.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.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import co.cask.tephra.TxConstants;
 import co.cask.tephra.hbase10.coprocessor.TransactionProcessor;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 public class TransactionIT extends BaseHBaseManagedTimeIT {
 	
@@ -70,6 +76,14 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     public void setUp() throws SQLException {
         ensureTableCreated(getUrl(), TRANSACTIONAL_DATA_TABLE);
     }
+    
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
 		
 	@Test
 	public void testReadOwnWrites() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
index 34869dd..e9ba592 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
@@ -74,8 +74,9 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
 	@BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.DEFAULT_TRANSACTIONAL_ATTRIB, Boolean.toString(true));
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
+        props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 	

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/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 3613c95..fb8ebfd 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
@@ -276,6 +276,8 @@ public enum SQLExceptionCode {
     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"),
     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"),
 
     /** Sequence related */
     SEQUENCE_ALREADY_EXIST(1200, "42Z00", "Sequence already exists.", new Factory() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index bc91c04..b925933 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -338,7 +338,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 User.login(config, HBASE_CLIENT_KEYTAB, HBASE_CLIENT_PRINCIPAL, null);
                 logger.info("Successfull login to secure cluster!!");
             }
-            initTxServiceClient();
+			boolean transactionsEnabled = config.getBoolean(
+					QueryServices.TRANSACTIONS_ENABLED,
+					QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
+			// only initialize the tx service client if needed
+			if (transactionsEnabled) {
+				initTxServiceClient();
+			}
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index ebdb8e1..ba77edb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -159,7 +159,8 @@ public interface QueryServices extends SQLCloseable {
     public static final String DELAY_FOR_SCHEMA_UPDATE_CHECK = "phoenix.schema.change.delay";
     public static final String DEFAULT_KEEP_DELETED_CELLS_ATTRIB = "phoenix.table.default.keep.deleted.cells";
     public static final String DEFAULT_STORE_NULLS_ATTRIB = "phoenix.table.default.store.nulls";
-    public static final String DEFAULT_TRANSACTIONAL_ATTRIB = "phoenix.transactions.default.enabled";
+    public static final String DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB = "phoenix.table.istransactional.default";
+    public static final String TRANSACTIONS_ENABLED = "phoenix.transactions.enabled";
     public static final String GLOBAL_METRICS_ENABLED = "phoenix.query.global.metrics.enabled";
     
     // rpc queue configs

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 8ada3d0..05d9064 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -194,6 +194,8 @@ public class QueryServicesOptions {
     // We'll also need this for transactions to work correctly
     public static final boolean DEFAULT_AUTO_COMMIT = false;
     public static final boolean DEFAULT_TRANSACTIONAL = false;
+    public static final boolean DEFAULT_TABLE_ISTRANSACTIONAL = false;
+    public static final boolean DEFAULT_TRANSACTIONS_ENABLED = false;
     public static final boolean DEFAULT_IS_GLOBAL_METRICS_ENABLED = true;
     
     private static final String DEFAULT_CLIENT_RPC_CONTROLLER_FACTORY = ClientRpcControllerFactory.class.getName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d559ea70/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 29e4af2..cabfd04 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
@@ -80,7 +80,9 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.TRANSACTIONS_ENABLED;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED;
 import static org.apache.phoenix.schema.PTable.ViewType.MAPPED;
 import static org.apache.phoenix.schema.PTableType.TABLE;
 import static org.apache.phoenix.schema.PTableType.VIEW;
@@ -451,7 +453,7 @@ public class MetaDataClient {
         }
         
         boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-							                QueryServices.DEFAULT_TRANSACTIONAL_ATTRIB,
+							                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
 							                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
         // TODO if system tables become transactional remove the check 
@@ -1689,12 +1691,21 @@ public class MetaDataClient {
             if (parent == null) {
                 if (transactionalProp == null) {
                     transactional = connection.getQueryServices().getProps().getBoolean(
-                                    QueryServices.DEFAULT_TRANSACTIONAL_ATTRIB,
+                                    QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
                                     QueryServicesOptions.DEFAULT_TRANSACTIONAL);
                 } else {
                     transactional = transactionalProp;
                 }
             }
+            boolean transactionsEnabled = connection.getQueryServices().getProps().getBoolean(
+											QueryServices.TRANSACTIONS_ENABLED,
+											QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
+            // can't create a transactional table if transactions are not enabled
+            if (!transactionsEnabled && transactional) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED)
+                .setSchemaName(schemaName).setTableName(tableName)
+                .build().buildException();
+            }
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
             if (transactional) {
                 // If TTL set, use Tephra TTL property name instead
@@ -2628,6 +2639,14 @@ public class MetaDataClient {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX)
                             .setSchemaName(schemaName).setTableName(tableName).build().buildException();
                         }
+                        // cannot create a transactional table if transactions are disabled
+                        boolean transactionsEnabled = connection.getQueryServices().getProps().getBoolean(
+								QueryServices.TRANSACTIONS_ENABLED,
+								QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
+                        if (!transactionsEnabled) {
+                        	throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED)
+                            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                        }
                         timeStamp = TransactionUtil.getTableTimestamp(connection, isTransactional);
                         changingPhoenixTableProperty = true;
                         nonTxToTx = true;