You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/12/24 10:08:26 UTC

[GitHub] [spark] HeartSaVioR opened a new pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

HeartSaVioR opened a new pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001
 
 
   ### What changes were proposed in this pull request?
   
   This patch fixes the intermittent test failure on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite, getting ConnectException when querying to thrift server.
   (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115646/testReport/)
   
   The relevant unit test log messages are following:
   
   ```
   19/12/23 13:33:01.875 pool-1-thread-1 INFO AbstractService: Service:ThriftBinaryCLIService is started.
   19/12/23 13:33:01.875 pool-1-thread-1 INFO AbstractService: Service:HiveServer2 is started.
   ...
   19/12/23 13:33:01.888 pool-1-thread-1 INFO ThriftServerWithSparkContextSuite: HiveThriftServer2 started successfully
   ...
   19/12/23 13:33:01.909 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO ThriftServerWithSparkContextSuite:
   
   ===== TEST OUTPUT FOR o.a.s.sql.hive.thriftserver.ThriftServerWithSparkContextSuite: 'SPARK-29911: Uncache cached tables when session closed' =====
   
   ...
   19/12/23 13:33:02.017 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO Utils: Supplied authorities: localhost:15441
   19/12/23 13:33:02.018 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO Utils: Resolved authority: localhost:15441
   19/12/23 13:33:02.078 HiveServer2-Background-Pool: Thread-213 INFO BaseSessionStateBuilder$$anon$2: Optimization rule 'org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation' is excluded from the optimizer.
   19/12/23 13:33:02.078 HiveServer2-Background-Pool: Thread-213 INFO BaseSessionStateBuilder$$anon$2: Optimization rule 'org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation' is excluded from the optimizer.
   19/12/23 13:33:02.121 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite WARN HiveConnection: Failed to connect to localhost:15441
   19/12/23 13:33:02.124 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO ThriftServerWithSparkContextSuite:
   
   ===== FINISHED o.a.s.sql.hive.thriftserver.ThriftServerWithSparkContextSuite: 'SPARK-29911: Uncache cached tables when session closed' =====
   
   19/12/23 13:33:02.143 Thread-35 INFO ThriftCLIService: Starting ThriftBinaryCLIService on port 15441 with 5...500 worker threads
   19/12/23 13:33:02.327 pool-1-thread-1 INFO HiveServer2: Shutting down HiveServer2
   19/12/23 13:33:02.328 pool-1-thread-1 INFO ThriftCLIService: Thrift server has stopped
   ```
   (Here the error is logged as `WARN HiveConnection: Failed to connect to localhost:15441` - the actual stack trace can be seen on Jenkins test summary.)
   
   The reason of test failure: Thrift(Binary|Http)CLIService prepare and launch the service asynchronously (in new thread), which suites are not waiting for completion and just start running tests, ends up with race condition.
   
   That can be easily reproduced, via adding artificial sleep in `ThriftBinaryCLIService.run()` here:
   https://github.com/apache/spark/blob/ba3f6330dd2b6054988f1f6f0ffe014fc4969088/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java#L49
   
   (Note that `sleep` should be added before initializing server socket. E.g. Line 57)
   
   This patch changes the test initialization logic to try executing simple query to wait until the service is available. The patch also refactors the code to apply the change both ThriftServerQueryTestSuite and ThriftServerWithSparkContextSuite easily.
   
   ### Why are the changes needed?
   
   This patch fixes the intermittent failure observed here:
   https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115646/testReport/
   
   ### Does this PR introduce any user-facing change?
   
   No
   
   ### How was this patch tested?
   
   Artificially made the test fail consistently (by the approach described above), and confirmed the patch fixed the test.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569049457
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/20602/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] maropu commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
maropu commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569045496
 
 
   The change looks reasonable to me.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568714814
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/20524/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568721852
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/115730/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568940535
 
 
   ThriftServerQueryTestSuite has been touched from various people, but let me first cc.ing some committers.
   cc. @wangyum @cloud-fan @maropu @HyukjinKwon 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569212370
 
 
   Thanks all for reviewing and merging!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568721852
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/115730/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569211058
 
 
   thanks, merging to master!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569049457
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/20602/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568714809
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569049453
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568715129
 
 
   This patch also resolve SPARK-28883 which error message is reported to ConnectException, but given there might be some failures not relevant to this, I'd defer to reviewers judging whether this patch also resolves SPARK-28883 entirely or partially.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568714809
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568721847
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568714510
 
 
   **[Test build #115730 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115730/testReport)** for PR 27001 at commit [`8309fb6`](https://github.com/apache/spark/commit/8309fb60899bffb3cbaa061b85ed2e7fbe0f31e4).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568714814
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/20524/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] maropu commented on a change in pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#discussion_r361437757
 
 

 ##########
 File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala
 ##########
 @@ -0,0 +1,86 @@
+/*
+ * 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.spark.sql.hive.thriftserver
+
+import java.sql.{DriverManager, Statement}
+
+import scala.concurrent.duration._
+import scala.util.{Random, Try}
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.test.SharedSparkSession
+
+trait SharedThriftServer extends QueryTest with SharedSparkSession {
+
+  private var hiveServer2: HiveThriftServer2 = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    // Chooses a random port between 10000 and 19999
+    var listeningPort = 10000 + Random.nextInt(10000)
+
+    // Retries up to 3 times with different port numbers if the server fails to start
+    (1 to 3).foldLeft(Try(startThriftServer(listeningPort, 0))) { case (started, attempt) =>
+      started.orElse {
+        listeningPort += 1
+        Try(startThriftServer(listeningPort, attempt))
+      }
+    }.recover {
+      case cause: Throwable =>
+        throw cause
+    }.get
+    logInfo("HiveThriftServer2 started successfully")
+  }
+
+  override def afterAll(): Unit = {
+    try {
+      hiveServer2.stop()
+    } finally {
+      super.afterAll()
+    }
+  }
+
+  protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = {
+    val user = System.getProperty("user.name")
+
+    val serverPort = hiveServer2.getHiveConf.get(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname)
+    val connections =
+      fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") }
+    val statements = connections.map(_.createStatement())
+
+    try {
+      statements.zip(fs).foreach { case (s, f) => f(s) }
+    } finally {
+      statements.foreach(_.close())
+      connections.foreach(_.close())
+    }
+  }
+
+  private def startThriftServer(port: Int, attempt: Int): Unit = {
+    logInfo(s"Trying to start HiveThriftServer2: port=$port, attempt=$attempt")
+    val sqlContext = spark.newSession().sqlContext
+    sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, port.toString)
+    hiveServer2 = HiveThriftServer2.startWithContext(sqlContext)
+
+    eventually(timeout(30.seconds), interval(1.seconds)) {
 
 Review comment:
   Can you leave some comments here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568721755
 
 
   **[Test build #115730 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115730/testReport)** for PR 27001 at commit [`8309fb6`](https://github.com/apache/spark/commit/8309fb60899bffb3cbaa061b85ed2e7fbe0f31e4).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `trait SharedThriftServer extends QueryTest with SharedSparkSession `
     * `class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer `
     * `class ThriftServerWithSparkContextSuite extends SharedThriftServer `

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] maropu commented on a change in pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#discussion_r361437718
 
 

 ##########
 File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala
 ##########
 @@ -0,0 +1,86 @@
+/*
+ * 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.spark.sql.hive.thriftserver
+
+import java.sql.{DriverManager, Statement}
+
+import scala.concurrent.duration._
+import scala.util.{Random, Try}
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.test.SharedSparkSession
+
+trait SharedThriftServer extends QueryTest with SharedSparkSession {
 
 Review comment:
   How about just moving `extends QueryTest with SharedSparkSession` into the two test suites since they are not dependent between each other?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568714295
 
 
   Here's diff file to reproduce the issue: 
   [SPARK-30345-reproducerThriftBinaryCLIService.diff.zip](https://github.com/apache/spark/files/3997961/SPARK-30345-reproducerThriftBinaryCLIService.diff.zip)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569201728
 
 
   > The fix itself LGTM, but do you know why Thrift(Binary|Http)CLIService prepare and launch the service asynchronously? Seems end-users can also hit this problem if they run queries right after the server is started.
   
   I don't know about history, but if I get the commit history correctly, codebase of `sql/hive-thiftserver` just came from Hive, and we just fixed minors. So that's basically inherited from Hive.
   
   Here we're embedding Hive server in test suites hence get a chance to encounter race condition, but most likely it will run as standalone app which end users would just think server is not ready and rerun the query when query fails for such reason. 
   
   Btw, there's another test suites which runs Hive server as external standalone app - it shouldn't encounter this issue but it would show longer startup and shutdown. 
   
   https://github.com/apache/spark/blob/9c046dc8084500860f3820bdbd2be04a6e491431/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala#L986-L1215

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR edited a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568714295
 
 
   Here's the diff file to reproduce the issue: 
   [SPARK-30345-reproducerThriftBinaryCLIService.diff.zip](https://github.com/apache/spark/files/3997961/SPARK-30345-reproducerThriftBinaryCLIService.diff.zip)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569049163
 
 
   Thanks for the reviewing, I've addressed review comments. Please take a look again. Thanks in advance!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569053086
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569053086
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569049453
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569199553
 
 
   The fix itself LGTM, but do you know why Thrift(Binary|Http)CLIService prepare and launch the service asynchronously? Seems end-users can also hit this problem if they run queries right after the server is started.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568714510
 
 
   **[Test build #115730 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115730/testReport)** for PR 27001 at commit [`8309fb6`](https://github.com/apache/spark/commit/8309fb60899bffb3cbaa061b85ed2e7fbe0f31e4).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569053089
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/115810/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569049219
 
 
   **[Test build #115810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115810/testReport)** for PR 27001 at commit [`160ca1a`](https://github.com/apache/spark/commit/160ca1a9f008ff4d7e17fff166e795ac17d20963).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569053089
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/115810/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] maropu commented on a change in pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#discussion_r361437718
 
 

 ##########
 File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala
 ##########
 @@ -0,0 +1,86 @@
+/*
+ * 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.spark.sql.hive.thriftserver
+
+import java.sql.{DriverManager, Statement}
+
+import scala.concurrent.duration._
+import scala.util.{Random, Try}
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.test.SharedSparkSession
+
+trait SharedThriftServer extends QueryTest with SharedSparkSession {
 
 Review comment:
   How about just moving `extends QueryTest with SharedSparkSession` into the two test suites since they are not dependent between each other?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569053035
 
 
   **[Test build #115810 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115810/testReport)** for PR 27001 at commit [`160ca1a`](https://github.com/apache/spark/commit/160ca1a9f008ff4d7e17fff166e795ac17d20963).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `trait SharedThriftServer extends SharedSparkSession `

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR edited a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569201728
 
 
   > The fix itself LGTM, but do you know why Thrift(Binary|Http)CLIService prepare and launch the service asynchronously? Seems end-users can also hit this problem if they run queries right after the server is started.
   
   I don't know about history, but if I get the commit history correctly, codebase of `sql/hive-thiftserver` just came from Hive, and we just fixed minors. So that's basically inherited from Hive.
   
   Here we're embedding Hive server in test suites hence get a chance to encounter race condition, but most likely it will run as standalone app which end users would just think server is not ready and rerun the query when query fails for such reason. (And the chance is rare, as it's even hard to reproduce without adding artificial sleep. It might be under 1 second.)
   
   Btw, there's another test suites which runs Hive server as external standalone app - it shouldn't encounter this issue but it would show longer startup and shutdown. 
   
   https://github.com/apache/spark/blob/9c046dc8084500860f3820bdbd2be04a6e491431/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala#L986-L1215

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] maropu commented on a change in pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
maropu commented on a change in pull request #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#discussion_r361439333
 
 

 ##########
 File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala
 ##########
 @@ -0,0 +1,86 @@
+/*
+ * 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.spark.sql.hive.thriftserver
+
+import java.sql.{DriverManager, Statement}
+
+import scala.concurrent.duration._
+import scala.util.{Random, Try}
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.test.SharedSparkSession
+
+trait SharedThriftServer extends QueryTest with SharedSparkSession {
 
 Review comment:
   How about just moving `QueryTest` into each test suite since they are not dependent between each other?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568721847
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-569049219
 
 
   **[Test build #115810 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115810/testReport)** for PR 27001 at commit [`160ca1a`](https://github.com/apache/spark/commit/160ca1a9f008ff4d7e17fff166e795ac17d20963).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR edited a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-568715129
 
 
   This patch also resolve [SPARK-28883](https://issues.apache.org/jira/browse/SPARK-28883) which error message is reported to ConnectException, but given there might be some failures not relevant to this, I'd defer to reviewers judging whether this patch also resolves SPARK-28883 entirely or partially.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] juliuszsompolski commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite

Posted by GitBox <gi...@apache.org>.
juliuszsompolski commented on issue #27001: [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
URL: https://github.com/apache/spark/pull/27001#issuecomment-586299066
 
 
   Thank you very much @HeartSaVioR for this trait cleanup! I missed this PR over Christmas and just now saw the new code. This trait that starts the Thriftserver with existing SparkContext instead of as a separate process can be very useful to increase thriftserver test coverage in other places - e.g. add tests validating the state of the thriftserver listener, session manager, query cancellation etc... - things to assert which the test needs to touch thriftserver internals to validate.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org