Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -395,12 +395,16 @@ private[spark] class PythonWorkerFactory(
}
}

private val workerLogCapture =
envVars.get("PYSPARK_SPARK_SESSION_UUID").map(new PythonWorkerLogCapture(_))

/**
* Redirect the given streams to our stderr in separate threads.
*/
private def redirectStreamsToStderr(stdout: InputStream, stderr: InputStream): Unit = {
try {
new RedirectThread(stdout, System.err, "stdout reader for " + pythonExec).start()
new RedirectThread(workerLogCapture.map(_.wrapInputStream(stdout)).getOrElse(stdout),
System.err, "stdout reader for " + pythonExec).start()
new RedirectThread(stderr, System.err, "stderr reader for " + pythonExec).start()
} catch {
case e: Exception =>
Expand Down Expand Up @@ -460,6 +464,7 @@ private[spark] class PythonWorkerFactory(
}

def stop(): Unit = {
workerLogCapture.foreach(_.closeAllWriters())
stopDaemon()
}

Expand Down
Original file line number Diff line number Diff line change
@@ -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.
*/

package org.apache.spark.api.python

import java.io.{BufferedReader, InputStream, InputStreamReader}
import java.nio.ByteBuffer
import java.nio.charset.StandardCharsets
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicLong

import scala.jdk.CollectionConverters._

import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging
import org.apache.spark.storage.{PythonWorkerLogBlockIdGenerator, PythonWorkerLogLine, RollingLogWriter}

/**
* Manages Python UDF log capture and routing to per-worker log writers.
*
* This class handles the parsing of Python worker output streams and routes
* log messages to appropriate rolling log writers based on worker PIDs.
* Works for both daemon and non-daemon modes.
*/
private[python] class PythonWorkerLogCapture(
sessionId: String,
logMarker: String = "PYTHON_WORKER_LOGGING") extends Logging {

// Map to track per-worker log writers: workerId(PID) -> (writer, sequenceId)
private val workerLogWriters = new ConcurrentHashMap[String, (RollingLogWriter, AtomicLong)]()

/**
* Creates an InputStream wrapper that captures Python UDF logs from the given stream.
*
* @param inputStream The input stream to wrap (typically daemon stdout or worker stdout)
* @return A wrapped InputStream that captures and routes log messages
*/
def wrapInputStream(inputStream: InputStream): InputStream = {
new CaptureWorkerLogsInputStream(inputStream)
}

/**
* Removes and closes the log writer for a specific worker.
*
* @param workerId The worker ID (typically PID as string)
*/
def removeAndCloseWorkerLogWriter(workerId: String): Unit = {
Option(workerLogWriters.remove(workerId)).foreach { case (writer, _) =>
try {
writer.close()
} catch {
case e: Exception =>
logWarning(s"Failed to close log writer for worker $workerId", e)
}
}
}

/**
* Closes all active worker log writers.
*/
def closeAllWriters(): Unit = {
workerLogWriters.values().asScala.foreach { case (writer, _) =>
try {
writer.close()
} catch {
case e: Exception =>
logWarning("Failed to close log writer", e)
}
}
workerLogWriters.clear()
}

/**
* Gets or creates a log writer for the specified worker.
*
* @param workerId Unique identifier for the worker (typically PID)
* @return Tuple of (RollingLogWriter, AtomicLong sequence counter)
*/
private def getOrCreateLogWriter(workerId: String): (RollingLogWriter, AtomicLong) = {
workerLogWriters.computeIfAbsent(workerId, _ => {
val logWriter = SparkEnv.get.blockManager.getRollingLogWriter(
new PythonWorkerLogBlockIdGenerator(sessionId, workerId)
)
(logWriter, new AtomicLong())
})
}

/**
* Processes a log line from a Python worker.
*
* @param line The complete line containing the log marker and JSON
* @return The prefix (non-log content) that should be passed through
*/
private def processLogLine(line: String): String = {
val markerIndex = line.indexOf(s"$logMarker:")
if (markerIndex >= 0) {
val prefix = line.substring(0, markerIndex)
val markerAndJson = line.substring(markerIndex)

// Parse: "PYTHON_UDF_LOGGING:12345:{json}"
val parts = markerAndJson.split(":", 3)
if (parts.length >= 3) {
val workerId = parts(1) // This is the PID from Python worker
val json = parts(2)

try {
if (json.isEmpty) {
removeAndCloseWorkerLogWriter(workerId)
} else {
val (writer, seqId) = getOrCreateLogWriter(workerId)
writer.writeLog(
PythonWorkerLogLine(System.currentTimeMillis(), seqId.getAndIncrement(), json)
Comment on lines +125 to +126
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we want to limit the number of lines written to block manager?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cc @ivoson

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cc @ivoson @cloud-fan this is important as we don't want users to write unlimited number of logs into block manager.

)
}
} catch {
case e: Exception =>
logWarning(s"Failed to write log for worker $workerId", e)
}
}
prefix
} else {
line + System.lineSeparator()
}
}

/**
* InputStream wrapper that captures and processes Python UDF logs.
*/
private class CaptureWorkerLogsInputStream(in: InputStream) extends InputStream {

private[this] val reader = new BufferedReader(
new InputStreamReader(in, StandardCharsets.ISO_8859_1))
private[this] val temp = new Array[Byte](1)
private[this] var buffer = ByteBuffer.allocate(0)

override def read(): Int = {
val n = read(temp)
if (n <= 0) {
-1
} else {
// Signed byte to unsigned integer
temp(0) & 0xff
}
}

override def read(b: Array[Byte], off: Int, len: Int): Int = {
if (buffer.hasRemaining) {
val buf = ByteBuffer.wrap(b, off, len)
val remaining = Math.min(buffer.remaining(), buf.remaining())
buf.put(buf.position(), buffer, buffer.position(), remaining)
buffer.position(buffer.position() + remaining)
remaining
} else {
val line = reader.readLine()
if (line == null) {
closeAllWriters()
-1
} else {
val processedContent = if (line.contains(s"$logMarker:")) {
processLogLine(line)
} else {
line + System.lineSeparator()
}

buffer = ByteBuffer.wrap(processedContent.getBytes(StandardCharsets.ISO_8859_1))
read(b, off, len)
}
}
}

override def close(): Unit = {
try {
reader.close()
} finally {
closeAllWriters()
}
}
}
}
38 changes: 33 additions & 5 deletions core/src/main/scala/org/apache/spark/storage/BlockId.scala
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,7 @@ case class PythonStreamBlockId(streamId: Int, uniqueId: Long) extends BlockId {
object LogBlockType extends Enumeration {
type LogBlockType = Value
val TEST = Value
val PYTHON_WORKER = Value
}

/**
Expand All @@ -188,30 +189,54 @@ object LogBlockType extends Enumeration {
* and log management.
* @param executorId the ID of the executor that produced this log block.
*/
abstract sealed class LogBlockId(
val lastLogTime: Long,
val executorId: String) extends BlockId {
abstract sealed class LogBlockId extends BlockId {
def lastLogTime: Long
def executorId: String
def logBlockType: LogBlockType
}

object LogBlockId {
def empty(logBlockType: LogBlockType): LogBlockId = {
logBlockType match {
case LogBlockType.TEST => TestLogBlockId(0L, "")
case LogBlockType.PYTHON_WORKER => PythonWorkerLogBlockId(0L, "", "", "")
case _ => throw new SparkException(s"Unsupported log block type: $logBlockType")
}
}
}

// Used for test purpose only.
case class TestLogBlockId(override val lastLogTime: Long, override val executorId: String)
extends LogBlockId(lastLogTime, executorId) {
case class TestLogBlockId(lastLogTime: Long, executorId: String)
extends LogBlockId {
override def name: String =
"test_log_" + lastLogTime + "_" + executorId

override def logBlockType: LogBlockType = LogBlockType.TEST
}

/**
* Identifies a block of Python worker log data.
*
* @param lastLogTime the timestamp of the last log entry in this block, used for filtering
* and log management.
* @param executorId the ID of the executor that produced this log block.
* @param sessionId the session ID to isolate the logs.
* @param workerId the worker ID to distinguish the Python worker process.
*/
@DeveloperApi
case class PythonWorkerLogBlockId(
lastLogTime: Long,
executorId: String,
sessionId: String,
workerId: String)
extends LogBlockId {
override def name: String = {
s"python_worker_log_${lastLogTime}_${executorId}_${sessionId}_$workerId"
}

override def logBlockType: LogBlockType = LogBlockType.PYTHON_WORKER
}

/** Id associated with temporary local data managed as blocks. Not serializable. */
private[spark] case class TempLocalBlockId(id: UUID) extends BlockId {
override def name: String = "temp_local_" + id
Expand Down Expand Up @@ -260,6 +285,7 @@ object BlockId {
val TEMP_SHUFFLE = "temp_shuffle_([-A-Fa-f0-9]+)".r
val TEST = "test_(.*)".r
val TEST_LOG_BLOCK = "test_log_([0-9]+)_(.*)".r
val PYTHON_WORKER_LOG_BLOCK = "python_worker_log_([0-9]+)_([^_]*)_([^_]*)_([^_]*)".r

def apply(name: String): BlockId = name match {
case RDD(rddId, splitIndex) =>
Expand Down Expand Up @@ -302,6 +328,8 @@ object BlockId {
TempShuffleBlockId(UUID.fromString(uuid))
case TEST_LOG_BLOCK(lastLogTime, executorId) =>
TestLogBlockId(lastLogTime.toLong, executorId)
case PYTHON_WORKER_LOG_BLOCK(lastLogTime, executorId, sessionId, workerId) =>
PythonWorkerLogBlockId(lastLogTime.toLong, executorId, sessionId, workerId)
case TEST(value) =>
TestBlockId(value)
case _ => throw SparkCoreErrors.unrecognizedBlockIdError(name)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,3 +47,14 @@ trait LogBlockIdGenerator {
blockId
}
}

class PythonWorkerLogBlockIdGenerator(
sessionId: String,
workerId: String)
extends LogBlockIdGenerator {

override def logBlockType: LogBlockType = LogBlockType.PYTHON_WORKER

override protected def genUniqueBlockId(lastLogTime: Long, executorId: String): LogBlockId =
PythonWorkerLogBlockId(lastLogTime, executorId, sessionId, workerId)
}
5 changes: 5 additions & 0 deletions core/src/main/scala/org/apache/spark/storage/LogLine.scala
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@ object LogLine {
logBlockType match {
case LogBlockType.TEST =>
classTag[TestLogLine]
case LogBlockType.PYTHON_WORKER =>
classTag[PythonWorkerLogLine]
case unsupportedLogBlockType =>
throw new RuntimeException("Not supported log type " + unsupportedLogBlockType)
}
Expand All @@ -46,3 +48,6 @@ object LogLine {
case class TestLogLine(eventTime: Long, sequenceId: Long, message: String)
extends LogLine {
}

case class PythonWorkerLogLine(eventTime: Long, sequenceId: Long, message: String)
extends LogLine
Loading