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 #171

Merged
merged 1 commit into from
Apr 26, 2017
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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,20 @@ 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 =>
val excChains = ExceptionUtils.getThrowables(e)
val exc = if (excChains.length == 1) excChains(0) else excChains(excChains.length - 2)
throw exc
case e : ExecutionError =>
val excChains = ExceptionUtils.getThrowables(e)
val exc = if (excChains.length == 1) excChains(0) else excChains(excChains.length - 2)
throw exc
}

/**
Expand Down Expand Up @@ -951,10 +966,14 @@ object CodeGenerator extends Logging {
evaluator.cook("generated.java", code.body)
recordCompilationStats(evaluator)
} catch {
case e: Exception =>
case e: JaninoRuntimeException =>
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.asInstanceOf[CompileException].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)
}

class InterpretedPredicate(expression: Expression) extends BasePredicate {
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,28 @@ 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
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)
} catch {
case e: JaninoRuntimeException if sqlContext == null || sqlContext.conf.wholeStageFallback =>
genInterpretedPredicate(expression, inputSchema)
case e: CompileException if sqlContext == null || sqlContext.conf.wholeStageFallback =>
genInterpretedPredicate(expression, inputSchema)
}
}

protected def newOrdering(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,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
12 changes: 12 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 @@ -24,6 +24,7 @@ import java.util.UUID

import scala.util.Random

import com.sun.net.httpserver.Authenticator.Retry
import org.scalatest.Matchers._

import org.apache.spark.SparkException
Expand Down Expand Up @@ -1703,4 +1704,15 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
val df = spark.range(1).selectExpr("CAST(id as DECIMAL) as x").selectExpr("percentile(x, 0.5)")
checkAnswer(df, Row(BigDecimal(0.0)) :: Nil)
}

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 @@ -1036,7 +1036,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
BoundReference(index, partitionSchema(index).dataType, nullable = true)
})
clientPrunedPartitions.filter { p =>
boundPredicate(p.toRow(partitionSchema, defaultTimeZoneId))
boundPredicate.eval(p.toRow(partitionSchema, defaultTimeZoneId))
}
} else {
client.getPartitions(catalogTable).map { part =>
Expand Down
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