Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/5096#discussion_r26809287
  
    --- Diff: core/src/main/scala/org/apache/spark/api/r/RRDD.scala ---
    @@ -0,0 +1,515 @@
    +/*
    + * 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.r
    +
    +import java.io._
    +import java.net.ServerSocket
    +import java.util.{Map => JMap}
    +
    +import scala.collection.JavaConversions._
    +import scala.io.Source
    +import scala.reflect.ClassTag
    +import scala.util.Try
    +
    +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext}
    +import org.apache.spark.broadcast.Broadcast
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark._
    +
    +private abstract class BaseRRDD[T: ClassTag, U: ClassTag](
    +    parent: RDD[T],
    +    numPartitions: Int,
    +    func: Array[Byte],
    +    deserializer: String,
    +    serializer: String,
    +    packageNames: Array[Byte],
    +    rLibDir: String,
    +    broadcastVars: Array[Broadcast[Object]])
    +  extends RDD[U](parent) with Logging {
    +  override def getPartitions = parent.partitions
    +
    +  override def compute(split: Partition, context: TaskContext): 
Iterator[U] = {
    +
    +    // The parent may be also an RRDD, so we should launch it first.
    +    val parentIterator = firstParent[T].iterator(split, context)
    +
    +    // we expect two connections
    +    val serverSocket = new ServerSocket(0, 2)
    +    val listenPort = serverSocket.getLocalPort()
    +
    +    // The stdout/stderr is shared by multiple tasks, because we use one 
daemon
    +    // to launch child process as worker.
    +    val errThread = RRDD.createRWorker(rLibDir, listenPort)
    +
    +    // We use two sockets to separate input and output, then it's easy to 
manage
    +    // the lifecycle of them to avoid deadlock.
    +    // TODO: optimize it to use one socket
    +
    +    // the socket used to send out the input of task
    +    serverSocket.setSoTimeout(10000)
    +    val inSocket = serverSocket.accept()
    +    startStdinThread(inSocket.getOutputStream(), parentIterator, 
split.index)
    +
    +    // the socket used to receive the output of task
    +    val outSocket = serverSocket.accept()
    +    val inputStream = new BufferedInputStream(outSocket.getInputStream)
    +    val dataStream = openDataStream(inputStream)
    +    serverSocket.close()
    +
    +    try {
    +
    +      return new Iterator[U] {
    +        def next(): U = {
    +          val obj = _nextObj
    +          if (hasNext) {
    +            _nextObj = read()
    +          }
    +          obj
    +        }
    +
    +        var _nextObj = read()
    +
    +        def hasNext(): Boolean = {
    +          val hasMore = (_nextObj != null)
    +          if (!hasMore) {
    +            dataStream.close()
    +          }
    +          hasMore
    +        }
    +      }
    +    } catch {
    +      case e: Exception =>
    +        throw new SparkException("R computation failed with\n " + 
errThread.getLines())
    +    }
    +  }
    +
    +  /**
    +   * Start a thread to write RDD data to the R process.
    +   */
    +  private def startStdinThread[T](
    +    output: OutputStream,
    +    iter: Iterator[T],
    +    splitIndex: Int) = {
    +
    +    val env = SparkEnv.get
    +    val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
    +    val stream = new BufferedOutputStream(output, bufferSize)
    +
    +    new Thread("writer for R") {
    +      override def run() {
    +        try {
    +          SparkEnv.set(env)
    +          val dataOut = new DataOutputStream(stream)
    +          dataOut.writeInt(splitIndex)
    +
    +          SerDe.writeString(dataOut, deserializer)
    +          SerDe.writeString(dataOut, serializer)
    +
    +          dataOut.writeInt(packageNames.length)
    +          dataOut.write(packageNames)
    +
    +          dataOut.writeInt(func.length)
    +          dataOut.write(func)
    +
    +          dataOut.writeInt(broadcastVars.length)
    +          broadcastVars.foreach { broadcast =>
    +            // TODO(shivaram): Read a Long in R to avoid this cast
    +            dataOut.writeInt(broadcast.id.toInt)
    +            // TODO: Pass a byte array from R to avoid this cast ?
    +            val broadcastByteArr = 
broadcast.value.asInstanceOf[Array[Byte]]
    +            dataOut.writeInt(broadcastByteArr.length)
    +            dataOut.write(broadcastByteArr)
    +          }
    +
    +          dataOut.writeInt(numPartitions)
    +
    +          if (!iter.hasNext) {
    +            dataOut.writeInt(0)
    +          } else {
    +            dataOut.writeInt(1)
    +          }
    +
    +          val printOut = new PrintStream(stream)
    +
    +          def writeElem(elem: Any): Unit = {
    +            if (deserializer == SerializationFormats.BYTE) {
    +              val elemArr = elem.asInstanceOf[Array[Byte]]
    +              dataOut.writeInt(elemArr.length)
    +              dataOut.write(elemArr)
    +            } else if (deserializer == SerializationFormats.ROW) {
    +              dataOut.write(elem.asInstanceOf[Array[Byte]])
    +            } else if (deserializer == SerializationFormats.STRING) {
    +              printOut.println(elem)
    +            }
    +          }
    +
    +          for (elem <- iter) {
    +            elem match {
    +              case (key, value) =>
    +                writeElem(key)
    +                writeElem(value)
    +              case _ =>
    +                writeElem(elem)
    +            }
    +          }
    +          stream.flush()
    +        } catch {
    +          // TODO: We should propogate this error to the task thread
    +          case e: Exception =>
    +            logError("R Writer thread got an exception", e)
    +        } finally {
    +          Try(output.close())
    +        }
    +      }
    +    }.start()
    +  }
    +
    +  protected def openDataStream(input: InputStream): Closeable
    +
    +  protected def read(): U
    +}
    +
    +/**
    + * Form an RDD[(Int, Array[Byte])] from key-value pairs returned from R.
    + * This is used by SparkR's shuffle operations.
    + */
    +private class PairwiseRRDD[T: ClassTag](
    +    parent: RDD[T],
    +    numPartitions: Int,
    +    hashFunc: Array[Byte],
    +    deserializer: String,
    +    packageNames: Array[Byte],
    +    rLibDir: String,
    +    broadcastVars: Array[Object])
    +  extends BaseRRDD[T, (Int, Array[Byte])](
    +    parent, numPartitions, hashFunc, deserializer,
    +    SerializationFormats.BYTE, packageNames, rLibDir,
    +    broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) {
    +
    +  private var dataStream: DataInputStream = _
    +
    +  override protected def openDataStream(input: InputStream) = {
    +    dataStream = new DataInputStream(input)
    +    dataStream
    +  }
    +
    +  override protected def read(): (Int, Array[Byte]) = {
    +    try {
    +      val length = dataStream.readInt()
    +
    +      length match {
    +        case length if length == 2 =>
    +          val hashedKey = dataStream.readInt()
    +          val contentPairsLength = dataStream.readInt()
    +          val contentPairs = new Array[Byte](contentPairsLength)
    +          dataStream.readFully(contentPairs)
    +          (hashedKey, contentPairs)
    +        case _ => null   // End of input
    +      }
    +    } catch {
    +      case eof: EOFException => {
    +        throw new SparkException("R worker exited unexpectedly (crashed)", 
eof)
    +      }
    +    }
    +  }
    +
    +  lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = 
JavaPairRDD.fromRDD(this)
    +}
    +
    +/**
    + * An RDD that stores serialized R objects as Array[Byte].
    + */
    +private class RRDD[T: ClassTag](
    +    parent: RDD[T],
    +    func: Array[Byte],
    +    deserializer: String,
    +    serializer: String,
    +    packageNames: Array[Byte],
    +    rLibDir: String,
    +    broadcastVars: Array[Object])
    +  extends BaseRRDD[T, Array[Byte]](parent, -1, func, deserializer,
    +                                   serializer, packageNames, rLibDir,
    +                                   broadcastVars.map(x => 
x.asInstanceOf[Broadcast[Object]])) {
    +
    +  private var dataStream: DataInputStream = _
    +
    +  override protected def openDataStream(input: InputStream) = {
    +    dataStream = new DataInputStream(input)
    +    dataStream
    +  }
    +
    +  override protected def read(): Array[Byte] = {
    +    try {
    +      val length = dataStream.readInt()
    +
    +      length match {
    +        case length if length > 0 =>
    +          val obj = new Array[Byte](length)
    +          dataStream.readFully(obj, 0, length)
    +          obj
    +        case _ => null
    +      }
    +    } catch {
    +      case eof: EOFException => {
    +        throw new SparkException("R worker exited unexpectedly (crashed)", 
eof)
    +      }
    +    }
    +  }
    +
    +  lazy val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
    +}
    +
    +/**
    + * An RDD that stores R objects as Array[String].
    + */
    +private class StringRRDD[T: ClassTag](
    +    parent: RDD[T],
    +    func: Array[Byte],
    +    deserializer: String,
    +    packageNames: Array[Byte],
    +    rLibDir: String,
    +    broadcastVars: Array[Object])
    +  extends BaseRRDD[T, String](parent, -1, func, deserializer, 
SerializationFormats.STRING,
    +                              packageNames, rLibDir,
    +                              broadcastVars.map(x => 
x.asInstanceOf[Broadcast[Object]])) {
    +
    +  private var dataStream: BufferedReader = _
    +
    +  override protected def openDataStream(input: InputStream) = {
    +    dataStream = new BufferedReader(new InputStreamReader(input))
    +    dataStream
    +  }
    +
    +  override protected def read(): String = {
    +    try {
    +      dataStream.readLine()
    +    } catch {
    +      case e: IOException => {
    +        throw new SparkException("R worker exited unexpectedly (crashed)", 
e)
    +      }
    +    }
    +  }
    +
    +  lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this)
    +}
    +
    +private[r] class BufferedStreamThread(
    +    in: InputStream,
    +    name: String,
    +    errBufferSize: Int) extends Thread(name) with Logging {
    +  val lines = new Array[String](errBufferSize)
    +  var lineIdx = 0
    +  override def run() {
    +    for (line <- Source.fromInputStream(in).getLines) {
    +      synchronized {
    +        lines(lineIdx) = line
    +        lineIdx = (lineIdx + 1) % errBufferSize
    +      }
    +      logInfo(line)
    +    }
    +  }
    +
    +  def getLines(): String = synchronized {
    +    (0 until errBufferSize).filter { x =>
    +      lines((x + lineIdx) % errBufferSize) != null
    +    }.map { x =>
    +      lines((x + lineIdx) % errBufferSize)
    +    }.mkString("\n")
    +  }
    +}
    +
    +private[r] object RRDD {
    +  // Because forking processes from Java is expensive, we prefer to launch
    +  // a single R daemon (daemon.R) and tell it to fork new workers for our 
tasks.
    +  // This daemon currently only works on UNIX-based systems now, so we 
should
    +  // also fall back to launching workers (worker.R) directly.
    +  val inWindows = System.getProperty("os.name").startsWith("Windows")
    --- End diff --
    
    I think we have a `Utils.isWindows` that we can use for this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to