diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala index a27a8a023..06d1e19a2 100644 --- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala +++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/ExecutePython.scala @@ -197,6 +197,10 @@ case class SessionPythonWorker( * @return the python response */ def runCode(code: String, internal: Boolean = false): Option[PythonResponse] = withLockRequired { + if (!workerProcess.isAlive) { + throw KyuubiSQLException("Python worker process has been exited, please check the error log" + + " and re-create the session to run python code.") + } val input = ExecutePython.toJson(Map("code" -> code, "cmd" -> "run_code")) // scalastyle:off println stdin.println(input) @@ -337,7 +341,7 @@ object ExecutePython extends Logging { val stderrThread = new Thread("process stderr thread") { override def run(): Unit = { val lines = scala.io.Source.fromInputStream(process.getErrorStream).getLines() - lines.foreach(logger.error) + lines.filter(_.trim.nonEmpty).foreach(logger.error) } } stderrThread.setDaemon(true)