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

    https://github.com/apache/spark/pull/3505#discussion_r22774231
  
    --- Diff: core/src/main/scala/org/apache/spark/rdd/SkewedJoinRDD.scala ---
    @@ -0,0 +1,345 @@
    +/*
    + * 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.rdd
    +
    +import java.io.{ObjectOutputStream, IOException}
    +
    +import scala.reflect.ClassTag
    +
    +import org.apache.spark._
    +import org.apache.spark.serializer.Serializer
    +import org.apache.spark.shuffle.ShuffleHandle
    +import org.apache.spark.util.Utils
    +import org.apache.spark.util.collection._
    +
    +private[spark] sealed trait JoinType[K, L, R, PAIR <: Product2[_, _]] 
extends Serializable {
    +
    +  def flatten(i: Iterator[(K, (Iterable[Chunk[L]], Iterable[Chunk[R]]))]): 
Iterator[(K, PAIR)]
    +
    +}
    +
    +private[spark] object JoinType {
    +
    +  def inner[K, L, R] = new JoinType[K, L, R, (L, R)] {
    +
    +    override def flatten(i: Iterator[(K, (Iterable[Chunk[L]], 
Iterable[Chunk[R]]))]) =
    +      i flatMap {
    +        case (key, pair) => {
    +          if (pair._1.size < pair._2.size) {
    +            yieldPair(pair._1, pair._2, key, (p1: L, p2: R) => (p1, p2))
    +          } else {
    +            yieldPair(pair._2, pair._1, key, (p2: R, p1: L) => (p1, p2))
    +          }
    +        }
    +      }
    +  }
    +
    +  def leftOuter[K, L, R] = new JoinType[K, L, R, (L, Option[R])] {
    +
    +    override def flatten(i: Iterator[(K, (Iterable[Chunk[L]], 
Iterable[Chunk[R]]))]) =
    +      i flatMap {
    +        case (key, pair) => {
    +          if (pair._2.size == 0) {
    +            for (chunk <- pair._1.iterator;
    +                 v <- chunk
    +            ) yield (key, (v, None)): (K, (L, Option[R]))
    +          }
    +          else if (pair._1.size < pair._2.size) {
    +            yieldPair(pair._1, pair._2, key, (p1: L, p2: R) => (p1, 
Some(p2)))
    +          } else {
    +            yieldPair(pair._2, pair._1, key, (p2: R, p1: L) => (p1, 
Some(p2)))
    +          }
    +        }
    +      }
    +  }
    +
    +  def rightOuter[K, L, R] = new JoinType[K, L, R, (Option[L], R)] {
    +
    +    override def flatten(i: Iterator[(K, (Iterable[Chunk[L]], 
Iterable[Chunk[R]]))]) =
    +      i flatMap {
    +        case (key, pair) => {
    +          if (pair._1.size == 0) {
    +            for (chunk <- pair._2.iterator;
    +                 v <- chunk
    +            ) yield (key, (None, v)): (K, (Option[L], R))
    +          }
    +          else if (pair._1.size < pair._2.size) {
    +            yieldPair(pair._1, pair._2, key, (p1: L, p2: R) => (Some(p1), 
p2))
    +          } else {
    +            yieldPair(pair._2, pair._1, key, (p2: R, p1: L) => (Some(p1), 
p2))
    +          }
    +        }
    +      }
    +  }
    +
    +  def fullOuter[K, L, R] = new JoinType[K, L, R, (Option[L], Option[R])] {
    +
    +    override def flatten(i: Iterator[(K, (Iterable[Chunk[L]], 
Iterable[Chunk[R]]))]) =
    +      i flatMap {
    +        case (key, pair) => {
    +          if (pair._1.size == 0) {
    +            for (chunk <- pair._2.iterator;
    +                 v <- chunk
    +            ) yield (key, (None, Some(v))): (K, (Option[L], Option[R]))
    +          }
    +          else if (pair._2.size == 0) {
    +            for (chunk <- pair._1.iterator;
    +                 v <- chunk
    +            ) yield (key, (Some(v), None)): (K, (Option[L], Option[R]))
    +          }
    +          else if (pair._1.size < pair._2.size) {
    +            yieldPair(pair._1, pair._2, key, (p1: L, p2: R) => (Some(p1), 
Some(p2)))
    +          } else {
    +            yieldPair(pair._2, pair._1, key, (p2: R, p1: L) => (Some(p1), 
Some(p2)))
    +          }
    +        }
    +      }
    +  }
    +
    +  private def yieldPair[K, OUT, IN, PAIR <: Product2[_, _]](
    +      outer: Iterable[Chunk[OUT]], inner: Iterable[Chunk[IN]], key: K, 
toPair: (OUT, IN) => PAIR) =
    +    for (
    +      outerChunk <- outer.iterator;
    +      innerChunk <- inner.iterator;
    +      outerValue <- outerChunk;
    +      innerValue <- innerChunk
    +    ) yield (key, toPair(outerValue, innerValue))
    +}
    +
    +private[spark] sealed trait JoinSplitDep extends Serializable
    +
    +private[spark] case class NarrowJoinSplitDep(
    +    rdd: RDD[_],
    +    splitIndex: Int,
    +    var split: Partition) extends JoinSplitDep {
    +
    +  @throws(classOf[IOException])
    +  private def writeObject(oos: ObjectOutputStream): Unit = 
Utils.tryOrIOException {
    +    // Update the reference to parent split at the time of task 
serialization
    +    split = rdd.partitions(splitIndex)
    +    oos.defaultWriteObject()
    +  }
    +}
    +
    +private[spark] case class ShuffleJoinSplitDep(handle: ShuffleHandle) 
extends JoinSplitDep
    +
    +private[spark] class JoinPartition(idx: Int, val left: JoinSplitDep, val 
right: JoinSplitDep)
    +  extends Partition with Serializable {
    +  override val index: Int = idx
    +
    +  override def hashCode(): Int = idx
    +}
    +
    +private[spark] class BufferWrapper private(val isChunkBuffer: Boolean, 
buffer: Iterable[Any])
    +  extends Serializable {
    +
    +  def this(buffer: CompactBuffer[_]) {
    +    this(false, buffer)
    +  }
    +
    +  def this(buffer: ChunkBuffer[_]) {
    +    this(true, buffer)
    +  }
    +
    +  def getChunkBuffer[T: ClassTag]: ChunkBuffer[T] = 
buffer.asInstanceOf[ChunkBuffer[T]]
    +
    +  def getCompactBuffer[T: ClassTag]: CompactBuffer[T] = 
buffer.asInstanceOf[CompactBuffer[T]]
    +
    +  def asChunkIterable[T: ClassTag]: Iterable[Chunk[T]] = {
    +    if (isChunkBuffer) {
    +      getChunkBuffer[T]
    +    }
    +    else {
    +      val buffer = getCompactBuffer[T]
    +      if (buffer.isEmpty) {
    +        Iterable[Chunk[T]]()
    +      }
    +      else {
    +        Iterable(new Chunk[T](buffer))
    +      }
    +    }
    +  }
    +}
    +
    +private[spark] class SkewedJoinRDD[K, L, R, PAIR <: Product2[_, _]](
    +    left: RDD[(K, L)], right: RDD[(K, R)], part: Partitioner, joinType: 
JoinType[K, L, R, PAIR])
    +    (implicit kt: ClassTag[K], lt: ClassTag[L], rt: ClassTag[R], ord: 
Ordering[K])
    --- End diff --
    
    no, i just replace ExternalAppendOnlyMap with ExternalOrderingAppendOnlyMap 
in Aggregator. but i find ExternalOrderingAppendOnlyMap is not common for 
skewedGroupby and skewedJoin. in skewedJoin key in 
ExternalOrderingAppendOnlyMap  is a ordering. how about we can make 
ExternalOrderingAppendOnlyMap be general, all of skewedJoin and  skewedGroupby 
can resolve skew problem using it? @zsxwing


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to