From 87fcc7d8ed05e817b36eae1d84cb71b5deea4869 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 20 Mar 2024 13:24:18 -0700 Subject: [PATCH 1/4] Pass metadata extractors to FileScanRDD --- .../com/nvidia/spark/rapids/SparkShims.scala | 3 ++- .../hive/rapids/GpuHiveTableScanExec.scala | 2 +- .../sql/rapids/GpuFileSourceScanExec.scala | 2 +- .../rapids/shims/Spark31Xuntil33XShims.scala | 3 ++- .../rapids/shims/Spark330PlusShims.scala | 3 ++- .../rapids/shims/Spark321PlusDBShims.scala | 1 + .../spark/rapids/shims/SparkShims.scala | 22 +++++++++++++++++-- 7 files changed, 29 insertions(+), 7 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index 8f863d2962f..91cab5cdf2a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.{ColumnarToRowTransition, SparkPlan} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.command.{DataWritingCommand, RunnableCommand} -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.{FilePartition, HadoopFsRelation, PartitionedFile, PartitioningAwareFileIndex} import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.internal.SQLConf @@ -78,6 +78,7 @@ trait SparkShims { readFunction: (PartitionedFile) => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, + relation: Option[HadoopFsRelation], metadataColumns: Seq[AttributeReference] = Seq.empty): RDD[InternalRow] def shouldFailDivOverflow: Boolean diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala index e4db0adccc1..02b7a560ef4 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala @@ -269,7 +269,7 @@ case class GpuHiveTableScanExec(requestedAttributes: Seq[Attribute], // TODO [future]: Handle small-file optimization. // (https://github.com/NVIDIA/spark-rapids/issues/7017) // Currently assuming per-file reading. - SparkShimImpl.getFileScanRDD(sparkSession, readFile, filePartitions, readSchema) + SparkShimImpl.getFileScanRDD(sparkSession, readFile, filePartitions, readSchema, None) .asInstanceOf[RDD[ColumnarBatch]] } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index 3e32399fb47..ce34daaf2a3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -603,7 +603,7 @@ case class GpuFileSourceScanExec( if (isPerFileReadEnabled) { logInfo("Using the original per file reader") SparkShimImpl.getFileScanRDD(relation.sparkSession, readFile.get, locatedPartitions, - requiredSchema) + requiredSchema, Some(relation)) } else { logDebug(s"Using Datasource RDD, files are: " + s"${prunedPartitions.flatMap(_.files).mkString(",")}") diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala index b9fca5af58a..c28ad39d0bc 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, DataWritingCommand, RunnableCommand} -import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile} +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HadoopFsRelation, PartitionedFile} import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.types.StructType @@ -50,6 +50,7 @@ trait Spark31Xuntil33XShims extends SparkShims { readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, + relation: Option[HadoopFsRelation], metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = { new FileScanRDD(sparkSession, readFunction, filePartitions) } diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala index fe6d9fb4c05..f672a04266f 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile} +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HadoopFsRelation, PartitionedFile} import org.apache.spark.sql.rapids.shims.{GpuDivideYMInterval, GpuMultiplyYMInterval} import org.apache.spark.sql.types.StructType @@ -50,6 +50,7 @@ trait Spark330PlusShims extends Spark321PlusShims with Spark320PlusNonDBShims { readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, + relation: Option[HadoopFsRelation], metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = { new FileScanRDD(sparkSession, readFunction, filePartitions, readDataSchema, metadataColumns) } diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala index 121fed93b26..f4a284a4eac 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala @@ -56,6 +56,7 @@ trait Spark321PlusDBShims extends SparkShims readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, + relation: Option[HadoopFsRelation], metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = { new GpuFileScanRDD(sparkSession, readFunction, filePartitions) } diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala index 5c8a2d48d4d..1d91f5f4b35 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala @@ -22,15 +22,33 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ -import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDAF, ToPrettyString} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, PythonUDAF, ToPrettyString} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.TableCacheQueryStageExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HadoopFsRelation, PartitionedFile} import org.apache.spark.sql.execution.window.WindowGroupLimitExec import org.apache.spark.sql.rapids.execution.python.GpuPythonUDAF -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.{StringType, StructType} object SparkShimImpl extends Spark340PlusNonDBShims { + override def getFileScanRDD( + sparkSession: SparkSession, + readFunction: PartitionedFile => Iterator[InternalRow], + filePartitions: Seq[FilePartition], + readDataSchema: StructType, + relation: Option[HadoopFsRelation], + metadataColumns: Seq[AttributeReference] = Seq.empty): RDD[InternalRow] = { + if (relation.isDefined) { + new FileScanRDD(sparkSession, readFunction, filePartitions, readDataSchema, metadataColumns, + metadataExtractors = relation.get.fileFormat.fileConstantMetadataExtractors) + } else { + new FileScanRDD(sparkSession, readFunction, filePartitions, readDataSchema, metadataColumns) + } + } override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { val shimExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( From b2d133b482d4cdeab84239a80eaee920fd14e8f2 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 20 Mar 2024 13:35:08 -0700 Subject: [PATCH 2/4] Signing off Signed-off-by: Raza Jafri From a641cffe70d6ec2c026f81ad16932eb44ef6bc25 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 20 Mar 2024 22:17:24 -0700 Subject: [PATCH 3/4] addressed review comments --- .../scala/com/nvidia/spark/rapids/SparkShims.scala | 6 +++--- .../spark/sql/hive/rapids/GpuHiveTableScanExec.scala | 2 +- .../spark/sql/rapids/GpuFileSourceScanExec.scala | 2 +- .../spark/rapids/shims/Spark31Xuntil33XShims.scala | 6 +++--- .../spark/rapids/shims/Spark330PlusShims.scala | 6 +++--- .../spark/rapids/shims/Spark321PlusDBShims.scala | 4 ++-- .../com/nvidia/spark/rapids/shims/SparkShims.scala | 12 ++++-------- 7 files changed, 17 insertions(+), 21 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala index 91cab5cdf2a..0c7ede5e3db 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SparkShims.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.{ColumnarToRowTransition, SparkPlan} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.command.{DataWritingCommand, RunnableCommand} -import org.apache.spark.sql.execution.datasources.{FilePartition, HadoopFsRelation, PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.{FileFormat, FilePartition, PartitionedFile, PartitioningAwareFileIndex} import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.internal.SQLConf @@ -78,8 +78,8 @@ trait SparkShims { readFunction: (PartitionedFile) => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, - relation: Option[HadoopFsRelation], - metadataColumns: Seq[AttributeReference] = Seq.empty): RDD[InternalRow] + metadataColumns: Seq[AttributeReference] = Seq.empty, + fileFormat: Option[FileFormat] = None): RDD[InternalRow] def shouldFailDivOverflow: Boolean diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala index 02b7a560ef4..e4db0adccc1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala @@ -269,7 +269,7 @@ case class GpuHiveTableScanExec(requestedAttributes: Seq[Attribute], // TODO [future]: Handle small-file optimization. // (https://github.com/NVIDIA/spark-rapids/issues/7017) // Currently assuming per-file reading. - SparkShimImpl.getFileScanRDD(sparkSession, readFile, filePartitions, readSchema, None) + SparkShimImpl.getFileScanRDD(sparkSession, readFile, filePartitions, readSchema) .asInstanceOf[RDD[ColumnarBatch]] } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index ce34daaf2a3..a326006ab83 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -603,7 +603,7 @@ case class GpuFileSourceScanExec( if (isPerFileReadEnabled) { logInfo("Using the original per file reader") SparkShimImpl.getFileScanRDD(relation.sparkSession, readFile.get, locatedPartitions, - requiredSchema, Some(relation)) + requiredSchema, fileFormat = Some(relation.fileFormat)) } else { logDebug(s"Using Datasource RDD, files are: " + s"${prunedPartitions.flatMap(_.files).mkString(",")}") diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala index c28ad39d0bc..31526f313ce 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, DataWritingCommand, RunnableCommand} -import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HadoopFsRelation, PartitionedFile} +import org.apache.spark.sql.execution.datasources.{FileFormat, FilePartition, FileScanRDD, PartitionedFile} import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.types.StructType @@ -50,8 +50,8 @@ trait Spark31Xuntil33XShims extends SparkShims { readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, - relation: Option[HadoopFsRelation], - metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = { + metadataColumns: Seq[AttributeReference], + fileFormat: Option[FileFormat]): RDD[InternalRow] = { new FileScanRDD(sparkSession, readFunction, filePartitions) } diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala index f672a04266f..aded93560f7 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HadoopFsRelation, PartitionedFile} +import org.apache.spark.sql.execution.datasources.{FileFormat, FilePartition, FileScanRDD, PartitionedFile} import org.apache.spark.sql.rapids.shims.{GpuDivideYMInterval, GpuMultiplyYMInterval} import org.apache.spark.sql.types.StructType @@ -50,8 +50,8 @@ trait Spark330PlusShims extends Spark321PlusShims with Spark320PlusNonDBShims { readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, - relation: Option[HadoopFsRelation], - metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = { + metadataColumns: Seq[AttributeReference], + fileFormat: Option[FileFormat]): RDD[InternalRow] = { new FileScanRDD(sparkSession, readFunction, filePartitions, readDataSchema, metadataColumns) } diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala index f4a284a4eac..5e5bbed2942 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala @@ -56,8 +56,8 @@ trait Spark321PlusDBShims extends SparkShims readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, - relation: Option[HadoopFsRelation], - metadataColumns: Seq[AttributeReference]): RDD[InternalRow] = { + metadataColumns: Seq[AttributeReference], + fileFormat: Option[FileFormat]): RDD[InternalRow] = { new GpuFileScanRDD(sparkSession, readFunction, filePartitions) } diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala index 1d91f5f4b35..6238cfbc159 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.TableCacheQueryStageExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HadoopFsRelation, PartitionedFile} +import org.apache.spark.sql.execution.datasources.{FileFormat, FilePartition, FileScanRDD, PartitionedFile} import org.apache.spark.sql.execution.window.WindowGroupLimitExec import org.apache.spark.sql.rapids.execution.python.GpuPythonUDAF import org.apache.spark.sql.types.{StringType, StructType} @@ -40,14 +40,10 @@ object SparkShimImpl extends Spark340PlusNonDBShims { readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], readDataSchema: StructType, - relation: Option[HadoopFsRelation], - metadataColumns: Seq[AttributeReference] = Seq.empty): RDD[InternalRow] = { - if (relation.isDefined) { + metadataColumns: Seq[AttributeReference] = Seq.empty, + fileFormat: Option[FileFormat]): RDD[InternalRow] = { new FileScanRDD(sparkSession, readFunction, filePartitions, readDataSchema, metadataColumns, - metadataExtractors = relation.get.fileFormat.fileConstantMetadataExtractors) - } else { - new FileScanRDD(sparkSession, readFunction, filePartitions, readDataSchema, metadataColumns) - } + metadataExtractors = fileFormat.map(_.fileConstantMetadataExtractors).getOrElse(Map.empty)) } override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { From 03b843f5e2fb96e6b45ad62f77e6716c098c42c4 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Thu, 21 Mar 2024 11:05:25 -0700 Subject: [PATCH 4/4] updated copyrights manually --- .../org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala | 2 +- .../com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala | 2 +- .../scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala index e4db0adccc1..be41b4ef82f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTableScanExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala index 31526f313ce..95ca4019b84 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/Spark31Xuntil33XShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala index aded93560f7..d048a10821b 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/Spark330PlusShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License.