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 2020/06/18 09:53:11 UTC

[GitHub] [spark] juliuszsompolski commented on a change in pull request #28835: [WIP][SPARK-31926][TESTS][FOLLOWUP] Fix concurrency issue for ThriftCLIService to getPortNumber

juliuszsompolski commented on a change in pull request #28835:
URL: https://github.com/apache/spark/pull/28835#discussion_r442106655



##########
File path: sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala
##########
@@ -114,10 +134,29 @@ private[thriftserver] trait ReflectedCompositeService { this: AbstractService =>
     invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED)
     setAncestorField(this, 3, "hiveConf", hiveConf)
     invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED)
-    if (HiveUtils.isHive23) {
-      getAncestorField[Logger](this, 3, "LOG").info(s"Service: $getName is inited.")
-    } else {
-      getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.")
+    logInfo(s"Service: $getName is inited.")
+  }
+
+  def startCompositeService(): Unit = {
+    // Emulating `CompositeService.init(hiveConf)`
+    val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList")
+    var serviceStartCount = 0
+    try {
+      serviceList.asScala.foreach { service =>
+        service.start()
+        serviceStartCount += 1
+      }
+    } catch {
+      case NonFatal(e) =>
+      logError(s"Error starting services $getName", e)
+      invoke(classOf[CompositeService], this, "stop",
+        classOf[Int] -> new Integer(serviceStartCount))
+      throw new ServiceException("Failed to Start " + getName, e)
     }
+
+    // Emulating `AbstractService.start`

Review comment:
       AbstractService.start does also `startTime = System.currentTimeMillis();`
   Lets set startTime as well, just in case.

##########
File path: sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala
##########
@@ -69,23 +89,42 @@ trait SharedThriftServer extends SharedSparkSession {
   }
 
   private def startThriftServer(attempt: Int): Unit = {
-    logInfo(s"Trying to start HiveThriftServer2:, attempt=$attempt")
+    logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt")
     val sqlContext = spark.newSession().sqlContext
-    // Set the HIVE_SERVER2_THRIFT_PORT to 0, so it could randomly pick any free port to use.
+    sqlContext.setConf(ConfVars.METASTORECONNECTURLKEY.varname,
+      s"jdbc:derby:;databaseName=$metastorePath;create=true")
+    sqlContext.setConf(ConfVars.METASTOREURIS.varname, "")

Review comment:
       What's this for?




----------------------------------------------------------------
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



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