spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From rxin <...@git.apache.org>
Subject [GitHub] spark pull request: [SPARK-13237] [SQL] generated broadcast outer ...
Date Sun, 14 Feb 2016 06:50:55 GMT
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11130#discussion_r52838672
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
---
    @@ -184,36 +240,106 @@ case class BroadcastHashJoin(
     
         if (broadcastRelation.value.isInstanceOf[UniqueHashedRelation]) {
           s"""
    -         | // generate join key
    -         | ${keyVal.code}
    -         | // find matches from HashedRelation
    -         | UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyVal.value});
    -         | if ($matched != null) {
    -         |   ${buildColumns.map(_.code).mkString("\n")}
    -         |   $outputCode
    -         | }
    -     """.stripMargin
    +         |// generate join key
    +         |${keyEv.code}
    +         |// find matches from HashedRelation
    +         |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value});
    +         |if ($matched != null) {
    +         |  ${buildVars.map(_.code).mkString("\n")}
    +         |  $outputCode
    +         |}
    +       """.stripMargin
    +
    +    } else {
    +      val matches = ctx.freshName("matches")
    +      val bufferType = classOf[CompactBuffer[UnsafeRow]].getName
    +      val i = ctx.freshName("i")
    +      val size = ctx.freshName("size")
    +      s"""
    +         |// generate join key
    +         |${keyEv.code}
    +         |// find matches from HashRelation
    +         |$bufferType $matches = $anyNull ? null : ($bufferType)$relationTerm.get(${keyEv.value});
    +         |if ($matches != null) {
    +         |  int $size = $matches.size();
    +         |  for (int $i = 0; $i < $size; $i++) {
    +         |    UnsafeRow $matched = (UnsafeRow) $matches.apply($i);
    +         |    ${buildVars.map(_.code).mkString("\n")}
    +         |    $outputCode
    +         |  }
    +         |}
    +       """.stripMargin
    +    }
    +  }
    +
    +
    +  private def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = {
    +    val (broadcastRelation, relationTerm) = prepareBroadcast(ctx)
    +    val (keyEv, anyNull) = genJoinKey(ctx, input)
    +    val matched = ctx.freshName("matched")
    +    val buildVars = genBuildSideVars(ctx, matched)
    +    val resultVars = buildSide match {
    +      case BuildLeft => buildVars ++ input
    +      case BuildRight => input ++ buildVars
    +    }
    +    val numOutput = metricTerm(ctx, "numOutputRows")
    +
    +    // filter the output via condition
    +    val conditionPassed = ctx.freshName("conditionPassed")
    +    val checkCondition = if (condition.isDefined) {
    +      ctx.currentVars = resultVars
    +      val ev = BindReferences.bindReference(condition.get, this.output).gen(ctx)
    +      s"""
    +         |boolean $conditionPassed = true;
    +         |if ($matched != null) {
    +         |  ${ev.code}
    +         |  $conditionPassed = !${ev.isNull} && ${ev.value};
    +         |}
    +       """.stripMargin
    +    } else {
    +      s"final boolean $conditionPassed = true;"
    +    }
    +
    +    if (broadcastRelation.value.isInstanceOf[UniqueHashedRelation]) {
    +      s"""
    +         |// generate join key
    +         |${keyEv.code}
    +         |// find matches from HashedRelation
    +         |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value});
    +         |${buildVars.map(_.code).mkString("\n")}
    +         |${checkCondition.trim}
    +         |if (!$conditionPassed) {
    +         |  // reset to null
    +         |  ${buildVars.map(v => s"${v.isNull} = true;").mkString("\n")}
    +         |}
    +         |$numOutput.add(1);
    +         |${consume(ctx, resultVars)}
    +       """.stripMargin
     
         } else {
           val matches = ctx.freshName("matches")
           val bufferType = classOf[CompactBuffer[UnsafeRow]].getName
           val i = ctx.freshName("i")
           val size = ctx.freshName("size")
    +      val found = ctx.freshName("found")
           s"""
    -         | // generate join key
    -         | ${keyVal.code}
    -         | // find matches from HashRelation
    -         | $bufferType $matches = ${anyNull} ? null :
    -         |  ($bufferType) $relationTerm.get(${keyVal.value});
    -         | if ($matches != null) {
    -         |   int $size = $matches.size();
    -         |   for (int $i = 0; $i < $size; $i++) {
    -         |     UnsafeRow $matched = (UnsafeRow) $matches.apply($i);
    -         |     ${buildColumns.map(_.code).mkString("\n")}
    -         |     $outputCode
    -         |   }
    -         | }
    -     """.stripMargin
    +         |// generate join key
    +         |${keyEv.code}
    +         |// find matches from HashRelation
    +         |$bufferType $matches = $anyNull ? null : ($bufferType)$relationTerm.get(${keyEv.value});
    +         |int $size = $matches != null ? $matches.size() : 0;
    +         |boolean $found = false;
    +         |for (int $i = 0; $i <= $size; $i++) {
    --- End diff --
    
    this is clever, but i think you need to document it (i.e. you are adding an extra iteration
at the end of the loop to handle null)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


Mime
View raw message