kiszk commented on a change in pull request #23171: [SPARK-26205][SQL] Optimize
In for bytes, shorts, ints
URL: https://github.com/apache/spark/pull/23171#discussion_r241939270
##########
File path:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
##########
@@ -335,6 +343,41 @@ case class In(value: Expression, list: Seq[Expression])
extends Predicate {
""".stripMargin)
}
+ private def genCodeWithSwitch(ctx: CodegenContext, ev: ExprCode): ExprCode =
{
+ val (nullLiterals, nonNullLiterals) = list.partition {
+ case Literal(null, _) => true
+ case _ => false
+ }
+ val listGen = nonNullLiterals.map(_.genCode(ctx))
+ val valueGen = value.genCode(ctx)
+
+ val caseBranches = listGen.map(literal =>
+ s"""
+ |case ${literal.value}:
+ | ${ev.value} = true;
+ | break;
+ """.stripMargin)
+
+ ev.copy(code =
+ code"""
+ |${valueGen.code}
+ |${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = ${valueGen.isNull};
+ |${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = false;
+ |if (!${valueGen.isNull}) {
+ | switch (${valueGen.value}) {
+ | ${caseBranches.mkString("")}
+ | default:
+ | ${ev.isNull} = ${nullLiterals.nonEmpty};
+ | }
+ |}
+ """.stripMargin)
+ }
+
+ private def isSwitchCompatible: Boolean = list.forall {
Review comment:
Could you please take care of the following limitation of Java switch
statement, too?
> npairs pairs of signed 32-bit values
https://docs.oracle.com/javase/specs/jvms/se8/html/jvms-6.html#jvms-6.5.lookupswitch
https://docs.oracle.com/javase/specs/jvms/se8/html/jvms-6.html#jvms-6.5.tableswitch
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]