Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-19372][SQL] Fix throwing a Java exception at df.fliter() due to 64KB bytecode size limit #17087

Closed
wants to merge 15 commits into from
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ object ExternalCatalogUtils {
})

inputPartitions.filter { p =>
boundPredicate(p.toRow(partitionSchema, defaultTimeZoneId))
boundPredicate.eval(p.toRow(partitionSchema, defaultTimeZoneId))
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,10 @@ import scala.language.existentials
import scala.util.control.NonFatal

import com.google.common.cache.{CacheBuilder, CacheLoader}
import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, SimpleCompiler}
import com.google.common.util.concurrent.{ExecutionError, UncheckedExecutionException}
import org.apache.commons.lang3.exception.ExceptionUtils
import org.codehaus.commons.compiler.CompileException
import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, JaninoRuntimeException, SimpleCompiler}
import org.codehaus.janino.util.ClassFile

import org.apache.spark.{SparkEnv, TaskContext, TaskKilledException}
Expand Down Expand Up @@ -899,8 +902,14 @@ object CodeGenerator extends Logging {
/**
* Compile the Java source code into a Java class, using Janino.
*/
def compile(code: CodeAndComment): GeneratedClass = {
def compile(code: CodeAndComment): GeneratedClass = try {
cache.get(code)
} catch {
// Cache.get() may wrap the original exception. See the following URL
// http://google.github.io/guava/releases/14.0/api/docs/com/google/common/cache/
// Cache.html#get(K,%20java.util.concurrent.Callable)
case e @ (_: UncheckedExecutionException | _: ExecutionError) =>
throw e.getCause
}

/**
Expand Down Expand Up @@ -951,10 +960,14 @@ object CodeGenerator extends Logging {
evaluator.cook("generated.java", code.body)
recordCompilationStats(evaluator)
} catch {
case e: Exception =>
case e: JaninoRuntimeException =>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like CompileException can be thrown from janino?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for pointing out . Now, CompileException can be caught.

val msg = s"failed to compile: $e\n$formatted"
logError(msg, e)
throw new Exception(msg, e)
throw new JaninoRuntimeException(msg, e)
case e: CompileException =>
val msg = s"failed to compile: $e\n$formatted"
logError(msg, e)
throw new CompileException(msg, e.getLocation)
}
evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass]
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,20 +20,22 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.expressions.codegen.{Predicate => BasePredicate}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.util.TypeUtils
import org.apache.spark.sql.types._


object InterpretedPredicate {
def create(expression: Expression, inputSchema: Seq[Attribute]): (InternalRow => Boolean) =
def create(expression: Expression, inputSchema: Seq[Attribute]): InterpretedPredicate =
create(BindReferences.bindReference(expression, inputSchema))

def create(expression: Expression): (InternalRow => Boolean) = {
(r: InternalRow) => expression.eval(r).asInstanceOf[Boolean]
}
def create(expression: Expression): InterpretedPredicate = new InterpretedPredicate(expression)
}

case class InterpretedPredicate(expression: Expression) extends BasePredicate {
override def eval(r: InternalRow): Boolean = expression.eval(r).asInstanceOf[Boolean]
}

/**
* An [[Expression]] that returns a boolean value.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, Da
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.ExecutionContext

import org.codehaus.commons.compiler.CompileException
import org.codehaus.janino.JaninoRuntimeException

import org.apache.spark.{broadcast, SparkEnv}
import org.apache.spark.internal.Logging
import org.apache.spark.io.CompressionCodec
Expand Down Expand Up @@ -353,9 +356,27 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
GenerateMutableProjection.generate(expressions, inputSchema, useSubexprElimination)
}

private def genInterpretedPredicate(
expression: Expression, inputSchema: Seq[Attribute]): InterpretedPredicate = {
val str = expression.toString
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think Expression.toString will truncate too big expression. Right?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, Expression.toString truncates something. However, it does not work for this case. Thus, I did not change here.

val logMessage = if (str.length > 256) {
str.substring(0, 256 - 3) + "..."
} else {
str
}
logWarning(s"Codegen disabled for this expression:\n $logMessage")
InterpretedPredicate.create(expression, inputSchema)
}

protected def newPredicate(
expression: Expression, inputSchema: Seq[Attribute]): GenPredicate = {
GeneratePredicate.generate(expression, inputSchema)
try {
GeneratePredicate.generate(expression, inputSchema)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we only do this fallback if sqlContext.conf.wholeStageFallback is turned on?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is good to control it using an option. This is not a part of the whole-stage codegen.
Is it better to use sqlContext.conf.wholeStageFallback or add sqlContext.conf.codegenFallback?
What do you think?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am wondering if it makes sense that wholeStageFallback is false and this new option is true, or vice verse.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, now look at sqlContext.conf.wholeStageFallback.

} catch {
case e @ (_: JaninoRuntimeException | _: CompileException)
if sqlContext == null || sqlContext.conf.wholeStageFallback =>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sqlContext.conf.wholeStageFallback is almost useless here, because almost all the cases this will be done in executors.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pretty risky if we always fallback. This might hide bugs.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let me fix it by another PR.

genInterpretedPredicate(expression, inputSchema)
}
}

protected def newOrdering(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,7 +177,7 @@ abstract class PartitioningAwareFileIndex(
})

val selected = partitions.filter {
case PartitionPath(values, _) => boundPredicate(values)
case PartitionPath(values, _) => boundPredicate.eval(values)
}
logInfo {
val total = partitions.length
Expand Down
11 changes: 11 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1732,4 +1732,15 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
.filter($"x1".isNotNull || !$"y".isin("a!"))
.count
}

test("SPARK-19372: Filter can be executed w/o generated code due to JVM code size limit") {
val N = 400
val rows = Seq(Row.fromSeq(Seq.fill(N)("string")))
val schema = StructType(Seq.tabulate(N)(i => StructField(s"_c$i", StringType)))
val df = spark.createDataFrame(spark.sparkContext.makeRDD(rows), schema)

val filter = (0 until N)
.foldLeft(lit(false))((e, index) => e.or(df.col(df.columns(index)) =!= "string"))
df.filter(filter).count
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister {
// `Cast`ed values are always of internal types (e.g. UTF8String instead of String)
Cast(Literal(value), dataType).eval()
})
}.filter(predicate).map(projection)
}.filter(predicate.eval).map(projection)

// Appends partition values
val fullOutput = requiredSchema.toAttributes ++ partitionSchema.toAttributes
Expand Down