You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by pb...@apache.org on 2018/04/13 22:40:09 UTC

[20/20] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)

PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-cdh5.11
Commit: 9fc11de1586c93c53d2fe994e374e2f15b3b3f05
Parents: 4022649
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Apr 13 22:30:00 2018 +0100
Committer: Pedro Boado <pb...@apache.org>
Committed: Fri Apr 13 23:31:51 2018 +0100

----------------------------------------------------------------------
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 --
 .../query/ConnectionQueryServicesImpl.java      | 41 ++++++++++++++------
 .../query/ConnectionlessQueryServicesImpl.java  | 35 +++++++++++++++--
 .../apache/phoenix/query/QueryConstants.java    | 14 +++----
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java     |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  4 +-
 7 files changed, 74 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 26f8198..36d6f0d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -72,10 +72,6 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
     public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-    public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-    public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
-    public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
-    public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
     public static final int DEFAULT_LOG_VERSIONS = 10;
     public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 7f97c74..02fcf24 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
@@ -720,10 +720,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private HColumnDescriptor generateColumnFamilyDescriptor(Pair<byte[],Map<String,Object>> family, PTableType tableType) throws SQLException {
         HColumnDescriptor columnDesc = new HColumnDescriptor(family.getFirst());
         if (tableType != PTableType.VIEW) {
-            if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
-                columnDesc.setKeepDeletedCells(props.getBoolean(
-                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
-            }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             columnDesc.setBloomFilterType(BloomType.NONE);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -2459,8 +2455,29 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     // Available for testing
-    protected String getSystemCatalogDML() {
-        return QueryConstants.CREATE_TABLE_METADATA;
+    protected String getSystemCatalogTableDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+    }
+
+    // Available for testing
+    protected String getFunctionTableDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+    }
+
+    // Available for testing
+    protected String getLogTableDDL() {
+        return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+    }
+
+    private String setSystemDDLProperties(String ddl) {
+        return String.format(ddl,
+          props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+          props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
+    }
+
+    private String setSystemLogDDLProperties(String ddl) {
+        return String.format(ddl,
+          props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
     }
 
     @Override
@@ -2503,7 +2520,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                          scnProps, newEmptyMetaData())) {
                                 try {
                                     metaConnection.setRunningUpgrade(true);
-                                    metaConnection.createStatement().executeUpdate(getSystemCatalogDML());
+                                    metaConnection.createStatement().executeUpdate(getSystemCatalogTableDDL());
                                 } catch (NewerTableAlreadyExistsException ignore) {
                                     // Ignore, as this will happen if the SYSTEM.CATALOG already exists at this fixed
                                     // timestamp. A TableAlreadyExistsException is not thrown, since the table only exists
@@ -2665,10 +2682,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             metaConnection.createStatement().execute(QueryConstants.CREATE_STATS_TABLE_METADATA);
         } catch (TableAlreadyExistsException ignore) {}
         try {
-            metaConnection.createStatement().execute(QueryConstants.CREATE_FUNCTION_METADATA);
+            metaConnection.createStatement().execute(getFunctionTableDDL());
         } catch (TableAlreadyExistsException ignore) {}
         try {
-            metaConnection.createStatement().execute(QueryConstants.CREATE_LOG_METADATA);
+            metaConnection.createStatement().execute(getLogTableDDL());
         } catch (TableAlreadyExistsException ignore) {}
         // Catch the IOException to log the error message and then bubble it up for the client to retry.
         try {
@@ -2971,7 +2988,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 createSysMutexTableIfNotExists(admin);
             }
             try {
-                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
+                metaConnection.createStatement().executeUpdate(getSystemCatalogTableDDL());
             } catch (NewerTableAlreadyExistsException ignore) {
                 // Ignore, as this will happen if the SYSTEM.CATALOG already exists at this fixed
                 // timestamp. A TableAlreadyExistsException is not thrown, since the table only exists
@@ -3093,10 +3110,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             try {
-                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
+                metaConnection.createStatement().executeUpdate(getFunctionTableDDL());
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
             try {
-                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_LOG_METADATA);
+                metaConnection.createStatement().executeUpdate(getLogTableDDL());
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
 
             // In case namespace mapping is enabled and system table to system namespace mapping is also enabled,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 14abd63..c6c2617 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -84,11 +84,13 @@ import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
 import org.apache.phoenix.transaction.PhoenixTransactionClient;
 import org.apache.phoenix.transaction.TransactionFactory.Provider;
+import org.apache.phoenix.util.ConfigUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.JDBCUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SequenceUtil;
 
@@ -106,6 +108,7 @@ import com.google.common.collect.Maps;
 public class ConnectionlessQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices  {
     private static ServerName SERVER_NAME = ServerName.parseServerName(HConstants.LOCALHOST + Addressing.HOSTNAME_PORT_SEPARATOR + HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT);
     
+    private final ReadOnlyProps props;
     private PMetaData metaData;
     private final Map<SequenceKey, SequenceInfo> sequenceMap = Maps.newHashMap();
     private final String userName;
@@ -143,12 +146,38 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         // on the server side during testing.
         this.config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration(config);
         this.guidePostsCache = new GuidePostsCache(this, config);
+        // set replication required parameter
+        ConfigUtil.setReplicationConfigIfAbsent(this.config);
+        this.props = new ReadOnlyProps(this.config.iterator());
     }
 
     private PMetaData newEmptyMetaData() {
         return new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps());
     }
 
+    protected String getSystemCatalogTableDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+    }
+
+    protected String getFunctionTableDDL() {
+        return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+    }
+
+    protected String getLogTableDDL() {
+        return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+    }
+
+    private String setSystemDDLProperties(String ddl) {
+        return String.format(ddl,
+          props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+          props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
+    }
+
+    private String setSystemLogDDLProperties(String ddl) {
+        return String.format(ddl,
+          props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
+    }
+
     @Override
     public ConnectionQueryServices getChildQueryServices(ImmutableBytesWritable childId) {
         return this; // Just reuse the same query services
@@ -307,7 +336,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                 metaConnection = new PhoenixConnection(this, globalUrl, scnProps, newEmptyMetaData());
                 metaConnection.setRunningUpgrade(true);
                 try {
-                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
+                    metaConnection.createStatement().executeUpdate(getSystemCatalogTableDDL());
                 } catch (TableAlreadyExistsException ignore) {
                     // Ignore, as this will happen if the SYSTEM.TABLE already exists at this fixed timestamp.
                     // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
@@ -330,11 +359,11 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                 }
                 
                 try {
-                   metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
+                    metaConnection.createStatement().executeUpdate(getFunctionTableDDL());
                 } catch (NewerTableAlreadyExistsException ignore) {
                 }
                 try {
-                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_LOG_METADATA);
+                    metaConnection.createStatement().executeUpdate(getLogTableDDL());
                 } catch (NewerTableAlreadyExistsException ignore) {}
             } catch (SQLException e) {
                 sqlE = e;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index d181fc8..22fa2f4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -343,8 +343,8 @@ public interface QueryConstants {
             TRANSACTION_PROVIDER + " TINYINT, " +
             "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
             + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
-            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n" +
+            HConstants.VERSIONS + "=%s,\n" +
+            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
             // Install split policy to prevent a tenant's metadata from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
@@ -361,8 +361,6 @@ public interface QueryConstants {
             "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY ("
             + PHYSICAL_NAME + ","
             + COLUMN_FAMILY + ","+ GUIDE_POST_KEY+"))\n" +
-            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_STAT_DATA_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_STATS_KEEP_DELETED_CELLS + ",\n" +
             // Install split policy to prevent a physical table's stats from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
@@ -382,8 +380,6 @@ public interface QueryConstants {
             CYCLE_FLAG + " BOOLEAN, \n" +
             LIMIT_REACHED_FLAG + " BOOLEAN \n" +
             " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + "," + SEQUENCE_SCHEMA + "," + SEQUENCE_NAME + "))\n" +
-            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n" +
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
     public static final String CREATE_SYSTEM_SCHEMA = "CREATE SCHEMA " + SYSTEM_CATALOG_SCHEMA;
     public static final String UPGRADE_TABLE_SNAPSHOT_PREFIX = "_UPGRADING_TABLE_";
@@ -407,8 +403,8 @@ public interface QueryConstants {
             MIN_VALUE + " VARCHAR, \n" +
             MAX_VALUE + " VARCHAR, \n" +
             " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ", " + FUNCTION_NAME + ", " + TYPE + ", " + ARG_POSITION + "))\n" +
-            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n"+
+            HConstants.VERSIONS + "=%s,\n" +
+            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n"+
             // Install split policy to prevent a tenant's metadata from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
@@ -434,7 +430,7 @@ public interface QueryConstants {
             SCAN_METRICS_JSON + " VARCHAR, \n" +
             " CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (QUERY_ID))\n" +
             HConstants.VERSIONS + "= " + MetaDataProtocol.DEFAULT_LOG_VERSIONS + ",\n" +
-            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n"+
+            HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n"+
             // Install split policy to prevent a tenant's metadata from being split across regions.
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "',\n" + 
             PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE+ ",\n" + 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 29d18d9..8cc156c 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
@@ -204,7 +204,6 @@ public interface QueryServices extends SQLCloseable {
     public static final String ALLOW_ONLINE_TABLE_SCHEMA_UPDATE = "hbase.online.schema.update.enable";
     public static final String NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK = "phoenix.schema.change.retries";
     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_TABLE_ISTRANSACTIONAL_ATTRIB = "phoenix.table.istransactional.default";
     public static final String DEFAULT_TRANSACTION_PROVIDER_ATTRIB = "phoenix.table.transaction.provider.default";
@@ -252,6 +251,10 @@ public interface QueryServices extends SQLCloseable {
     public static final String QUERY_SERVER_REMOTEUSEREXTRACTOR_PARAM = "phoenix.queryserver.remoteUserExtractor.param";
     public static final String QUERY_SERVER_DISABLE_KERBEROS_LOGIN = "phoenix.queryserver.disable.kerberos.login";
 
+    // metadata configs
+    public static final String DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB = "phoenix.system.default.keep.deleted.cells";
+    public static final String DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB = "phoenix.system.default.max.versions";
+
     public static final String RENEW_LEASE_ENABLED = "phoenix.scanner.lease.renew.enabled";
     public static final String RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS = "phoenix.scanner.lease.renew.interval";
     public static final String RENEW_LEASE_THRESHOLD_MILLISECONDS = "phoenix.scanner.lease.threshold";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 70ac11b..23aed7c 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
@@ -213,6 +213,8 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_ALLOW_LOCAL_INDEX = true;
     public static final int DEFAULT_INDEX_HANDLER_COUNT = 30;
     public static final int DEFAULT_METADATA_HANDLER_COUNT = 30;
+    public static final int DEFAULT_SYSTEM_MAX_VERSIONS = 1;
+    public static final boolean DEFAULT_SYSTEM_KEEP_DELETED_CELLS = false;
 
     // Retries when doing server side writes to SYSTEM.CATALOG
     // 20 retries with 100 pause = 230 seconds total retry time
@@ -257,7 +259,6 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_ALLOW_ONLINE_TABLE_SCHEMA_UPDATE = true;
     public static final int DEFAULT_RETRIES_FOR_SCHEMA_UPDATE_CHECK = 10;
     public static final long DEFAULT_DELAY_FOR_SCHEMA_UPDATE_CHECK = 5 * 1000; // 5 seconds.
-    public static final boolean DEFAULT_KEEP_DELETED_CELLS = false;
     public static final boolean DEFAULT_STORE_NULLS = false;
 
     // TODO Change this to true as part of PHOENIX-1543

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9fc11de1/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 741bdff..63f2183 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
@@ -223,7 +223,7 @@ public class UpgradeUtil {
 
         Scan scan = new Scan();
         scan.setRaw(true);
-        scan.setMaxVersions(MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS);
+        scan.setMaxVersions();
         ResultScanner scanner = null;
         HTableInterface source = null;
         HTableInterface target = null;
@@ -698,7 +698,7 @@ public class UpgradeUtil {
                 boolean success = false;
                 Scan scan = new Scan();
                 scan.setRaw(true);
-                scan.setMaxVersions(MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS);
+                scan.setMaxVersions();
                 HTableInterface seqTable = conn.getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                 try {
                     boolean committed = false;