You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ya...@apache.org on 2021/01/21 19:17:48 UTC

[phoenix] branch 4.16 updated (3cb7bb2 -> 8458175)

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

yanxinyi pushed a change to branch 4.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git.


    from 3cb7bb2  PHOENIX-6307 Build and release official binary distributions with each HBase profile (addendum: include hbase-compat in phoenix-server)
     new 5f9aef7  PHOENIX-6327 Consolidate Junit test categories
     new e58e26f  PHOENIX-6327 Consolidate Junit test categories (addendum: restore -Xmx3000 option for ParallelStatsDisabledTest)
     new 8458175  PHOENIX-6330 SystemCatalogRegionObserver isn't added when cluster is initialized with isNamespaceMappingEnabled=true

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../phoenix/end2end/BaseHBaseManagedTimeIT.java    | 69 ---------------------
 .../phoenix/end2end/HBaseManagedTimeTest.java      | 43 -------------
 .../phoenix/end2end/NeedsOwnMiniClusterTest.java   |  8 +--
 .../phoenix/end2end/ParallelStatsDisabledTest.java |  4 +-
 .../phoenix/end2end/ParallelStatsEnabledTest.java  |  4 +-
 .../phoenix/end2end/SplitSystemCatalogIT.java      |  2 +-
 .../phoenix/end2end/SplitSystemCatalogTests.java   | 28 ---------
 .../end2end/SystemCatalogRegionObserverIT.java     | 72 ++++++++++++++++++++++
 .../phoenix/query/ConnectionQueryServicesImpl.java |  3 +-
 .../java/org/apache/phoenix/query/BaseTest.java    | 17 +++--
 .../org/apache/phoenix/pherf/ResultBaseTestIT.java | 26 +++++++-
 .../org/apache/phoenix/pherf/SchemaReaderIT.java   | 32 +++++++++-
 pom.xml                                            | 40 +-----------
 13 files changed, 147 insertions(+), 201 deletions(-)
 delete mode 100644 phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
 delete mode 100644 phoenix-core/src/it/java/org/apache/phoenix/end2end/HBaseManagedTimeTest.java
 delete mode 100644 phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
 create mode 100644 phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogRegionObserverIT.java


[phoenix] 03/03: PHOENIX-6330 SystemCatalogRegionObserver isn't added when cluster is initialized with isNamespaceMappingEnabled=true

Posted by ya...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yanxinyi pushed a commit to branch 4.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 8458175ee7e928009cd2751537a53356b7159b22
Author: Istvan Toth <st...@apache.org>
AuthorDate: Wed Jan 20 18:02:55 2021 +0100

    PHOENIX-6330 SystemCatalogRegionObserver isn't added when cluster is initialized with isNamespaceMappingEnabled=true
---
 .../end2end/SystemCatalogRegionObserverIT.java     | 72 ++++++++++++++++++++++
 .../phoenix/query/ConnectionQueryServicesImpl.java |  3 +-
 2 files changed, 73 insertions(+), 2 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogRegionObserverIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogRegionObserverIT.java
new file mode 100644
index 0000000..a7a549b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogRegionObserverIT.java
@@ -0,0 +1,72 @@
+/*
+ * 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.end2end;
+
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.phoenix.coprocessor.SystemCatalogRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.thirdparty.com.google.common.collect.Maps;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class SystemCatalogRegionObserverIT extends BaseUniqueNamesOwnClusterIT {
+
+    public SystemCatalogRegionObserverIT() {
+    }
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+        serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+
+    protected Connection getConnection() throws SQLException{
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
+        return DriverManager.getConnection(getUrl(),props);
+    }
+
+    @Test
+    public void testSystemCatalogRegionObserverWasAdded() throws Exception {
+        try (Connection conn = getConnection()){
+            PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
+            Table syscatTable = phoenixConn.getQueryServices().getTable(
+                SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, true).getName());
+            assertTrue("SystemCatalogRegionObserver was not added to SYSTEM.CATALOG",
+                syscatTable.getTableDescriptor().getCoprocessors().contains(
+                SystemCatalogRegionObserver.class.getName()));
+        }
+    }
+}
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 9403cbe..7fdffdd 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
@@ -1152,8 +1152,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             PhoenixTTLRegionObserver.class.getName(), null, priority-2, null);
                 }
             }
-
-            if (Arrays.equals(tableName, SYSTEM_CATALOG_NAME_BYTES)) {
+            if (Arrays.equals(tableName, SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, props).getName())) {
                 if (!descriptor.hasCoprocessor(SystemCatalogRegionObserver.class.getName())) {
                     descriptor.addCoprocessor(
                             SystemCatalogRegionObserver.class.getName(), null, priority, null);


[phoenix] 01/03: PHOENIX-6327 Consolidate Junit test categories

Posted by ya...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yanxinyi pushed a commit to branch 4.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 5f9aef74015c51c6ca6e3d2413551a1a868d3174
Author: Istvan Toth <st...@apache.org>
AuthorDate: Tue Jan 19 08:28:47 2021 +0100

    PHOENIX-6327 Consolidate Junit test categories
---
 .../phoenix/end2end/BaseHBaseManagedTimeIT.java    | 69 ----------------------
 .../phoenix/end2end/HBaseManagedTimeTest.java      | 43 --------------
 .../phoenix/end2end/NeedsOwnMiniClusterTest.java   |  8 +--
 .../phoenix/end2end/ParallelStatsDisabledTest.java |  4 +-
 .../phoenix/end2end/ParallelStatsEnabledTest.java  |  4 +-
 .../phoenix/end2end/SplitSystemCatalogIT.java      |  2 +-
 .../phoenix/end2end/SplitSystemCatalogTests.java   | 28 ---------
 .../java/org/apache/phoenix/query/BaseTest.java    | 17 ++++--
 .../org/apache/phoenix/pherf/ResultBaseTestIT.java | 26 +++++++-
 .../org/apache/phoenix/pherf/SchemaReaderIT.java   | 32 +++++++++-
 pom.xml                                            | 40 +------------
 11 files changed, 74 insertions(+), 199 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
deleted file mode 100644
index 0f455fd..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.end2end;
-
-import javax.annotation.concurrent.NotThreadSafe;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.phoenix.query.BaseTest;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.experimental.categories.Category;
-
-/**
- * Base class for tests that let HBase set timestamps.
- * We need to separate these from tests that rely on clients 
- * to set timestamps, because we create/destroy the Phoenix tables
- * between tests and only allow a table time stamp to increase.
- * Without this separation table deletion/creation would fail.
- *
- * Try and use (@link BaseHBaseManagedTimeTableReuseIT) and only
- * extend this class if really necessary.  It is very slow as
- * we have to delete tables which takes up to 10 seconds per test case.
- * All tests extending this class use the mini cluster that is
- * different from the mini cluster used by test classes extending 
- * {@link BaseClientManagedTimeIT}.
- * 
- * @since 0.1
- */
-@NotThreadSafe
-@Category(HBaseManagedTimeTest.class)
-public abstract class BaseHBaseManagedTimeIT extends BaseTest {
-    protected static Configuration getTestClusterConfig() {
-        // don't want callers to modify config.
-        return new Configuration(config);
-    }
-    
-    @BeforeClass
-    public static synchronized void doSetup() throws Exception {
-        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS);
-    }
-    
-    @AfterClass
-    public static synchronized void doTeardown() throws Exception {
-        dropNonSystemTables();
-    }
-    
-    @After
-    public void cleanUpAfterTest() throws Exception {
-        deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getUrl());
-    }
-}
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HBaseManagedTimeTest.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HBaseManagedTimeTest.java
deleted file mode 100644
index c9eee86..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HBaseManagedTimeTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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 maynot 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 applicablelaw 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.end2end;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Inherited;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * 
- * Annotation to denote that the test lets HBase manage timestamp 
- * itself.
- * 
- * Tests using a mini cluster need to be classified either 
- * as {@link HBaseManagedTimeTest} or {@link ClientManagedTimeTest} 
- * or {@link NeedsOwnMiniClusterTest} or {@link ParallelStatsDisabledTest}
- * otherwise they won't be run when one runs mvn verify or mvn install.
- * 
- * @since 4.1
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-@Inherited
-public @interface HBaseManagedTimeTest {
-
-}
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NeedsOwnMiniClusterTest.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NeedsOwnMiniClusterTest.java
index 6ea4c4e..aa37c88 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NeedsOwnMiniClusterTest.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NeedsOwnMiniClusterTest.java
@@ -26,12 +26,12 @@ import java.lang.annotation.Target;
  * 
  * Annotation to denote that the test needs to run its own 
  * mini cluster that is separate from the clusters used by 
- * tests annotated as {@link HBaseManagedTimeTest} or 
- * {@link ClientManagedTimeTest}.
+ * tests annotated as {@link ParallelStatsDisabledTest} or 
+ * {@link ParallelStatsEnabledTest}.
  * 
  * As much as possible, tests should be able to run in one of the
- * mini clusters used by {@link HBaseManagedTimeTest} or 
- * {@link ClientManagedTimeTest}. In the *rare* case it can't
+ * mini clusters used by {@link ParallelStatsDisabledTest} or 
+ * {@link ParallelStatsEnabledTest}. In the *rare* case it can't
  * you would need to annotate the test as {@link NeedsOwnMiniClusterTest}
  * otherwise the test won't be executed when you run mvn verify or mvn install.
  * 
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledTest.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledTest.java
index bb8c19f..d807b8d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledTest.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledTest.java
@@ -32,8 +32,8 @@ import org.apache.phoenix.query.BaseTest;
  * these tests use random table names generated by {@link BaseTest#generateRandomString}.
  * 
  * Tests using a mini cluster need to be classified either 
- * as {@link ParallelStatsDisabledTest} or {@link ClientManagedTimeTest}
- * or {@link NeedsOwnMiniClusterTest} or {@link HBaseManagedTimeTest} otherwise they won't be run
+ * as {@link ParallelStatsDisabledTest} or {@link ParallelStatsEnabledTest}
+ * or {@link NeedsOwnMiniClusterTest} otherwise they won't be run
  * when one runs mvn verify or mvn install.
  * 
  * @since 4.1
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledTest.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledTest.java
index 6607b05..9fcffbc 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledTest.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledTest.java
@@ -32,8 +32,8 @@ import org.apache.phoenix.query.BaseTest;
  * these tests use random table names generated by {@link BaseTest#generateRandomString}.
  * 
  * Tests using a mini cluster need to be classified either 
- * as {@link ParallelStatsEnabledTest} or {@link ClientManagedTimeTest}
- * or {@link NeedsOwnMiniClusterTest} or {@link HBaseManagedTimeTest} otherwise they won't be run
+ * as {@link ParallelStatsDisabledTest} or {@link ParallelStatsEnabledTest}
+ * or {@link NeedsOwnMiniClusterTest} otherwise they won't be run
  * when one runs mvn verify or mvn install.
  * 
  * @since 4.1
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java
index b2075a7..ccf2eb2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java
@@ -37,7 +37,7 @@ import com.google.common.collect.Maps;
  * Base class for tests that run with split SYSTEM.CATALOG.
  * 
  */
-@Category(SplitSystemCatalogTests.class)
+@Category(NeedsOwnMiniClusterTest.class)
 public class SplitSystemCatalogIT extends BaseTest {
 
     protected static String SCHEMA1 = "SCHEMA1";
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
deleted file mode 100644
index 67d3fd3..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogTests.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.end2end;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface SplitSystemCatalogTests {
-}
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 f326f9b..1537ce6 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
@@ -134,7 +134,6 @@ import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.phoenix.SystemExitRule;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -170,13 +169,19 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * Base class that contains all the methods needed by
  * client-time and hbase-time managed tests.
  * 
+ *  * Tests using a mini cluster need to be classified either 
+ * as {@link ParallelStatsDisabledTest} or {@link ParallelStatsEnabledTest}
+ * or {@link NeedsOwnMiniClusterTest} otherwise they won't be run
+ * when one runs mvn verify or mvn install.
+ * 
  * For tests needing connectivity to a cluster, please use
- * {@link BaseHBaseManagedTimeIT}.
+ * {@link ParallelStatsDisabledIt} or {@link ParallelStatsEnabledIt}.
  * 
- * In the rare case when a test can't share the same mini cluster as the 
- * ones used by {@link BaseHBaseManagedTimeIT},
- * one could extend this class and spin up your own mini cluster. Please 
- * make sure to shutdown the mini cluster in a method annotated by @AfterClass.  
+ * In the case when a test can't share the same mini cluster as the 
+ * ones used by {@link ParallelStatsDisabledIt} or {@link ParallelStatsEnabledIt},
+ * one could extend this class and spin up your own mini cluster. Please
+ * make sure to annotate such clesses with {@link NeedsOwnMiniClusterTest} and
+ * shutdown the mini cluster in a method annotated by @AfterClass.
  *
  */
 
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
index a1ca6fa..fe1f2ea 100644
--- a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
@@ -22,15 +22,22 @@ import java.nio.file.Path;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.pherf.configuration.XMLConfigParser;
 import org.apache.phoenix.pherf.result.ResultUtil;
 import org.apache.phoenix.pherf.schema.SchemaReader;
 import org.apache.phoenix.pherf.util.PhoenixUtil;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
 
-public class ResultBaseTestIT extends BaseHBaseManagedTimeIT {
+@Category(NeedsOwnMiniClusterTest.class)
+public class ResultBaseTestIT extends BaseTest {
     protected static final String matcherScenario = ".*scenario/.*test.*xml";
     protected static final String matcherSchema = ".*datamodel/.*test.*sql";
 
@@ -41,6 +48,11 @@ public class ResultBaseTestIT extends BaseHBaseManagedTimeIT {
     protected static List<Path> resources;
     protected static ResultUtil resultUtil = new ResultUtil();
 
+    protected static Configuration getTestClusterConfig() {
+        // don't want callers to modify config.
+        return new Configuration(config);
+    }
+
     @BeforeClass public static synchronized void setUp() throws Exception {
 
         PherfConstants constants = PherfConstants.create();
@@ -51,9 +63,17 @@ public class ResultBaseTestIT extends BaseHBaseManagedTimeIT {
         PhoenixUtil.setZookeeper("localhost");
         reader = new SchemaReader(util, matcherSchema);
         parser = new XMLConfigParser(matcherScenario);
+
+        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS);
     }
 
     @AfterClass public static synchronized void tearDown() throws Exception {
-    	resultUtil.deleteDir(properties.getProperty("pherf.default.results.dir"));
+        dropNonSystemTables();
+        resultUtil.deleteDir(properties.getProperty("pherf.default.results.dir"));
+    }
+
+    @After
+    public void cleanUpAfterTest() throws Exception {
+        deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getUrl());
     }
 }
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java
index a897a46..901c92f 100644
--- a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java
@@ -30,19 +30,45 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataModel;
 import org.apache.phoenix.pherf.configuration.Scenario;
 import org.apache.phoenix.pherf.configuration.XMLConfigParser;
 import org.apache.phoenix.pherf.schema.SchemaReader;
 import org.apache.phoenix.pherf.util.PhoenixUtil;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-
-public class SchemaReaderIT extends BaseHBaseManagedTimeIT {
+@Category(NeedsOwnMiniClusterTest.class)
+public class SchemaReaderIT extends BaseTest {
     protected static PhoenixUtil util = PhoenixUtil.create(true);
 
+    protected static Configuration getTestClusterConfig() {
+        // don't want callers to modify config.
+        return new Configuration(config);
+    }
+
+    @BeforeClass public static synchronized void setUp() throws Exception {
+        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS);
+    }
+
+    @AfterClass public static synchronized void tearDown() throws Exception {
+        dropNonSystemTables();
+    }
+
+    @After
+    public void cleanUpAfterTest() throws Exception {
+        deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getUrl());
+    }
+
     @Test 
     public void testSchemaReader() {
         // Test for the unit test version of the schema files.
diff --git a/pom.xml b/pom.xml
index 5417668..766aec6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -315,9 +315,7 @@
               <id>ParallelStatsEnabledTest</id>
               <configuration>
                 <reuseForks>true</reuseForks>
-                <!--parallel>methods</parallel>
-                <threadCount>20</threadCount-->
-                <argLine>@{jacocoArgLine} -Xmx2000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}" -XX:NewRatio=4 -XX:SurvivorRatio=8 -XX:+UseCompressedOops -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+DisableExplicitGC -XX:+UseCMSInitiatingOccupancyOnly -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:CMSInitiatingOccupancyFraction=68 -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDum [...]
+                <argLine>@{jacocoArgLine} -Xmx2000m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}" -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
                 <groups>org.apache.phoenix.end2end.ParallelStatsEnabledTest</groups>
               </configuration>
               <goals>
@@ -329,17 +327,7 @@
               <id>ParallelStatsDisabledTest</id>
               <configuration>
                 <reuseForks>true</reuseForks>
-                <!--parallel>methods</parallel>
-                <threadCount>20</threadCount-->
-                <!-- We're intermittantly hitting this assertion when running in parallel:
-                     Caused by: java.lang.AssertionError: we should never remove a different context
-	                 at org.apache.hadoop.hbase.regionserver.HRegion$RowLockContext.cleanUp(HRegion.java:5206)
-	                 at org.apache.hadoop.hbase.regionserver.HRegion$RowLockImpl.release(HRegion.java:5246)
-	                 at org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2898)
-	                 at org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2835)
-	                 at org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:490) -->
-		<!--enableAssertions>false</enableAssertions-->
-                <argLine>@{jacocoArgLine} -Xmx3000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}" -XX:NewRatio=4 -XX:SurvivorRatio=8 -XX:+UseCompressedOops -XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+DisableExplicitGC -XX:+UseCMSInitiatingOccupancyOnly -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:CMSInitiatingOccupancyFraction=68 -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDum [...]
+                <argLine>@{jacocoArgLine} -Xmx2000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"  -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
                 <groups>org.apache.phoenix.end2end.ParallelStatsDisabledTest</groups>
               </configuration>
               <goals>
@@ -348,18 +336,6 @@
               </goals>
             </execution>
             <execution>
-              <id>HBaseManagedTimeTests</id>
-              <configuration>
-                <reuseForks>true</reuseForks>
-                <argLine>@{jacocoArgLine} -enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}" -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
-                <groups>org.apache.phoenix.end2end.HBaseManagedTimeTest</groups>
-              </configuration>
-              <goals>
-                <goal>integration-test</goal>
-                <goal>verify</goal>
-              </goals>
-            </execution>
-	    <execution>
               <id>NeedTheirOwnClusterTests</id>
               <configuration>
                  <reuseForks>false</reuseForks>
@@ -371,18 +347,6 @@
                  <goal>verify</goal>
               </goals>
             </execution>
-<execution>
-              <id>SplitSystemCatalogTests</id>
-              <configuration>
-                 <reuseForks>false</reuseForks>
-                 <argLine>@{jacocoArgLine} -enableassertions -Xmx2000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}" -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
-                 <groups>org.apache.phoenix.end2end.SplitSystemCatalogTests</groups>
-              </configuration>
-              <goals>
-                 <goal>integration-test</goal>
-                 <goal>verify</goal>
-              </goals>
-            </execution>
           </executions>
         </plugin>
         <plugin>


[phoenix] 02/03: PHOENIX-6327 Consolidate Junit test categories (addendum: restore -Xmx3000 option for ParallelStatsDisabledTest)

Posted by ya...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yanxinyi pushed a commit to branch 4.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit e58e26f2d6ff398b5d006c014a1f583af22e3083
Author: Istvan Toth <st...@apache.org>
AuthorDate: Wed Jan 20 07:08:40 2021 +0100

    PHOENIX-6327 Consolidate Junit test categories (addendum: restore -Xmx3000 option for ParallelStatsDisabledTest)
---
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pom.xml b/pom.xml
index 766aec6..9b70ea0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -327,7 +327,7 @@
               <id>ParallelStatsDisabledTest</id>
               <configuration>
                 <reuseForks>true</reuseForks>
-                <argLine>@{jacocoArgLine} -Xmx2000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"  -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
+                <argLine>@{jacocoArgLine} -Xmx3000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"  -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
                 <groups>org.apache.phoenix.end2end.ParallelStatsDisabledTest</groups>
               </configuration>
               <goals>