You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ch...@apache.org on 2020/10/19 18:50:00 UTC

[phoenix] branch 4.x updated: PHOENIX-6125 : Disable region split for SYSTEM.TASK

This is an automated email from the ASF dual-hosted git repository.

chinmayskulkarni pushed a commit to branch 4.x
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x by this push:
     new 4d2cf8c  PHOENIX-6125 : Disable region split for SYSTEM.TASK
4d2cf8c is described below

commit 4d2cf8c4c7e8e2bff4738843281ca1ba2bb49115
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Mon Oct 19 23:07:21 2020 +0530

    PHOENIX-6125 : Disable region split for SYSTEM.TASK
    
    Signed-off-by: Chinmay Kulkarni <ch...@apache.org>
---
 .../phoenix/end2end/BackwardCompatibilityIT.java   | 23 +++++++-
 ...ogUpgradeIT.java => SystemTablesUpgradeIT.java} | 42 ++++++++++----
 .../InvalidRegionSplitPolicyException.java         | 49 ++++++++++++++++
 .../apache/phoenix/exception/SQLExceptionCode.java |  6 +-
 .../phoenix/query/ConnectionQueryServicesImpl.java | 65 +++++++++++++++++++++-
 .../org/apache/phoenix/query/QueryConstants.java   |  9 ++-
 .../phoenix/schema/SystemTaskSplitPolicy.java      | 28 ++++++++++
 .../query/ConnectionQueryServicesImplTest.java     | 47 +++++++++++++++-
 8 files changed, 248 insertions(+), 21 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java
index e8ef84f..d20610a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java
@@ -49,6 +49,7 @@ import org.apache.curator.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -59,6 +60,7 @@ import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.SystemTaskSplitPolicy;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.After;
@@ -329,10 +331,27 @@ public class BackwardCompatibilityIT {
         executeQueriesWithCurrentVersion(QUERY_ADD_DELETE);
         assertExpectedOutput(QUERY_ADD_DELETE);
     }
-    
+
+    @Test
+    public void testUpdatedSplitPolicyForSysTask() throws Exception {
+        executeQueryWithClientVersion(compatibleClientVersion,
+            CREATE_DIVERGED_VIEW);
+        executeQueriesWithCurrentVersion(QUERY_CREATE_DIVERGED_VIEW);
+        try (org.apache.hadoop.hbase.client.Connection conn =
+                hbaseTestUtil.getConnection(); Admin admin = conn.getAdmin()) {
+            HTableDescriptor tableDescriptor = admin.getTableDescriptor(
+                TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_TASK_NAME));
+            assertEquals("split policy not updated with compatible client version: "
+                + compatibleClientVersion,
+                tableDescriptor.getRegionSplitPolicyClassName(),
+                SystemTaskSplitPolicy.class.getName());
+        }
+        assertExpectedOutput(QUERY_CREATE_DIVERGED_VIEW);
+    }
+
     private void checkForPreConditions() throws Exception {
         // For the first code cut of any major version, there wouldn't be any backward compatible
-        // clients. Hence the test wouldn't run and just return true when the client  
+        // clients. Hence the test wouldn't run and just return true when the client
         // version to be tested is same as current version
         assumeFalse(compatibleClientVersion.contains(MetaDataProtocol.CURRENT_CLIENT_VERSION));
         // Make sure that cluster is clean before test execution with no system tables
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogUpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesUpgradeIT.java
similarity index 84%
rename from phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogUpgradeIT.java
rename to phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesUpgradeIT.java
index 3fcef36..e38c5e6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogUpgradeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesUpgradeIT.java
@@ -25,8 +25,11 @@ import java.sql.SQLException;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
 import org.apache.phoenix.query.BaseTest;
@@ -34,52 +37,56 @@ import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.ConnectionQueryServicesImpl;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesTestImpl;
+import org.apache.phoenix.schema.SystemTaskSplitPolicy;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
-public class SystemCatalogUpgradeIT extends BaseTest {
+/**
+ * Tests for upgrades of System tables.
+ */
+public class SystemTablesUpgradeIT extends BaseTest {
     private static boolean reinitialize;
     private static int countUpgradeAttempts;
     private static long systemTableVersion = MetaDataProtocol.getPriorVersion();
-    
+
     private static class PhoenixUpgradeCountingServices extends ConnectionQueryServicesImpl {
         public PhoenixUpgradeCountingServices(QueryServices services, ConnectionInfo connectionInfo, Properties info) {
             super(services, connectionInfo, info);
         }
-        
+
         @Override
         protected void setUpgradeRequired() {
             super.setUpgradeRequired();
             countUpgradeAttempts++;
         }
-        
+
         @Override
         protected long getSystemTableVersion() {
             return systemTableVersion;
         }
-        
+
         @Override
         protected boolean isInitialized() {
             return !reinitialize && super.isInitialized();
         }
     }
-    
+
     public static class PhoenixUpgradeCountingDriver extends PhoenixTestDriver {
         private ConnectionQueryServices cqs;
         private final ReadOnlyProps overrideProps;
-        
+
         public PhoenixUpgradeCountingDriver(ReadOnlyProps props) {
             overrideProps = props;
         }
-        
+
         @Override
         public boolean acceptsURL(String url) throws SQLException {
             return true;
         }
-        
+
         @Override // public for testing
         public synchronized ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException {
             if (cqs == null) {
@@ -92,7 +99,7 @@ public class SystemCatalogUpgradeIT extends BaseTest {
             return cqs;
         }
     }
-    
+
     @BeforeClass
     public static synchronized void doSetup() throws Exception {
         Map<String, String> props = Maps.newConcurrentMap();
@@ -115,7 +122,18 @@ public class SystemCatalogUpgradeIT extends BaseTest {
         // Confirm that another connection does not increase the number of times upgrade was attempted
         DriverManager.getConnection(getUrl());
         assertEquals(wasTimestampChanged ? 1 : 0, countUpgradeAttempts);
+        // Additional test for PHOENIX-6125
+        // Confirm that SYSTEM.TASK has split policy set as
+        // SystemTaskSplitPolicy (which is extending DisabledRegionSplitPolicy
+        // as of this writing)
+        try (Admin admin = services.getAdmin()) {
+            String taskSplitPolicy = admin
+                .getTableDescriptor(TableName.valueOf(
+                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME))
+                .getRegionSplitPolicyClassName();
+            assertEquals(SystemTaskSplitPolicy.class.getName(),
+                taskSplitPolicy);
+        }
     }
 
-
-}
+}
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/InvalidRegionSplitPolicyException.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/InvalidRegionSplitPolicyException.java
new file mode 100644
index 0000000..ddfa3ee
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/InvalidRegionSplitPolicyException.java
@@ -0,0 +1,49 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.exception;
+
+import java.sql.SQLException;
+import java.util.List;
+
+/**
+ * Invalid region split policy for given table
+ */
+public class InvalidRegionSplitPolicyException extends SQLException {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final SQLExceptionCode EXCEPTION_CODE =
+        SQLExceptionCode.INVALID_REGION_SPLIT_POLICY;
+    private static final String ERROR_MSG = "Region split policy for table %s"
+        + " is expected to be among: %s , actual split policy: %s";
+
+    public InvalidRegionSplitPolicyException(final String schemaName,
+          final String tableName, final List<String> expectedSplitPolicies,
+          final String actualSplitPolicy) {
+        super(new SQLExceptionInfo.Builder(EXCEPTION_CODE)
+                .setSchemaName(schemaName)
+                .setTableName(tableName)
+                .setMessage(String.format(ERROR_MSG, tableName,
+                    expectedSplitPolicies, actualSplitPolicy))
+                .build().toString(),
+            EXCEPTION_CODE.getSQLState(), EXCEPTION_CODE.getErrorCode(), null);
+    }
+
+}
\ No newline at end of file
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 1763082..015519f 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
@@ -549,7 +549,11 @@ public enum SQLExceptionCode {
 
     INCORRECT_INDEX_NAME(906, "43M17", "The list contains one or more incorrect index name(s)"),
 
-    NOT_SUPPORTED_CASCADE_FEATURE_LOCAL_INDEX(907, "43M18", "CASCADE INDEX feature is not supported for local index");
+    NOT_SUPPORTED_CASCADE_FEATURE_LOCAL_INDEX(907, "43M18",
+        "CASCADE INDEX feature is not supported for local index"),
+
+    INVALID_REGION_SPLIT_POLICY(908, "43M19",
+        "REGION SPLIT POLICY is incorrect.");
 
     private final int errorCode;
     private final String sqlState;
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 da7f3bc..f9011bd 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
@@ -114,6 +114,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.annotation.concurrent.GuardedBy;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -188,6 +189,7 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRespons
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
+import org.apache.phoenix.exception.InvalidRegionSplitPolicyException;
 import org.apache.phoenix.exception.PhoenixIOException;
 import org.apache.phoenix.exception.RetriableUpgradeException;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -244,6 +246,7 @@ import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.SystemFunctionSplitPolicy;
 import org.apache.phoenix.schema.SystemStatsSplitPolicy;
+import org.apache.phoenix.schema.SystemTaskSplitPolicy;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableProperty;
@@ -1452,6 +1455,50 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return null; // will never make it here
     }
 
+    /**
+     * If given HTableDescriptor belongs to SYSTEM.TASK and if the table
+     * still does not have split policy setup as SystemTaskSplitPolicy, set
+     * it up and return true, else return false. This method is expected
+     * to return true only if it updated split policy (which should happen
+     * once during initial upgrade).
+     *
+     * @param htd table descriptor
+     * @return return true if split policy of SYSTEM.TASK is updated to
+     *     SystemTaskSplitPolicy.
+     * @throws SQLException If SYSTEM.TASK already has custom split policy
+     *     set up other than SystemTaskSplitPolicy
+     */
+    @VisibleForTesting
+    public boolean updateAndConfirmSplitPolicyForTask(
+            final HTableDescriptor htd) throws SQLException {
+        boolean isTaskTable = false;
+        TableName sysTaskTable = SchemaUtil
+            .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_TASK_NAME,
+                props);
+        if (htd.getTableName().equals(sysTaskTable)) {
+            isTaskTable = true;
+        }
+        if (isTaskTable) {
+            final String actualSplitPolicy = htd
+                .getRegionSplitPolicyClassName();
+            final String targetSplitPolicy =
+                SystemTaskSplitPolicy.class.getName();
+            if (!targetSplitPolicy.equals(actualSplitPolicy)) {
+                if (StringUtils.isNotEmpty(actualSplitPolicy)) {
+                    // Rare possibility. pre-4.16 create DDL query
+                    // doesn't have any split policy setup for SYSTEM.TASK
+                    throw new InvalidRegionSplitPolicyException(
+                      QueryConstants.SYSTEM_SCHEMA_NAME, SYSTEM_TASK_TABLE,
+                      ImmutableList.of("null", targetSplitPolicy),
+                      actualSplitPolicy);
+                }
+                htd.setRegionSplitPolicyClassName(targetSplitPolicy);
+                return true;
+            }
+        }
+        return false;
+    }
+
     private static boolean hasTxCoprocessor(HTableDescriptor descriptor) {
         for (TransactionFactory.Provider provider : TransactionFactory.Provider.values()) {
             Class<? extends RegionObserver> coprocessorClass = provider.getTransactionProvider().getCoprocessor();
@@ -3957,7 +4004,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private PhoenixConnection upgradeSystemTask(PhoenixConnection metaConnection)
-    throws SQLException {
+            throws SQLException, IOException {
         try {
             metaConnection.createStatement().executeUpdate(getTaskDDL());
         } catch (NewerTableAlreadyExistsException ignored) {
@@ -3984,6 +4031,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         "ALTER TABLE " + taskTableFullName + " SET " + TTL + "=" + TASK_TABLE_TTL);
                 clearCache();
             }
+            // If SYSTEM.TASK does not have disabled regions split policy,
+            // set it up here while upgrading it
+            try (HBaseAdmin admin = metaConnection.getQueryServices().getAdmin()) {
+                HTableDescriptor td;
+                TableName tableName = SchemaUtil.getPhysicalTableName(
+                    PhoenixDatabaseMetaData.SYSTEM_TASK_NAME, props);
+                td = admin.getTableDescriptor(tableName);
+                if (updateAndConfirmSplitPolicyForTask(td)) {
+                    admin.modifyTable(tableName, td);
+                    pollForUpdatedTableDescriptor(admin,
+                        td, tableName.getName());
+                }
+            } catch (InterruptedException | TimeoutException ite) {
+                throw new SQLException(PhoenixDatabaseMetaData.SYSTEM_TASK_NAME
+                    + " Upgrade is not confirmed");
+            }
         }
         return metaConnection;
     }
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 5ecfa46..4f50337 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
@@ -35,6 +35,7 @@ import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.SystemFunctionSplitPolicy;
 import org.apache.phoenix.schema.SystemStatsSplitPolicy;
+import org.apache.phoenix.schema.SystemTaskSplitPolicy;
 import org.apache.phoenix.schema.TableProperty;
 
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
@@ -492,9 +493,9 @@ public interface QueryConstants {
             HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
             TRANSACTIONAL + "=" + Boolean.FALSE + ",\n" +
             HColumnDescriptor.TTL + "=" + TTL_FOR_MUTEX;
-    
-	String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" +
-            SYSTEM_TASK_TABLE + "\"(\n" +
+
+    String CREATE_TASK_METADATA = "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\""
+        + SYSTEM_TASK_TABLE + "\"(\n" +
             // PK columns
             TASK_TYPE + " UNSIGNED_TINYINT NOT NULL," +
             TASK_TS + " TIMESTAMP NOT NULL," +
@@ -512,6 +513,8 @@ public interface QueryConstants {
             HConstants.VERSIONS + "=%s,\n" +
             HColumnDescriptor.KEEP_DELETED_CELLS + "=%s,\n" +
             HColumnDescriptor.TTL + "=" + TASK_TABLE_TTL + ",\n" +     // 10 days
+            HTableDescriptor.SPLIT_POLICY + "='"
+                + SystemTaskSplitPolicy.class.getName() + "',\n" +
             TRANSACTIONAL + "=" + Boolean.FALSE + ",\n" +
             STORE_NULLS + "=" + Boolean.TRUE;
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SystemTaskSplitPolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SystemTaskSplitPolicy.java
new file mode 100644
index 0000000..c626f44
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SystemTaskSplitPolicy.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.schema;
+
+import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
+
+/**
+ * Split policy for SYSTEM.TASK table
+ */
+public class SystemTaskSplitPolicy extends DisabledRegionSplitPolicy {
+    // empty
+}
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
index 366451a..62299e1 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionQueryServicesImplTest.java
@@ -24,6 +24,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_FOR_MUTEX;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doCallRealMethod;
@@ -33,6 +34,8 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.sql.SQLException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -47,6 +50,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.PhoenixIOException;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.Before;
 import org.junit.Test;
@@ -64,14 +68,33 @@ public class ConnectionQueryServicesImplTest {
 
     @Mock
     private ConnectionQueryServicesImpl mockCqs;
+
     @Mock
     private HBaseAdmin mockAdmin;
 
+    @Mock
+    private ReadOnlyProps readOnlyProps;
+
+    public static final HTableDescriptor SYS_TASK_TDB =
+        new HTableDescriptor(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_TASK_NAME));
+    public static final HTableDescriptor SYS_TASK_TDB_SP =
+        new HTableDescriptor(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_TASK_NAME))
+            .setRegionSplitPolicyClassName("abc");
+
     @Before
-    public void reset() throws IOException {
+    public void setup() throws IOException, NoSuchFieldException,
+            IllegalAccessException, SQLException {
         MockitoAnnotations.initMocks(this);
+        Field props = ConnectionQueryServicesImpl.class
+            .getDeclaredField("props");
+        props.setAccessible(true);
+        props.set(mockCqs, readOnlyProps);
         when(mockCqs.checkIfSysMutexExistsAndModifyTTLIfRequired(mockAdmin))
-                .thenCallRealMethod();
+            .thenCallRealMethod();
+        when(mockCqs.updateAndConfirmSplitPolicyForTask(SYS_TASK_TDB))
+            .thenCallRealMethod();
+        when(mockCqs.updateAndConfirmSplitPolicyForTask(SYS_TASK_TDB_SP))
+            .thenCallRealMethod();
     }
 
     @SuppressWarnings("unchecked")
@@ -146,4 +169,24 @@ public class ConnectionQueryServicesImplTest {
         verify(mockAdmin, Mockito.times(0)).modifyTable(
                 any(TableName.class), any(HTableDescriptor.class));
     }
+
+    @Test
+    public void testSysTaskSplitPolicy() throws Exception {
+        assertTrue(mockCqs.updateAndConfirmSplitPolicyForTask(SYS_TASK_TDB));
+        assertFalse(mockCqs.updateAndConfirmSplitPolicyForTask(SYS_TASK_TDB));
+    }
+
+    @Test
+    public void testSysTaskSplitPolicyWithError() {
+        try {
+            mockCqs.updateAndConfirmSplitPolicyForTask(SYS_TASK_TDB_SP);
+            fail("Split policy for SYSTEM.TASK cannot be updated");
+        } catch (SQLException e) {
+            assertEquals("ERROR 908 (43M19): REGION SPLIT POLICY is incorrect."
+                + " Region split policy for table TASK is expected to be "
+                + "among: [null, org.apache.phoenix.schema.SystemTaskSplitPolicy]"
+                + " , actual split policy: abc tableName=SYSTEM.TASK",
+                e.getMessage());
+        }
+    }
 }