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 2016/02/27 09:15:31 UTC

phoenix git commit: PHOENIX-2718 Prevent stats creation for stats table

Repository: phoenix
Updated Branches:
  refs/heads/master 29ca38b85 -> 4f743230b


PHOENIX-2718 Prevent stats creation for stats table


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

Branch: refs/heads/master
Commit: 4f743230bcf91630cff0ae1bb0e86bc9680787ce
Parents: 29ca38b
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Feb 27 00:15:22 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Feb 27 00:15:22 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/PhoenixRuntimeIT.java       |  4 +-
 .../phoenix/end2end/index/LocalIndexIT.java     |  2 +-
 .../phoenix/end2end/index/ViewIndexIT.java      |  2 +-
 .../ipc/controller/MetadataRpcController.java   |  2 +-
 .../coprocessor/SequenceRegionObserver.java     | 12 ++---
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   | 17 ++++----
 .../query/ConnectionQueryServicesImpl.java      | 20 ++++-----
 .../org/apache/phoenix/schema/Sequence.java     | 46 ++++++++++----------
 .../stats/StatisticsCollectorFactory.java       |  3 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |  2 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |  4 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    | 18 ++++----
 .../java/org/apache/phoenix/query/BaseTest.java |  2 +-
 .../apache/phoenix/util/PhoenixRuntimeTest.java |  2 +-
 14 files changed, 69 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
index 234ea1c..e748825 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
@@ -107,8 +107,8 @@ public class PhoenixRuntimeIT extends BaseHBaseManagedTimeIT {
         props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "t1");
         Connection tsconn = DriverManager.getConnection(getUrl(), props);
         tsconn.createStatement().execute("CREATE SEQUENCE s1");
-        Expression e1 = PhoenixRuntime.getTenantIdExpression(tsconn, PhoenixDatabaseMetaData.SEQUENCE_FULLNAME);
-        HTableInterface htable1 = tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+        Expression e1 = PhoenixRuntime.getTenantIdExpression(tsconn, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME);
+        HTableInterface htable1 = tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
         assertTenantIds(e1, htable1, new FirstKeyOnlyFilter(), new String[] {"", "t1"} );
         
         tsconn.createStatement().execute("CREATE VIEW A.BAR(V1 VARCHAR) AS SELECT * FROM " + tableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index a79d3d1..d0d56b8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -184,7 +184,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         ResultSet rs = conn2.createStatement().executeQuery("SELECT "
                 + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
                 + PhoenixDatabaseMetaData.SEQUENCE_NAME
-                + " FROM " + PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_ESCAPED);
+                + " FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE);
         assertFalse("View index sequences should be deleted.", rs.next());
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
index fe8ea77..79b776c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
@@ -83,7 +83,7 @@ public class ViewIndexIT extends BaseHBaseManagedTimeIT {
         ResultSet rs = conn2.createStatement().executeQuery("SELECT "
                 + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
                 + PhoenixDatabaseMetaData.SEQUENCE_NAME
-                + " FROM " + PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_ESCAPED);
+                + " FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE);
         assertFalse("View index sequences should be deleted.", rs.next());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
index 2d51d08..ab463e9 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
@@ -35,7 +35,7 @@ class MetadataRpcController extends DelegatingPayloadCarryingRpcController {
 	private static final List<String> SYSTEM_TABLE_NAMES = new ImmutableList.Builder<String>()
 			.add(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME)
 			.add(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME)
-			.add(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME)
+			.add(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME)
 			.add(PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME).build();
 
 	public MetadataRpcController(PayloadCarryingRpcController delegate,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
index bb6c4b5..49962e0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
@@ -85,7 +85,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
         PInteger.INSTANCE.getCodec().encodeInt(errorCode, errorCodeBuf, 0);
         return  Result.create(Collections.singletonList(
                 (Cell)KeyValueUtil.newKeyValue(row, 
-                        PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, 
+                        PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, 
                         QueryConstants.EMPTY_COLUMN_BYTES, timestamp, errorCodeBuf)));
     }
     
@@ -154,7 +154,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
                 long timestamp = currentValueKV.getTimestamp();
 				Put put = new Put(row, timestamp);
                 
-				int numIncrementKVs = increment.getFamilyCellMap().get(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES).size();
+				int numIncrementKVs = increment.getFamilyCellMap().get(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES).size();
                 // creates the list of KeyValues used for the Result that will be returned
                 List<Cell> cells = Sequence.getCells(result, numIncrementKVs);
                 
@@ -298,7 +298,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
 	KeyValue createKeyValue(byte[] key, byte[] cqBytes, long value, long timestamp) {
 		byte[] valueBuffer = new byte[PLong.INSTANCE.getByteSize()];
     PLong.INSTANCE.getCodec().encodeLong(value, valueBuffer, 0);
-		return KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, cqBytes, timestamp, valueBuffer);
+		return KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, cqBytes, timestamp, valueBuffer);
 	}
     
 	/**
@@ -312,7 +312,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
 	 */
 	private KeyValue createKeyValue(byte[] key, byte[] cqBytes, boolean value, long timestamp) throws IOException {
 		// create new key value for put
-		return KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, cqBytes, 
+		return KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, cqBytes, 
 				timestamp, value ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
 	}
 
@@ -399,7 +399,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
                     // Timestamp should match exactly, or we may have the wrong sequence
                     if (expectedValue != value || currentValueKV.getTimestamp() != clientTimestamp) {
                         return Result.create(Collections.singletonList(
-                          (Cell)KeyValueUtil.newKeyValue(row, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, 
+                          (Cell)KeyValueUtil.newKeyValue(row, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, 
                             QueryConstants.EMPTY_COLUMN_BYTES, currentValueKV.getTimestamp(), ByteUtil.EMPTY_BYTE_ARRAY)));
                     }
                     m = new Put(row, currentValueKV.getTimestamp());
@@ -427,7 +427,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
                 // the client cares about is the timestamp, which is the timestamp of
                 // when the mutation was actually performed (useful in the case of .
                 return Result.create(Collections.singletonList(
-                  (Cell)KeyValueUtil.newKeyValue(row, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, serverTimestamp, SUCCESS_VALUE)));
+                  (Cell)KeyValueUtil.newKeyValue(row, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, serverTimestamp, SUCCESS_VALUE)));
             } finally {
                 region.releaseRowLocks(locks);
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index fabd949..26d57fa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -104,6 +104,15 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final byte[] SYSTEM_STATS_TABLE_BYTES = Bytes.toBytes(SYSTEM_STATS_TABLE);
     public static final String SYSTEM_CATALOG_ALIAS = "\"SYSTEM.TABLE\"";
 
+    public static final byte[] SYSTEM_SEQUENCE_FAMILY_BYTES = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
+    public static final String SYSTEM_SEQUENCE_SCHEMA = SYSTEM_CATALOG_SCHEMA;
+    public static final byte[] SYSTEM_SEQUENCE_SCHEMA_BYTES = Bytes.toBytes(SYSTEM_SEQUENCE_SCHEMA);
+    public static final String SYSTEM_SEQUENCE_TABLE = "SEQUENCE";
+    public static final byte[] SYSTEM_SEQUENCE_TABLE_BYTES = Bytes.toBytes(SYSTEM_SEQUENCE_TABLE);
+    public static final String SYSTEM_SEQUENCE = SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_SEQUENCE_TABLE + "\"";
+    public static final String SYSTEM_SEQUENCE_NAME = SchemaUtil.getTableName(SYSTEM_SEQUENCE_SCHEMA, SYSTEM_SEQUENCE_TABLE);
+    public static final byte[] SYSTEM_SEQUENCE_NAME_BYTES = Bytes.toBytes(SYSTEM_SEQUENCE_NAME);
+    
     public static final String TABLE_NAME = "TABLE_NAME";
     public static final byte[] TABLE_NAME_BYTES = Bytes.toBytes(TABLE_NAME);
     public static final String TABLE_TYPE = "TABLE_TYPE";
@@ -223,14 +232,6 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final String NUM_ARGS = "NUM_ARGS";
     public static final byte[] NUM_ARGS_BYTES = Bytes.toBytes(NUM_ARGS);
     
-    public static final byte[] SEQUENCE_FAMILY_BYTES = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
-    public static final String SEQUENCE_SCHEMA_NAME = SYSTEM_CATALOG_SCHEMA;
-    public static final byte[] SEQUENCE_SCHEMA_NAME_BYTES = Bytes.toBytes(SEQUENCE_SCHEMA_NAME);
-    public static final String SEQUENCE_TABLE_NAME = TYPE_SEQUENCE;
-    public static final byte[] SEQUENCE_TABLE_NAME_BYTES = Bytes.toBytes(SEQUENCE_TABLE_NAME);
-    public static final String SEQUENCE_FULLNAME_ESCAPED = SYSTEM_CATALOG_SCHEMA + ".\"" + TYPE_SEQUENCE + "\"";
-    public static final String SEQUENCE_FULLNAME = SchemaUtil.getTableName(SEQUENCE_SCHEMA_NAME, SEQUENCE_TABLE_NAME);
-    public static final byte[] SEQUENCE_FULLNAME_BYTES = Bytes.toBytes(SEQUENCE_FULLNAME);
     public static final String SEQUENCE_SCHEMA = "SEQUENCE_SCHEMA";
     public static final String SEQUENCE_NAME = "SEQUENCE_NAME";
     public static final String CURRENT_VALUE = "CURRENT_VALUE";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/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 3327861..39118a3 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
@@ -2417,14 +2417,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1) {
                                     if (UpgradeUtil.upgradeSequenceTable(metaConnection, nSaltBuckets, e.getTable())) {
                                         metaConnection.removeTable(null,
-                                                PhoenixDatabaseMetaData.SEQUENCE_SCHEMA_NAME,
-                                                PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME,
+                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
+                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                         clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
-                                                PhoenixDatabaseMetaData.SEQUENCE_SCHEMA_NAME_BYTES,
-                                                PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES,
+                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
+                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
-                                        clearTableRegionCache(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+                                        clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                                     }
                                     nSequenceSaltBuckets = nSaltBuckets;
                                 } else { 
@@ -2773,7 +2773,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // Now that we have the lock we need, create the sequence
             Append append = sequence.createSequence(startWith, incrementBy, cacheSize, timestamp, minValue, maxValue, cycle);
             HTableInterface htable =
-                    this.getTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+                    this.getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
             htable.setAutoFlush(true);
             try {
                 Result result = htable.append(append);
@@ -2800,7 +2800,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             sequence.getLock().lock();
             // Now that we have the lock we need, create the sequence
             Append append = sequence.dropSequence(timestamp);
-            HTableInterface htable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+            HTableInterface htable = this.getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
             try {
                 Result result = htable.append(append);
                 return sequence.dropSequence(result);
@@ -2896,7 +2896,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (toIncrementList.isEmpty()) {
                 return;
             }
-            HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+            HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
             Object[] resultObjects = null;
             SQLException sqlE = null;
             try {
@@ -3019,7 +3019,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (toReturnList.isEmpty()) {
                 return;
             }
-            HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+            HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
             Object[] resultObjects = null;
             SQLException sqlE = null;
             try {
@@ -3072,7 +3072,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (mutations.isEmpty()) {
             return;
         }
-        HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+        HTableInterface hTable = this.getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
         SQLException sqlE = null;
         try {
             hTable.batch(mutations);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
index a2041f7..9c1a74b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
@@ -25,7 +25,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY_BYTES
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_VALUE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MIN_VALUE_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES;
 
 import java.io.IOException;
 import java.sql.SQLException;
@@ -66,13 +66,13 @@ public class Sequence {
     public enum MetaOp {CREATE_SEQUENCE, DROP_SEQUENCE, RETURN_SEQUENCE};
     
     // create empty Sequence key values used while created a sequence row
-    private static final KeyValue CURRENT_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, CURRENT_VALUE_BYTES);
-    private static final KeyValue INCREMENT_BY_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, INCREMENT_BY_BYTES);
-    private static final KeyValue CACHE_SIZE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, CACHE_SIZE_BYTES);
-    private static final KeyValue MIN_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, MIN_VALUE_BYTES);
-    private static final KeyValue MAX_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, MAX_VALUE_BYTES);
-    private static final KeyValue CYCLE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, CYCLE_FLAG_BYTES);
-    private static final KeyValue LIMIT_REACHED_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SEQUENCE_FAMILY_BYTES, LIMIT_REACHED_FLAG_BYTES);
+    private static final KeyValue CURRENT_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SYSTEM_SEQUENCE_FAMILY_BYTES, CURRENT_VALUE_BYTES);
+    private static final KeyValue INCREMENT_BY_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SYSTEM_SEQUENCE_FAMILY_BYTES, INCREMENT_BY_BYTES);
+    private static final KeyValue CACHE_SIZE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SYSTEM_SEQUENCE_FAMILY_BYTES, CACHE_SIZE_BYTES);
+    private static final KeyValue MIN_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SYSTEM_SEQUENCE_FAMILY_BYTES, MIN_VALUE_BYTES);
+    private static final KeyValue MAX_VALUE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SYSTEM_SEQUENCE_FAMILY_BYTES, MAX_VALUE_BYTES);
+    private static final KeyValue CYCLE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SYSTEM_SEQUENCE_FAMILY_BYTES, CYCLE_FLAG_BYTES);
+    private static final KeyValue LIMIT_REACHED_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, SYSTEM_SEQUENCE_FAMILY_BYTES, LIMIT_REACHED_FLAG_BYTES);
     private static final List<KeyValue> SEQUENCE_KV_COLUMNS = Arrays.<KeyValue>asList(
             CURRENT_VALUE_KV,
             INCREMENT_BY_KV,
@@ -300,9 +300,9 @@ public class Sequence {
         append.setAttribute(SequenceRegionObserver.OPERATION_ATTRIB, opBuf);
         append.setAttribute(SequenceRegionObserver.CURRENT_VALUE_ATTRIB, PLong.INSTANCE.toBytes(value.nextValue));
         Map<byte[], List<Cell>> familyMap = append.getFamilyCellMap();
-        familyMap.put(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, Arrays.<Cell>asList(
-        		(Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CURRENT_VALUE_BYTES, value.timestamp, PLong.INSTANCE.toBytes(value.currentValue)),
-        		(Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG_BYTES, value.timestamp, PBoolean.INSTANCE.toBytes(value.limitReached))
+        familyMap.put(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, Arrays.<Cell>asList(
+        		(Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CURRENT_VALUE_BYTES, value.timestamp, PLong.INSTANCE.toBytes(value.currentValue)),
+        		(Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG_BYTES, value.timestamp, PBoolean.INSTANCE.toBytes(value.limitReached))
                 ));
         return append;
     }
@@ -550,16 +550,16 @@ public class Sequence {
         }
         Map<byte[], List<Cell>> familyMap = append.getFamilyCellMap();
         byte[] startWithBuf = PLong.INSTANCE.toBytes(startWith);
-        familyMap.put(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, Arrays.<Cell>asList(
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timestamp, ByteUtil.EMPTY_BYTE_ARRAY),
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CURRENT_VALUE_BYTES, timestamp, startWithBuf),
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.START_WITH_BYTES, timestamp, startWithBuf),
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.INCREMENT_BY_BYTES, timestamp, PLong.INSTANCE.toBytes(incrementBy)),
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CACHE_SIZE_BYTES, timestamp, PLong.INSTANCE.toBytes(cacheSize)),
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.MIN_VALUE_BYTES, timestamp, PLong.INSTANCE.toBytes(minValue)),
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.MAX_VALUE_BYTES, timestamp, PLong.INSTANCE.toBytes(maxValue)),
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CYCLE_FLAG_BYTES, timestamp, PBoolean.INSTANCE.toBytes(cycle)),
-                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG_BYTES, timestamp, PDataType.FALSE_BYTES)
+        familyMap.put(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, Arrays.<Cell>asList(
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timestamp, ByteUtil.EMPTY_BYTE_ARRAY),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CURRENT_VALUE_BYTES, timestamp, startWithBuf),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.START_WITH_BYTES, timestamp, startWithBuf),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.INCREMENT_BY_BYTES, timestamp, PLong.INSTANCE.toBytes(incrementBy)),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CACHE_SIZE_BYTES, timestamp, PLong.INSTANCE.toBytes(cacheSize)),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.MIN_VALUE_BYTES, timestamp, PLong.INSTANCE.toBytes(minValue)),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.MAX_VALUE_BYTES, timestamp, PLong.INSTANCE.toBytes(maxValue)),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CYCLE_FLAG_BYTES, timestamp, PBoolean.INSTANCE.toBytes(cycle)),
+                KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG_BYTES, timestamp, PDataType.FALSE_BYTES)
                 ));
         return append;
     }
@@ -588,8 +588,8 @@ public class Sequence {
             append.setAttribute(SequenceRegionObserver.MAX_TIMERANGE_ATTRIB, Bytes.toBytes(timestamp));
         }
         Map<byte[], List<Cell>> familyMap = append.getFamilyCellMap();
-        familyMap.put(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, Arrays.<Cell>asList(
-                (Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timestamp, ByteUtil.EMPTY_BYTE_ARRAY)));
+        familyMap.put(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, Arrays.<Cell>asList(
+                (Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timestamp, ByteUtil.EMPTY_BYTE_ARRAY)));
         return append;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectorFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectorFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectorFactory.java
index 964bb28..dcffa41 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectorFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectorFactory.java
@@ -61,7 +61,8 @@ public class StatisticsCollectorFactory {
     static {
         DISABLE_STATS.add(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME));
         DISABLE_STATS.add(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME));
-        DISABLE_STATS.add(TableName.valueOf(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME));
+        DISABLE_STATS.add(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME));
+        DISABLE_STATS.add(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME));
     }
     
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 6d5903d..5127f5b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -375,7 +375,7 @@ public class MetaDataUtil {
 
     public static void deleteViewIndexSequences(PhoenixConnection connection, PName name) throws SQLException {
         String schemaName = getViewIndexSchemaName(name);
-        connection.createStatement().executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_ESCAPED + 
+        connection.createStatement().executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE + 
                 " WHERE " + PhoenixDatabaseMetaData.TENANT_ID + " IS NULL AND " + 
                 PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + " = '" + schemaName + "'");
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index de92046..523f684 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -426,11 +426,11 @@ public class SchemaUtil {
     }
     
     public static boolean isSequenceTable(byte[] tableName) {
-        return Bytes.compareTo(tableName, PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES) == 0;
+        return Bytes.compareTo(tableName, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES) == 0;
     }
 
     public static boolean isSequenceTable(PTable table) {
-        return PhoenixDatabaseMetaData.SEQUENCE_FULLNAME.equals(table.getName().getString());
+        return PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME.equals(table.getName().getString());
     }
 
     public static boolean isMetaTable(PTable table) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index ac4ff7a..ba33a2c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -128,17 +128,17 @@ public class UpgradeUtil {
     }
 
     private static byte[] getSequenceSnapshotName() {
-        return Bytes.toBytes("_BAK_" + PhoenixDatabaseMetaData.SEQUENCE_FULLNAME);
+        return Bytes.toBytes("_BAK_" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME);
     }
     
     private static void createSequenceSnapshot(HBaseAdmin admin, PhoenixConnection conn) throws SQLException {
         byte[] tableName = getSequenceSnapshotName();
-        HColumnDescriptor columnDesc = new HColumnDescriptor(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES);
+        HColumnDescriptor columnDesc = new HColumnDescriptor(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_FAMILY_BYTES);
         HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
         desc.addFamily(columnDesc);
         try {
             admin.createTable(desc);
-            copyTable(conn, PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES, tableName);
+            copyTable(conn, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES, tableName);
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
         }
@@ -146,7 +146,7 @@ public class UpgradeUtil {
     
     private static void restoreSequenceSnapshot(HBaseAdmin admin, PhoenixConnection conn) throws SQLException {
         byte[] tableName = getSequenceSnapshotName();
-        copyTable(conn, tableName, PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+        copyTable(conn, tableName, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
     }
     
     private static void deleteSequenceSnapshot(HBaseAdmin admin) throws SQLException {
@@ -237,11 +237,11 @@ public class UpgradeUtil {
                 return;
             }
             logger.warn("Pre-splitting SYSTEM.SEQUENCE table " + nSaltBuckets + "-ways. This may take some time - please do not close window.");
-            HTableDescriptor desc = admin.getTableDescriptor(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+            HTableDescriptor desc = admin.getTableDescriptor(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
             createSequenceSnapshot(admin, conn);
             snapshotCreated = true;
-            admin.disableTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME);
-            admin.deleteTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME);
+            admin.disableTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME);
+            admin.deleteTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME);
             byte[][] splitPoints = SaltingUtil.getSalteByteSplitPoints(nSaltBuckets);
             admin.createTable(desc, splitPoints);
             restoreSequenceSnapshot(admin, conn);
@@ -272,7 +272,7 @@ public class UpgradeUtil {
     public static boolean upgradeSequenceTable(PhoenixConnection conn, int nSaltBuckets, PTable oldTable) throws SQLException {
         logger.info("Upgrading SYSTEM.SEQUENCE table");
 
-        byte[] seqTableKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SEQUENCE_SCHEMA_NAME, PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME);
+        byte[] seqTableKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE);
         HTableInterface sysTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
         try {
             logger.info("Setting SALT_BUCKETS property of SYSTEM.SEQUENCE to " + SaltingUtil.MAX_BUCKET_NUM);
@@ -326,7 +326,7 @@ public class UpgradeUtil {
                 Scan scan = new Scan();
                 scan.setRaw(true);
                 scan.setMaxVersions(MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS);
-                HTableInterface seqTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_BYTES);
+                HTableInterface seqTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                 try {
                     boolean committed = false;
                     logger.info("Adding salt byte to all SYSTEM.SEQUENCE rows");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/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 1192059..8b9b74f 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
@@ -943,7 +943,7 @@ public abstract class BaseTest {
                 + PhoenixDatabaseMetaData.TENANT_ID + ","
                 + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + "," 
                 + PhoenixDatabaseMetaData.SEQUENCE_NAME 
-                + " FROM " + PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_ESCAPED);
+                + " FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE);
         String lastTenantId = null;
         Connection conn = globalConn;
         while (rs.next()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4f743230/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
index b6465c3..895e29e 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
@@ -171,7 +171,7 @@ public class PhoenixRuntimeTest extends BaseConnectionlessQueryTest {
         Expression e2 = PhoenixRuntime.getTenantIdExpression(conn, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
         assertNotNull(e2);
 
-        Expression e3 = PhoenixRuntime.getTenantIdExpression(conn, PhoenixDatabaseMetaData.SEQUENCE_FULLNAME);
+        Expression e3 = PhoenixRuntime.getTenantIdExpression(conn, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME);
         assertNotNull(e3);
         
         conn.createStatement().execute("CREATE TABLE FOO (k VARCHAR PRIMARY KEY)");