You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2016/02/24 22:13:41 UTC

[23/50] [abbrv] phoenix git commit: PHOENIX-2671 System.STATS table getting truncated every time on new client connection(Ankit Singhal)

PHOENIX-2671 System.STATS table getting truncated every time on new client connection(Ankit Singhal)


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

Branch: refs/heads/calcite
Commit: 6881aef0cfaae8643303f0612a2a4b997b8a5138
Parents: cdaca28
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Feb 15 12:04:20 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon Feb 15 12:04:20 2016 +0530

----------------------------------------------------------------------
 .../query/ConnectionQueryServicesImpl.java      | 48 ++++++--------------
 .../org/apache/phoenix/util/UpgradeUtil.java    | 16 +++----
 2 files changed, 22 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6881aef0/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 897c207..9a385b2 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
@@ -188,11 +188,6 @@ import org.apache.twill.zookeeper.ZKClients;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.distributed.PooledClientProvider;
-import co.cask.tephra.distributed.TransactionServiceClient;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Throwables;
@@ -205,6 +200,11 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.distributed.PooledClientProvider;
+import co.cask.tephra.distributed.TransactionServiceClient;
+
 
 public class ConnectionQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices {
     private static final Logger logger = LoggerFactory.getLogger(ConnectionQueryServicesImpl.class);
@@ -2370,14 +2370,16 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     // parts we haven't yet done).
                                     metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
                                             PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                    metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1, 
-                                            PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
+                                    // Drop old stats table so that new stats table is created
+                                    metaConnection = dropStatsTable(metaConnection,
+                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
+                                    metaConnection = addColumnsIfNotExists(metaConnection,
+                                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0,
+                                            PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " "
+                                                    + PLong.INSTANCE.getSqlTypeName());
                                     setImmutableTableIndexesImmutable(metaConnection);
-									// Drop old stats table so that new stats table is created
-									metaConnection = dropStatsTable(metaConnection,
-											MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
-									// Clear the server cache so the above changes make it over to any clients
-									// that already have cached data.
+                                    // that already have cached data.
 									clearCache();
                                 }
                                 
@@ -2511,7 +2513,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
 			throws SQLException, IOException {
 		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp-1));
+		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
 		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
 		SQLException sqlE = null;
 		boolean wasCommit = metaConnection.getAutoCommit();
@@ -2540,26 +2542,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 				throw sqlE;
 			}
 		}
-
-		oldMetaConnection = metaConnection;
-		props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-		try {
-			metaConnection = new PhoenixConnection(oldMetaConnection, ConnectionQueryServicesImpl.this, props);
-		} finally {
-			try {
-				oldMetaConnection.close();
-			} catch (SQLException e) {
-				if (sqlE != null) {
-					sqlE.setNextException(e);
-				} else {
-					sqlE = e;
-				}
-			}
-			if (sqlE != null) {
-				throw sqlE;
-			}
-		}
 		return metaConnection;
 	}
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6881aef0/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 1d6f438..ac4ff7a 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
@@ -1221,18 +1221,16 @@ public class UpgradeUtil {
 
     public static boolean truncateStats(HTableInterface metaTable, HTableInterface statsTable)
             throws IOException, InterruptedException {
-        List<Cell> columnCells = metaTable
-                .get(new Get(SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
-                        PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE)))
+        byte[] statsTableKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
+                PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE);
+        List<Cell> columnCells = metaTable.get(new Get(statsTableKey))
                 .getColumnCells(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES);
-        if (!columnCells.isEmpty()
-                && columnCells.get(0).getTimestamp() < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
+        long timestamp;
+        if (!columnCells.isEmpty() && (timestamp = columnCells.get(0)
+                .getTimestamp()) < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
 
-            byte[] statsTableKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
-                    PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE);
             KeyValue upgradeKV = KeyValueUtil.newKeyValue(statsTableKey, PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
-                    UPGRADE_TO_4_7_COLUMN_NAME, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1,
-                    ByteUtil.EMPTY_BYTE_ARRAY);
+                    UPGRADE_TO_4_7_COLUMN_NAME, timestamp, PBoolean.INSTANCE.toBytes(true));
             Put upgradePut = new Put(statsTableKey);
             upgradePut.add(upgradeKV);