Skip to content

Commit

Permalink
Dynamically load hive and avro using reflection to avoid potential cl…
Browse files Browse the repository at this point in the history
…ass not found exception

Signed-off-by: Chong Gao <res_life@163.com>
  • Loading branch information
Chong Gao committed Jun 2, 2022
1 parent ee42b11 commit c89b407
Show file tree
Hide file tree
Showing 4 changed files with 292 additions and 150 deletions.
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (c) 2020-2021, NVIDIA CORPORATION.
* Copyright (c) 2020-2022, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -16,12 +16,18 @@

package org.apache.spark.sql.hive.rapids

import com.nvidia.spark.RapidsUDF
import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, RapidsConf, RepeatingParamCheck, ShimLoader, TypeSig}
import com.nvidia.spark.rapids.GpuUserDefinedFunction.udfTypeSig
import com.nvidia.spark.rapids.{ExprRule, ShimLoader}

import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.hive.{HiveGenericUDF, HiveSimpleUDF}

/**
* The subclass of HiveProvider imports spark-hive classes. This file should not imports
* spark-hive because `class not found` exception may throw if spark-hive does not exist at
* runtime. Details see: https://github.com/NVIDIA/spark-rapids/issues/5648
*/
trait HiveProvider {
def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]]
}

object GpuHiveOverrides {
def isSparkHiveAvailable: Boolean = {
Expand All @@ -39,94 +45,11 @@ object GpuHiveOverrides {
* mapping if spark-hive is unavailable.
*/
def exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = {
if (!isSparkHiveAvailable) {
return Map.empty
if (isSparkHiveAvailable) {
val className = "org.apache.spark.sql.hive.rapids.HiveSourceProvider"
ShimLoader.newInstanceOf[HiveProvider](className).getExprs
} else {
Map.empty
}

Seq(
GpuOverrides.expr[HiveSimpleUDF](
"Hive UDF, the UDF can choose to implement a RAPIDS accelerated interface to" +
" get better performance",
ExprChecks.projectOnly(
udfTypeSig,
TypeSig.all,
repeatingParamCheck = Some(RepeatingParamCheck("param", udfTypeSig, TypeSig.all))),
(a, conf, p, r) => new ExprMeta[HiveSimpleUDF](a, conf, p, r) {
private val opRapidsFunc = a.function match {
case rapidsUDF: RapidsUDF => Some(rapidsUDF)
case _ => None
}

override def tagExprForGpu(): Unit = {
if (opRapidsFunc.isEmpty && !conf.isCpuBasedUDFEnabled) {
willNotWorkOnGpu(s"Hive SimpleUDF ${a.name} implemented by " +
s"${a.funcWrapper.functionClassName} does not provide a GPU implementation " +
s"and CPU-based UDFs are not enabled by `${RapidsConf.ENABLE_CPU_BASED_UDF.key}`")
}
}

override def convertToGpu(): GpuExpression = {
opRapidsFunc.map { _ =>
// We use the original HiveGenericUDF `deterministic` method as a proxy
// for simplicity.
GpuHiveSimpleUDF(
a.name,
a.funcWrapper,
childExprs.map(_.convertToGpu()),
a.dataType,
a.deterministic)
}.getOrElse {
// This `require` is just for double check.
require(conf.isCpuBasedUDFEnabled)
GpuRowBasedHiveSimpleUDF(
a.name,
a.funcWrapper,
childExprs.map(_.convertToGpu()))
}
}
}),
GpuOverrides.expr[HiveGenericUDF](
"Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to" +
" get better performance",
ExprChecks.projectOnly(
udfTypeSig,
TypeSig.all,
repeatingParamCheck = Some(RepeatingParamCheck("param", udfTypeSig, TypeSig.all))),
(a, conf, p, r) => new ExprMeta[HiveGenericUDF](a, conf, p, r) {
private val opRapidsFunc = a.function match {
case rapidsUDF: RapidsUDF => Some(rapidsUDF)
case _ => None
}

override def tagExprForGpu(): Unit = {
if (opRapidsFunc.isEmpty && !conf.isCpuBasedUDFEnabled) {
willNotWorkOnGpu(s"Hive GenericUDF ${a.name} implemented by " +
s"${a.funcWrapper.functionClassName} does not provide a GPU implementation " +
s"and CPU-based UDFs are not enabled by `${RapidsConf.ENABLE_CPU_BASED_UDF.key}`")
}
}

override def convertToGpu(): GpuExpression = {
opRapidsFunc.map { _ =>
// We use the original HiveGenericUDF `deterministic` method as a proxy
// for simplicity.
GpuHiveGenericUDF(
a.name,
a.funcWrapper,
childExprs.map(_.convertToGpu()),
a.dataType,
a.deterministic,
a.foldable)
}.getOrElse {
// This `require` is just for double check.
require(conf.isCpuBasedUDFEnabled)
GpuRowBasedHiveGenericUDF(
a.name,
a.funcWrapper,
childExprs.map(_.convertToGpu()))
}
}
})
).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
/*
* Copyright (c) 2022, NVIDIA CORPORATION.
*
* Licensed 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.sql.hive.rapids

import com.nvidia.spark.RapidsUDF
import com.nvidia.spark.rapids.{ExprChecks, ExprMeta, ExprRule, GpuExpression, GpuOverrides, RapidsConf, RepeatingParamCheck, TypeSig}
import com.nvidia.spark.rapids.GpuUserDefinedFunction.udfTypeSig

import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.hive.{HiveGenericUDF, HiveSimpleUDF}

class HiveSourceProvider extends HiveProvider {

/**
* Builds the rules that are specific to spark-hive Catalyst nodes. This will return an empty
* mapping if spark-hive is unavailable.
*/
override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = {
Seq(
GpuOverrides.expr[HiveSimpleUDF](
"Hive UDF, the UDF can choose to implement a RAPIDS accelerated interface to" +
" get better performance",
ExprChecks.projectOnly(
udfTypeSig,
TypeSig.all,
repeatingParamCheck = Some(RepeatingParamCheck("param", udfTypeSig, TypeSig.all))),
(a, conf, p, r) => new ExprMeta[HiveSimpleUDF](a, conf, p, r) {
private val opRapidsFunc = a.function match {
case rapidsUDF: RapidsUDF => Some(rapidsUDF)
case _ => None
}

override def tagExprForGpu(): Unit = {
if (opRapidsFunc.isEmpty && !conf.isCpuBasedUDFEnabled) {
willNotWorkOnGpu(s"Hive SimpleUDF ${a.name} implemented by " +
s"${a.funcWrapper.functionClassName} does not provide a GPU implementation " +
s"and CPU-based UDFs are not enabled by `${RapidsConf.ENABLE_CPU_BASED_UDF.key}`")
}
}

override def convertToGpu(): GpuExpression = {
opRapidsFunc.map { _ =>
// We use the original HiveGenericUDF `deterministic` method as a proxy
// for simplicity.
GpuHiveSimpleUDF(
a.name,
a.funcWrapper,
childExprs.map(_.convertToGpu()),
a.dataType,
a.deterministic)
}.getOrElse {
// This `require` is just for double check.
require(conf.isCpuBasedUDFEnabled)
GpuRowBasedHiveSimpleUDF(
a.name,
a.funcWrapper,
childExprs.map(_.convertToGpu()))
}
}
}),
GpuOverrides.expr[HiveGenericUDF](
"Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to" +
" get better performance",
ExprChecks.projectOnly(
udfTypeSig,
TypeSig.all,
repeatingParamCheck = Some(RepeatingParamCheck("param", udfTypeSig, TypeSig.all))),
(a, conf, p, r) => new ExprMeta[HiveGenericUDF](a, conf, p, r) {
private val opRapidsFunc = a.function match {
case rapidsUDF: RapidsUDF => Some(rapidsUDF)
case _ => None
}

override def tagExprForGpu(): Unit = {
if (opRapidsFunc.isEmpty && !conf.isCpuBasedUDFEnabled) {
willNotWorkOnGpu(s"Hive GenericUDF ${a.name} implemented by " +
s"${a.funcWrapper.functionClassName} does not provide a GPU implementation " +
s"and CPU-based UDFs are not enabled by `${RapidsConf.ENABLE_CPU_BASED_UDF.key}`")
}
}

override def convertToGpu(): GpuExpression = {
opRapidsFunc.map { _ =>
// We use the original HiveGenericUDF `deterministic` method as a proxy
// for simplicity.
GpuHiveGenericUDF(
a.name,
a.funcWrapper,
childExprs.map(_.convertToGpu()),
a.dataType,
a.deterministic,
a.foldable)
}.getOrElse {
// This `require` is just for double check.
require(conf.isCpuBasedUDFEnabled)
GpuRowBasedHiveGenericUDF(
a.name,
a.funcWrapper,
childExprs.map(_.convertToGpu()))
}
}
})
).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* Copyright (c) 2022, NVIDIA CORPORATION.
*
* Licensed 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.sql.rapids

import com.nvidia.spark.rapids._

import org.apache.spark.broadcast.Broadcast
import org.apache.spark.sql.avro.{AvroFileFormat, AvroOptions}
import org.apache.spark.sql.connector.read.{PartitionReaderFactory, Scan}
import org.apache.spark.sql.execution.FileSourceScanExec
import org.apache.spark.sql.execution.datasources.FileFormat
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.v2.avro.AvroScan
import org.apache.spark.util.SerializableConfiguration

class AvroSourceProvider extends AvroProvider {

/** If the file format is supported as an external source */
def isSupportedFormat(format: FileFormat): Boolean = {
format match {
case _: AvroFileFormat => true
case _ => false
}
}

def isPerFileReadEnabledForFormat(format: FileFormat, conf: RapidsConf): Boolean = {
format match {
case _: AvroFileFormat => conf.isAvroPerFileReadEnabled
case _ => false
}
}

def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = {
meta.wrapped.relation.fileFormat match {
case _: AvroFileFormat => GpuReadAvroFileFormat.tagSupport(meta)
case f =>
meta.willNotWorkOnGpu(s"unsupported file format: ${f.getClass.getCanonicalName}")
}
}

/**
* Get a read file format for the input format.
* Better to check if the format is supported first by calling 'isSupportedFormat'
*/
def getReadFileFormat(format: FileFormat): FileFormat = {
format match {
case _: AvroFileFormat => new GpuReadAvroFileFormat
case f =>
throw new IllegalArgumentException(s"${f.getClass.getCanonicalName} is not supported")
}
}

/**
* Create a multi-file reader factory for the input format.
* Better to check if the format is supported first by calling 'isSupportedFormat'
*/
def createMultiFileReaderFactory(
format: FileFormat,
broadcastedConf: Broadcast[SerializableConfiguration],
pushedFilters: Array[Filter],
fileScan: GpuFileSourceScanExec): PartitionReaderFactory = {
format match {
case _: AvroFileFormat =>
GpuAvroMultiFilePartitionReaderFactory(
fileScan.relation.sparkSession.sessionState.conf,
fileScan.rapidsConf,
broadcastedConf,
fileScan.relation.dataSchema,
fileScan.requiredSchema,
fileScan.relation.partitionSchema,
new AvroOptions(fileScan.relation.options, broadcastedConf.value.value),
fileScan.allMetrics,
pushedFilters,
fileScan.queryUsesInputFile)
case _ =>
// never reach here
throw new RuntimeException(s"File format $format is not supported yet")
}
}

def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = {
Seq(
GpuOverrides.scan[AvroScan](
"Avro parsing",
(a, conf, p, r) => new ScanMeta[AvroScan](a, conf, p, r) {
override def tagSelfForGpu(): Unit = GpuAvroScan.tagSupport(this)

override def convertToGpu(): Scan =
GpuAvroScan(a.sparkSession,
a.fileIndex,
a.dataSchema,
a.readDataSchema,
a.readPartitionSchema,
a.options,
a.pushedFilters,
conf,
a.partitionFilters,
a.dataFilters)
})
).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap
}
}
Loading

0 comments on commit c89b407

Please sign in to comment.