Browse Source

[livy] Capture the output from batch yarn jobs

Erick Tryzelaar 10 years ago
parent
commit
7d5ce8679d

+ 16 - 34
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/LineBufferedProcess.scala

@@ -18,53 +18,35 @@
 
 package com.cloudera.hue.livy
 
-import scala.io.Source
-
 class LineBufferedProcess(process: Process) extends Logging {
 
-  private[this] var _stdoutLines: IndexedSeq[String] = IndexedSeq()
-  private[this] var _stderrLines: IndexedSeq[String] = IndexedSeq()
-
-  private val stdoutThread = new Thread {
-    override def run() = {
-      val lines = Source.fromInputStream(process.getInputStream).getLines()
-      for (line <- lines) {
-        trace("stdout: ", line)
-        _stdoutLines +:= line
-      }
-    }
-  }
-  stdoutThread.setDaemon(true)
-  stdoutThread.start()
-
-  private val stderrThread = new Thread {
-    override def run() = {
-      val lines = Source.fromInputStream(process.getErrorStream).getLines()
-      for (line <- lines) {
-        trace("stderr: ", line)
-        _stderrLines +:= line
-      }
-    }
-  }
-  stderrThread.setDaemon(true)
-  stderrThread.start()
+  private[this] val _inputStream = new LineBufferedStream(process.getInputStream)
+  private[this] val _errorStream = new LineBufferedStream(process.getErrorStream)
 
-  def stdoutLines: IndexedSeq[String] = _stdoutLines
+  def inputLines: IndexedSeq[String] = _inputStream.lines
+  def errorLines: IndexedSeq[String] = _errorStream.lines
 
-  def stderrLines: IndexedSeq[String] = _stderrLines
+  def inputIterator: Iterator[String] = _inputStream.iterator
+  def errorIterator: Iterator[String] = _errorStream.iterator
 
   def destroy(): Unit = {
     process.destroy()
   }
 
+  /** Returns if the process is still actively running. */
+  def isAlive: Boolean = try {
+    exitValue()
+    false
+  } catch {
+    case _: IllegalStateException => true
+  }
+
   def exitValue(): Int = {
     process.exitValue()
   }
 
   def waitFor(): Int = {
-    val output = process.waitFor()
-    stdoutThread.join()
-    stderrThread.join()
-    output
+    process.waitFor()
   }
 }
+

+ 94 - 0
apps/spark/java/livy-core/src/main/scala/com/cloudera/hue/livy/LineBufferedStream.scala

@@ -0,0 +1,94 @@
+/*
+ * Licensed to Cloudera, Inc. under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Cloudera, Inc. 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 com.cloudera.hue.livy
+
+import java.io.InputStream
+import java.util.concurrent.locks.ReentrantLock
+
+import scala.io.Source
+
+class LineBufferedStream(inputStream: InputStream) extends Logging {
+
+  private[this] var _lines: IndexedSeq[String] = IndexedSeq()
+
+  private[this] val _lock = new ReentrantLock()
+  private[this] val _condition = _lock.newCondition()
+  private[this] var _finished = false
+
+  private val thread = new Thread {
+    override def run() = {
+      val lines = Source.fromInputStream(inputStream).getLines()
+      for (line <- lines) {
+        _lock.lock()
+        try {
+          trace("stdout: ", line)
+          _lines = _lines :+ line
+          _condition.signalAll()
+        } finally {
+          _lock.unlock()
+        }
+      }
+
+      _lock.lock()
+      try {
+        _finished = true
+        _condition.signalAll()
+      } finally {
+        _lock.unlock()
+      }
+    }
+  }
+  thread.setDaemon(true)
+  thread.start()
+
+  def lines: IndexedSeq[String] = _lines
+
+  def iterator: Iterator[String] = {
+    new LinesIterator
+  }
+
+  private class LinesIterator extends Iterator[String] {
+    private[this] var index = 0
+
+    override def hasNext: Boolean = {
+      if (index < _lines.length) {
+        true
+      } else {
+        // Otherwise we might still have more data.
+        _lock.lock()
+        try {
+          if (_finished) {
+            false
+          } else {
+            _condition.await()
+            index < _lines.length
+          }
+        } finally {
+          _lock.unlock()
+        }
+      }
+    }
+
+    override def next(): String = {
+      val line = _lines(index)
+      index += 1
+      line
+    }
+  }
+}

+ 1 - 1
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchProcess.scala

@@ -72,7 +72,7 @@ private class BatchProcess(val id: Int,
     Success()
   }
 
-  override def lines: IndexedSeq[String] = process.stdoutLines
+  override def lines: IndexedSeq[String] = process.inputLines
 
   override def stop(): Future[Unit] = {
     Future {

+ 8 - 5
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/batch/BatchYarn.scala

@@ -20,7 +20,7 @@ package com.cloudera.hue.livy.server.batch
 
 import java.lang.ProcessBuilder.Redirect
 
-import com.cloudera.hue.livy.LivyConf
+import com.cloudera.hue.livy.{LineBufferedProcess, LivyConf}
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
 import com.cloudera.hue.livy.yarn._
 
@@ -35,8 +35,11 @@ object BatchYarn {
   def apply(livyConf: LivyConf, client: Client, id: Int, createBatchRequest: CreateBatchRequest): Batch = {
     val builder = sparkBuilder(createBatchRequest)
 
-    val process = builder.start(createBatchRequest.file, createBatchRequest.args)
-    new BatchYarn(id, Future { client.getJobFromProcess(process) })
+    val process = new LineBufferedProcess(builder.start(createBatchRequest.file, createBatchRequest.args))
+    val job = Future {
+      client.getJobFromProcess(process)
+    }
+    new BatchYarn(id, process, job)
   }
 
   private def sparkBuilder(createBatchRequest: CreateBatchRequest): SparkSubmitProcessBuilder = {
@@ -61,7 +64,7 @@ object BatchYarn {
   }
 }
 
-private class BatchYarn(val id: Int, jobFuture: Future[Job]) extends Batch {
+private class BatchYarn(val id: Int, process: LineBufferedProcess, jobFuture: Future[Job]) extends Batch {
 
   implicit def executor: ExecutionContextExecutor = ExecutionContext.global
 
@@ -107,5 +110,5 @@ private class BatchYarn(val id: Int, jobFuture: Future[Job]) extends Batch {
     }
   }
 
-  override def lines: IndexedSeq[String] = IndexedSeq()
+  override def lines: IndexedSeq[String] = process.inputLines
 }

+ 8 - 2
apps/spark/java/livy-server/src/main/scala/com/cloudera/hue/livy/server/sessions/YarnSession.scala

@@ -22,7 +22,7 @@ import java.lang.ProcessBuilder.Redirect
 import java.util.concurrent.TimeUnit
 
 import com.cloudera.hue.livy.spark.SparkSubmitProcessBuilder
-import com.cloudera.hue.livy.{Utils, LivyConf}
+import com.cloudera.hue.livy.{LineBufferedProcess, Utils, LivyConf}
 import com.cloudera.hue.livy.sessions.{Kind, Error}
 import com.cloudera.hue.livy.yarn.{Client, Job}
 
@@ -52,7 +52,13 @@ object YarnSession {
     val process = builder.start(livyJar(livyConf), List(kind.toString))
 
     val job = Future {
-      client.getJobFromProcess(process)
+      val proc = new LineBufferedProcess(process)
+      val job = client.getJobFromProcess(proc)
+
+      // We don't need the process anymore.
+      proc.destroy()
+
+      job
     }
 
     new YarnSession(id, kind, proxyUser, job)

+ 6 - 12
apps/spark/java/livy-yarn/src/main/scala/com/cloudera/hue/livy/yarn/Client.scala

@@ -21,7 +21,7 @@ package com.cloudera.hue.livy.yarn
 import java.io.{InputStream, BufferedReader, InputStreamReader}
 
 import com.cloudera.hue.livy.yarn.Client._
-import com.cloudera.hue.livy.{LivyConf, Logging}
+import com.cloudera.hue.livy.{Utils, LineBufferedProcess, LivyConf, Logging}
 import org.apache.hadoop.yarn.api.records.{ApplicationId, FinalApplicationStatus, YarnApplicationState}
 import org.apache.hadoop.yarn.client.api.YarnClient
 import org.apache.hadoop.yarn.conf.YarnConfiguration
@@ -54,14 +54,11 @@ class Client(livyConf: LivyConf) extends Logging {
   yarnClient.init(yarnConf)
   yarnClient.start()
 
-  def getJobFromProcess(process: Process): Job = {
-    val lines = Source.fromInputStream(process.getInputStream).getLines()
-    val applicationId = parseApplicationId(lines).getOrElse(throw new FailedToSubmitApplication)
-
-    // Application has been submitted, so we don't need to keep the process around anymore.
-    process.destroy()
-
-    new Job(yarnClient, ConverterUtils.toApplicationId(applicationId))
+  def getJobFromProcess(process: LineBufferedProcess): Job = {
+    parseApplicationId(process.inputIterator) match {
+      case Some(appId) => new Job(yarnClient, ConverterUtils.toApplicationId(appId))
+      case None => throw new FailedToSubmitApplication
+    }
   }
 
   def close() = {
@@ -72,9 +69,6 @@ class Client(livyConf: LivyConf) extends Logging {
   private def parseApplicationId(lines: Iterator[String]): Option[String] = {
     if (lines.hasNext) {
       val line = lines.next()
-
-      info(f"shell output: $line")
-
       line match {
         case regex(applicationId) => Some(applicationId)
         case _ => parseApplicationId(lines)