Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Next Next commit
SPARK-25004: Access SparkEnv.get.conf in PythonRunner.
  • Loading branch information
rdblue committed Aug 23, 2018
commit fcee94c22bebf1d55ba8ab9091b4bcb9852717b2
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ private[spark] class PythonRDD(
val asJavaRDD: JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)

override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
val runner = PythonRunner(func, SparkEnv.get.conf)
val runner = PythonRunner(func)
runner.compute(firstParent.iterator(split, context), split.index, context)
}

Expand Down
20 changes: 8 additions & 12 deletions core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@ import scala.collection.JavaConverters._

import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.security.SocketAuthHelper
import org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY
import org.apache.spark.security.SocketAuthHelper
import org.apache.spark.util._


Expand Down Expand Up @@ -64,12 +64,12 @@ private[spark] object PythonEvalType {
private[spark] abstract class BasePythonRunner[IN, OUT](
funcs: Seq[ChainedPythonFunctions],
evalType: Int,
argOffsets: Array[Array[Int]],
conf: SparkConf)
argOffsets: Array[Array[Int]])
extends Logging {

require(funcs.length == argOffsets.length, "argOffsets should have the same length as funcs")

private val conf = SparkEnv.get.conf
private val bufferSize = conf.getInt("spark.buffer.size", 65536)
private val reuseWorker = conf.getBoolean("spark.python.worker.reuse", true)
// each python worker gets an equal part of the allocation. the worker pool will grow to the
Expand All @@ -89,7 +89,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
private[spark] var serverSocket: Option[ServerSocket] = None

// Authentication helper used when serving method calls via socket from Python side.
private lazy val authHelper = new SocketAuthHelper(SparkEnv.get.conf)
private lazy val authHelper = new SocketAuthHelper(conf)

def compute(
inputIterator: Iterator[IN],
Expand Down Expand Up @@ -495,21 +495,17 @@ private[spark] abstract class BasePythonRunner[IN, OUT](

private[spark] object PythonRunner {

def apply(
func: PythonFunction,
conf: SparkConf): PythonRunner = {
new PythonRunner(Seq(ChainedPythonFunctions(Seq(func))), conf)
def apply(func: PythonFunction): PythonRunner = {
new PythonRunner(Seq(ChainedPythonFunctions(Seq(func))))
}
}

/**
* A helper class to run Python mapPartition in Spark.
*/
private[spark] class PythonRunner(
funcs: Seq[ChainedPythonFunctions],
conf: SparkConf)
private[spark] class PythonRunner(funcs: Seq[ChainedPythonFunctions])
extends BasePythonRunner[Array[Byte], Array[Byte]](
funcs, PythonEvalType.NON_UDF, Array(Array(0)), conf) {
funcs, PythonEvalType.NON_UDF, Array(Array(0))) {

protected override def newWriterThread(
env: SparkEnv,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -139,8 +139,7 @@ case class AggregateInPandasExec(
argOffsets,
aggInputSchema,
sessionLocalTimeZone,
pythonRunnerConf,
SparkEnv.get.conf).compute(projectedRowIter, context.partitionId(), context)
pythonRunnerConf).compute(projectedRowIter, context.partitionId(), context)

val joinedAttributes =
groupingExpressions.map(_.toAttribute) ++ udfExpressions.map(_.resultAttribute)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.python

import scala.collection.JavaConverters._

import org.apache.spark.{SparkEnv, TaskContext}
import org.apache.spark.TaskContext
import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
Expand Down Expand Up @@ -84,8 +84,7 @@ case class ArrowEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi
argOffsets,
schema,
sessionLocalTimeZone,
pythonRunnerConf,
SparkEnv.get.conf).compute(batchIter, context.partitionId(), context)
pythonRunnerConf).compute(batchIter, context.partitionId(), context)

new Iterator[InternalRow] {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,10 +43,9 @@ class ArrowPythonRunner(
argOffsets: Array[Array[Int]],
schema: StructType,
timeZoneId: String,
conf: Map[String, String],
sparkConf: SparkConf)
conf: Map[String, String])
extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch](
funcs, evalType, argOffsets, sparkConf) {
funcs, evalType, argOffsets) {

protected override def newWriterThread(
env: SparkEnv,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._

import net.razorvine.pickle.{Pickler, Unpickler}

import org.apache.spark.{SparkEnv, TaskContext}
import org.apache.spark.TaskContext
import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
Expand Down Expand Up @@ -66,8 +66,7 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi
}.grouped(100).map(x => pickle.dumps(x.toArray))

// Output iterator for results from Python.
val outputIterator = new PythonUDFRunner(
funcs, PythonEvalType.SQL_BATCHED_UDF, argOffsets, SparkEnv.get.conf)
val outputIterator = new PythonUDFRunner(funcs, PythonEvalType.SQL_BATCHED_UDF, argOffsets)
.compute(inputIterator, context.partitionId(), context)

val unpickle = new Unpickler
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.python
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer

import org.apache.spark.{SparkEnv, TaskContext}
import org.apache.spark.TaskContext
import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
Expand Down Expand Up @@ -143,8 +143,7 @@ case class FlatMapGroupsInPandasExec(
argOffsets,
dedupSchema,
sessionLocalTimeZone,
pythonRunnerConf,
SparkEnv.get.conf).compute(grouped, context.partitionId(), context)
pythonRunnerConf).compute(grouped, context.partitionId(), context)

columnarBatchIter.flatMap(_.rowIterator.asScala).map(UnsafeProjection.create(output, output))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ class PythonForeachWriter(func: PythonFunction, schema: StructType)
}

private lazy val pythonRunner = {
PythonRunner(func, SparkEnv.get.conf)
PythonRunner(func)
}

private lazy val outputIterator =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,10 +30,9 @@ import org.apache.spark.api.python._
class PythonUDFRunner(
funcs: Seq[ChainedPythonFunctions],
evalType: Int,
argOffsets: Array[Array[Int]],
conf: SparkConf)
argOffsets: Array[Array[Int]])
extends BasePythonRunner[Array[Byte], Array[Byte]](
funcs, evalType, argOffsets, conf) {
funcs, evalType, argOffsets) {

protected override def newWriterThread(
env: SparkEnv,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -158,8 +158,7 @@ case class WindowInPandasExec(
argOffsets,
windowInputSchema,
sessionLocalTimeZone,
pythonRunnerConf,
SparkEnv.get.conf).compute(pythonInput, context.partitionId(), context)
pythonRunnerConf).compute(pythonInput, context.partitionId(), context)

val joined = new JoinedRow
val resultProj = createResultProjection(expressions)
Expand Down