You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2020/05/29 11:22:30 UTC

[hive] branch master updated: HIVE-22942: Replace PTest with an alternative

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

kgyrtkirk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 8443e50  HIVE-22942: Replace PTest with an alternative
8443e50 is described below

commit 8443e50fdfa284531300f3ab283a7e4959dba623
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Fri May 29 10:21:41 2020 +0000

    HIVE-22942: Replace PTest with an alternative
    
    Closes apache/hive#948
---
 Jenkinsfile                                        | 193 +++++++++++++++++++++
 .../listener/TestDbNotificationListener.java       |   2 +
 .../parse/TestScheduledReplicationScenarios.java   |   2 +
 .../apache/hive/beeline/TestBeeLineWithArgs.java   |  10 +-
 .../org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java |   4 +-
 .../org/apache/hive/jdbc/TestActivePassiveHA.java  |   4 +-
 .../hive/jdbc/TestJdbcGenericUDTFGetSplits.java    |   2 +
 .../hive/jdbc/TestJdbcGenericUDTFGetSplits2.java   |   2 +
 .../apache/hive/jdbc/TestJdbcWithMiniLlapRow.java  |   1 +
 .../hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java |   1 +
 .../hive/jdbc/TestJdbcWithServiceDiscovery.java    |   2 +
 .../apache/hive/jdbc/TestNewGetSplitsFormat.java   |   1 +
 .../jdbc/TestNewGetSplitsFormatReturnPath.java     |   8 +
 .../jdbc/TestTriggersTezSessionPoolManager.java    |   2 +
 itests/qtest/pom.xml                               |   2 -
 .../hive/kafka/TransactionalKafkaWriterTest.java   |   2 +
 .../hive/llap/registry/impl/TestSlotZnode.java     |   4 +-
 .../llap/daemon/impl/TestTaskExecutorService.java  |  16 +-
 .../authorization_disallow_transform.q             |   1 +
 ql/src/test/queries/clientnegative/masking_mv.q    |   2 +-
 .../test/queries/clientnegative/strict_pruning.q   |   1 +
 .../test/queries/clientnegative/strict_pruning_2.q |   1 +
 .../clientpositive/authorization_show_grant.q      |   1 +
 .../druid_materialized_view_rewrite_ssb.q          |   1 +
 .../clientpositive/druidkafkamini_delimited.q      |   1 +
 .../clientpositive/merge_test_dummy_operator.q     |   1 +
 .../clientpositive/results_cache_invalidation2.q   |   2 +-
 ...schema_evol_par_vec_table_dictionary_encoding.q |   2 +
 ...ma_evol_par_vec_table_non_dictionary_encoding.q |   1 +
 .../special_character_in_tabnames_1.q              |   1 +
 .../queries/clientpositive/stats_list_bucket.q     |   1 +
 .../temp_table_multi_insert_partitioned.q          |   1 +
 .../llap/results_cache_invalidation2.q.out         |   4 +-
 .../metastore/txn/TestAcidTxnCleanerService.java   |   2 +
 34 files changed, 259 insertions(+), 22 deletions(-)

diff --git a/Jenkinsfile b/Jenkinsfile
new file mode 100644
index 0000000..65cc65e
--- /dev/null
+++ b/Jenkinsfile
@@ -0,0 +1,193 @@
+/*
+ * 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.
+ */
+
+properties([
+    // max 5 build/branch/day
+    rateLimitBuilds(throttle: [count: 5, durationName: 'day', userBoost: true]),
+    // do not run multiple testruns on the same branch
+    disableConcurrentBuilds(),
+    parameters([
+        string(name: 'SPLIT', defaultValue: '20', description: 'Number of buckets to split tests into.'),
+        string(name: 'OPTS', defaultValue: '', description: 'additional maven opts'),
+    ])
+])
+
+def setPrLabel(String prLabel) {
+  if (env.CHANGE_ID) {
+   def mapping=[
+    "SUCCESS":"tests passed",
+    "UNSTABLE":"tests unstable",
+    "FAILURE":"tests failed",
+    "PENDING":"tests pending",
+   ]
+   def newLabels = []
+   for( String l : pullRequest.labels )
+     newLabels.add(l)
+   for( String l : mapping.keySet() )
+     newLabels.remove(mapping[l])
+   newLabels.add(mapping[prLabel])
+   echo ('' +newLabels)
+   pullRequest.labels=newLabels
+  }
+}
+
+setPrLabel("PENDING");
+
+def executorNode(run) {
+  hdbPodTemplate {
+      node(POD_LABEL) {
+        container('hdb') {
+          run()
+        }
+    }
+  }
+}
+
+def buildHive(args) {
+  configFileProvider([configFile(fileId: 'artifactory', variable: 'SETTINGS')]) {
+    withEnv(["MULTIPLIER=$params.MULTIPLIER","M_OPTS=$params.OPTS"]) {
+      sh '''#!/bin/bash -e
+ls -l
+set -x
+. /etc/profile.d/confs.sh
+export USER="`whoami`"
+export MAVEN_OPTS="-Xmx2g"
+export -n HIVE_CONF_DIR
+OPTS=" -s $SETTINGS -B -Dmaven.test.failure.ignore -Dtest.groups= "
+OPTS+=" -Pitests,qsplits"
+OPTS+=" -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugin.surefire.SurefirePlugin=INFO"
+OPTS+=" -Dmaven.repo.local=$PWD/.m2"
+OPTS+=" $M_OPTS "
+if [ -s inclusions.txt ]; then OPTS+=" -Dsurefire.includesFile=$PWD/inclusions.txt";fi
+if [ -s exclusions.txt ]; then OPTS+=" -Dsurefire.excludesFile=$PWD/exclusions.txt";fi
+mvn $OPTS '''+args+'''
+du -h --max-depth=1
+'''
+    }
+  }
+}
+
+def hdbPodTemplate(closure) {
+  podTemplate(
+  containers: [
+    containerTemplate(name: 'hdb', image: 'kgyrtkirk/hive-dev-box:executor', ttyEnabled: true, command: 'cat',
+        alwaysPullImage: true,
+        resourceRequestCpu: '1800m',
+        resourceLimitCpu: '3000m',
+        resourceRequestMemory: '6400Mi',
+        resourceLimitMemory: '12000Mi'
+    ),
+  ], yaml:'''
+spec:
+  securityContext:
+    fsGroup: 1000
+  tolerations:
+    - key: "type"
+      operator: "Equal"
+      value: "slave"
+      effect: "PreferNoSchedule"
+    - key: "type"
+      operator: "Equal"
+      value: "slave"
+      effect: "NoSchedule"
+  nodeSelector:
+    type: slave
+''') {
+    closure();
+  }
+}
+
+def jobWrappers(closure) {
+  try {
+    // allocate 1 precommit token for the execution
+    lock(label:'hive-precommit', quantity:1, variable: 'LOCKED_RESOURCE')  {
+      timestamps {
+        echo env.LOCKED_RESOURCE
+        closure()
+      }
+    }
+  } finally {
+    setPrLabel(currentBuild.currentResult)
+  }
+}
+
+def saveWS() {
+  sh '''#!/bin/bash -e
+    tar --exclude=archive.tar -cf archive.tar .
+    ls -l archive.tar
+    rsync -rltDq --stats archive.tar rsync://rsync/data/$LOCKED_RESOURCE'''
+}
+
+def loadWS() {
+  sh '''#!/bin/bash -e
+    rsync -rltDq --stats rsync://rsync/data/$LOCKED_RESOURCE archive.tar
+    tar -xf archive.tar'''
+}
+
+jobWrappers {
+
+  def splits
+  executorNode {
+    container('hdb') {
+      stage('Checkout') {
+        checkout scm
+      }
+      stage('Compile') {
+        buildHive("install -Dtest=noMatches")
+        sh '''#!/bin/bash -e
+            # make parallel-test-execution plugins source scanner happy ~ better results for 1st run
+            find . -name '*.java'|grep /Test|grep -v src/test/java|grep org/apache|while read f;do t="`echo $f|sed 's|.*org/apache|happy/src/test/java/org/apache|'`";mkdir -p  "${t%/*}";touch "$t";done
+        '''
+      }
+      stage('Upload') {
+        saveWS()
+        splits = splitTests parallelism: count(Integer.parseInt(params.SPLIT)), generateInclusions: true, estimateTestsFromFiles: true
+      }
+    }
+  }
+
+  stage('Testing') {
+
+    def branches = [:]
+    for (int i = 0; i < splits.size(); i++) {
+      def num = i
+      def split = splits[num]
+      def splitName=String.format("split-%02d",num+1)
+      branches[splitName] = {
+        executorNode {
+          stage('Prepare') {
+              loadWS();
+              writeFile file: (split.includes ? "inclusions.txt" : "exclusions.txt"), text: split.list.join("\n")
+              writeFile file: (split.includes ? "exclusions.txt" : "inclusions.txt"), text: ''
+              sh '''echo "@INC";cat inclusions.txt;echo "@EXC";cat exclusions.txt;echo "@END"'''
+          }
+          try {
+            stage('Test') {
+              buildHive("install -q")
+            }
+          } finally {
+            stage('Archive') {
+              junit '**/TEST-*.xml'
+            }
+          }
+        }
+      }
+    }
+    parallel branches
+  }
+}
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index 7d61173..b948727 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -114,6 +114,7 @@ import org.junit.Test;
 import org.junit.rules.TestRule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.junit.Ignore;
 
 /**
  * Tests DbNotificationListener when used as a transactional event listener
@@ -1410,6 +1411,7 @@ public class TestDbNotificationListener {
   }
 
   @Test
+  @Ignore("HIVE-23401")
   public void sqlInsertTable() throws Exception {
     String defaultDbName = "default";
     String tblName = "sqlins";
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestScheduledReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestScheduledReplicationScenarios.java
index d99bf54..a13d842 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestScheduledReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestScheduledReplicationScenarios.java
@@ -33,6 +33,7 @@ import org.junit.Before;
 import org.junit.After;
 import org.junit.Test;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 
 import java.io.File;
 import java.io.IOException;
@@ -102,6 +103,7 @@ public class TestScheduledReplicationScenarios extends BaseReplicationScenariosA
   }
 
   @Test
+  @Ignore("HIVE-23395")
   public void testAcidTablesReplLoadBootstrapIncr() throws Throwable {
     // Bootstrap
     primary.run("use " + primaryDbName)
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
index 73e6262..2a38c7e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
@@ -176,9 +176,9 @@ import org.junit.Test;
     }
     String[] args = argList.toArray(new String[argList.size()]);
     beeLine.begin(args, inputStream);
-    String output = os.toString("UTF8");
-
     beeLine.close();
+    beelineOutputStream.close();
+    String output = os.toString("UTF8");
     return output;
   }
 
@@ -1156,14 +1156,16 @@ import org.junit.Test;
   }
 
   @Test
+  @Ignore("HIVE-23398")
   public void testRowsAffected() throws Throwable {
     final String SCRIPT_TEXT = "drop table if exists new_table;\n create table new_table(foo int);\n "
       + "insert into new_table values (1);\n";
     final String EXPECTED_PATTERN = "1 row affected";
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
-    testScriptFile(SCRIPT_TEXT, argList, OutStream.ERR, EXPECTED_PATTERN, true);
+    testScriptFile(SCRIPT_TEXT, argList, OutStream.ERR,
+        Collections.singletonList(new Tuple<>(EXPECTED_PATTERN, true)),
+        Arrays.asList(Modes.SCRIPT));
   }
-
   /**
    * Test 'describe extended' on tables that have special white space characters in the row format.
    */
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java
index 45b22f9..20682ff 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/BaseJdbcWithMiniLlap.java
@@ -203,7 +203,7 @@ public abstract class BaseJdbcWithMiniLlap {
     }
   }
 
-  @Test(timeout = 60000)
+  @Test(timeout = 120000)
   public void testLlapInputFormatEndToEnd() throws Exception {
     createTestTable("testtab1");
 
@@ -616,7 +616,7 @@ public abstract class BaseJdbcWithMiniLlap {
   }
 
 
-  @Test(timeout = 60000)
+  @Test(timeout = 120000)
   public void testComplexQuery() throws Exception {
     createTestTable("testtab1");
 
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestActivePassiveHA.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestActivePassiveHA.java
index bf24ebf..a965be6 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestActivePassiveHA.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestActivePassiveHA.java
@@ -72,7 +72,9 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.Ignore;
 
+@Ignore("HIVE-23548")
 public class TestActivePassiveHA {
   private MiniHS2 miniHS2_1 = null;
   private MiniHS2 miniHS2_2 = null;
@@ -726,4 +728,4 @@ public class TestActivePassiveHA {
     }
     assertEquals(false, caughtException);
   }
-}
\ No newline at end of file
+}
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits.java
index fccf5ed..6ca5276 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hive.llap.LlapBaseInputFormat;
 import org.apache.hadoop.hive.llap.LlapInputSplit;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.mapred.JobConf;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.sql.ResultSet;
@@ -37,6 +38,7 @@ import static org.junit.Assert.assertTrue;
 public class TestJdbcGenericUDTFGetSplits extends AbstractTestJdbcGenericUDTFGetSplits {
 
   @Test(timeout = 200000)
+  @Ignore("HIVE-23394")
   public void testGenericUDTFOrderBySplitCount1() throws Exception {
     super.testGenericUDTFOrderBySplitCount1("get_splits", new int[]{10, 1, 0, 2, 2, 2, 1, 10});
   }
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits2.java
index d296d56..14d5e62 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcGenericUDTFGetSplits2.java
@@ -16,6 +16,7 @@
 
 package org.apache.hive.jdbc;
 
+import org.junit.Ignore;
 import org.junit.Test;
 
 /**
@@ -24,6 +25,7 @@ import org.junit.Test;
 public class TestJdbcGenericUDTFGetSplits2 extends AbstractTestJdbcGenericUDTFGetSplits {
 
   @Test(timeout = 200000)
+  @Ignore("HIVE-23394")
   public void testGenericUDTFOrderBySplitCount1() throws Exception {
     super.testGenericUDTFOrderBySplitCount1("get_llap_splits", new int[]{12, 3, 1, 4, 4, 4, 3, 12});
   }
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapRow.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapRow.java
index 7fd1992..2c2836e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapRow.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapRow.java
@@ -32,6 +32,7 @@ import org.junit.Ignore;
 /**
  * TestJdbcWithMiniLlap for llap Row format.
  */
+@Ignore("HIVE-23549")
 public class TestJdbcWithMiniLlapRow extends BaseJdbcWithMiniLlap {
 
   @BeforeClass
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java
index 683ba48..6b7bb24 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniLlapVectorArrow.java
@@ -46,6 +46,7 @@ import org.junit.Test;
 /**
  * TestJdbcWithMiniLlap for Arrow format with vectorized output sink
  */
+@Ignore("unstable HIVE-23549")
 public class TestJdbcWithMiniLlapVectorArrow extends BaseJdbcWithMiniLlap {
 
 
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithServiceDiscovery.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithServiceDiscovery.java
index 6ff4dae..9f2fb323 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithServiceDiscovery.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithServiceDiscovery.java
@@ -27,6 +27,7 @@ import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.Ignore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,6 +50,7 @@ import static org.junit.Assert.assertTrue;
 /**
  * Test JDBC driver when two HS2 instance is running with service discovery enabled.
  */
+@Ignore("unstable HIVE-23528")
 public class TestJdbcWithServiceDiscovery {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestJdbcWithServiceDiscovery.class);
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormat.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormat.java
index 5aac2a5..b275982 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormat.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormat.java
@@ -38,6 +38,7 @@ import java.util.UUID;
 /**
  * TestNewGetSplitsFormat.
  */
+@Ignore("test unstable HIVE-23524")
 public class TestNewGetSplitsFormat extends BaseJdbcWithMiniLlap {
 
   @BeforeClass public static void beforeTest() throws Exception {
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormatReturnPath.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormatReturnPath.java
index 3edfabf..83abffb 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormatReturnPath.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestNewGetSplitsFormatReturnPath.java
@@ -21,6 +21,7 @@ package org.apache.hive.jdbc;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
+import org.junit.Test;
 
 /**
  * TestNewGetSplitsFormatReturnPath.
@@ -37,7 +38,14 @@ public class TestNewGetSplitsFormatReturnPath extends TestNewGetSplitsFormat {
 
   @Override
   @Ignore
+  @Test
   public void testMultipleBatchesOfComplexTypes() {
     // ToDo: FixMe
   }
+
+  @Override
+  @Ignore("HIVE-23524 flaky")
+  @Test
+  public void testLlapInputFormatEndToEndWithMultipleBatches() {
+  }
 }
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersTezSessionPoolManager.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersTezSessionPoolManager.java
index 4372967..2e84c4c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersTezSessionPoolManager.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersTezSessionPoolManager.java
@@ -31,10 +31,12 @@ import org.apache.hadoop.hive.ql.wm.Trigger;
 
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.Ignore;
 import org.junit.rules.TestName;
 
 import com.google.common.collect.Lists;
 
+@Ignore("test unstable HIVE-23523")
 public class TestTriggersTezSessionPoolManager extends AbstractJdbcTriggersTest {
   @Rule
   public TestName testName = new TestName();
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index e969f1f..404fafb 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -30,8 +30,6 @@
 
   <properties>
     <hive.path.to.root>../..</hive.path.to.root>
-    <!-- Determines the log level of the console logger, hive.log is independent of this-->
-    <test.console.log.level>OFF</test.console.log.level>
 <!--    <initScript></initScript>-->
     <qfile></qfile>
     <qfile_regex></qfile_regex>
diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/TransactionalKafkaWriterTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/TransactionalKafkaWriterTest.java
index 7c9ca37..07a3b5a 100644
--- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/TransactionalKafkaWriterTest.java
+++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/TransactionalKafkaWriterTest.java
@@ -37,6 +37,7 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.Ignore;
 import org.junit.rules.TemporaryFolder;
 import org.mockito.Mockito;
 
@@ -202,6 +203,7 @@ public class TransactionalKafkaWriterTest {
     checkData();
   }
 
+  @Ignore("HIVE-23400 flaky")
   @Test(expected = IOException.class) public void writerFencedOut() throws IOException {
     TransactionalKafkaWriter
         writer =
diff --git a/llap-client/src/test/org/apache/hadoop/hive/llap/registry/impl/TestSlotZnode.java b/llap-client/src/test/org/apache/hadoop/hive/llap/registry/impl/TestSlotZnode.java
index 0569505..22dd583 100644
--- a/llap-client/src/test/org/apache/hadoop/hive/llap/registry/impl/TestSlotZnode.java
+++ b/llap-client/src/test/org/apache/hadoop/hive/llap/registry/impl/TestSlotZnode.java
@@ -33,6 +33,7 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,7 +43,6 @@ import java.net.BindException;
 import java.util.Collection;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.FutureTask;
@@ -158,11 +158,13 @@ public class TestSlotZnode {
     }
   }
 
+  @Ignore("HIVE-23564")
   @Test
   public void testConcurrencyAndFallback() throws Exception {
     concurrencyTest(100, true);
   }
 
+  @Ignore("HIVE-23564")
   @Test
   public void testConcurrencyNoFallback() throws Exception {
     concurrencyTest(100, false);
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java
index ff61fdd..9fc1796 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TestTaskExecutorService.java
@@ -66,7 +66,7 @@ public class TestTaskExecutorService {
     initMocks(this);
   }
 
-  @Test(timeout = 5000)
+  @Test(timeout = 10000)
   public void testPreemptionQueueComparator() throws InterruptedException {
     TaskWrapper r1 = createTaskWrapper(
         createSubmitWorkRequestProto(1, 2, 100, 200, false), false, 100000);
@@ -665,7 +665,7 @@ public class TestTaskExecutorService {
     }
   }
 
-  @Test(timeout = 1000)
+  @Test(timeout = 10000)
   public void testZeroCapacity() throws InterruptedException {
     TaskExecutorServiceForTest taskExecutorService =
         new TaskExecutorServiceForTest(1, 1, ShortestJobFirstComparator.class.getName(), true, mockMetrics);
@@ -705,35 +705,35 @@ public class TestTaskExecutorService {
     }
   }
 
-  @Test(timeout = 1000, expected = IllegalArgumentException.class)
+  @Test(timeout = 10000, expected = IllegalArgumentException.class)
   public void testSetCapacityHighExecutors() {
     TaskExecutorServiceForTest taskExecutorService =
         new TaskExecutorServiceForTest(2, 3, ShortestJobFirstComparator.class.getName(), true, mockMetrics);
     taskExecutorService.setCapacity(3, 3);
   }
 
-  @Test(timeout = 1000, expected = IllegalArgumentException.class)
+  @Test(timeout = 10000, expected = IllegalArgumentException.class)
   public void testSetCapacityHighQueueSize() {
     TaskExecutorServiceForTest taskExecutorService =
         new TaskExecutorServiceForTest(2, 3, ShortestJobFirstComparator.class.getName(), true, mockMetrics);
     taskExecutorService.setCapacity(2, 5);
   }
 
-  @Test(timeout = 1000, expected = IllegalArgumentException.class)
+  @Test(timeout = 10000, expected = IllegalArgumentException.class)
   public void testSetCapacityNegativeExecutors() {
     TaskExecutorServiceForTest taskExecutorService =
         new TaskExecutorServiceForTest(2, 3, ShortestJobFirstComparator.class.getName(), true, mockMetrics);
     taskExecutorService.setCapacity(-3, 3);
   }
 
-  @Test(timeout = 1000, expected = IllegalArgumentException.class)
+  @Test(timeout = 10000, expected = IllegalArgumentException.class)
   public void testSetCapacityNegativeQueueSize() {
     TaskExecutorServiceForTest taskExecutorService =
         new TaskExecutorServiceForTest(2, 3, ShortestJobFirstComparator.class.getName(), true, mockMetrics);
     taskExecutorService.setCapacity(2, -5);
   }
 
-  @Test(timeout = 1000)
+  @Test(timeout = 10000)
   public void testCapacityMetricsInitial() {
     TaskExecutorServiceForTest taskExecutorService = new TaskExecutorServiceForTest(2, 10,
         ShortestJobFirstComparator.class.getName(), true, mockMetrics);
@@ -742,7 +742,7 @@ public class TestTaskExecutorService {
     verify(mockMetrics).setWaitQueueSize(10);
   }
 
-  @Test(timeout = 1000)
+  @Test(timeout = 10000)
   public void testCapacityMetricsModification() {
     TaskExecutorServiceForTest taskExecutorService = new TaskExecutorServiceForTest(2, 10,
         ShortestJobFirstComparator.class.getName(), true, mockMetrics);
diff --git a/ql/src/test/queries/clientnegative/authorization_disallow_transform.q b/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
index 35c0653..af94f7c 100644
--- a/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
+++ b/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
@@ -1,3 +1,4 @@
+--! qt:disabled:flaky/bad/? HIVE-23567
 set hive.test.authz.sstd.hs2.mode=true;
 set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
 set hive.security.authorization.enabled=true;
diff --git a/ql/src/test/queries/clientnegative/masking_mv.q b/ql/src/test/queries/clientnegative/masking_mv.q
index deb5c4a..2ceafdb 100644
--- a/ql/src/test/queries/clientnegative/masking_mv.q
+++ b/ql/src/test/queries/clientnegative/masking_mv.q
@@ -1,5 +1,5 @@
---! qt:dataset:srcpart
 --! qt:dataset:src
+
 set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.mapred.mode=nonstrict;
diff --git a/ql/src/test/queries/clientnegative/strict_pruning.q b/ql/src/test/queries/clientnegative/strict_pruning.q
index 137fd7b..84af061 100644
--- a/ql/src/test/queries/clientnegative/strict_pruning.q
+++ b/ql/src/test/queries/clientnegative/strict_pruning.q
@@ -1,3 +1,4 @@
+--! qt:disabled:flaky HIVE-23320
 --! qt:dataset:srcpart
 set hive.strict.checks.bucketing=false; 
 
diff --git a/ql/src/test/queries/clientnegative/strict_pruning_2.q b/ql/src/test/queries/clientnegative/strict_pruning_2.q
index 3a11218..a6f498b 100644
--- a/ql/src/test/queries/clientnegative/strict_pruning_2.q
+++ b/ql/src/test/queries/clientnegative/strict_pruning_2.q
@@ -1,3 +1,4 @@
+--! qt:disabled:flaky HIVE-23320
 --! qt:dataset:srcpart
 reset hive.mapred.mode;
 set hive.strict.checks.no.partition.filter=true;
diff --git a/ql/src/test/queries/clientpositive/authorization_show_grant.q b/ql/src/test/queries/clientpositive/authorization_show_grant.q
index 3e10225..c1b9258 100644
--- a/ql/src/test/queries/clientpositive/authorization_show_grant.q
+++ b/ql/src/test/queries/clientpositive/authorization_show_grant.q
@@ -1,3 +1,4 @@
+--! qt:dataset::ONLY
 --! qt:authorizer
 
 set user.name=hive_admin_user;
diff --git a/ql/src/test/queries/clientpositive/druid_materialized_view_rewrite_ssb.q b/ql/src/test/queries/clientpositive/druid_materialized_view_rewrite_ssb.q
index c173fb8..b64cc49 100644
--- a/ql/src/test/queries/clientpositive/druid_materialized_view_rewrite_ssb.q
+++ b/ql/src/test/queries/clientpositive/druid_materialized_view_rewrite_ssb.q
@@ -1,3 +1,4 @@
+--! qt:disabled:unstable; fails sometimes HIVE-23450
 --! qt:dataset:part
 
 set hive.support.concurrency=true;
diff --git a/ql/src/test/queries/clientpositive/druidkafkamini_delimited.q b/ql/src/test/queries/clientpositive/druidkafkamini_delimited.q
index 91e279d..f7c64f9 100644
--- a/ql/src/test/queries/clientpositive/druidkafkamini_delimited.q
+++ b/ql/src/test/queries/clientpositive/druidkafkamini_delimited.q
@@ -1,3 +1,4 @@
+--! qt:disabled:this started falling after the druid upgrade just like HIVE-23450
 SET hive.vectorized.execution.enabled=false;
 
 CREATE EXTERNAL TABLE druid_kafka_test_delimited(`__time` timestamp , `page` string, `user` string, `language` string,
diff --git a/ql/src/test/queries/clientpositive/merge_test_dummy_operator.q b/ql/src/test/queries/clientpositive/merge_test_dummy_operator.q
index f11e22c..3ff885f 100644
--- a/ql/src/test/queries/clientpositive/merge_test_dummy_operator.q
+++ b/ql/src/test/queries/clientpositive/merge_test_dummy_operator.q
@@ -1,3 +1,4 @@
+--! qt:disabled:unstable HIVE-23552
 --! qt:dataset:src
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
diff --git a/ql/src/test/queries/clientpositive/results_cache_invalidation2.q b/ql/src/test/queries/clientpositive/results_cache_invalidation2.q
index b360c85..e915dc3 100644
--- a/ql/src/test/queries/clientpositive/results_cache_invalidation2.q
+++ b/ql/src/test/queries/clientpositive/results_cache_invalidation2.q
@@ -42,7 +42,7 @@ select count(*) from tab2 where key > 0;
 insert into tab1 select * from src;
 
 -- Run a query long enough that the invalidation check can run.
-select reflect("java.lang.Thread", 'sleep', cast(4000 as bigint));
+select reflect("java.lang.Thread", 'sleep', cast(10000 as bigint));
 
 set test.comment="Cached entry should be invalidated - query should not use cache";
 set test.comment;
diff --git a/ql/src/test/queries/clientpositive/schema_evol_par_vec_table_dictionary_encoding.q b/ql/src/test/queries/clientpositive/schema_evol_par_vec_table_dictionary_encoding.q
index fbae934..11a3186 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_par_vec_table_dictionary_encoding.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_par_vec_table_dictionary_encoding.q
@@ -1,3 +1,5 @@
+--! qt:disabled:falky HIVE-23566
+
 set hive.fetch.task.conversion=none;
 set hive.vectorized.execution.enabled=true;
 set parquet.enable.dictionary=true;
diff --git a/ql/src/test/queries/clientpositive/schema_evol_par_vec_table_non_dictionary_encoding.q b/ql/src/test/queries/clientpositive/schema_evol_par_vec_table_non_dictionary_encoding.q
index a8a16d2..c822f6f 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_par_vec_table_non_dictionary_encoding.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_par_vec_table_non_dictionary_encoding.q
@@ -1,3 +1,4 @@
+--! qt:disabled:HIVE-23566
 set hive.fetch.task.conversion=none;
 set hive.vectorized.execution.enabled=true;
 set parquet.enable.dictionary=false;
diff --git a/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q b/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
index 2159052..5ba6c5b 100644
--- a/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
+++ b/ql/src/test/queries/clientpositive/special_character_in_tabnames_1.q
@@ -1,3 +1,4 @@
+--! qt:disabled:flaky HIVE-23568
 --! qt:dataset:src1
 --! qt:dataset:src
 --! qt:dataset:lineitem
diff --git a/ql/src/test/queries/clientpositive/stats_list_bucket.q b/ql/src/test/queries/clientpositive/stats_list_bucket.q
index bbb4206..dff588a 100644
--- a/ql/src/test/queries/clientpositive/stats_list_bucket.q
+++ b/ql/src/test/queries/clientpositive/stats_list_bucket.q
@@ -1,3 +1,4 @@
+--! qt:disabled:breaks sysdb.q HIVE-23370
 --! qt:dataset:src
 
 
diff --git a/ql/src/test/queries/clientpositive/temp_table_multi_insert_partitioned.q b/ql/src/test/queries/clientpositive/temp_table_multi_insert_partitioned.q
index 68d0483..9b00a83 100644
--- a/ql/src/test/queries/clientpositive/temp_table_multi_insert_partitioned.q
+++ b/ql/src/test/queries/clientpositive/temp_table_multi_insert_partitioned.q
@@ -1,3 +1,4 @@
+--! qt:disabled:multi_insert_stuff HIVE-23565
 --! qt:dataset:src
 set hive.stats.column.autogather=false;
 set hive.mapred.mode=nonstrict;
diff --git a/ql/src/test/results/clientpositive/llap/results_cache_invalidation2.q.out b/ql/src/test/results/clientpositive/llap/results_cache_invalidation2.q.out
index 6796d5f..170fff1 100644
--- a/ql/src/test/results/clientpositive/llap/results_cache_invalidation2.q.out
+++ b/ql/src/test/results/clientpositive/llap/results_cache_invalidation2.q.out
@@ -171,11 +171,11 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: default@tab1
 POSTHOOK: Lineage: tab1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: tab1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: select reflect("java.lang.Thread", 'sleep', cast(4000 as bigint))
+PREHOOK: query: select reflect("java.lang.Thread", 'sleep', cast(10000 as bigint))
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-POSTHOOK: query: select reflect("java.lang.Thread", 'sleep', cast(4000 as bigint))
+POSTHOOK: query: select reflect("java.lang.Thread", 'sleep', cast(10000 as bigint))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/TestAcidTxnCleanerService.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/TestAcidTxnCleanerService.java
index ba8ba73..99c045d 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/TestAcidTxnCleanerService.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/txn/TestAcidTxnCleanerService.java
@@ -37,6 +37,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Ignore;
 
 
 import static java.util.Collections.singletonList;
@@ -45,6 +46,7 @@ import static java.util.Collections.singletonList;
  * Testing whether AcidTxnCleanerService removes the correct records
  * from the TXNS table (via TxnStore).
  */
+@Ignore("test unstable HIVE-23525")
 public class TestAcidTxnCleanerService {
 
   private AcidTxnCleanerService underTest;