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

    https://github.com/apache/spark/pull/21560#discussion_r198222671
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala
 ---
    @@ -0,0 +1,108 @@
    +/*
    + * 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.sql.execution.streaming.continuous
    +
    +import org.apache.spark._
    +import org.apache.spark.rdd.{CoalescedRDDPartition, RDD}
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.expressions.UnsafeRow
    +import org.apache.spark.sql.execution.streaming.continuous.shuffle._
    +import org.apache.spark.util.ThreadUtils
    +
    +case class ContinuousCoalesceRDDPartition(index: Int) extends Partition {
    +  // This flag will be flipped on the executors to indicate that the 
threads processing
    +  // partitions of the write-side RDD have been started. These will run 
indefinitely
    +  // asynchronously as epochs of the coalesce RDD complete on the read 
side.
    +  private[continuous] var writersInitialized: Boolean = false
    +}
    +
    +/**
    + * RDD for continuous coalescing. Asynchronously writes all partitions of 
`prev` into a local
    + * continuous shuffle, and then reads them in the task thread using 
`reader`.
    + */
    +class ContinuousCoalesceRDD(
    +    context: SparkContext,
    +    numPartitions: Int,
    +    readerQueueSize: Int,
    +    epochIntervalMs: Long,
    +    readerEndpointName: String,
    +    prev: RDD[InternalRow])
    +  extends RDD[InternalRow](context, Nil) {
    +
    +  override def getPartitions: Array[Partition] = 
Array(ContinuousCoalesceRDDPartition(0))
    +
    + val readerRDD = new ContinuousShuffleReadRDD(
    +    sparkContext,
    +    numPartitions,
    +    readerQueueSize,
    +    prev.getNumPartitions,
    +    epochIntervalMs,
    +    Seq(readerEndpointName))
    +
    +  private lazy val threadPool = ThreadUtils.newDaemonFixedThreadPool(
    +    prev.getNumPartitions,
    +    this.name)
    +
    +  override def compute(split: Partition, context: TaskContext): 
Iterator[InternalRow] = {
    +    assert(split.index == 0)
    +    // lazy initialize endpoint so writer can send to it
    +    
readerRDD.partitions(0).asInstanceOf[ContinuousShuffleReadPartition].endpoint
    +
    +    if 
(!split.asInstanceOf[ContinuousCoalesceRDDPartition].writersInitialized) {
    +      val rpcEnv = SparkEnv.get.rpcEnv
    +      val outputPartitioner = new HashPartitioner(1)
    +      val endpointRefs = readerRDD.endpointNames.map { endpointName =>
    +          rpcEnv.setupEndpointRef(rpcEnv.address, endpointName)
    +      }
    +
    +      val runnables = prev.partitions.map { prevSplit =>
    +        new Runnable() {
    +          override def run(): Unit = {
    +            TaskContext.setTaskContext(context)
    +
    +            val writer: ContinuousShuffleWriter = new 
RPCContinuousShuffleWriter(
    +              prevSplit.index, outputPartitioner, endpointRefs.toArray)
    +
    +            EpochTracker.initializeCurrentEpoch(
    +              
context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong)
    +            while (!context.isInterrupted() && !context.isCompleted()) {
    +              writer.write(prev.compute(prevSplit, 
context).asInstanceOf[Iterator[UnsafeRow]])
    +              // Note that current epoch is a non-inheritable thread 
local, so each writer thread
    +              // can properly increment its own epoch without affecting 
the main task thread.
    +              EpochTracker.incrementCurrentEpoch()
    +            }
    +          }
    +        }
    +      }
    +
    +      context.addTaskCompletionListener { ctx =>
    +        threadPool.shutdownNow()
    +      }
    +
    +      
split.asInstanceOf[ContinuousCoalesceRDDPartition].writersInitialized = true
    +
    +      runnables.foreach(threadPool.execute)
    +    }
    +
    +    readerRDD.compute(readerRDD.partitions(split.index), context)
    --- End diff --
    
    Agree. Also the 2*numShuffle partition threads here looks like an overhead. 
Maybe ok for now but the CoalesceRDD iterator could just iterate over the 
parent RDD partitions tracking the epochs, returning the rows and terminating 
when the epoch is received from all its parents.


---

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

Reply via email to