@@ -28,7 +28,7 @@ import org.apache.hadoop.hive.serde.serdeConstants
28
28
import org .apache .hadoop .hive .serde2 .AbstractSerDe
29
29
import org .apache .hadoop .hive .serde2 .objectinspector ._
30
30
31
- import org .apache .spark .Logging
31
+ import org .apache .spark .{ TaskContext , Logging }
32
32
import org .apache .spark .rdd .RDD
33
33
import org .apache .spark .sql .catalyst .InternalRow
34
34
import org .apache .spark .sql .catalyst .CatalystTypeConverters
@@ -98,7 +98,8 @@ case class ScriptTransformation(
98
98
ioschema,
99
99
outputStream,
100
100
proc,
101
- stderrBuffer
101
+ stderrBuffer,
102
+ TaskContext .get()
102
103
)
103
104
104
105
// This nullability is a performance optimization in order to avoid an Option.foreach() call
@@ -221,7 +222,8 @@ private class ScriptTransformationWriterThread(
221
222
ioschema : HiveScriptIOSchema ,
222
223
outputStream : OutputStream ,
223
224
proc : Process ,
224
- stderrBuffer : CircularBuffer
225
+ stderrBuffer : CircularBuffer ,
226
+ taskContext : TaskContext
225
227
) extends Thread (" Thread-ScriptTransformation-Feed" ) with Logging {
226
228
227
229
setDaemon(true )
@@ -232,6 +234,8 @@ private class ScriptTransformationWriterThread(
232
234
def exception : Option [Throwable ] = Option (_exception)
233
235
234
236
override def run (): Unit = Utils .logUncaughtExceptions {
237
+ TaskContext .setTaskContext(taskContext)
238
+
235
239
val dataOutputStream = new DataOutputStream (outputStream)
236
240
237
241
// We can't use Utils.tryWithSafeFinally here because we also need a `catch` block, so
0 commit comments