diff --git a/.github/workflows/auto-merge.yml b/.github/workflows/auto-merge.yml index a54b1cae0e1..efcb45a937f 100644 --- a/.github/workflows/auto-merge.yml +++ b/.github/workflows/auto-merge.yml @@ -18,7 +18,7 @@ name: auto-merge HEAD to BASE on: pull_request_target: branches: - - branch-23.08 + - branch-23.10 types: [closed] jobs: @@ -29,13 +29,13 @@ jobs: steps: - uses: actions/checkout@v3 with: - ref: branch-23.08 # force to fetch from latest upstream instead of PR ref + ref: branch-23.10 # force to fetch from latest upstream instead of PR ref - name: auto-merge job uses: ./.github/workflows/auto-merge env: OWNER: NVIDIA REPO_NAME: spark-rapids - HEAD: branch-23.08 - BASE: branch-23.10 + HEAD: branch-23.10 + BASE: branch-23.12 AUTOMERGE_TOKEN: ${{ secrets.AUTOMERGE_TOKEN }} # use to merge PR diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 514e21386c8..c80a4e7c3aa 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -42,6 +42,7 @@ jobs: jbrennan333, \ jlowe,\ krajendrannv,\ + kuhushukla,\ mythrocks,\ nartal1,\ nvdbaranec,\ @@ -68,6 +69,8 @@ jobs: YanxuanLiu,\ cindyyuanjiang,\ thirtiseven,\ + winningsix,\ + viadea,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' steps: - name: Check if comment is issued by authorized person diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 5d6c0933f1f..b4d7cc040cf 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -144,8 +144,9 @@ specifying the environment variable `BUILD_PARALLEL=`. ### Building against different CUDA Toolkit versions -You can build against different versions of the CUDA Toolkit by using one of the following profiles: -* `-Pcuda11` (CUDA 11.0/11.1/11.2, default) +You can build against different versions of the CUDA Toolkit by modifying the variable `cuda.version`: +* `-Dcuda.version=cuda11` (CUDA 11.x, default) +* `-Dcuda.version=cuda12` (CUDA 12.x) ### Building a Distribution for a Single Spark Release diff --git a/aggregator/pom.xml b/aggregator/pom.xml index a001b0695d5..12cf62e6954 100644 --- a/aggregator/pom.xml +++ b/aggregator/pom.xml @@ -534,7 +534,7 @@ com.nvidia - rapids-4-spark-delta-stub_${scala.binary.version} + rapids-4-spark-delta-24x_${scala.binary.version} ${project.version} ${spark.version.classifier} diff --git a/api_validation/pom.xml b/api_validation/pom.xml index 892bd83ef24..cbd72317451 100644 --- a/api_validation/pom.xml +++ b/api_validation/pom.xml @@ -100,7 +100,7 @@ com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} provided @@ -115,7 +115,7 @@ com.nvidia rapids-4-spark_${scala.binary.version} ${project.version} - ${cuda.version} + ${jni.classifier} provided diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DeltaProviderImpl.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DeltaProviderImpl.scala index 6f225810b6e..9f7e0896ec6 100644 --- a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DeltaProviderImpl.scala +++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DeltaProviderImpl.scala @@ -16,14 +16,15 @@ package com.nvidia.spark.rapids.delta -import com.databricks.sql.transaction.tahoe.DeltaLog +import com.databricks.sql.transaction.tahoe.{DeltaLog, DeltaParquetFileFormat} import com.databricks.sql.transaction.tahoe.commands.{DeleteCommand, DeleteCommandEdge, MergeIntoCommand, MergeIntoCommandEdge, UpdateCommand, UpdateCommandEdge} import com.databricks.sql.transaction.tahoe.sources.DeltaDataSource import com.nvidia.spark.rapids._ import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand +import org.apache.spark.sql.execution.datasources.{FileFormat, SaveIntoDataSourceCommand} import org.apache.spark.sql.rapids.ExternalSource import org.apache.spark.sql.sources.CreatableRelationProvider @@ -72,6 +73,25 @@ object DeltaProviderImpl extends DeltaProviderImplBase { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = { + format == classOf[DeltaParquetFileFormat] + } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + GpuDeltaParquetFileFormat.tagSupportForGpuFileSourceScan(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDeltaParquetFileFormat.convertToGpu(cpuFormat) + } } class DeltaCreatableRelationProviderMeta( diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala new file mode 100644 index 00000000000..598cf8248f4 --- /dev/null +++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala @@ -0,0 +1,84 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMapping, DeltaColumnMappingMode, NoMapping} +import com.nvidia.spark.rapids.{GpuMetric, GpuParquetMultiFilePartitionReaderFactory, GpuReadParquetFileFormat} +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.rapids.GpuFileSourceScanExec +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +abstract class GpuDeltaParquetFileFormatBase extends GpuReadParquetFileFormat { + val columnMappingMode: DeltaColumnMappingMode + val referenceSchema: StructType + + def prepareSchema(inputSchema: StructType): StructType = { + DeltaColumnMapping.createPhysicalSchema(inputSchema, referenceSchema, columnMappingMode) + } + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuParquetMultiFilePartitionReaderFactory( + fileScan.conf, + broadcastedConf, + prepareSchema(fileScan.relation.dataSchema), + prepareSchema(fileScan.requiredSchema), + prepareSchema(fileScan.readPartitionSchema), + pushedFilters, + fileScan.rapidsConf, + fileScan.allMetrics, + fileScan.queryUsesInputFile, + fileScan.alluxioPathsMap) + } + + override def buildReaderWithPartitionValuesAndMetrics( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration, + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] = { + super.buildReaderWithPartitionValuesAndMetrics( + sparkSession, + prepareSchema(dataSchema), + prepareSchema(partitionSchema), + prepareSchema(requiredSchema), + filters, + options, + hadoopConf, + metrics, + alluxioPathReplacementMap) + } + + override def supportFieldName(name: String): Boolean = { + if (columnMappingMode != NoMapping) true else super.supportFieldName(name) + } +} diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala index 22f5485092d..17511390fbe 100644 --- a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala +++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala @@ -33,11 +33,11 @@ object RapidsDeltaUtils { options: Map[String, String], spark: SparkSession): Unit = { FileFormatChecks.tag(meta, schema, DeltaFormatType, WriteFileOp) - DeltaLogShim.fileFormat(deltaLog) match { - case _: DeltaParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) - case f => - meta.willNotWorkOnGpu(s"file format $f is not supported") + val format = DeltaLogShim.fileFormat(deltaLog) + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) + } else { + meta.willNotWorkOnGpu(s"file format $format is not supported") } checkIncompatibleConfs(meta, schema, deltaLog, spark.sessionState.conf, options) } diff --git a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/DeltaIOProvider.scala b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/DeltaIOProvider.scala index 0b7da564a93..64ad89490dd 100644 --- a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/DeltaIOProvider.scala +++ b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/DeltaIOProvider.scala @@ -21,10 +21,10 @@ import scala.util.Try import com.nvidia.spark.rapids._ import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.{DeltaLog, DeltaParquetFileFormat} import org.apache.spark.sql.delta.rapids.DeltaRuntimeShim import org.apache.spark.sql.delta.sources.DeltaDataSource -import org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand +import org.apache.spark.sql.execution.datasources.{FileFormat, SaveIntoDataSourceCommand} import org.apache.spark.sql.rapids.ExternalSource import org.apache.spark.sql.rapids.execution.UnshimmedTrampolineUtil import org.apache.spark.sql.sources.CreatableRelationProvider @@ -44,6 +44,10 @@ abstract class DeltaIOProvider extends DeltaProviderImplBase { }) ).map(r => (r.getClassFor.asSubclass(classOf[CreatableRelationProvider]), r)).toMap } + + override def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = { + format == classOf[DeltaParquetFileFormat] + } } class DeltaCreatableRelationProviderMeta( diff --git a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..96cb4b6d552 --- /dev/null +++ b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,84 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta + +import com.nvidia.spark.rapids.{GpuMetric, GpuParquetMultiFilePartitionReaderFactory, GpuReadParquetFileFormat} +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaColumnMappingMode, NoMapping} +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.rapids.GpuFileSourceScanExec +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +trait GpuDeltaParquetFileFormat extends GpuReadParquetFileFormat { + val columnMappingMode: DeltaColumnMappingMode + val referenceSchema: StructType + + def prepareSchema(inputSchema: StructType): StructType = { + DeltaColumnMapping.createPhysicalSchema(inputSchema, referenceSchema, columnMappingMode) + } + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuParquetMultiFilePartitionReaderFactory( + fileScan.conf, + broadcastedConf, + prepareSchema(fileScan.relation.dataSchema), + prepareSchema(fileScan.requiredSchema), + prepareSchema(fileScan.readPartitionSchema), + pushedFilters, + fileScan.rapidsConf, + fileScan.allMetrics, + fileScan.queryUsesInputFile, + fileScan.alluxioPathsMap) + } + + override def buildReaderWithPartitionValuesAndMetrics( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration, + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] = { + super.buildReaderWithPartitionValuesAndMetrics( + sparkSession, + prepareSchema(dataSchema), + prepareSchema(partitionSchema), + prepareSchema(requiredSchema), + filters, + options, + hadoopConf, + metrics, + alluxioPathReplacementMap) + } + + override def supportFieldName(name: String): Boolean = { + if (columnMappingMode != NoMapping) true else super.supportFieldName(name) + } +} diff --git a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala index 75a6af75ae4..e0c5725f91d 100644 --- a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala +++ b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala @@ -32,11 +32,11 @@ object RapidsDeltaUtils { options: Map[String, String], spark: SparkSession): Unit = { FileFormatChecks.tag(meta, schema, DeltaFormatType, WriteFileOp) - DeltaRuntimeShim.fileFormatFromLog(deltaLog) match { - case _: DeltaParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) - case f => - meta.willNotWorkOnGpu(s"file format $f is not supported") + val format = DeltaRuntimeShim.fileFormatFromLog(deltaLog) + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) + } else { + meta.willNotWorkOnGpu(s"file format $format is not supported") } checkIncompatibleConfs(meta, deltaLog, spark.sessionState.conf, options) } diff --git a/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/Delta20xProvider.scala b/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/Delta20xProvider.scala index 256ebd10c48..1e8cef6cca7 100644 --- a/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/Delta20xProvider.scala +++ b/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/Delta20xProvider.scala @@ -16,11 +16,14 @@ package com.nvidia.spark.rapids.delta.delta20x -import com.nvidia.spark.rapids.{GpuOverrides, RunnableCommandRule} +import com.nvidia.spark.rapids.{GpuOverrides, GpuReadParquetFileFormat, RunnableCommandRule, SparkPlanMeta} import com.nvidia.spark.rapids.delta.DeltaIOProvider +import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.commands.{DeleteCommand, MergeIntoCommand, UpdateCommand} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat object Delta20xProvider extends DeltaIOProvider { @@ -41,4 +44,18 @@ object Delta20xProvider extends DeltaIOProvider { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDelta20xParquetFileFormat(cpuFormat.columnMappingMode, cpuFormat.referenceSchema) + } } diff --git a/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/GpuDelta20xParquetFileFormat.scala b/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/GpuDelta20xParquetFileFormat.scala new file mode 100644 index 00000000000..05b764eed22 --- /dev/null +++ b/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/GpuDelta20xParquetFileFormat.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta.delta20x + +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat + +import org.apache.spark.sql.delta.DeltaColumnMappingMode +import org.apache.spark.sql.types.StructType + +case class GpuDelta20xParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormat { +} diff --git a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala index ec7380d5344..282368fc95b 100644 --- a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala @@ -71,7 +71,7 @@ class GpuOptimisticTransaction * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } @@ -200,9 +200,11 @@ class GpuOptimisticTransaction }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/Delta21xProvider.scala b/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/Delta21xProvider.scala index 3ada9d2087e..934fe7eb95b 100644 --- a/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/Delta21xProvider.scala +++ b/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/Delta21xProvider.scala @@ -16,11 +16,14 @@ package com.nvidia.spark.rapids.delta.delta21x -import com.nvidia.spark.rapids.{GpuOverrides, RunnableCommandRule} +import com.nvidia.spark.rapids.{GpuOverrides, GpuReadParquetFileFormat, RunnableCommandRule, SparkPlanMeta} import com.nvidia.spark.rapids.delta.DeltaIOProvider +import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.commands.{DeleteCommand, MergeIntoCommand, UpdateCommand} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat object Delta21xProvider extends DeltaIOProvider { @@ -41,4 +44,18 @@ object Delta21xProvider extends DeltaIOProvider { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDelta21xParquetFileFormat(cpuFormat.columnMappingMode, cpuFormat.referenceSchema) + } } diff --git a/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/GpuDelta21xParquetFileFormat.scala b/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/GpuDelta21xParquetFileFormat.scala new file mode 100644 index 00000000000..accbde4de3a --- /dev/null +++ b/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/GpuDelta21xParquetFileFormat.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta.delta21x + +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat + +import org.apache.spark.sql.delta.DeltaColumnMappingMode +import org.apache.spark.sql.types.StructType + +case class GpuDelta21xParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormat { +} diff --git a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala index b2be6747e44..9acded983d0 100644 --- a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala @@ -71,7 +71,7 @@ class GpuOptimisticTransaction * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } @@ -200,9 +200,11 @@ class GpuOptimisticTransaction }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/Delta22xProvider.scala b/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/Delta22xProvider.scala index db8a8c69f84..d5b5a79c7bc 100644 --- a/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/Delta22xProvider.scala +++ b/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/Delta22xProvider.scala @@ -16,11 +16,14 @@ package com.nvidia.spark.rapids.delta.delta22x -import com.nvidia.spark.rapids.{GpuOverrides, RunnableCommandRule} +import com.nvidia.spark.rapids.{GpuOverrides, GpuReadParquetFileFormat, RunnableCommandRule, SparkPlanMeta} import com.nvidia.spark.rapids.delta.DeltaIOProvider +import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.commands.{DeleteCommand, MergeIntoCommand, UpdateCommand} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat object Delta22xProvider extends DeltaIOProvider { @@ -41,4 +44,18 @@ object Delta22xProvider extends DeltaIOProvider { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDelta22xParquetFileFormat(cpuFormat.columnMappingMode, cpuFormat.referenceSchema) + } } diff --git a/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/GpuDelta22xParquetFileFormat.scala b/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/GpuDelta22xParquetFileFormat.scala new file mode 100644 index 00000000000..3dfb55eb209 --- /dev/null +++ b/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/GpuDelta22xParquetFileFormat.scala @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta.delta22x + +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.{DeltaColumnMappingMode, IdMapping} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +case class GpuDelta22xParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormat { + + if (columnMappingMode == IdMapping) { + val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredReadConf)), + s"${requiredReadConf.key} must be enabled to support Delta id column mapping mode") + val requiredWriteConf = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredWriteConf)), + s"${requiredWriteConf.key} must be enabled to support Delta id column mapping mode") + } +} diff --git a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala index 3efdb384be5..3ec00c5cb62 100644 --- a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala @@ -71,7 +71,7 @@ class GpuOptimisticTransaction * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } @@ -222,9 +222,11 @@ class GpuOptimisticTransaction }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala index 369e54af4bb..1f21c2a3e02 100644 --- a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala +++ b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala @@ -16,11 +16,15 @@ package com.nvidia.spark.rapids.delta.delta24x -import com.nvidia.spark.rapids.{GpuOverrides, RunnableCommandRule} +import com.nvidia.spark.rapids.{GpuOverrides, GpuReadParquetFileFormat, RunnableCommandRule, SparkPlanMeta} import com.nvidia.spark.rapids.delta.DeltaIOProvider +import org.apache.spark.sql.delta.DeltaParquetFileFormat +import org.apache.spark.sql.delta.DeltaParquetFileFormat.{IS_ROW_DELETED_COLUMN_NAME, ROW_INDEX_COLUMN_NAME} import org.apache.spark.sql.delta.commands.{DeleteCommand, MergeIntoCommand, UpdateCommand} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat object Delta24xProvider extends DeltaIOProvider { @@ -41,4 +45,31 @@ object Delta24xProvider extends DeltaIOProvider { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + val deltaFormat = format.asInstanceOf[DeltaParquetFileFormat] + val requiredSchema = meta.wrapped.requiredSchema + if (requiredSchema.exists(_.name == IS_ROW_DELETED_COLUMN_NAME)) { + meta.willNotWorkOnGpu( + s"reading metadata column $IS_ROW_DELETED_COLUMN_NAME is not supported") + } + if (requiredSchema.exists(_.name == ROW_INDEX_COLUMN_NAME)) { + meta.willNotWorkOnGpu( + s"reading metadata column $ROW_INDEX_COLUMN_NAME is not supported") + } + if (deltaFormat.hasDeletionVectorMap()) { + meta.willNotWorkOnGpu("deletion vectors are not supported") + } + GpuReadParquetFileFormat.tagSupport(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDelta24xParquetFileFormat(cpuFormat.metadata, cpuFormat.isSplittable) + } } diff --git a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala new file mode 100644 index 00000000000..ab8b6afe1ce --- /dev/null +++ b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala @@ -0,0 +1,48 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta.delta24x + +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.{DeltaColumnMappingMode, IdMapping} +import org.apache.spark.sql.delta.actions.Metadata +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +case class GpuDelta24xParquetFileFormat( + metadata: Metadata, + isSplittable: Boolean) extends GpuDeltaParquetFileFormat { + + override val columnMappingMode: DeltaColumnMappingMode = metadata.columnMappingMode + override val referenceSchema: StructType = metadata.schema + + if (columnMappingMode == IdMapping) { + val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredReadConf)), + s"${requiredReadConf.key} must be enabled to support Delta id column mapping mode") + val requiredWriteConf = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredWriteConf)), + s"${requiredWriteConf.key} must be enabled to support Delta id column mapping mode") + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = isSplittable +} diff --git a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala index a089e72f462..bc95a656271 100644 --- a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala @@ -71,7 +71,7 @@ class GpuOptimisticTransaction * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } @@ -224,9 +224,11 @@ class GpuOptimisticTransaction }.toMap } - val gpuFileFormat = DeltaRuntimeShim.fileFormatFromLog(deltaLog) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = DeltaRuntimeShim.fileFormatFromLog(deltaLog) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index 07f2298af66..03f3592af5a 100644 --- a/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -33,6 +33,7 @@ import com.databricks.sql.transaction.tahoe.schema.InvariantViolationException import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.delta._ +import com.nvidia.spark.rapids.shims.ParquetFieldIdShims import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.fs.Path @@ -62,7 +63,7 @@ class GpuOptimisticTransaction( * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } @@ -198,6 +199,10 @@ class GpuOptimisticTransaction( val hadoopConf = spark.sessionState.newHadoopConfWithOptions( metadata.configuration ++ deltaLog.options) + if (metadata.columnMappingMode == IdMapping) { + // Need Parquet field IDs when doing column ID mapping + ParquetFieldIdShims.setWriteIdOverride(hadoopConf, true) + } if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { val serializableHadoopConf = new SerializableConfiguration(hadoopConf) @@ -218,9 +223,11 @@ class GpuOptimisticTransaction( case Some(writeOptions) => writeOptions.options } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..773da5eb0bf --- /dev/null +++ b/delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMappingMode, DeltaParquetFileFormat} +import com.nvidia.spark.rapids.SparkPlanMeta + +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.types.StructType + +case class GpuDeltaParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormatBase { +} + +object GpuDeltaParquetFileFormat { + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = {} + + def convertToGpu(format: DeltaParquetFileFormat): GpuDeltaParquetFileFormat = { + GpuDeltaParquetFileFormat(format.columnMappingMode, format.referenceSchema) + } +} diff --git a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index 9bfd3347f00..9dff950ca5b 100644 --- a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -32,6 +32,7 @@ import com.databricks.sql.transaction.tahoe.schema.InvariantViolationException import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.delta._ +import com.nvidia.spark.rapids.shims.ParquetFieldIdShims import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.fs.Path @@ -72,7 +73,7 @@ class GpuOptimisticTransaction( * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } @@ -210,6 +211,10 @@ class GpuOptimisticTransaction( val hadoopConf = spark.sessionState.newHadoopConfWithOptions( metadata.configuration ++ deltaLog.options) + if (metadata.columnMappingMode == IdMapping) { + // Need Parquet field IDs when doing column ID mapping + ParquetFieldIdShims.setWriteIdOverride(hadoopConf, true) + } if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { val serializableHadoopConf = new SerializableConfiguration(hadoopConf) @@ -236,9 +241,11 @@ class GpuOptimisticTransaction( }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..773da5eb0bf --- /dev/null +++ b/delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMappingMode, DeltaParquetFileFormat} +import com.nvidia.spark.rapids.SparkPlanMeta + +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.types.StructType + +case class GpuDeltaParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormatBase { +} + +object GpuDeltaParquetFileFormat { + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = {} + + def convertToGpu(format: DeltaParquetFileFormat): GpuDeltaParquetFileFormat = { + GpuDeltaParquetFileFormat(format.columnMappingMode, format.referenceSchema) + } +} diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index b4f454ba920..fb9d3840917 100644 --- a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -72,7 +72,7 @@ class GpuOptimisticTransaction( * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } @@ -238,9 +238,11 @@ class GpuOptimisticTransaction( }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..969d005b573 --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,65 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids.delta + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMappingMode, DeltaParquetFileFormat, IdMapping} +import com.databricks.sql.transaction.tahoe.DeltaParquetFileFormat.IS_ROW_DELETED_COLUMN_NAME +import com.nvidia.spark.rapids.SparkPlanMeta +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +case class GpuDeltaParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType, + isSplittable: Boolean) extends GpuDeltaParquetFileFormatBase { + + if (columnMappingMode == IdMapping) { + val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredReadConf)), + s"${requiredReadConf.key} must be enabled to support Delta id column mapping mode") + val requiredWriteConf = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredWriteConf)), + s"${requiredWriteConf.key} must be enabled to support Delta id column mapping mode") + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = isSplittable +} + +object GpuDeltaParquetFileFormat { + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat.asInstanceOf[DeltaParquetFileFormat] + val requiredSchema = meta.wrapped.requiredSchema + if (requiredSchema.exists(_.name == IS_ROW_DELETED_COLUMN_NAME)) { + meta.willNotWorkOnGpu( + s"reading metadata column $IS_ROW_DELETED_COLUMN_NAME is not supported") + } + if (format.hasDeletionVectorMap()) { + meta.willNotWorkOnGpu("deletion vectors are not supported") + } + } + + def convertToGpu(fmt: DeltaParquetFileFormat): GpuDeltaParquetFileFormat = { + GpuDeltaParquetFileFormat(fmt.columnMappingMode, fmt.referenceSchema, fmt.isSplittable) + } +} diff --git a/dist/pom.xml b/dist/pom.xml index e40de9e7d33..171592ea691 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -40,7 +40,7 @@ - ${project.build.directory}/${project.build.finalName}-${cuda.version}.jar + ${project.build.directory}/${project.build.finalName}-${jni.classifier}.jar jar:file:${dist.jar.name}!/META-INF/maven/${project.groupId}/${project.artifactId}/pom.xml none @@ -223,7 +223,7 @@ org.apache.maven.plugins maven-jar-plugin - ${cuda.version} + ${jni.classifier} @@ -235,7 +235,7 @@ ${dist.jar.compress} ${project.build.directory}/parallel-world - ${cuda.version} + ${jni.classifier} @@ -412,7 +412,7 @@ self.log("... OK") com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} META-INF/** ${project.build.directory}/parallel-world true @@ -447,7 +447,7 @@ self.log("... OK") ${dist.jar.name} ${project.artifactId} - ${cuda.version} + ${jni.classifier} ${project.groupId} ${project.version} jar @@ -477,7 +477,7 @@ self.log("... OK") ${dist.jar.name} file://${java.io.tmpdir}/m2-repo ${project.artifactId} - ${cuda.version} + ${jni.classifier} ${project.groupId} jar - GdsTest true UTF-8 @@ -699,11 +700,10 @@ 333, 330cdh, 340, - 341 + 341, + 350 - - 321db, @@ -768,7 +768,7 @@ com.nvidia spark-rapids-jni ${spark-rapids-jni.version} - ${cuda.version} + ${jni.classifier} org.openucx @@ -1322,9 +1322,9 @@ - org.scalastyle + com.beautiful-scala scalastyle_${scala.binary.version} - 1.0.0 + 1.5.1 diff --git a/scalastyle-config.xml b/scalastyle-config.xml index af30a096710..acea68d8996 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -104,6 +104,23 @@ You can also disable only one rule, by specifying its rule id, as specified in: Use Javadoc style indentation for multiline comments + + + + + + + + def this\((.*)\) \{ + false + + procedure syntax is deprecated for constructors in Scala 2.13: add `=`, as in method definition + + + + procedure syntax is deprecated in Scala 2.13: add return type `: Unit` and `=` + + diff --git a/shuffle-plugin/pom.xml b/shuffle-plugin/pom.xml index 8ff9c2e8686..be645a73b2a 100644 --- a/shuffle-plugin/pom.xml +++ b/shuffle-plugin/pom.xml @@ -40,7 +40,7 @@ com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} org.scala-lang diff --git a/sql-plugin-api/pom.xml b/sql-plugin-api/pom.xml index 72940548d3b..564a98795a2 100644 --- a/sql-plugin-api/pom.xml +++ b/sql-plugin-api/pom.xml @@ -37,7 +37,7 @@ com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 11cbdca82fa..02c8534f1aa 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -39,7 +39,7 @@ com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} com.nvidia diff --git a/sql-plugin/src/main/java/com/nvidia/spark/Retryable.java b/sql-plugin/src/main/java/com/nvidia/spark/Retryable.java new file mode 100644 index 00000000000..db1bb0e1fe6 --- /dev/null +++ b/sql-plugin/src/main/java/com/nvidia/spark/Retryable.java @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark; + +/** + * An interface that can be used to retry the processing on non-deterministic + * expressions on the GPU. + * + * GPU memory is a limited resource. When it runs out the RAPIDS Accelerator + * for Apache Spark will use several different strategies to try and free more + * GPU memory to let the query complete. + * One of these strategies is to roll back the processioning for one task, pause + * the task thread, then retry the task when more memory is available. This + * works transparently for any stateless deterministic processing. But technically + * an expression/UDF can be non-deterministic and/or keep state in between calls. + * This interface provides a checkpoint method to save any needed state, and a + * restore method to reset the state in the case of a retry. + * + * Please note that a retry is not isolated to a single expression, so a restore can + * be called even after the expression returned one or more batches of results. And + * each time checkpoint it called any previously saved state can be overwritten. + */ +public interface Retryable { + /** + * Save the state, so it can be restored in the case of a retry. + * (This is called inside a Spark task context on executors.) + */ + void checkpoint(); + + /** + * Restore the state that was saved by calling to "checkpoint". + * (This is called inside a Spark task context on executors.) + */ + void restore(); +} diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java index e624852e19d..12236dcd54d 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java @@ -23,6 +23,7 @@ import java.util.NoSuchElementException; import ai.rapids.cudf.Scalar; +import com.nvidia.spark.rapids.CastOptions$; import com.nvidia.spark.rapids.GpuCast; import com.nvidia.spark.rapids.GpuColumnVector; import com.nvidia.spark.rapids.GpuScalar; @@ -158,7 +159,7 @@ static ColumnarBatch addUpcastsIfNeeded(ColumnarBatch batch, Schema expectedSche GpuColumnVector oldColumn = columns[i]; columns[i] = GpuColumnVector.from( GpuCast.doCast(oldColumn.getBase(), oldColumn.dataType(), expectedSparkType, - false, false, false), expectedSparkType); + CastOptions$.MODULE$.DEFAULT_CAST_OPTIONS()), expectedSparkType); } ColumnarBatch newBatch = new ColumnarBatch(columns, batch.numRows()); columns = null; diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java index b89ad62163e..c20300c1441 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -368,7 +368,7 @@ public String toString() { } /** Return a copy of "this" but with "queryUsesInputFile = true" */ - public GpuSparkBatchQueryScan copyWithInputFileTrue() { + public GpuSparkBatchQueryScan withInputFile() { return new GpuSparkBatchQueryScan(SparkSession.active(), table(), this.scan, readConf(), expectedSchema(), filterExpressions(), rapidsConf(), true // queryUsesInputFile diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java index 62cbd3d14c2..1c6ae738e1a 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java @@ -23,9 +23,9 @@ import java.util.stream.Collectors; import com.nvidia.spark.rapids.GpuMetric; +import com.nvidia.spark.rapids.GpuScanWrapper; import com.nvidia.spark.rapids.MultiFileReaderUtils; import com.nvidia.spark.rapids.RapidsConf; -import com.nvidia.spark.rapids.ScanWithMetricsWrapper; import com.nvidia.spark.rapids.iceberg.spark.Spark3Util; import com.nvidia.spark.rapids.iceberg.spark.SparkReadConf; import com.nvidia.spark.rapids.iceberg.spark.SparkSchemaUtil; @@ -67,7 +67,7 @@ * GPU-accelerated Iceberg Scan. * This is derived from Apache Iceberg's SparkScan class. */ -abstract class GpuSparkScan extends ScanWithMetricsWrapper +abstract class GpuSparkScan extends GpuScanWrapper implements Scan, SupportsReportStatistics { private static final Logger LOG = LoggerFactory.getLogger(GpuSparkScan.class); diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractHostByteBufferIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractHostByteBufferIterator.scala new file mode 100644 index 00000000000..24f3bacf28b --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractHostByteBufferIterator.scala @@ -0,0 +1,103 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids + +import java.nio.ByteBuffer + +import ai.rapids.cudf.{Cuda, HostMemoryBuffer, MemoryBuffer} + +abstract class AbstractHostByteBufferIterator + extends Iterator[ByteBuffer] { + private[this] var nextBufferStart: Long = 0L + + val totalLength: Long + + protected val limit: Long = Integer.MAX_VALUE + + def getByteBuffer(offset: Long, length: Long): ByteBuffer + + override def hasNext: Boolean = nextBufferStart < totalLength + + override def next(): ByteBuffer = { + val offset = nextBufferStart + val length = Math.min(totalLength - nextBufferStart, limit) + nextBufferStart += length + getByteBuffer(offset, length) + } +} + +/** + * Create an iterator that will emit ByteBuffer instances sequentially + * to work around the 2GB ByteBuffer size limitation. This allows + * the entire address range of a >2GB host buffer to be covered + * by a sequence of ByteBuffer instances. + *

NOTE: It is the caller's responsibility to ensure this iterator + * does not outlive the host buffer. The iterator DOES NOT increment + * the reference count of the host buffer to ensure it remains valid. + * + * @param hostBuffer host buffer to iterate + * @return ByteBuffer iterator + */ +class HostByteBufferIterator(hostBuffer: HostMemoryBuffer) + extends AbstractHostByteBufferIterator { + override protected val limit: Long = Integer.MAX_VALUE + + override val totalLength: Long = if (hostBuffer == null) { + 0 + } else { + hostBuffer.getLength + } + + override def getByteBuffer(offset: Long, length: Long): ByteBuffer = { + hostBuffer.asByteBuffer(offset, length.toInt) + } +} + +/** + * Create an iterator that will emit ByteBuffer instances sequentially + * to work around the 2GB ByteBuffer size limitation after copying a `MemoryBuffer` + * (which is likely a `DeviceMemoryBuffer`) to a host-backed bounce buffer + * that is likely smaller than 2GB. + * @note It is the caller's responsibility to ensure this iterator + * does not outlive `memoryBuffer`. The iterator DOES NOT increment + * the reference count of `memoryBuffer` to ensure it remains valid. + * @param memoryBuffer memory buffer to copy. This is likely a DeviceMemoryBuffer + * @param bounceBuffer a host bounce buffer that will be used to stage copies onto the host + * @param stream stream to synchronize on after staging to bounceBuffer + * @return ByteBuffer iterator + */ +class MemoryBufferToHostByteBufferIterator( + memoryBuffer: MemoryBuffer, + bounceBuffer: HostMemoryBuffer, + stream: Cuda.Stream) + extends AbstractHostByteBufferIterator { + override val totalLength: Long = if (memoryBuffer == null) { + 0 + } else { + memoryBuffer.getLength + } + + override protected val limit: Long = + Math.min(bounceBuffer.getLength, Integer.MAX_VALUE) + + override def getByteBuffer(offset: Long, length: Long): ByteBuffer = { + bounceBuffer + .copyFromMemoryBufferAsync(0, memoryBuffer, offset, length, stream) + stream.sync() + bounceBuffer.asByteBuffer(0, length.toInt) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala index 03bf3362062..3355a270b77 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,19 +16,13 @@ package com.nvidia.spark.rapids -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.connector.read.{PartitionReaderFactory, Scan} +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.rapids.GpuFileSourceScanExec -import org.apache.spark.sql.sources.Filter -import org.apache.spark.util.SerializableConfiguration trait AvroProvider { /** If the file format is supported as an external source */ - def isSupportedFormat(format: FileFormat): Boolean - - def isPerFileReadEnabledForFormat(format: FileFormat, conf: RapidsConf): Boolean + def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit @@ -38,19 +32,5 @@ trait AvroProvider { */ def getReadFileFormat(format: FileFormat): FileFormat - /** - * 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 - def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] - - def isSupportedScan(scan: Scan): Boolean - - def copyScanWithInputFileTrue(scan: Scan): Scan } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala index b258d892e36..5bd51bd9d33 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala @@ -21,14 +21,15 @@ import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import org.apache.spark.TaskContext /** - * Helper iterator that wraps a BufferedIterator of AutoCloseable subclasses. + * Helper iterator that wraps an Iterator of AutoCloseable subclasses. * This iterator also implements AutoCloseable, so it can be closed in case - * of exceptions. + * of exceptions and when close is called on it, its buffered item will be + * closed as well. * - * @param wrapped the buffered iterator + * @param wrapped the iterator we are wrapping for buffering * @tparam T an AutoCloseable subclass */ -class CloseableBufferedIterator[T <: AutoCloseable](wrapped: BufferedIterator[T]) +class CloseableBufferedIterator[T <: AutoCloseable](wrapped: Iterator[T]) extends BufferedIterator[T] with AutoCloseable { // Don't install the callback if in a unit test Option(TaskContext.get()).foreach { tc => @@ -38,13 +39,38 @@ class CloseableBufferedIterator[T <: AutoCloseable](wrapped: BufferedIterator[T] } private[this] var isClosed = false - override def head: T = wrapped.head - override def headOption: Option[T] = wrapped.headOption - override def next: T = wrapped.next - override def hasNext: Boolean = wrapped.hasNext + + private var hd: Option[T] = None + + def head: T = { + if (hd.isEmpty) { + hd = Some(next()) + } + hd.get + } + + override def headOption: Option[T] = { + if (hasNext) { + Some(head) + } else { + None + } + } + + override def next: T = if (hd.isDefined) { + val res = hd.get + hd = None + res + } else { + wrapped.next + } + + override def hasNext: Boolean = !isClosed && (hd.isDefined || wrapped.hasNext) + override def close(): Unit = { if (!isClosed) { - headOption.foreach(_.close()) + hd.foreach(_.close()) // close a buffered head item + hd = None isClosed = true } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala index 278b19bb661..69157c046b6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala @@ -21,6 +21,7 @@ import java.io.OutputStream import scala.collection.mutable import ai.rapids.cudf.{HostBufferConsumer, HostMemoryBuffer, NvtxColor, NvtxRange, TableWriter} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRestoreOnRetry, withRetry, withRetryNoSplit} @@ -186,7 +187,7 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, /** Apply any necessary casts before writing batch out */ def transformAndClose(cb: ColumnarBatch): ColumnarBatch = cb - private val checkpointRestore = new CheckpointRestore { + private val checkpointRestore = new Retryable { override def checkpoint(): Unit = () override def restore(): Unit = dropBufferedData() } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala index 8a4b46626d4..2cfcc542315 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.rapids.execution.TrampolineUtil * RMM event handler to trigger spilling from the device memory store. * @param store device memory store that will be triggered to spill * @param oomDumpDir local directory to create heap dumps on GPU OOM - * @param isGdsSpillEnabled true if GDS is enabled for device->disk spill * @param maxFailedOOMRetries maximum number of retries for OOMs after * depleting the device store */ @@ -39,7 +38,6 @@ class DeviceMemoryEventHandler( catalog: RapidsBufferCatalog, store: RapidsDeviceMemoryStore, oomDumpDir: Option[String], - isGdsSpillEnabled: Boolean, maxFailedOOMRetries: Int) extends RmmEventHandler with Logging { // Flag that ensures we dump stack traces once and not for every allocation @@ -159,11 +157,7 @@ class DeviceMemoryEventHandler( catalog.synchronousSpill(store, targetSize, Cuda.DEFAULT_STREAM) maybeAmountSpilled.foreach { amountSpilled => logInfo(s"Spilled $amountSpilled bytes from the device store") - if (isGdsSpillEnabled) { - TrampolineUtil.incTaskMetricsDiskBytesSpilled(amountSpilled) - } else { - TrampolineUtil.incTaskMetricsMemoryBytesSpilled(amountSpilled) - } + TrampolineUtil.incTaskMetricsMemoryBytesSpilled(amountSpilled) } true } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuApproximatePercentile.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuApproximatePercentile.scala index 2c3c98d20b9..1a188d47660 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuApproximatePercentile.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuApproximatePercentile.scala @@ -144,9 +144,7 @@ case class ApproxPercentileFromTDigestExpr( // array and return that (after converting from Double to finalDataType) withResource(cv.getBase.approxPercentile(Array(p))) { percentiles => withResource(percentiles.extractListElement(0)) { childView => - withResource(doCast(childView, DataTypes.DoubleType, finalDataType, - ansiMode = false, legacyCastToString = false, - stringToDateAnsiModeEnabled = false)) { childCv => + withResource(doCast(childView, DataTypes.DoubleType, finalDataType)) { childCv => GpuColumnVector.from(childCv.copyToColumnVector(), dataType) } } @@ -159,9 +157,7 @@ case class ApproxPercentileFromTDigestExpr( GpuColumnVector.from(percentiles.incRefCount(), dataType) } else { withResource(percentiles.getChildColumnView(0)) { childView => - withResource(doCast(childView, DataTypes.DoubleType, finalDataType, - ansiMode = false, legacyCastToString = false, - stringToDateAnsiModeEnabled = false)) { childCv => + withResource(doCast(childView, DataTypes.DoubleType, finalDataType)) { childCv => withResource(percentiles.replaceListChild(childCv)) { x => GpuColumnVector.from(x.copyToColumnVector(), dataType) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index db323c36eff..8b77ef6c71c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -51,9 +51,6 @@ trait ScanWithMetrics { var metrics : Map[String, GpuMetric] = Map.empty } -// Allows use of ScanWithMetrics from Java code -class ScanWithMetricsWrapper extends ScanWithMetrics - object GpuCSVScan { def tagSupport(scanMeta: ScanMeta[CSVScan]) : Unit = { val scan = scanMeta.wrapped @@ -213,7 +210,7 @@ case class GpuCSVScan( dataFilters: Seq[Expression], maxReaderBatchSizeRows: Integer, maxReaderBatchSizeBytes: Long) - extends TextBasedFileScan(sparkSession, options) with ScanWithMetrics { + extends TextBasedFileScan(sparkSession, options) with GpuScan { private lazy val parsedOptions: CSVOptions = new CSVOptions( options.asScala.toMap, @@ -251,6 +248,8 @@ case class GpuCSVScan( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) + override def withInputFile(): GpuScan = this + override def equals(obj: Any): Boolean = obj match { case c: GpuCSVScan => super.equals(c) && dataSchema == c.dataSchema && options == c.options && diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 573f3ced068..d9dded085c5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -50,24 +50,46 @@ final class CastExprMeta[INPUT <: UnaryExpression with TimeZoneAwareExpression w // We do not want to create a shim class for this small change stringToAnsiDate: Boolean, toTypeOverride: Option[DataType] = None) - extends UnaryExprMeta[INPUT](cast, conf, parent, rule) { + extends CastExprMetaBase(cast, conf, parent, rule, doFloatToIntCheck) { + + val legacyCastComplexTypesToString: Boolean = + SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING) + override val toType: DataType = toTypeOverride.getOrElse(cast.dataType) + + override def tagExprForGpu(): Unit = { + if (evalMode == GpuEvalMode.TRY) { + willNotWorkOnGpu("try_cast is not supported on the GPU") + } + recursiveTagExprForGpuCheck() + } def withToTypeOverride(newToType: DecimalType): CastExprMeta[INPUT] = new CastExprMeta[INPUT](cast, evalMode, conf, parent, rule, doFloatToIntCheck, stringToAnsiDate, Some(newToType)) + override def convertToGpu(child: Expression): GpuExpression = + GpuCast(child, toType, evalMode == GpuEvalMode.ANSI, cast.timeZoneId, + legacyCastComplexTypesToString, stringToAnsiDate) + +} + +/** Meta-data for cast, ansi_cast and ToPrettyString */ +abstract class CastExprMetaBase[INPUT <: UnaryExpression with TimeZoneAwareExpression]( + cast: INPUT, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule, + doFloatToIntCheck: Boolean = false) + extends UnaryExprMeta[INPUT](cast, conf, parent, rule) { + val fromType: DataType = cast.child.dataType - val toType: DataType = toTypeOverride.getOrElse(cast.dataType) - val legacyCastToString: Boolean = SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING) + val toType: DataType = cast.dataType override def tagExprForGpu(): Unit = { - if (evalMode == GpuEvalMode.TRY) { - willNotWorkOnGpu("try_cast is not supported on the GPU") - } recursiveTagExprForGpuCheck() } - private def recursiveTagExprForGpuCheck( + protected def recursiveTagExprForGpuCheck( fromDataType: DataType = fromType, toDataType: DataType = toType, depth: Int = 0): Unit = { @@ -155,14 +177,99 @@ final class CastExprMeta[INPUT <: UnaryExpression with TimeZoneAwareExpression w s"${entry.doc}. To enable this operation on the GPU, set ${entry.key} to true." } - override def convertToGpu(child: Expression): GpuExpression = - GpuCast(child, toType, evalMode == GpuEvalMode.ANSI, cast.timeZoneId, legacyCastToString, - stringToAnsiDate) - // timezone tagging in type checks is good enough, so always false override protected val needTimezoneTagging: Boolean = false } +object CastOptions { + val DEFAULT_CAST_OPTIONS = new CastOptions(false, false, false) + val ARITH_ANSI_OPTIONS = new CastOptions(false, true, false) + val TO_PRETTY_STRING_OPTIONS = ToPrettyStringOptions + + def getArithmeticCastOptions(failOnError: Boolean): CastOptions = + if (failOnError) ARITH_ANSI_OPTIONS else DEFAULT_CAST_OPTIONS + + object ToPrettyStringOptions extends CastOptions(false, false, false) { + override val leftBracket: String = "{" + + override val rightBracket: String = "}" + + override val nullString: String = "NULL" + + override val useDecimalPlainString: Boolean = true + + override val useHexFormatForBinary: Boolean = true + } +} + +/** + * This class is used to encapsulate parameters to use to help determine how to + * cast + * + * @param legacyCastComplexTypesToString If we should use legacy casting method + * @param ansiMode Whether the cast should be ANSI compliant + * @param stringToDateAnsiMode Whether to cast String to Date using ANSI compliance + */ +class CastOptions( + legacyCastComplexTypesToString: Boolean, + ansiMode: Boolean, + stringToDateAnsiMode: Boolean) extends Serializable { + + /** + * Retuns the left bracket to use when surrounding brackets when converting + * map or struct types to string + * example: + * [ "a" -> "b"] when legacyCastComplexTypesToString is enabled + * otherwise { "a" -> "b" } + */ + val leftBracket: String = if (legacyCastComplexTypesToString) "[" else "{" + + /** + * Returns the right bracket to use when surrounding brackets when converting + * map or struct types to string + * example: + * [ "a" -> "b"] when legacyCastComplexTypesToString is enabled + * otherwise { "a" -> "b" } + */ + val rightBracket: String = if (legacyCastComplexTypesToString) "]" else "}" + + /** + * Returns the string value to use to represent null elements in array/struct/map. + */ + val nullString: String = if (legacyCastComplexTypesToString) "" else "null" + + /** + * Returns whether a decimal value with exponents should be + * converted to a plain string, exactly like Java BigDecimal.toPlainString() + * example: + * plain string value of decimal 1.23E+7 is 12300000 + */ + val useDecimalPlainString: Boolean = ansiMode + + /** + * Returns whether the binary data should be printed as hex values + * instead of ascii values + */ + val useHexFormatForBinary: Boolean = false + + /** + * Returns whether we should cast using ANSI compliance + */ + val isAnsiMode: Boolean = ansiMode + + /** + * Returns whether we should use ANSI compliance when casting a String + * to Date + */ + val useAnsiStringToDateMode: Boolean = stringToDateAnsiMode + + /** + * Returns whether we should use legacy behavior to convert complex types + * like structs/maps to a String + */ + val useLegacyComplexTypesToString: Boolean = legacyCastComplexTypesToString +} + object GpuCast { private val DATE_REGEX_YYYY_MM_DD = "\\A\\d{4}\\-\\d{1,2}\\-\\d{1,2}([ T](:?[\\r\\n]|.)*)?\\Z" @@ -191,14 +298,13 @@ object GpuCast { input: ColumnView, fromDataType: DataType, toDataType: DataType, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { - + options: CastOptions = CastOptions.DEFAULT_CAST_OPTIONS): ColumnVector = { if (DataType.equalsStructurally(fromDataType, toDataType)) { return input.copyToColumnVector() } + val ansiMode = options.isAnsiMode + (fromDataType, toDataType) match { case (NullType, to) => GpuColumnVector.columnVectorFromNull(input.getRowCount.toInt, to) @@ -206,8 +312,12 @@ object GpuCast { case (DateType, BooleanType | _: NumericType) => // casts from date type to numerics are always null GpuColumnVector.columnVectorFromNull(input.getRowCount.toInt, toDataType) - case (DateType, StringType) => - input.asStrings("%Y-%m-%d") + + // Cast to String + case (DateType | TimestampType | FloatType | DoubleType | BinaryType | + _: DecimalType | _: ArrayType | _: MapType | _: StructType, StringType) => + castToString(input, fromDataType, options) + case (TimestampType, FloatType | DoubleType) => withResource(input.castTo(DType.INT64)) { asLongs => @@ -226,13 +336,13 @@ object GpuCast { toDataType match { case IntegerType => assertValuesInRange[Long](cv, Int.MinValue.toLong, - Int.MaxValue.toLong, errorMsg = GpuCast.OVERFLOW_MESSAGE) + Int.MaxValue.toLong, errorMsg = OVERFLOW_MESSAGE) case ShortType => assertValuesInRange[Long](cv, Short.MinValue.toLong, - Short.MaxValue.toLong, errorMsg = GpuCast.OVERFLOW_MESSAGE) + Short.MaxValue.toLong, errorMsg = OVERFLOW_MESSAGE) case ByteType => assertValuesInRange[Long](cv, Byte.MinValue.toLong, - Byte.MaxValue.toLong, errorMsg = GpuCast.OVERFLOW_MESSAGE) + Byte.MaxValue.toLong, errorMsg = OVERFLOW_MESSAGE) } } cv.castTo(GpuColumnVector.getNonNestedRapidsType(toDataType)) @@ -245,13 +355,6 @@ object GpuCast { asLongs.floorDiv(microsPerSec, GpuColumnVector.getNonNestedRapidsType(toDataType)) } } - case (TimestampType, StringType) => - castTimestampToString(input) - - case (StructType(fields), StringType) => - castStructToString(input, fields, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) - // ansi cast from larger-than-long integral-like types, to long case (dt: DecimalType, LongType) if ansiMode => // This is a work around for https://github.com/rapidsai/cudf/issues/9282 @@ -264,7 +367,7 @@ object GpuCast { withResource(input.max()) { maxInput => if (minInput.isValid && minInput.getBigDecimal().compareTo(min) == -1 || maxInput.isValid && maxInput.getBigDecimal().compareTo(max) == 1) { - throw new ArithmeticException(GpuCast.OVERFLOW_MESSAGE) + throw new ArithmeticException(OVERFLOW_MESSAGE) } } } @@ -427,9 +530,7 @@ object GpuCast { withResource(FloatUtils.nanToZero(input)) { inputWithNansToZero => inputWithNansToZero.castTo(GpuColumnVector.getNonNestedRapidsType(toDataType)) } - case (FloatType | DoubleType, StringType) => - castFloatingTypeToString(input) - case (StringType, ByteType | ShortType | IntegerType | LongType ) => + case (StringType, ByteType | ShortType | IntegerType | LongType) => CastStrings.toInteger(input, ansiMode, GpuColumnVector.getNonNestedRapidsType(toDataType)) case (StringType, FloatType | DoubleType) => @@ -441,7 +542,7 @@ object GpuCast { case BooleanType => castStringToBool(trimmed, ansiMode) case DateType => - if (stringToDateAnsiModeEnabled) { + if (options.useAnsiStringToDateMode) { castStringToDateAnsi(trimmed, ansiMode) } else { castStringToDate(trimmed) @@ -459,34 +560,18 @@ object GpuCast { case (ShortType | IntegerType | LongType | ByteType | StringType, BinaryType) => input.asByteList(true) - case (BinaryType, StringType) => - castBinToString(input) - - case (_: DecimalType, StringType) => - GpuCastShims.CastDecimalToString(input, ansiMode) - case (ArrayType(nestedFrom, _), ArrayType(nestedTo, _)) => withResource(input.getChildColumnView(0)) { childView => - withResource(doCast(childView, nestedFrom, nestedTo, - ansiMode, legacyCastToString, stringToDateAnsiModeEnabled)) { childColumnVector => + withResource(doCast(childView, nestedFrom, nestedTo, options)) { childColumnVector => withResource(input.replaceListChild(childColumnVector))(_.copyToColumnVector()) } } - case (ArrayType(elementType, _), StringType) => - castArrayToString( - input, elementType, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled - ) - case (from: StructType, to: StructType) => - castStructToStruct(from, to, input, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) + castStructToStruct(from, to, input, options) case (from: MapType, to: MapType) => - castMapToMap(from, to, input, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) - - case (from: MapType, _: StringType) => - castMapToString(input, from, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + castMapToMap(from, to, input, options) case (dayTime: DataType, _: StringType) if GpuTypeShims.isSupportedDayTimeType(dayTime) => GpuIntervalUtils.toDayTimeIntervalString(input, dayTime) @@ -548,7 +633,7 @@ object GpuCast { maxValue: T, inclusiveMin: Boolean = true, inclusiveMax: Boolean = true, - errorMsg:String = GpuCast.OVERFLOW_MESSAGE) + errorMsg: String = OVERFLOW_MESSAGE) (implicit ord: Ordering[T]): Unit = { def throwIfAnyNan(): Unit = { @@ -635,6 +720,25 @@ object GpuCast { } } + def castToString( + input: ColumnView, + fromDataType: DataType, options: CastOptions): ColumnVector = fromDataType match { + case StringType => input.copyToColumnVector() + case DateType => input.asStrings("%Y-%m-%d") + case TimestampType => castTimestampToString(input) + case FloatType | DoubleType => castFloatingTypeToString(input) + case BinaryType => castBinToString(input, options) + case _: DecimalType => GpuCastShims.CastDecimalToString(input, options.useDecimalPlainString) + case StructType(fields) => castStructToString(input, fields, options) + + case ArrayType(elementType, _) => + castArrayToString(input, elementType, options) + case from: MapType => + castMapToString(input, from, options) + case _ => + input.castTo(GpuColumnVector.getNonNestedRapidsType(StringType)) + } + private def castTimestampToString(input: ColumnView): ColumnVector = { // the complexity in this function is due to Spark's rules for truncating // the fractional part of the timestamp string. Any trailing decimal place @@ -684,16 +788,21 @@ object GpuCast { */ private def concatenateStringArrayElements( input: ColumnView, - legacyCastToString: Boolean): ColumnVector = { + options: CastOptions, + castingBinaryData: Boolean = false): ColumnVector = { + + import options._ + val emptyStr = "" val spaceStr = " " - val nullStr = if (legacyCastToString) "" else "null" - val sepStr = if (legacyCastToString) "," else ", " + val sepStr = if (useHexFormatForBinary && castingBinaryData) spaceStr + else if (useLegacyComplexTypesToString) "," else ", " + withResource( - Seq(emptyStr, spaceStr, nullStr, sepStr).safeMap(Scalar.fromString) - ){ case Seq(empty, space, nullRep, sep) => + Seq(emptyStr, spaceStr, nullString, sepStr).safeMap(Scalar.fromString) + ) { case Seq(empty, space, nullRep, sep) => - val withSpacesIfLegacy = if (!legacyCastToString) { + val withSpacesIfLegacy = if (!useLegacyComplexTypesToString) { withResource(input.getChildColumnView(0)) { _.replaceNulls(nullRep) } @@ -724,7 +833,7 @@ object GpuCast { val strCol = withResource(concatenated) { _.replaceNulls(empty) } - if (!legacyCastToString) { + if (!useLegacyComplexTypesToString) { strCol } else { // If the first char of a string is ' ', remove it (only for legacyCastToString = true) @@ -741,26 +850,24 @@ object GpuCast { private def castArrayToString(input: ColumnView, elementType: DataType, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { - - val (leftStr, rightStr) = ("[", "]") + options: CastOptions, + castingBinaryData: Boolean = false): ColumnVector = { + // We use square brackets for arrays regardless + val (leftStr, rightStr) = ("[", "]") val emptyStr = "" - val nullStr = if (legacyCastToString) "" else "null" val numRows = input.getRowCount.toInt withResource( - Seq(leftStr, rightStr, emptyStr, nullStr).safeMap(Scalar.fromString) + Seq(leftStr, rightStr, emptyStr, options.nullString).safeMap(Scalar.fromString) ){ case Seq(left, right, empty, nullRep) => val strChildContainsNull = withResource(input.getChildColumnView(0)) {child => doCast( - child, elementType, StringType, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + child, elementType, StringType, options) } val concatenated = withResource(strChildContainsNull) { _ => withResource(input.replaceListChild(strChildContainsNull)) { - concatenateStringArrayElements(_, legacyCastToString) + concatenateStringArrayElements(_, options, castingBinaryData) } } @@ -782,45 +889,43 @@ object GpuCast { private def castMapToString( input: ColumnView, from: MapType, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { val numRows = input.getRowCount.toInt val (arrowStr, emptyStr, spaceStr) = ("->", "", " ") - val (leftStr, rightStr, nullStr) = - if (legacyCastToString) ("[", "]", "") else ("{", "}", "null") // cast the key column and value column to string columns val (strKey, strValue) = withResource(input.getChildColumnView(0)) { kvStructColumn => val strKey = withResource(kvStructColumn.getChildColumnView(0)) { keyColumn => - doCast( - keyColumn, from.keyType, StringType, ansiMode, - legacyCastToString, stringToDateAnsiModeEnabled) + castToString(keyColumn, from.keyType, options) } - val strValue = closeOnExcept(strKey) {_ => + val strValue = closeOnExcept(strKey) { _ => withResource(kvStructColumn.getChildColumnView(1)) { valueColumn => - doCast( - valueColumn, from.valueType, StringType, ansiMode, - legacyCastToString, stringToDateAnsiModeEnabled) + castToString(valueColumn, from.valueType, options) } } (strKey, strValue) } + import options._ // concatenate the key-value pairs to string // Example: ("key", "value") -> "key -> value" withResource( - Seq(leftStr, rightStr, arrowStr, emptyStr, nullStr, spaceStr).safeMap(Scalar.fromString) + Seq(leftBracket, + rightBracket, + arrowStr, + emptyStr, + nullString, + spaceStr).safeMap(Scalar.fromString) ) { case Seq(leftScalar, rightScalar, arrowScalar, emptyScalar, nullScalar, spaceScalar) => val strElements = withResource(Seq(strKey, strValue)) { case Seq(strKey, strValue) => val numElements = strKey.getRowCount.toInt withResource(Seq(spaceScalar, arrowScalar).safeMap(ColumnVector.fromScalar(_, numElements)) - ) {case Seq(spaceCol, arrowCol) => - if (legacyCastToString) { + ) { case Seq(spaceCol, arrowCol) => + if (useLegacyComplexTypesToString) { withResource( spaceCol.mergeAndSetValidity(BinaryOp.BITWISE_AND, strValue) - ) {spaceBetweenSepAndVal => + ) { spaceBetweenSepAndVal => ColumnVector.stringConcatenate( emptyScalar, nullScalar, Array(strKey, spaceCol, arrowCol, spaceBetweenSepAndVal, strValue)) @@ -835,7 +940,7 @@ object GpuCast { // concatenate elements val strCol = withResource(strElements) { _ => withResource(input.replaceListChild(strElements)) { - concatenateStringArrayElements(_, legacyCastToString) + concatenateStringArrayElements(_, options) } } val resPreValidityFix = withResource(strCol) { _ => @@ -855,14 +960,12 @@ object GpuCast { private def castStructToString( input: ColumnView, inputSchema: Array[StructField], - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { + + import options._ - val (leftStr, rightStr) = if (legacyCastToString) ("[", "]") else ("{", "}") val emptyStr = "" - val nullStr = if (legacyCastToString) "" else "null" - val separatorStr = if (legacyCastToString) "," else ", " + val separatorStr = if (useLegacyComplexTypesToString) "," else ", " val spaceStr = " " val numRows = input.getRowCount.toInt val numInputColumns = input.getNumChildren @@ -879,8 +982,7 @@ object GpuCast { // 3.1+: {firstCol columns += leftColumn.incRefCount() withResource(input.getChildColumnView(0)) { firstColumnView => - columns += doCast(firstColumnView, inputSchema.head.dataType, StringType, - ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + columns += castToString(firstColumnView, inputSchema.head.dataType, options) } for (nonFirstIndex <- 1 until numInputColumns) { withResource(input.getChildColumnView(nonFirstIndex)) { nonFirstColumnView => @@ -888,9 +990,8 @@ object GpuCast { // 3.1+: ", " columns += sepColumn.incRefCount() val nonFirstColumn = doCast(nonFirstColumnView, - inputSchema(nonFirstIndex).dataType, StringType, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) - if (legacyCastToString) { + inputSchema(nonFirstIndex).dataType, StringType, options) + if (useLegacyComplexTypesToString) { // " " if non-null columns += spaceColumn.mergeAndSetValidity(BinaryOp.BITWISE_AND, nonFirstColumnView) } @@ -905,8 +1006,8 @@ object GpuCast { } } - withResource(Seq(emptyStr, nullStr, separatorStr, spaceStr, leftStr, rightStr) - .safeMap(Scalar.fromString)) { + withResource(Seq(emptyStr, nullString, separatorStr, spaceStr, leftBracket, rightBracket) + .safeMap(Scalar.fromString)) { case Seq(emptyScalar, nullScalar, columnScalars@_*) => withResource( @@ -955,7 +1056,7 @@ object GpuCast { if (ansiEnabled) { withResource(validBools.all()) { isAllBool => if (isAllBool.isValid && !isAllBool.getBoolean) { - throw new IllegalStateException(GpuCast.INVALID_INPUT_MESSAGE) + throw new IllegalStateException(INVALID_INPUT_MESSAGE) } } } @@ -996,8 +1097,8 @@ object GpuCast { } } - /** This method does not close the `input` ColumnVector. */ - def convertDateOr( + /** This method does not close the `input` ColumnVector. */ + def convertDateOr( input: ColumnVector, regex: String, cudfFormat: String, @@ -1216,20 +1317,16 @@ object GpuCast { from: MapType, to: MapType, input: ColumnView, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { // For cudf a map is a list of (key, value) structs, but lets keep it in ColumnView as much // as possible withResource(input.getChildColumnView(0)) { kvStructColumn => val castKey = withResource(kvStructColumn.getChildColumnView(0)) { keyColumn => - doCast(keyColumn, from.keyType, to.keyType, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) + doCast(keyColumn, from.keyType, to.keyType, options) } withResource(castKey) { castKey => val castValue = withResource(kvStructColumn.getChildColumnView(1)) { valueColumn => - doCast(valueColumn, from.valueType, to.valueType, - ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + doCast(valueColumn, from.valueType, to.valueType, options) } withResource(castValue) { castValue => withResource(ColumnView.makeStructView(castKey, castValue)) { castKvStructColumn => @@ -1248,17 +1345,13 @@ object GpuCast { from: StructType, to: StructType, input: ColumnView, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { withResource(new ArrayBuffer[ColumnVector](from.length)) { childColumns => from.indices.foreach { index => childColumns += doCast( input.getChildColumnView(index), from(index).dataType, - to(index).dataType, - ansiMode, - legacyCastToString, stringToDateAnsiModeEnabled) + to(index).dataType, options) } withResource(ColumnView.makeStructView(childColumns: _*)) { casted => if (input.getNullCount == 0) { @@ -1274,17 +1367,27 @@ object GpuCast { } } - private def castBinToString(input: ColumnView): ColumnVector = { - // Spark interprets the binary as UTF-8 bytes. So the layout of the - // binary and the layout of the string are the same. We just need to play some games with - // the CPU side metadata to make CUDF think it is a String. - // Sadly there is no simple CUDF API to do this, so for now we pull it apart and put - // it back together again - withResource(input.getChildColumnView(0)) { dataCol => - withResource(new ColumnView(DType.STRING, input.getRowCount, - Optional.of[java.lang.Long](input.getNullCount), - dataCol.getData, input.getValid, input.getOffsets)) { cv => - cv.copyToColumnVector() + private def castBinToString(input: ColumnView, options: CastOptions): ColumnVector = { + if (options.useHexFormatForBinary) { + withResource(input.getChildColumnView(0)) { dataCol => + withResource(dataCol.toHex()) { stringCol => + withResource(input.replaceListChild(stringCol)) { cv => + castArrayToString(cv, DataTypes.StringType, options, true) + } + } + } + } else { + // Spark interprets the binary as UTF-8 bytes. So the layout of the + // binary and the layout of the string are the same. We just need to play some games with + // the CPU side metadata to make CUDF think it is a String. + // Sadly there is no simple CUDF API to do this, so for now we pull it apart and put + // it back together again + withResource(input.getChildColumnView(0)) { dataCol => + withResource(new ColumnView(DType.STRING, input.getRowCount, + Optional.of[java.lang.Long](input.getNullCount), + dataCol.getData, input.getValid, input.getOffsets)) { cv => + cv.copyToColumnVector() + } } } } @@ -1372,7 +1475,7 @@ object GpuCast { if (ansiMode) { withResource(outOfBounds.any()) { isAny => if (isAny.isValid && isAny.getBoolean) { - throw RapidsErrorUtils.arithmeticOverflowError(GpuCast.OVERFLOW_MESSAGE) + throw RapidsErrorUtils.arithmeticOverflowError(OVERFLOW_MESSAGE) } } input.copyToColumnVector() @@ -1473,7 +1576,7 @@ object GpuCast { val cv = withResource(updatedMaxRet) { updatedMax => withResource(Seq(minSeconds, Long.MinValue).safeMap(Scalar.fromLong)) { case Seq(minSecondsS, longMinS) => - withResource(longInput.lessThan(minSecondsS)){ + withResource(longInput.lessThan(minSecondsS)) { _.ifElse(longMinS, updatedMax) } } @@ -1492,12 +1595,15 @@ case class GpuCast( dataType: DataType, ansiMode: Boolean = false, timeZoneId: Option[String] = None, - legacyCastToString: Boolean = false, + legacyCastComplexTypesToString: Boolean = false, stringToDateAnsiModeEnabled: Boolean = false) extends GpuUnaryExpression with TimeZoneAwareExpression with NullIntolerant { import GpuCast._ + private val options: CastOptions = + new CastOptions(legacyCastComplexTypesToString, ansiMode, stringToDateAnsiModeEnabled) + // when ansi mode is enabled, some cast expressions can throw exceptions on invalid inputs override def hasSideEffects: Boolean = super.hasSideEffects || { (child.dataType, dataType) match { @@ -1563,7 +1669,5 @@ case class GpuCast( } override def doColumnar(input: GpuColumnVector): ColumnVector = - doCast(input.getBase, input.dataType(), dataType, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) - + doCast(input.getBase, input.dataType(), dataType, options) } \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala index 13928fceb89..e7acd575285 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala @@ -409,6 +409,26 @@ abstract class AbstractGpuCoalesceIterator( } } + /** + * A Simple wrapper around a ColumnarBatch to let us avoid closing it in some cases. + */ + private class BatchWrapper(var cb: ColumnarBatch) extends AutoCloseable { + def get: ColumnarBatch = cb + + def release: ColumnarBatch = { + val tmp = cb + cb = null + tmp + } + + override def close(): Unit = { + if (cb != null) { + cb.close() + cb = null + } + } + } + /** * Add input batches to the `batches` collection up to the limit specified * by the goal. Note: for a size goal, if any incoming batch is greater than this size @@ -447,13 +467,13 @@ abstract class AbstractGpuCoalesceIterator( } while(maybeFilteredIter.hasNext) { - var cb = maybeFilteredIter.next() + val cb = new BatchWrapper(maybeFilteredIter.next()) closeOnExcept(cb) { _ => - val nextRows = cb.numRows() + val nextRows = cb.get.numRows() // filter out empty batches if (nextRows > 0) { numInputRows += nextRows - val nextBytes = getBatchDataSize(cb) + val nextBytes = getBatchDataSize(cb.get) // calculate the new sizes based on this input batch being added to the current // output batch @@ -488,9 +508,10 @@ abstract class AbstractGpuCoalesceIterator( } } // 2) Filter the incoming batch. - val filteredCbIter = GpuFilter.filterAndClose(cb, filterTier, + // filterAndClose takes ownership of CB so we should not close it on a failure + // anymore... + val filteredCbIter = GpuFilter.filterAndClose(cb.release, filterTier, NoopMetric, NoopMetric, opTime) - cb = null // null out `cb` to prevent multiple close calls while (filteredCbIter.hasNext) { closeOnExcept(filteredCbIter.next()) { filteredCb => val filteredWouldBeRows = filteredNumRows + filteredCb.numRows() @@ -521,23 +542,23 @@ abstract class AbstractGpuCoalesceIterator( s"At least $wouldBeRows are in this partition, even after filtering " + s"nulls. Please try increasing your partition count.") } - case _ => saveOnDeck(cb) // not a single batch requirement + case _ => saveOnDeck(cb.get) // not a single batch requirement } } else if (batchRowLimit > 0 && wouldBeRows > batchRowLimit) { - saveOnDeck(cb) + saveOnDeck(cb.get) } else if (wouldBeBytes > goal.targetSizeBytes && numBytes > 0) { // There are no explicit checks for the concatenate result exceeding the cudf 2^31 // row count limit for any column. We are relying on cudf's concatenate to throw // an exception if this occurs and limiting performance-oriented goals to under // 2GB data total to avoid hitting that error. - saveOnDeck(cb) + saveOnDeck(cb.get) } else { - addBatch(cb) + addBatch(cb.get) numRows = wouldBeRows numBytes = wouldBeBytes } } else { - cleanupInputBatch(cb) + cleanupInputBatch(cb.get) } } // end of closeOnExcept(cb) } // end of while(maybeFilteredIter.hasNext) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala index 6ccf12af5ae..291171ccebb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala @@ -223,7 +223,7 @@ class ColumnarToRowIterator(batches: Iterator[ColumnarBatch], opTime: GpuMetric, streamTime: GpuMetric, nullSafe: Boolean = false, - releaseSemaphore: Boolean = true) extends Iterator[InternalRow] { + releaseSemaphore: Boolean = true) extends Iterator[InternalRow] with AutoCloseable { // GPU batches read in must be closed by the receiver (us) @transient private var cb: ColumnarBatch = null private var it: java.util.Iterator[InternalRow] = null @@ -241,6 +241,8 @@ class ColumnarToRowIterator(batches: Iterator[ColumnarBatch], } } + override def close(): Unit = closeCurrentBatch() + private def closeCurrentBatch(): Unit = { if (cb != null) { cb.close() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala index 96a50e78cd4..c46862ab2aa 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{ast, BinaryOp, BinaryOperable, ColumnVector, DType, Scalar, UnaryOp} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{withResource, withResourceIfAllowed} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.{ShimBinaryExpression, ShimExpression, ShimTernaryExpression, ShimUnaryExpression} @@ -90,6 +91,19 @@ object GpuExpressionsUtils { case ga: GpuAlias => extractGpuLit(ga.child) case _ => None } + + /** + * Collect the Retryables from a Seq of expression. + */ + def collectRetryables(expressions: Seq[Expression]): Seq[Retryable] = { + // There should be no dependence between expression and its children for + // the checkpoint and restore operations. + expressions.flatMap { expr => + expr.collect { + case r: Retryable => r + } + } + } } /** @@ -169,10 +183,49 @@ trait GpuExpression extends Expression { * this is seen. */ def disableTieredProjectCombine: Boolean = hasSideEffects + + /** + * Whether an expression itself is non-deterministic when its "deterministic" is false, + * no matter whether it has any non-deterministic children. + * An expression is actually a tree, and deterministic being false means there is at + * least one tree node is non-deterministic, but we need to know the exact nodes which + * are non-deterministic to check if it implements the Retryable. + * + * Default to false because Spark checks only children by default in Expression. So it + * is non-deterministic iff it has non-deterministic children. + * + * NOTE When overriding "deterministic", this should be taken care of. + */ + val selfNonDeterministic: Boolean = false + + /** + * true means this expression can be used inside a retry block, otherwise false. + * An expression is retryable when + * - it is deterministic, or + * - when being non-deterministic, it is a Retryable and its children are all retryable. + */ + lazy val retryable: Boolean = deterministic || { + val childrenAllRetryable = children.forall(_.asInstanceOf[GpuExpression].retryable) + if (selfNonDeterministic || children.forall(_.deterministic)) { + // self is non-deterministic, so need to check if it is a Retryable. + // + // "selfNonDeterministic" should be reliable enough, but it is still good to + // do this check for one case we are 100% sure self is non-deterministic (its + // "deterministic" is false but its children are all deterministic). This can + // minimize the possibility of missing expressions that happen to forget + // overriding "selfNonDeterministic" correctly. + this.isInstanceOf[Retryable] && childrenAllRetryable + } else { + childrenAllRetryable + } + } } abstract class GpuLeafExpression extends GpuExpression with ShimExpression { override final def children: Seq[Expression] = Nil + + /* no children, so only self can be non-deterministic */ + override final val selfNonDeterministic: Boolean = !deterministic } trait GpuUnevaluable extends GpuExpression { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 919bd539abe..f971ddd2aa4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -84,7 +84,7 @@ case class GpuOrcScan( dataFilters: Seq[Expression], rapidsConf: RapidsConf, queryUsesInputFile: Boolean = false) - extends ScanWithMetrics with FileScan with Logging { + extends FileScan with GpuScan with Logging { override def isSplitable(path: Path): Boolean = true @@ -124,6 +124,8 @@ case class GpuOrcScan( def withFilters( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) + + override def withInputFile(): GpuScan = copy(queryUsesInputFile = true) } object GpuOrcScan { @@ -381,8 +383,7 @@ object GpuOrcScan { case (f: DType, t: DType) if f.isDecimalType && t.isDecimalType => val fromDataType = DecimalType(f.getDecimalMaxPrecision, -f.getScale) val toDataType = DecimalType(t.getDecimalMaxPrecision, -t.getScale) - GpuCast.doCast(col, fromDataType, toDataType, ansiMode=false, legacyCastToString = false, - stringToDateAnsiModeEnabled = false) + GpuCast.doCast(col, fromDataType, toDataType) case (DType.STRING, DType.STRING) if originalFromDt.isInstanceOf[CharType] => // Trim trailing whitespace off of output strings, to match CPU output. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 7c60b561393..bb8668cbb9b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -324,24 +324,23 @@ final class InsertIntoHadoopFsRelationCommandMeta( } val spark = SparkSession.active - - fileFormat = cmd.fileFormat match { - case _: CSVFileFormat => - willNotWorkOnGpu("CSV output is not supported") - None - case _: JsonFileFormat => - willNotWorkOnGpu("JSON output is not supported") - None - case f if GpuOrcFileFormat.isSparkOrcFormat(f) => - GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema) - case _: ParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema) - case _: TextFileFormat => - willNotWorkOnGpu("text output is not supported") - None - case f => - willNotWorkOnGpu(s"unknown file format: ${f.getClass.getCanonicalName}") - None + val formatCls = cmd.fileFormat.getClass + fileFormat = if (formatCls == classOf[CSVFileFormat]) { + willNotWorkOnGpu("CSV output is not supported") + None + } else if (formatCls == classOf[JsonFileFormat]) { + willNotWorkOnGpu("JSON output is not supported") + None + } else if (GpuOrcFileFormat.isSparkOrcFormat(formatCls)) { + GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema) + } else if (formatCls == classOf[ParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema) + } else if (formatCls == classOf[TextFileFormat]) { + willNotWorkOnGpu("text output is not supported") + None + } else { + willNotWorkOnGpu(s"unknown file format: ${formatCls.getCanonicalName}") + None } } @@ -393,7 +392,7 @@ trait GpuOverridesListener { def optimizedPlan( plan: SparkPlanMeta[SparkPlan], sparkPlan: SparkPlan, - costOptimizations: Seq[Optimization]) + costOptimizations: Seq[Optimization]): Unit } sealed trait FileFormatType @@ -3086,6 +3085,35 @@ object GpuOverrides extends Logging { |For instance decimal strings not longer than 18 characters / hexadecimal strings |not longer than 15 characters disregarding the sign cannot cause an overflow. """.stripMargin.replaceAll("\n", " ")), + expr[FormatNumber]( + "Formats the number x like '#,###,###.##', rounded to d decimal places.", + ExprChecks.binaryProject(TypeSig.STRING, TypeSig.STRING, + ("x", TypeSig.gpuNumeric, TypeSig.cpuNumeric), + ("d", TypeSig.lit(TypeEnum.INT), TypeSig.INT+TypeSig.STRING)), + (in, conf, p, r) => new BinaryExprMeta[FormatNumber](in, conf, p, r) { + override def tagExprForGpu(): Unit = { + in.children.head.dataType match { + case _: FloatType | DoubleType => { + if (!conf.isFloatFormatNumberEnabled) { + willNotWorkOnGpu("format_number with floating point types on the GPU returns " + + "results that have a different precision than the default results of Spark. " + + "To enable this operation on the GPU, set" + + s" ${RapidsConf.ENABLE_FLOAT_FORMAT_NUMBER} to true.") + } + } + case dt: DecimalType => { + if (dt.scale > 32) { + willNotWorkOnGpu("format_number will generate results mismatched from Spark " + + "when the scale is larger than 32.") + } + } + case _ => + } + } + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = + GpuFormatNumber(lhs, rhs) + } + ), expr[MapConcat]( "Returns the union of all the given maps", ExprChecks.projectOnly(TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + @@ -3619,7 +3647,7 @@ object GpuOverrides extends Logging { (a, conf, p, r) => new ScanMeta[CSVScan](a, conf, p, r) { override def tagSelfForGpu(): Unit = GpuCSVScan.tagSupport(this) - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuCSVScan(a.sparkSession, a.fileIndex, a.dataSchema, @@ -3636,7 +3664,7 @@ object GpuOverrides extends Logging { (a, conf, p, r) => new ScanMeta[JsonScan](a, conf, p, r) { override def tagSelfForGpu(): Unit = GpuJsonScan.tagSupport(this) - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuJsonScan(a.sparkSession, a.fileIndex, a.dataSchema, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 0bed9b47b4f..cff38e3e3ec 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -250,7 +250,7 @@ class GpuParquetFileFormat extends ColumnarFileFormat with Logging { conf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key, outputTimestampType.toString) ParquetFieldIdShims.setupParquetFieldIdWriteConfig(conf, sqlConf) - val parquetFieldIdWriteEnabled = ParquetFieldIdShims.getParquetIdWriteEnabled(sqlConf) + val parquetFieldIdWriteEnabled = ParquetFieldIdShims.getParquetIdWriteEnabled(conf, sqlConf) ParquetTimestampNTZShims.setupTimestampNTZConfig(conf, sqlConf) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index d13b9617ae9..53b783581a4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -108,7 +108,7 @@ case class GpuParquetScan( dataFilters: Seq[Expression], rapidsConf: RapidsConf, queryUsesInputFile: Boolean = false) - extends ScanWithMetrics with FileScan with Logging { + extends FileScan with GpuScan with Logging { override def isSplitable(path: Path): Boolean = true @@ -146,6 +146,8 @@ case class GpuParquetScan( def withFilters( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) + + override def withInputFile(): GpuScan = copy(queryUsesInputFile = true) } object GpuParquetScan { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala index 7cf2a9fbd6e..79403bae6b5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,12 +19,15 @@ package com.nvidia.spark.rapids import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -66,6 +69,15 @@ class GpuReadCSVFileFormat extends CSVFileFormat with GpuReadFileFormatWithMetri options) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = true + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + throw new IllegalStateException("CSV format does not support multifile reads") + } } object GpuReadCSVFileFormat { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala index 488aea02039..9b384916ab5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2021, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -22,11 +22,15 @@ package com.nvidia.spark.rapids import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.Job +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration trait GpuReadFileFormatWithMetrics extends FileFormat { final override def supportBatch(spark: SparkSession, dataSchema: StructType): Boolean = true @@ -62,4 +66,11 @@ trait GpuReadFileFormatWithMetrics extends FileFormat { metrics: Map[String, GpuMetric], alluxioPathReplacementMap: Option[Map[String, String]]) : PartitionedFile => Iterator[InternalRow] + + def isPerFileReadEnabled(conf: RapidsConf): Boolean + + def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala index b6dd08856a0..1792f36e09b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,11 +19,14 @@ package com.nvidia.spark.rapids import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -58,6 +61,24 @@ class GpuReadOrcFileFormat extends OrcFileFormat with GpuReadFileFormatWithMetri options) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = conf.isOrcPerFileReadEnabled + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuOrcMultiFilePartitionReaderFactory( + fileScan.conf, + broadcastedConf, + fileScan.relation.dataSchema, + fileScan.requiredSchema, + fileScan.readPartitionSchema, + pushedFilters, + fileScan.rapidsConf, + fileScan.allMetrics, + fileScan.queryUsesInputFile) + } } object GpuReadOrcFileFormat { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala index 8718219222c..e9b8beba036 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,11 +19,14 @@ package com.nvidia.spark.rapids import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -59,6 +62,25 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW alluxioPathReplacementMap) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = conf.isParquetPerFileReadEnabled + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuParquetMultiFilePartitionReaderFactory( + fileScan.conf, + broadcastedConf, + fileScan.relation.dataSchema, + fileScan.requiredSchema, + fileScan.readPartitionSchema, + pushedFilters, + fileScan.rapidsConf, + fileScan.allMetrics, + fileScan.queryUsesInputFile, + fileScan.alluxioPathsMap) + } } object GpuReadParquetFileFormat { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala index d41bc12e07c..e3869960fc4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala @@ -75,7 +75,7 @@ object GpuRunnableCommand { } } - def assertEmptyRootPath(tablePath: URI, saveMode: SaveMode, hadoopConf: Configuration) { + def assertEmptyRootPath(tablePath: URI, saveMode: SaveMode, hadoopConf: Configuration): Unit = { if (saveMode == SaveMode.ErrorIfExists && !getAllowNonEmptyLocationInCTAS) { val filePath = new org.apache.hadoop.fs.Path(tablePath) val fs = filePath.getFileSystem(hadoopConf) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuScan.scala new file mode 100644 index 00000000000..eb3e8a71e2d --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuScan.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids + +import org.apache.spark.sql.connector.read.Scan + +trait GpuScan extends Scan with ScanWithMetrics { + /** Create a version of this scan with input file name support */ + def withInputFile(): GpuScan +} + +// Allows use of GpuScan from Java code +abstract class GpuScanWrapper extends GpuScan diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala index 1222422393b..3d904c421b1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala @@ -253,7 +253,7 @@ object GpuShuffledHashJoinExec extends Logging { (Either[ColumnarBatch, Iterator[ColumnarBatch]], Iterator[ColumnarBatch]) = { val buildTime = coalesceMetrics(GpuMetric.BUILD_TIME) val buildTypes = buildOutput.map(_.dataType).toArray - closeOnExcept(new CloseableBufferedIterator(buildIter.buffered)) { bufBuildIter => + closeOnExcept(new CloseableBufferedIterator(buildIter)) { bufBuildIter => val startTime = System.nanoTime() // Batches type detection val isBuildSerialized = bufBuildIter.hasNext && isBatchSerialized(bufBuildIter.head) @@ -401,7 +401,7 @@ object GpuShuffledHashJoinExec extends Logging { // will grab the semaphore when putting the first stream batch on the GPU, and // then we bring the build batch to the GPU and return. withResource(hostConcatResult) { _ => - closeOnExcept(new CloseableBufferedIterator(streamIter.buffered)) { bufStreamIter => + closeOnExcept(new CloseableBufferedIterator(streamIter)) { bufStreamIter => withResource(new NvtxRange("first stream batch", NvtxColor.RED)) { _ => if (bufStreamIter.hasNext) { bufStreamIter.head diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala index b980be75d4f..b80913aa64a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedC import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExecBase, DropTableExec, ShowTablesExec} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, Exchange, ReusedExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashedRelationBroadcastMode} -import org.apache.spark.sql.rapids.{ExternalSource, GpuDataSourceScanExec, GpuFileSourceScanExec, GpuInputFileBlockLength, GpuInputFileBlockStart, GpuInputFileName, GpuShuffleEnv, GpuTaskMetrics} +import org.apache.spark.sql.rapids.{GpuDataSourceScanExec, GpuFileSourceScanExec, GpuInputFileBlockLength, GpuInputFileBlockStart, GpuInputFileName, GpuShuffleEnv, GpuTaskMetrics} import org.apache.spark.sql.rapids.execution.{ExchangeMappingCache, GpuBroadcastExchangeExec, GpuBroadcastExchangeExecBase, GpuBroadcastToRowExec, GpuCustomShuffleReaderExec, GpuHashJoin, GpuShuffleExchangeExecBase} import org.apache.spark.sql.types.StructType @@ -362,20 +362,13 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { disableUntilInput: Boolean = false): SparkPlan = { plan match { case batchScan: GpuBatchScanExec => - if ((batchScan.scan.isInstanceOf[GpuParquetScan] || - batchScan.scan.isInstanceOf[GpuOrcScan] || - ExternalSource.isSupportedScan(batchScan.scan)) && - (disableUntilInput || disableScanUntilInput(batchScan))) { - val scanCopy = batchScan.scan match { - case parquetScan: GpuParquetScan => - parquetScan.copy(queryUsesInputFile = true) - case orcScan: GpuOrcScan => - orcScan.copy(queryUsesInputFile = true) - case eScan if ExternalSource.isSupportedScan(eScan) => - ExternalSource.copyScanWithInputFileTrue(eScan) - case _ => throw new RuntimeException("Wrong format") // never reach here + if (disableUntilInput || disableScanUntilInput(batchScan)) { + val newScan = batchScan.scan.withInputFile() + if (newScan ne batchScan.scan) { + batchScan.copy(scan = newScan) + } else { + batchScan } - batchScan.copy(scan = scanCopy) } else { batchScan } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuUserDefinedFunction.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuUserDefinedFunction.scala index 7f362c68746..59f96cf3cdb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuUserDefinedFunction.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuUserDefinedFunction.scala @@ -45,6 +45,7 @@ trait GpuUserDefinedFunction extends GpuExpression override def hasSideEffects: Boolean = true override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) + override val selfNonDeterministic: Boolean = !udfDeterministic private[this] val nvtxRangeName = s"UDF: $name" private[this] lazy val funcCls = TrampolineUtil.getSimpleName(function.getClass) @@ -107,6 +108,7 @@ trait GpuRowBasedUserDefinedFunction extends GpuExpression private[this] lazy val outputType = dataType.catalogString override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) + override val selfNonDeterministic: Boolean = !udfDeterministic override def hasSideEffects: Boolean = true override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala index c5d9a8bce9f..b52670d6c1e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala @@ -689,12 +689,13 @@ object GroupedAggregations { private def getWindowOptions( orderSpec: Seq[SortOrder], orderPositions: Seq[Int], - frame: GpuSpecifiedWindowFrame): WindowOptions = { + frame: GpuSpecifiedWindowFrame, + minPeriods: Int): WindowOptions = { frame.frameType match { case RowFrame => withResource(getRowBasedLower(frame)) { lower => withResource(getRowBasedUpper(frame)) { upper => - val builder = WindowOptions.builder().minPeriods(1) + val builder = WindowOptions.builder().minPeriods(minPeriods) if (isUnbounded(frame.lower)) builder.unboundedPreceding() else builder.preceding(lower) if (isUnbounded(frame.upper)) builder.unboundedFollowing() else builder.following(upper) builder.build @@ -718,7 +719,7 @@ object GroupedAggregations { withResource(asScalarRangeBoundary(orderType, lower)) { preceding => withResource(asScalarRangeBoundary(orderType, upper)) { following => val windowOptionBuilder = WindowOptions.builder() - .minPeriods(1) + .minPeriods(1) // Does not currently support custom minPeriods. .orderByColumnIndex(orderByIndex) if (preceding.isEmpty) { @@ -929,13 +930,18 @@ class GroupedAggregations { if (frameSpec.frameType == frameType) { // For now I am going to assume that we don't need to combine calls across frame specs // because it would just not help that much - val result = withResource( - getWindowOptions(boundOrderSpec, orderByPositions, frameSpec)) { windowOpts => - val allAggs = functions.map { - case (winFunc, _) => winFunc.aggOverWindow(inputCb, windowOpts) - }.toSeq - withResource(GpuColumnVector.from(inputCb)) { initProjTab => - aggIt(initProjTab.groupBy(partByPositions: _*), allAggs) + val result = { + val allWindowOpts = functions.map { f => + getWindowOptions(boundOrderSpec, orderByPositions, frameSpec, + f._1.windowFunc.getMinPeriods) + } + withResource(allWindowOpts.toSeq) { allWindowOpts => + val allAggs = allWindowOpts.zip(functions).map { case (windowOpt, f) => + f._1.aggOverWindow(inputCb, windowOpt) + } + withResource(GpuColumnVector.from(inputCb)) { initProjTab => + aggIt(initProjTab.groupBy(partByPositions: _*), allAggs) + } } } withResource(result) { result => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index 98e47d93f42..9d9f4101e26 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -20,6 +20,7 @@ import java.util.concurrent.TimeUnit import ai.rapids.cudf import ai.rapids.cudf.{BinaryOp, ColumnVector, DType, GroupByScanAggregation, RollingAggregation, RollingAggregationOnColumn, Scalar, ScanAggregation} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.GpuOverrides.wrapExpr import com.nvidia.spark.rapids.shims.{GpuWindowUtil, ShimExpression} @@ -28,6 +29,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, Average, CollectList, CollectSet, Count, Max, Min, Sum} import org.apache.spark.sql.rapids.{AddOverflowChecks, GpuAggregateExpression, GpuCount, GpuCreateNamedStruct, GpuDivide, GpuSubtract} import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ @@ -81,13 +83,25 @@ abstract class GpuWindowExpressionMetaBase( case _: Lead | _: Lag => // ignored we are good case _ => // need to be sure that the lower/upper are acceptable - if (lower > 0) { - willNotWorkOnGpu(s"lower-bounds ahead of current row is not supported. " + - s"Found $lower") + // Negative bounds are allowed, so long as lower does not exceed upper. + if (upper < lower) { + willNotWorkOnGpu("upper-bounds must equal or exceed the lower bounds. " + + s"Found lower=$lower, upper=$upper ") } - if (upper < 0) { - willNotWorkOnGpu(s"upper-bounds behind the current row is not supported. " + - s"Found $upper") + // Also check for negative offsets. + if (upper < 0 || lower > 0) { + windowFunction.asInstanceOf[AggregateExpression].aggregateFunction match { + case _: Average => // Supported + case _: CollectList => // Supported + case _: CollectSet => // Supported + case _: Count => // Supported + case _: Max => // Supported + case _: Min => // Supported + case _: Sum => // Supported + case f: AggregateFunction => + willNotWorkOnGpu("negative row bounds unsupported for specified " + + s"aggregation: ${f.prettyName}") + } } } case RangeFrame => @@ -649,7 +663,15 @@ case class GpuSpecialFrameBoundary(boundary : SpecialFrameBoundary) // This is here for now just to tag an expression as being a GpuWindowFunction and match // Spark. This may expand in the future if other types of window functions show up. -trait GpuWindowFunction extends GpuUnevaluable with ShimExpression +trait GpuWindowFunction extends GpuUnevaluable with ShimExpression { + /** + * Get "min-periods" value, i.e. the minimum number of periods/rows + * above which a non-null value is returned for the function. + * Otherwise, null is returned. + * @return Non-negative value for min-periods. + */ + def getMinPeriods: Int = 1 +} /** * This is a special window function that simply replaces itself with one or more @@ -814,7 +836,7 @@ trait GpuRunningWindowFunction extends GpuWindowFunction { * * which can be output. */ -trait BatchedRunningWindowFixer extends AutoCloseable with CheckpointRestore { +trait BatchedRunningWindowFixer extends AutoCloseable with Retryable { /** * Fix up `windowedColumnOutput` with any stored state from previous batches. * Like all window operations the input data will have been sorted by the partition @@ -994,6 +1016,55 @@ class CountUnboundedToUnboundedFixer(errorOnOverflow: Boolean) } } +class BatchedUnboundedToUnboundedBinaryFixer(val binOp: BinaryOp, val dataType: DataType) + extends BatchedUnboundedToUnboundedWindowFixer { + private var previousResult: Option[Scalar] = None + + override def updateState(scalar: Scalar): Unit = previousResult match { + case None => + previousResult = Some(scalar.incRefCount()) + case Some(prev) => + // This is ugly, but for now it is simple to make it work + val result = withResource(ColumnVector.fromScalar(prev, 1)) { p1 => + withResource(p1.binaryOp(binOp, scalar, prev.getType)) { result1 => + result1.getScalarElement(0) + } + } + closeOnExcept(result) { _ => + previousResult.foreach(_.close) + previousResult = Some(result) + } + } + + override def fixUp(samePartitionMask: Either[ColumnVector, Boolean], + column: ColumnVector): ColumnVector = { + val scalar = previousResult match { + case Some(value) => + value.incRefCount() + case None => + GpuScalar.from(null, dataType) + } + + withResource(scalar) { scalar => + samePartitionMask match { + case scala.Left(cv) => + cv.ifElse(scalar, column) + case scala.Right(true) => + ColumnVector.fromScalar(scalar, column.getRowCount.toInt) + case _ => + column.incRefCount() + } + } + } + + override def close(): Unit = reset() + + override def reset(): Unit = { + previousResult.foreach(_.close()) + previousResult = None + } +} + /** * This class fixes up batched running windows by performing a binary op on the previous value and * those in the the same partition by key group. It does not deal with nulls, so it works for things diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala index 4e26e2c2f5e..587bffe7ebc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala @@ -284,7 +284,7 @@ private class HostAlloc(nonPinnedLimit: Long) { private def canNeverSucceed(amount: Long, preferPinned: Boolean): Boolean = { val pinnedFailed = (isPinnedOnly || preferPinned) && (amount > pinnedLimit) val nonPinnedFailed = isPinnedOnly || (amount > nonPinnedLimit) - pinnedFailed && nonPinnedFailed + !isUnlimited && pinnedFailed && nonPinnedFailed } private def checkSize(amount: Long, preferPinned: Boolean): Unit = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostByteBufferIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostByteBufferIterator.scala deleted file mode 100644 index 5df9f220435..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostByteBufferIterator.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright (c) 2020, 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 com.nvidia.spark.rapids - -import java.nio.ByteBuffer - -import ai.rapids.cudf.HostMemoryBuffer - -/** - * Create an iterator that will emit ByteBuffer instances sequentially - * to work around the 2GB ByteBuffer size limitation. This allows - * the entire address range of a >2GB host buffer to be covered - * by a sequence of ByteBuffer instances. - *

NOTE: It is the caller's responsibility to ensure this iterator - * does not outlive the host buffer. The iterator DOES NOT increment - * the reference count of the host buffer to ensure it remains valid. - * - * @param hostBuffer host buffer to iterate - * @return ByteBuffer iterator - */ -class HostByteBufferIterator(hostBuffer: HostMemoryBuffer) - extends Iterator[ByteBuffer] { - private[this] var nextBufferStart: Long = 0L - - override def hasNext: Boolean = hostBuffer != null && nextBufferStart < hostBuffer.getLength - - override def next(): ByteBuffer = { - val offset = nextBufferStart - val length = Math.min(hostBuffer.getLength - nextBufferStart, Integer.MAX_VALUE) - nextBufferStart += length - hostBuffer.asByteBuffer(offset, length.toInt) - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/JoinGatherer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/JoinGatherer.scala index 57260b61c4a..3a55ad94bfd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/JoinGatherer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/JoinGatherer.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{ColumnVector, ColumnView, DeviceMemoryBuffer, DType, GatherMap, NvtxColor, NvtxRange, OrderByArg, OutOfBoundsPolicy, Scalar, Table} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import org.apache.spark.TaskContext @@ -34,7 +35,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch * If the data is needed after `allowSpilling` is called the implementations should get the data * back and cache it again until allowSpilling is called once more. */ -trait LazySpillable extends AutoCloseable with CheckpointRestore { +trait LazySpillable extends AutoCloseable with Retryable { /** * Indicate that we are done using the data for now and it can be spilled. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala index ed0699e92f0..9e7a0eb7a47 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala @@ -61,7 +61,6 @@ object StorageTier extends Enumeration { val DEVICE: StorageTier = Value(0, "device memory") val HOST: StorageTier = Value(1, "host memory") val DISK: StorageTier = Value(2, "local disk") - val GDS: StorageTier = Value(3, "GPUDirect Storage") } /** @@ -207,14 +206,11 @@ class RapidsBufferCopyIterator(buffer: RapidsBuffer) } override def close(): Unit = { - val hasNextBeforeClose = hasNext val toClose = new ArrayBuffer[AutoCloseable]() toClose.appendAll(chunkedPacker) toClose.appendAll(Option(singleShotBuffer)) toClose.safeClose() - require(!hasNextBeforeClose, - "RapidsBufferCopyIterator was closed before exhausting") } } @@ -230,14 +226,14 @@ trait RapidsBuffer extends AutoCloseable { * * @note Do not use this size to allocate a target buffer to copy, always use `getPackedSize.` */ - def getMemoryUsedBytes: Long + val memoryUsedBytes: Long /** * The size of this buffer if it has already gone through contiguous_split. * * @note Use this function when allocating a target buffer for spill or shuffle purposes. */ - def getPackedSizeBytes: Long = getMemoryUsedBytes + def getPackedSizeBytes: Long = memoryUsedBytes /** * At spill time, obtain an iterator used to copy this buffer to a different tier. @@ -304,7 +300,7 @@ trait RapidsBuffer extends AutoCloseable { * @param stream CUDA stream to use */ def copyToMemoryBuffer( - srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, length: Long, stream: Cuda.Stream) + srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, length: Long, stream: Cuda.Stream): Unit /** * Get the device memory buffer from the underlying storage. If the buffer currently resides @@ -389,7 +385,7 @@ sealed class DegenerateRapidsBuffer( override val id: RapidsBufferId, override val meta: TableMeta) extends RapidsBuffer { - override def getMemoryUsedBytes: Long = 0L + override val memoryUsedBytes: Long = 0L override val storageTier: StorageTier = StorageTier.DEVICE @@ -451,7 +447,7 @@ trait RapidsHostBatchBuffer extends AutoCloseable { */ def getHostColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch - def getMemoryUsedBytes(): Long + val memoryUsedBytes: Long } trait RapidsBufferChannelWritable { @@ -459,7 +455,10 @@ trait RapidsBufferChannelWritable { * At spill time, write this buffer to an nio WritableByteChannel. * @param writableChannel that this buffer can just write itself to, either byte-for-byte * or via serialization if needed. + * @param stream the Cuda.Stream for the spilling thread. If the `RapidsBuffer` that + * implements this method is on the device, synchronization may be needed + * for staged copies. * @return the amount of bytes written to the channel */ - def writeToChannel(writableChannel: WritableByteChannel): Long + def writeToChannel(writableChannel: WritableByteChannel, stream: Cuda.Stream): Long } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala index 45fa981cff5..61a636c1708 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala @@ -19,9 +19,7 @@ package com.nvidia.spark.rapids import java.util.concurrent.ConcurrentHashMap import java.util.function.BiFunction -import scala.collection.mutable.ArrayBuffer - -import ai.rapids.cudf.{ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, NvtxColor, NvtxRange, Rmm, Table} +import ai.rapids.cudf.{ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, Rmm, Table} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.RapidsBufferCatalog.getExistingRapidsBufferAndAcquire import com.nvidia.spark.rapids.RapidsPluginImplicits._ @@ -32,7 +30,6 @@ import com.nvidia.spark.rapids.jni.RmmSpark import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.sql.rapids.{RapidsDiskBlockManager, TempSpillBufferId} -import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -378,7 +375,28 @@ class RapidsBufferCatalog( table: Table, initialSpillPriority: Long, needsSync: Boolean = true): RapidsBufferHandle = { - val id = TempSpillBufferId() + addTable(TempSpillBufferId(), table, initialSpillPriority, needsSync) + } + + /** + * Adds a table to the device storage. + * + * This takes ownership of the table. The reason for this is that tables + * don't have a reference count, so we cannot cleanly capture ownership by increasing + * ref count and decreasing from the caller. + * + * @param id specific RapidsBufferId to use for this table + * @param table table that will be owned by the store + * @param initialSpillPriority starting spill priority value + * @param needsSync whether the spill framework should stream synchronize while adding + * this table (defaults to true) + * @return RapidsBufferHandle handle for this RapidsBuffer + */ + def addTable( + id: RapidsBufferId, + table: Table, + initialSpillPriority: Long, + needsSync: Boolean): RapidsBufferHandle = { val rapidsBuffer = deviceStorage.addTable( id, table, @@ -442,7 +460,7 @@ class RapidsBufferCatalog( */ def acquireBuffer(handle: RapidsBufferHandle): RapidsBuffer = { val id = handle.id - (0 until RapidsBufferCatalog.MAX_BUFFER_LOOKUP_ATTEMPTS).foreach { _ => + def lookupAndReturn: Option[RapidsBuffer] = { val buffers = bufferMap.get(id) if (buffers == null || buffers.isEmpty) { throw new NoSuchElementException( @@ -450,7 +468,27 @@ class RapidsBufferCatalog( } val buffer = buffers.head if (buffer.addReference()) { - return buffer + Some(buffer) + } else { + None + } + } + + // fast path + (0 until RapidsBufferCatalog.MAX_BUFFER_LOOKUP_ATTEMPTS).foreach { _ => + val mayBuffer = lookupAndReturn + if (mayBuffer.isDefined) { + return mayBuffer.get + } + } + + // try one last time after locking the catalog (slow path) + // if there is a lot of contention here, I would rather lock the world than + // have tasks error out with "Unable to acquire" + synchronized { + val mayBuffer = lookupAndReturn + if (mayBuffer.isDefined) { + return mayBuffer.get } } throw new IllegalStateException(s"Unable to acquire buffer for ID: $id") @@ -552,14 +590,9 @@ class RapidsBufferCatalog( store: RapidsBufferStore, targetTotalSize: Long, stream: Cuda.Stream = Cuda.DEFAULT_STREAM): Option[Long] = { - val spillStore = store.spillStore - if (spillStore == null) { + if (store.spillStore == null) { throw new OutOfMemoryError("Requested to spill without a spill store") } - - // total amount spilled in this invocation - var totalSpilled: Long = 0 - require(targetTotalSize >= 0, s"Negative spill target size: $targetTotalSize") val mySpillCount = spillCount @@ -573,124 +606,21 @@ class RapidsBufferCatalog( // None which lets the calling code know that rmm should retry allocation None } else { - // this thread win the race and should spill + // this thread wins the race and should spill spillCount += 1 - - logWarning(s"Targeting a ${store.name} size of $targetTotalSize. " + - s"Current total ${store.currentSize}. " + - s"Current spillable ${store.currentSpillableSize}") - - if (store.currentSpillableSize > targetTotalSize) { - withResource(new NvtxRange(s"${store.name} sync spill", NvtxColor.ORANGE)) { _ => - logWarning(s"${store.name} store spilling to reduce usage from " + - s"${store.currentSize} total (${store.currentSpillableSize} spillable) " + - s"to $targetTotalSize bytes") - - // If the store has 0 spillable bytes left, it has exhausted. - var exhausted = false - - val buffersToFree = new ArrayBuffer[RapidsBuffer]() - - try { - while (!exhausted && - store.currentSpillableSize > targetTotalSize) { - val nextSpillable = store.nextSpillable() - if (nextSpillable != null) { - // we have a buffer (nextSpillable) to spill - spillBuffer(nextSpillable, spillStore, stream) - .foreach(buffersToFree.append(_)) - totalSpilled += nextSpillable.getMemoryUsedBytes - } - } - if (totalSpilled <= 0) { - // we didn't spill in this iteration, exit loop - exhausted = true - logWarning("Unable to spill enough to meet request. " + - s"Total=${store.currentSize} " + - s"Spillable=${store.currentSpillableSize} " + - s"Target=$targetTotalSize") - } - } finally { - if (buffersToFree.nonEmpty) { - // This is a hack in order to completely synchronize with the GPU before we free - // a buffer. It is necessary because of non-synchronous cuDF calls that could fall - // behind where the CPU is. Freeing a rapids buffer in these cases needs to wait for - // all launched GPU work, otherwise crashes or data corruption could occur. - // A more performant implementation would be to synchronize on the thread that read - // the buffer via events. - // https://github.com/NVIDIA/spark-rapids/issues/8610 - Cuda.deviceSynchronize() - buffersToFree.safeFree() - } - } - } - } - Some(totalSpilled) + Some(store.synchronousSpill(targetTotalSize, this, stream)) } } } - /** - * Given a specific `RapidsBuffer` spill it to `spillStore` - * @return the buffer, if successfully spilled, in order for the caller to free it - * @note called with catalog lock held - */ - private def spillBuffer( - buffer: RapidsBuffer, - spillStore: RapidsBufferStore, - stream: Cuda.Stream): Option[RapidsBuffer] = { - if (buffer.addReference()) { - withResource(buffer) { _ => - logDebug(s"Spilling $buffer ${buffer.id} to ${spillStore.name}") - val bufferHasSpilled = isBufferSpilled(buffer.id, buffer.storageTier) - if (!bufferHasSpilled) { - // if the spillStore specifies a maximum size spill taking this ceiling - // into account before trying to create a buffer there - // TODO: we may need to handle what happens if we can't spill anymore - // because all host buffers are being referenced. - trySpillToMaximumSize(buffer, spillStore, stream) - - // copy the buffer to spillStore - val newBuffer = spillStore.copyBuffer(buffer, stream) - - // once spilled, we get back a new RapidsBuffer instance in this new tier - registerNewBuffer(newBuffer) - } else { - logDebug(s"Skipping spilling $buffer ${buffer.id} to ${spillStore.name} as it is " + - s"already stored in multiple tiers") - } - } - // we can now remove the old tier linkage - removeBufferTier(buffer.id, buffer.storageTier) - - // return the buffer - Some(buffer) - } else { - None - } - } - - /** - * If `spillStore` defines a maximum size, spill to make room for `buffer`. - */ - private def trySpillToMaximumSize( - buffer: RapidsBuffer, - spillStore: RapidsBufferStore, - stream: Cuda.Stream): Unit = { - val spillStoreMaxSize = spillStore.getMaxSize - if (spillStoreMaxSize.isDefined) { - // this spillStore has a maximum size requirement (host only). We need to spill from it - // in order to make room for `buffer`. - val targetTotalSize = - math.max(spillStoreMaxSize.get - buffer.getMemoryUsedBytes, 0) - val maybeAmountSpilled = synchronousSpill(spillStore, targetTotalSize, stream) - maybeAmountSpilled.foreach { amountSpilled => - if (amountSpilled != 0) { - logInfo(s"Spilled $amountSpilled bytes from the ${spillStore.name} store") - TrampolineUtil.incTaskMetricsDiskBytesSpilled(amountSpilled) - } - } - } + def updateTiers(bufferSpill: BufferSpill): Long = bufferSpill match { + case BufferSpill(spilledBuffer, maybeNewBuffer) => + logDebug(s"Spilled ${spilledBuffer.id} from tier ${spilledBuffer.storageTier}. " + + s"Removing. Registering ${maybeNewBuffer.map(_.id).getOrElse ("None")} " + + s"${maybeNewBuffer}") + maybeNewBuffer.foreach(registerNewBuffer) + removeBufferTier(spilledBuffer.id, spilledBuffer.storageTier) + spilledBuffer.memoryUsedBytes } /** @@ -707,10 +637,12 @@ class RapidsBufferCatalog( // do not create a new one, else add a reference acquireBuffer(buffer.id, StorageTier.DEVICE) match { case None => - val newBuffer = deviceStorage.copyBuffer(buffer, stream) - newBuffer.addReference() // add a reference since we are about to use it - registerNewBuffer(newBuffer) - newBuffer + val maybeNewBuffer = deviceStorage.copyBuffer(buffer, this, stream) + maybeNewBuffer.map { newBuffer => + newBuffer.addReference() // add a reference since we are about to use it + registerNewBuffer(newBuffer) + newBuffer + }.get // the GPU store has to return a buffer here for now, or throw OOM case Some(existingBuffer) => existingBuffer } } @@ -764,14 +696,12 @@ class RapidsBufferCatalog( } object RapidsBufferCatalog extends Logging { - private val MAX_BUFFER_LOOKUP_ATTEMPTS = 100 private var deviceStorage: RapidsDeviceMemoryStore = _ private var hostStorage: RapidsHostMemoryStore = _ private var diskBlockManager: RapidsDiskBlockManager = _ private var diskStorage: RapidsDiskStore = _ - private var gdsStorage: RapidsGdsStore = _ private var memoryEventHandler: DeviceMemoryEventHandler = _ private var _shouldUnspill: Boolean = _ private var _singleton: RapidsBufferCatalog = null @@ -841,30 +771,29 @@ object RapidsBufferCatalog extends Logging { // We are going to re-initialize so make sure all of the old things were closed... closeImpl() assert(memoryEventHandler == null) - deviceStorage = new RapidsDeviceMemoryStore(rapidsConf.chunkedPackBounceBufferSize) + deviceStorage = new RapidsDeviceMemoryStore( + rapidsConf.chunkedPackBounceBufferSize, + rapidsConf.spillToDiskBounceBufferSize) diskBlockManager = new RapidsDiskBlockManager(conf) - if (rapidsConf.isGdsSpillEnabled) { - gdsStorage = new RapidsGdsStore(diskBlockManager, rapidsConf.gdsSpillBatchWriteBufferSize) - deviceStorage.setSpillStore(gdsStorage) + val hostSpillStorageSize = if (rapidsConf.offHeapLimitEnabled) { + // Disable the limit because it is handled by the RapidsHostMemoryStore + None + } else if (rapidsConf.hostSpillStorageSize == -1) { + // + 1 GiB by default to match backwards compatibility + Some(rapidsConf.pinnedPoolSize + (1024 * 1024 * 1024)) } else { - val hostSpillStorageSize = if (rapidsConf.hostSpillStorageSize == -1) { - // + 1 GiB by default to match backwards compatibility - rapidsConf.pinnedPoolSize + (1024 * 1024 * 1024) - } else { - rapidsConf.hostSpillStorageSize - } - hostStorage = new RapidsHostMemoryStore(hostSpillStorageSize) - diskStorage = new RapidsDiskStore(diskBlockManager) - deviceStorage.setSpillStore(hostStorage) - hostStorage.setSpillStore(diskStorage) + Some(rapidsConf.hostSpillStorageSize) } + hostStorage = new RapidsHostMemoryStore(hostSpillStorageSize) + diskStorage = new RapidsDiskStore(diskBlockManager) + deviceStorage.setSpillStore(hostStorage) + hostStorage.setSpillStore(diskStorage) logInfo("Installing GPU memory handler for spill") memoryEventHandler = new DeviceMemoryEventHandler( singleton, deviceStorage, rapidsConf.gpuOomDumpDir, - rapidsConf.isGdsSpillEnabled, rapidsConf.gpuOomMaxRetries) if (rapidsConf.sparkRmmStateEnable) { @@ -920,10 +849,6 @@ object RapidsBufferCatalog extends Logging { diskStorage.close() diskStorage = null } - if (gdsStorage != null) { - gdsStorage.close() - gdsStorage = null - } } def getDeviceStorage: RapidsDeviceMemoryStore = deviceStorage diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala index ecbd8e64259..a58a8415cbf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala @@ -21,7 +21,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock import scala.collection.mutable -import ai.rapids.cudf.{BaseDeviceMemoryBuffer, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer} +import ai.rapids.cudf.{BaseDeviceMemoryBuffer, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, NvtxColor, NvtxRange} import com.nvidia.spark.rapids.Arm._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.StorageTier.{DEVICE, StorageTier} @@ -32,6 +32,15 @@ import org.apache.spark.sql.rapids.GpuTaskMetrics import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch +/** + * A helper case class that contains the buffer we spilled from our current tier + * and likely a new buffer created in a spill store tier, but it can be set to None. + * If the buffer already exists in the target spill store, `newBuffer` will be None. + * @param spilledBuffer a `RapidsBuffer` we spilled from this store + * @param newBuffer an optional `RapidsBuffer` in the target spill store. + */ +case class BufferSpill(spilledBuffer: RapidsBuffer, newBuffer: Option[RapidsBuffer]) + /** * Base class for all buffer store types. * @@ -67,14 +76,14 @@ abstract class RapidsBufferStore(val tier: StorageTier) if (old != null) { throw new DuplicateBufferException(s"duplicate buffer registered: ${buffer.id}") } - totalBytesStored += buffer.getMemoryUsedBytes + totalBytesStored += buffer.memoryUsedBytes // device buffers "spillability" is handled via DeviceMemoryBuffer ref counting // so spillableOnAdd should be false, all other buffer tiers are spillable at // all times. - if (spillableOnAdd) { + if (spillableOnAdd && buffer.memoryUsedBytes > 0) { if (spillable.offer(buffer)) { - totalBytesSpillable += buffer.getMemoryUsedBytes + totalBytesSpillable += buffer.memoryUsedBytes } } } @@ -84,9 +93,9 @@ abstract class RapidsBufferStore(val tier: StorageTier) spilling.remove(id) val obj = buffers.remove(id) if (obj != null) { - totalBytesStored -= obj.getMemoryUsedBytes + totalBytesStored -= obj.memoryUsedBytes if (spillable.remove(obj)) { - totalBytesSpillable -= obj.getMemoryUsedBytes + totalBytesSpillable -= obj.memoryUsedBytes } } } @@ -115,19 +124,19 @@ abstract class RapidsBufferStore(val tier: StorageTier) * @param isSpillable whether the buffer should now be spillable */ def setSpillable(buffer: RapidsBufferBase, isSpillable: Boolean): Unit = synchronized { - if (isSpillable) { + if (isSpillable && buffer.memoryUsedBytes > 0) { // if this buffer is in the store and isn't currently spilling if (!spilling.contains(buffer.id) && buffers.containsKey(buffer.id)) { // try to add it to the spillable collection if (spillable.offer(buffer)) { - totalBytesSpillable += buffer.getMemoryUsedBytes + totalBytesSpillable += buffer.memoryUsedBytes logDebug(s"Buffer ${buffer.id} is spillable. " + s"total=${totalBytesStored} spillable=${totalBytesSpillable}") } // else it was already there (unlikely) } } else { if (spillable.remove(buffer)) { - totalBytesSpillable -= buffer.getMemoryUsedBytes + totalBytesSpillable -= buffer.memoryUsedBytes logDebug(s"Buffer ${buffer.id} is not spillable. " + s"total=${totalBytesStored}, spillable=${totalBytesSpillable}") } // else it was already removed @@ -139,8 +148,8 @@ abstract class RapidsBufferStore(val tier: StorageTier) if (buffer != null) { // mark the id as "spilling" (this buffer is in the middle of a spill operation) spilling.add(buffer.id) - totalBytesSpillable -= buffer.getMemoryUsedBytes - logDebug(s"Spilling buffer ${buffer.id}. size=${buffer.getMemoryUsedBytes} " + + totalBytesSpillable -= buffer.memoryUsedBytes + logDebug(s"Spilling buffer ${buffer.id}. size=${buffer.memoryUsedBytes} " + s"total=${totalBytesStored}, new spillable=${totalBytesSpillable}") } buffer @@ -196,15 +205,19 @@ abstract class RapidsBufferStore(val tier: StorageTier) * (i.e.: this method will not take ownership of the incoming buffer object). * This does not need to update the catalog, the caller is responsible for that. * @param buffer data from another store + * @param catalog RapidsBufferCatalog we may need to modify during this copy * @param stream CUDA stream to use for copy or null * @return the new buffer that was created */ def copyBuffer( buffer: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { - freeOnExcept(createBuffer(buffer, stream)) { newBuffer => - addBuffer(newBuffer) - newBuffer + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { + createBuffer(buffer, catalog, stream).map { newBuffer => + freeOnExcept(newBuffer) { newBuffer => + addBuffer(newBuffer) + newBuffer + } } } @@ -220,12 +233,14 @@ abstract class RapidsBufferStore(val tier: StorageTier) * @note DO NOT close the buffer unless adding a reference! * @note `createBuffer` impls should synchronize against `stream` before returning, if needed. * @param buffer data from another store + * @param catalog RapidsBufferCatalog we may need to modify during this create * @param stream CUDA stream to use or null * @return the new buffer that was created. */ protected def createBuffer( buffer: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] /** Update bookkeeping for a new buffer */ protected def addBuffer(buffer: RapidsBufferBase): Unit = { @@ -255,6 +270,129 @@ abstract class RapidsBufferStore(val tier: StorageTier) buffers.nextSpillableBuffer() } + def synchronousSpill( + targetTotalSize: Long, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream = Cuda.DEFAULT_STREAM): Long = { + if (currentSpillableSize > targetTotalSize) { + logWarning(s"Targeting a ${name} size of $targetTotalSize. " + + s"Current total ${currentSize}. " + + s"Current spillable ${currentSpillableSize}") + val bufferSpills = new mutable.ArrayBuffer[BufferSpill]() + withResource(new NvtxRange(s"${name} sync spill", NvtxColor.ORANGE)) { _ => + logWarning(s"${name} store spilling to reduce usage from " + + s"${currentSize} total (${currentSpillableSize} spillable) " + + s"to $targetTotalSize bytes") + + // If the store has 0 spillable bytes left, it has exhausted. + try { + var exhausted = false + var totalSpilled = 0L + while (!exhausted && + currentSpillableSize > targetTotalSize) { + val nextSpillableBuffer = nextSpillable() + if (nextSpillableBuffer != null) { + if (nextSpillableBuffer.addReference()) { + withResource(nextSpillableBuffer) { _ => + val bufferHasSpilled = + catalog.isBufferSpilled( + nextSpillableBuffer.id, + nextSpillableBuffer.storageTier) + val bufferSpill = if (!bufferHasSpilled) { + spillBuffer( + nextSpillableBuffer, this, catalog, stream) + } else { + // if `nextSpillableBuffer` already spilled, we still need to + // remove it from our tier and call free on it, but set + // `newBuffer` to None because there's nothing to register + // as it has already spilled. + BufferSpill(nextSpillableBuffer, None) + } + totalSpilled += bufferSpill.spilledBuffer.memoryUsedBytes + bufferSpills.append(bufferSpill) + catalog.updateTiers(bufferSpill) + } + } + } + } + if (totalSpilled <= 0) { + // we didn't spill in this iteration, exit loop + exhausted = true + logWarning("Unable to spill enough to meet request. " + + s"Total=${currentSize} " + + s"Spillable=${currentSpillableSize} " + + s"Target=$targetTotalSize") + } + totalSpilled + } finally { + if (bufferSpills.nonEmpty) { + // This is a hack in order to completely synchronize with the GPU before we free + // a buffer. It is necessary because of non-synchronous cuDF calls that could fall + // behind where the CPU is. Freeing a rapids buffer in these cases needs to wait for + // all launched GPU work, otherwise crashes or data corruption could occur. + // A more performant implementation would be to synchronize on the thread that read + // the buffer via events. + // https://github.com/NVIDIA/spark-rapids/issues/8610 + Cuda.deviceSynchronize() + bufferSpills.foreach(_.spilledBuffer.safeFree()) + } + } + } + } else { + 0L // nothing spilled + } + } + + /** + * Given a specific `RapidsBuffer` spill it to `spillStore` + * + * @return a `BufferSpill` instance with the target buffer in this store, and an optional + * new `RapidsBuffer` in the target spill store if this rapids buffer hadn't already + * spilled. + * @note called with catalog lock held + */ + private def spillBuffer( + buffer: RapidsBuffer, + store: RapidsBufferStore, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): BufferSpill = { + // copy the buffer to spillStore + var maybeNewBuffer: Option[RapidsBuffer] = None + var lastTier: Option[StorageTier] = None + var nextSpillStore = store.spillStore + while (maybeNewBuffer.isEmpty && nextSpillStore != null) { + lastTier = Some(nextSpillStore.tier) + // copy buffer if it fits + maybeNewBuffer = nextSpillStore.copyBuffer(buffer, catalog, stream) + + // if it didn't fit, we can try a lower tier that has more space + if (maybeNewBuffer.isEmpty) { + nextSpillStore = nextSpillStore.spillStore + } + } + if (maybeNewBuffer.isEmpty) { + throw new IllegalStateException( + s"Unable to spill buffer ${buffer.id} of size ${buffer.memoryUsedBytes} " + + s"to tier ${lastTier}") + } + // return the buffer to free and the new buffer to register + BufferSpill(buffer, maybeNewBuffer) + } + + /** + * Tries to make room for `buffer` in the host store by spilling. + * + * @param buffer buffer that will be copied to the host store if it fits + * @param stream CUDA stream to synchronize for memory operations + * @return true if the buffer fits after a potential spill + */ + protected def trySpillToMaximumSize( + buffer: RapidsBuffer, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Boolean = { + true // default to success, HostMemoryStore overrides this + } + /** Base class for all buffers in this store. */ abstract class RapidsBufferBase( override val id: RapidsBufferId, @@ -411,7 +549,7 @@ abstract class RapidsBufferStore(val tier: StorageTier) freeBuffer() } } else { - logWarning(s"Trying to free an invalid buffer => $id, size = ${getMemoryUsedBytes}, $this") + logWarning(s"Trying to free an invalid buffer => $id, size = ${memoryUsedBytes}, $this") } } @@ -453,13 +591,13 @@ abstract class RapidsBufferStore(val tier: StorageTier) releaseResources() } - override def toString: String = s"$name buffer size=${getMemoryUsedBytes}" + override def toString: String = s"$name buffer size=${memoryUsedBytes}" } } /** * Buffers that inherit from this type do not support changing the spillable status - * of a `RapidsBuffer`. This is only used right now for disk and GDS. + * of a `RapidsBuffer`. This is only used right now for disk. * @param tier storage tier of this store */ abstract class RapidsBufferStoreWithoutSpill(override val tier: StorageTier) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 1a2dc1e36d1..e80b53b5541 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -513,25 +513,6 @@ object RapidsConf { .booleanConf .createWithDefault(false) - val GDS_SPILL = conf("spark.rapids.memory.gpu.direct.storage.spill.enabled") - .doc("Should GPUDirect Storage (GDS) be used to spill GPU memory buffers directly to disk. " + - "GDS must be enabled and the directory `spark.local.dir` must support GDS. This is an " + - "experimental feature. For more information on GDS, see " + - "https://docs.nvidia.com/gpudirect-storage/.") - .startupOnly() - .booleanConf - .createWithDefault(false) - - val GDS_SPILL_BATCH_WRITE_BUFFER_SIZE = - conf("spark.rapids.memory.gpu.direct.storage.spill.batchWriteBuffer.size") - .doc("The size of the GPU memory buffer used to batch small buffers when spilling to GDS. " + - "Note that this buffer is mapped to the PCI Base Address Register (BAR) space, which may " + - "be very limited on some GPUs (e.g. the NVIDIA T4 only has 256 MiB), and it is also used " + - "by UCX bounce buffers.") - .startupOnly() - .bytesConf(ByteUnit.BYTE) - .createWithDefault(ByteUnit.MiB.toBytes(8)) - val POOLED_MEM = conf("spark.rapids.memory.gpu.pooling.enabled") .doc("Should RMM act as a pooling allocator for GPU memory, or should it just pass " + "through to CUDA memory allocation directly. DEPRECATED: please use " + @@ -771,6 +752,12 @@ object RapidsConf { .booleanConf .createWithDefault(true) + val ENABLE_FLOAT_FORMAT_NUMBER = conf("spark.rapids.sql.formatNumberFloat.enabled") + .doc("format_number with floating point types on the GPU returns results that have " + + "a different precision than the default results of Spark.") + .booleanConf + .createWithDefault(false) + val ENABLE_CAST_FLOAT_TO_INTEGRAL_TYPES = conf("spark.rapids.sql.castFloatToIntegralTypes.enabled") .doc("Casting from floating point types to integral types on the GPU supports a " + @@ -1660,14 +1647,17 @@ object RapidsConf { val SHUFFLE_MULTITHREADED_MAX_BYTES_IN_FLIGHT = conf("spark.rapids.shuffle.multiThreaded.maxBytesInFlight") - .doc("The size limit, in bytes, that the RAPIDS shuffle manager configured in " + - "\"MULTITHREADED\" mode will allow to be deserialized concurrently per task. This is " + - "also the maximum amount of memory that will be used per task. This should ideally be " + - "at least the same size as the batch size so we don't have to wait to process a " + - "single batch.") + .doc( + "The size limit, in bytes, that the RAPIDS shuffle manager configured in " + + "\"MULTITHREADED\" mode will allow to be deserialized concurrently per task. This is " + + "also the maximum amount of memory that will be used per task. This should be set larger " + + "than Spark's default maxBytesInFlight (48MB). The larger this setting is, the " + + "more compressed shuffle chunks are processed concurrently. In practice, " + + "care needs to be taken to not go over the amount of off-heap memory that Netty has " + + "available. See https://github.com/NVIDIA/spark-rapids/issues/9153.") .startupOnly() .bytesConf(ByteUnit.BYTE) - .createWithDefault(Integer.MAX_VALUE) + .createWithDefault(128 * 1024 * 1024) val SHUFFLE_MULTITHREADED_WRITER_THREADS = conf("spark.rapids.shuffle.multiThreaded.writer.threads") @@ -2031,6 +2021,16 @@ object RapidsConf { "The chunked pack bounce buffer must be at least 1MB in size") .createWithDefault(128L * 1024 * 1024) + val SPILL_TO_DISK_BOUNCE_BUFFER_SIZE = + conf("spark.rapids.memory.host.spillToDiskBounceBufferSize") + .doc("Amount of host memory (in bytes) to set aside at startup for the " + + "bounce buffer used for gpu to disk spill that bypasses the host store.") + .internal() + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v >= 1, + "The gpu to disk spill bounce buffer must have a positive size") + .createWithDefault(128L * 1024 * 1024) + val SPLIT_UNTIL_SIZE_OVERRIDE = conf("spark.rapids.sql.test.overrides.splitUntilSize") .doc("Only for tests: override the value of GpuDeviceManager.splitUntilSize") .internal() @@ -2325,10 +2325,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isUnspillEnabled: Boolean = get(UNSPILL) - lazy val isGdsSpillEnabled: Boolean = get(GDS_SPILL) - - lazy val gdsSpillBatchWriteBufferSize: Long = get(GDS_SPILL_BATCH_WRITE_BUFFER_SIZE) - lazy val needDecimalGuarantees: Boolean = get(NEED_DECIMAL_OVERFLOW_GUARANTEES) lazy val gpuTargetBatchSizeBytes: Long = get(GPU_BATCH_SIZE_BYTES) @@ -2391,6 +2387,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isCastFloatToStringEnabled: Boolean = get(ENABLE_CAST_FLOAT_TO_STRING) + lazy val isFloatFormatNumberEnabled: Boolean = get(ENABLE_FLOAT_FORMAT_NUMBER) + lazy val isCastStringToTimestampEnabled: Boolean = get(ENABLE_CAST_STRING_TO_TIMESTAMP) lazy val hasExtendedYearValues: Boolean = get(HAS_EXTENDED_YEAR_VALUES) @@ -2725,6 +2723,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val chunkedPackBounceBufferSize: Long = get(CHUNKED_PACK_BOUNCE_BUFFER_SIZE) + lazy val spillToDiskBounceBufferSize: Long = get(SPILL_TO_DISK_BOUNCE_BUFFER_SIZE) + lazy val splitUntilSizeOverride: Option[Long] = get(SPLIT_UNTIL_SIZE_OVERRIDE) private val optimizerDefaults = Map( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala index 7b0f07bf876..bc09752bfdf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala @@ -16,16 +16,18 @@ package com.nvidia.spark.rapids +import java.nio.channels.WritableByteChannel import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable import ai.rapids.cudf.{ColumnVector, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, Table} import com.nvidia.spark.rapids.Arm._ +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableSeq import com.nvidia.spark.rapids.StorageTier.StorageTier import com.nvidia.spark.rapids.format.TableMeta -import org.apache.spark.sql.rapids.TempSpillBufferId +import org.apache.spark.sql.rapids.storage.RapidsStorageUtils import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -35,7 +37,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch * during spill in chunked_pack. The parameter defaults to 128MB, * with a rule-of-thumb of 1MB per SM. */ -class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024) +class RapidsDeviceMemoryStore( + chunkedPackBounceBufferSize: Long = 128L*1024*1024, + hostBounceBufferSize: Long = 128L*1024*1024) extends RapidsBufferStore(StorageTier.DEVICE) { // The RapidsDeviceMemoryStore handles spillability via ref counting @@ -45,9 +49,13 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 private var chunkedPackBounceBuffer: DeviceMemoryBuffer = DeviceMemoryBuffer.allocate(chunkedPackBounceBufferSize) + private var hostSpillBounceBuffer: HostMemoryBuffer = + HostMemoryBuffer.allocate(hostBounceBufferSize) + override protected def createBuffer( other: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { val memoryBuffer = withResource(other.getCopyIterator) { copyIterator => copyIterator.next() } @@ -64,12 +72,12 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 case b => throw new IllegalStateException(s"Unrecognized buffer: $b") } } - new RapidsDeviceMemoryBuffer( + Some(new RapidsDeviceMemoryBuffer( other.id, deviceBuffer.getLength, other.meta, deviceBuffer, - other.getSpillPriority) + other.getSpillPriority)) } } @@ -127,7 +135,7 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 * @return the RapidsBuffer instance that was added. */ def addTable( - id: TempSpillBufferId, + id: RapidsBufferId, table: Table, initialSpillPriority: Long, needsSync: Boolean): RapidsBuffer = { @@ -209,13 +217,14 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 * @param spillPriority a starting spill priority */ class RapidsTable( - id: TempSpillBufferId, + id: RapidsBufferId, table: Table, spillPriority: Long) extends RapidsBufferBase( id, null, - spillPriority) { + spillPriority) + with RapidsBufferChannelWritable { /** The storage tier for this buffer */ override val storageTier: StorageTier = StorageTier.DEVICE @@ -256,7 +265,7 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 chunkedPacker.getMeta } - override def getMemoryUsedBytes: Long = unpackedSizeInBytes + override val memoryUsedBytes: Long = unpackedSizeInBytes override def getPackedSizeBytes: Long = getChunkedPacker.getTotalContiguousSize @@ -371,6 +380,32 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 } } } + + override def writeToChannel(outputChannel: WritableByteChannel, stream: Cuda.Stream): Long = { + var written: Long = 0L + withResource(getCopyIterator) { copyIter => + while(copyIter.hasNext) { + withResource(copyIter.next()) { slice => + val iter = + new MemoryBufferToHostByteBufferIterator( + slice, + hostSpillBounceBuffer, + stream) + iter.foreach { bb => + try { + while (bb.hasRemaining) { + written += outputChannel.write(bb) + } + } finally { + RapidsStorageUtils.dispose(bb) + } + } + } + } + written + } + } + } class RapidsDeviceMemoryBuffer( @@ -380,9 +415,10 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 contigBuffer: DeviceMemoryBuffer, spillPriority: Long) extends RapidsBufferBase(id, meta, spillPriority) - with MemoryBuffer.EventHandler { + with MemoryBuffer.EventHandler + with RapidsBufferChannelWritable { - override def getMemoryUsedBytes(): Long = size + override val memoryUsedBytes: Long = size override val storageTier: StorageTier = StorageTier.DEVICE @@ -456,10 +492,32 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 } super.free() } + + override def writeToChannel(outputChannel: WritableByteChannel, stream: Cuda.Stream): Long = { + var written: Long = 0L + val iter = new MemoryBufferToHostByteBufferIterator( + contigBuffer, + hostSpillBounceBuffer, + stream) + iter.foreach { bb => + try { + while (bb.hasRemaining) { + written += outputChannel.write(bb) + } + } finally { + RapidsStorageUtils.dispose(bb) + } + } + written + } } override def close(): Unit = { - super.close() - chunkedPackBounceBuffer.close() - chunkedPackBounceBuffer = null + try { + super.close() + } finally { + Seq(chunkedPackBounceBuffer, hostSpillBounceBuffer).safeClose() + chunkedPackBounceBuffer = null + hostSpillBounceBuffer = null + } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala index ffb8960ccb1..63f1b723ff1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala @@ -21,7 +21,7 @@ import java.nio.channels.FileChannel.MapMode import java.util.concurrent.ConcurrentHashMap import ai.rapids.cudf.{Cuda, HostMemoryBuffer, MemoryBuffer} -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.StorageTier.StorageTier import com.nvidia.spark.rapids.format.TableMeta @@ -37,7 +37,8 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) override protected def createBuffer( incoming: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { // assuming that the disk store gets contiguous buffers val id = incoming.id val path = if (id.canShareDiskPaths) { @@ -49,14 +50,14 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) val (fileOffset, diskLength) = if (id.canShareDiskPaths) { // only one writer at a time for now when using shared files path.synchronized { - writeToFile(incoming, path, append = true) + writeToFile(incoming, path, append = true, stream) } } else { - writeToFile(incoming, path, append = false) + writeToFile(incoming, path, append = false, stream) } logDebug(s"Spilled to $path $fileOffset:$diskLength") - incoming match { + val buff = incoming match { case _: RapidsHostBatchBuffer => new RapidsDiskColumnarBatch( id, @@ -73,19 +74,26 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) incoming.meta, incoming.getSpillPriority) } + Some(buff) } /** Copy a host buffer to a file, returning the file offset at which the data was written. */ private def writeToFile( incoming: RapidsBuffer, path: File, - append: Boolean): (Long, Long) = { + append: Boolean, + stream: Cuda.Stream): (Long, Long) = { incoming match { case fileWritable: RapidsBufferChannelWritable => withResource(new FileOutputStream(path, append)) { fos => withResource(fos.getChannel) { outputChannel => val startOffset = outputChannel.position() - val writtenBytes = fileWritable.writeToChannel(outputChannel) + val writtenBytes = fileWritable.writeToChannel(outputChannel, stream) + if (writtenBytes == 0) { + throw new IllegalStateException( + s"Buffer ${fileWritable} wrote 0 bytes disk on spill. This is not supported!" + ) + } (startOffset, writtenBytes) } } @@ -109,16 +117,17 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) id, meta, spillPriority) { private[this] var hostBuffer: Option[HostMemoryBuffer] = None - override def getMemoryUsedBytes(): Long = size + override val memoryUsedBytes: Long = size override val storageTier: StorageTier = StorageTier.DISK override def getMemoryBuffer: MemoryBuffer = synchronized { if (hostBuffer.isEmpty) { + require(size > 0, + s"$this attempted an invalid 0-byte mmap of a file") val path = id.getDiskPath(diskBlockManager) val mappedBuffer = HostMemoryBuffer.mapFile(path, MapMode.READ_WRITE, fileOffset, size) - logDebug(s"Created mmap buffer for $path $fileOffset:$size") hostBuffer = Some(mappedBuffer) } hostBuffer.foreach(_.incRefCount()) @@ -183,7 +192,7 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) val path = id.getDiskPath(diskBlockManager) withResource(new FileInputStream(path)) { fis => val (header, hostBuffer) = SerializedHostTableUtils.readTableHeaderAndBuffer(fis) - val hostCols = closeOnExcept(hostBuffer) { _ => + val hostCols = withResource(hostBuffer) { _ => SerializedHostTableUtils.buildHostColumns(header, hostBuffer, sparkTypes) } new ColumnarBatch(hostCols.toArray, header.getNumRows) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala deleted file mode 100644 index 1030d6e4f75..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala +++ /dev/null @@ -1,290 +0,0 @@ -/* - * Copyright (c) 2020-2023, 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 com.nvidia.spark.rapids - -import java.io.File -import java.util.concurrent.ConcurrentHashMap -import java.util.function.BiFunction - -import scala.collection.mutable.ArrayBuffer - -import ai.rapids.cudf._ -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} -import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta - -import org.apache.spark.sql.rapids.{RapidsDiskBlockManager, TempSpillBufferId} - -/** A buffer store using GPUDirect Storage (GDS). */ -class RapidsGdsStore( - diskBlockManager: RapidsDiskBlockManager, - batchWriteBufferSize: Long) - extends RapidsBufferStoreWithoutSpill(StorageTier.GDS) { - private[this] val batchSpiller = new BatchSpiller() - - override protected def createBuffer( - other: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { - // assume that we get 1 buffer - val otherBuffer = withResource(other.getCopyIterator) { it => - it.next() - } - - withResource(otherBuffer) { _ => - val deviceBuffer = otherBuffer match { - case d: BaseDeviceMemoryBuffer => d - case _ => throw new IllegalStateException("copying from buffer without device memory") - } - if (deviceBuffer.getLength < batchWriteBufferSize) { - batchSpiller.spill(other, deviceBuffer) - } else { - singleShotSpill(other, deviceBuffer) - } - } - } - - override def close(): Unit = { - super.close() - batchSpiller.close() - } - - abstract class RapidsGdsBuffer( - override val id: RapidsBufferId, - val size: Long, - override val meta: TableMeta, - spillPriority: Long) - extends RapidsBufferBase(id, meta, spillPriority) { - override val storageTier: StorageTier = StorageTier.GDS - - override def getMemoryUsedBytes(): Long = size - - override def getMemoryBuffer: MemoryBuffer = getDeviceMemoryBuffer - } - - class RapidsGdsSingleShotBuffer( - id: RapidsBufferId, path: File, fileOffset: Long, size: Long, meta: TableMeta, - spillPriority: Long) - extends RapidsGdsBuffer(id, size, meta, spillPriority) { - - override def materializeMemoryBuffer: MemoryBuffer = { - closeOnExcept(DeviceMemoryBuffer.allocate(size)) { buffer => - CuFile.readFileToDeviceBuffer(buffer, path, fileOffset) - logDebug(s"Created device buffer for $path $fileOffset:$size via GDS") - buffer - } - } - - override def copyToMemoryBuffer(srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, - length: Long, stream: Cuda.Stream): Unit = { - dst match { - case dmOriginal: BaseDeviceMemoryBuffer => - val sliced = dmOriginal.slice(dstOffset, length).asInstanceOf[BaseDeviceMemoryBuffer] - withResource(sliced) { dm => - // TODO: switch to async API when it's released, using the passed in CUDA stream. - stream.sync() - CuFile.readFileToDeviceBuffer(dm, path, fileOffset + srcOffset) - logDebug(s"Created device buffer for $path ${fileOffset + srcOffset}:$length via GDS") - } - case _ => throw new IllegalStateException( - s"GDS can only copy to device buffer, not ${dst.getClass}") - } - } - - override protected def releaseResources(): Unit = { - if (id.canShareDiskPaths) { - // Buffers that share paths must be cleaned up elsewhere - } else { - if (!path.delete() && path.exists()) { - logWarning(s"Unable to delete GDS spill path $path") - } - } - } - } - - private def singleShotSpill(other: RapidsBuffer, deviceBuffer: BaseDeviceMemoryBuffer) - : RapidsBufferBase = { - val id = other.id - val path = id.getDiskPath(diskBlockManager) - // When sharing files, append to the file; otherwise, write from the beginning. - val fileOffset = if (id.canShareDiskPaths) { - // only one writer at a time for now when using shared files - path.synchronized { - CuFile.appendDeviceBufferToFile(path, deviceBuffer) - } - } else { - CuFile.writeDeviceBufferToFile(path, 0, deviceBuffer) - 0 - } - logDebug(s"Spilled to $path $fileOffset:${deviceBuffer.getLength} via GDS") - new RapidsGdsSingleShotBuffer( - id, - path, - fileOffset, - deviceBuffer.getLength, - other.meta, - other.getSpillPriority) - } - - class BatchSpiller() extends AutoCloseable { - private val blockSize = 4096 - private[this] val spilledBuffers = new ConcurrentHashMap[File, Set[RapidsBufferId]] - private[this] val pendingBuffers = ArrayBuffer.empty[RapidsGdsBatchedBuffer] - private[this] val batchWriteBuffer = CuFileBuffer.allocate(batchWriteBufferSize, true) - private[this] var currentFile = TempSpillBufferId().getDiskPath(diskBlockManager) - private[this] var currentOffset = 0L - - override def close(): Unit = { - pendingBuffers.safeFree() - pendingBuffers.clear() - batchWriteBuffer.close() - } - - def spill(other: RapidsBuffer, deviceBuffer: BaseDeviceMemoryBuffer): RapidsBufferBase = - this.synchronized { - if (deviceBuffer.getLength > batchWriteBufferSize - currentOffset) { - val path = currentFile.getAbsolutePath - withResource(new CuFileWriteHandle(path)) { handle => - handle.write(batchWriteBuffer, batchWriteBufferSize, 0) - logDebug(s"Spilled to $path 0:$currentOffset via GDS") - } - pendingBuffers.foreach(_.unsetPending()) - pendingBuffers.clear - currentFile = TempSpillBufferId().getDiskPath(diskBlockManager) - currentOffset = 0 - } - - batchWriteBuffer.copyFromMemoryBuffer( - currentOffset, deviceBuffer, 0, deviceBuffer.getLength, Cuda.DEFAULT_STREAM) - - val id = other.id - addBuffer(currentFile, id) - val gdsBuffer = new RapidsGdsBatchedBuffer( - id, - currentFile, - currentOffset, - deviceBuffer.getLength, - other.meta, - other.getSpillPriority) - currentOffset += alignUp(deviceBuffer.getLength) - pendingBuffers += gdsBuffer - gdsBuffer - } - - private def alignUp(length: Long): Long = { - (length + blockSize - 1) & ~(blockSize - 1) - } - - private def copyToBuffer( - buffer: MemoryBuffer, offset: Long, size: Long, stream: Cuda.Stream): Unit = { - buffer.copyFromMemoryBuffer(0, batchWriteBuffer, offset, size, stream) - } - - private def addBuffer(path: File, id: RapidsBufferId): Set[RapidsBufferId] = { - val updater = new BiFunction[File, Set[RapidsBufferId], Set[RapidsBufferId]] { - override def apply(key: File, value: Set[RapidsBufferId]): Set[RapidsBufferId] = { - if (value == null) { - Set(id) - } else { - value + id - } - } - } - spilledBuffers.compute(path, updater) - } - - private def removeBuffer(path: File, id: RapidsBufferId): Set[RapidsBufferId] = { - val updater = new BiFunction[File, Set[RapidsBufferId], Set[RapidsBufferId]] { - override def apply(key: File, value: Set[RapidsBufferId]): Set[RapidsBufferId] = { - val newValue = value - id - if (newValue.isEmpty) { - null - } else { - newValue - } - } - } - spilledBuffers.computeIfPresent(path, updater) - } - - class RapidsGdsBatchedBuffer( - id: RapidsBufferId, - path: File, - fileOffset: Long, - size: Long, - meta: TableMeta, - spillPriority: Long, - var isPending: Boolean = true) - extends RapidsGdsBuffer(id, size, meta, spillPriority) { - - override def getMemoryUsedBytes(): Long = size - - override def materializeMemoryBuffer: MemoryBuffer = this.synchronized { - closeOnExcept(DeviceMemoryBuffer.allocate(size)) { buffer => - if (isPending) { - copyToBuffer(buffer, fileOffset, size, Cuda.DEFAULT_STREAM) - Cuda.DEFAULT_STREAM.sync() - logDebug(s"Created device buffer $size from batch write buffer") - } else { - CuFile.readFileToDeviceBuffer(buffer, path, fileOffset) - logDebug(s"Created device buffer for $path $fileOffset:$size via GDS") - } - buffer - } - } - - override def copyToMemoryBuffer(srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, - length: Long, stream: Cuda.Stream): Unit = this.synchronized { - dst match { - case dmOriginal: BaseDeviceMemoryBuffer => - val sliced = dmOriginal.slice(dstOffset, length).asInstanceOf[BaseDeviceMemoryBuffer] - withResource(sliced) { dm => - if (isPending) { - copyToBuffer(dm, fileOffset + srcOffset, length, stream) - stream.sync() - logDebug(s"Created device buffer $length from batch write buffer") - } else { - // TODO: switch to async API when it's released, using the passed in CUDA stream. - stream.sync() - CuFile.readFileToDeviceBuffer(dm, path, fileOffset + srcOffset) - logDebug(s"Created device buffer for $path ${fileOffset + srcOffset}:$length " + - s"via GDS") - } - } - case _ => throw new IllegalStateException( - s"GDS can only copy to device buffer, not ${dst.getClass}") - } - } - - /** - * Mark this buffer as disk based, no longer in device memory. - */ - def unsetPending(): Unit = this.synchronized { - isPending = false - } - - override protected def releaseResources(): Unit = { - val ids = removeBuffer(path, id) - if (ids == null) { - if (!path.delete() && path.exists()) { - logWarning(s"Unable to delete GDS spill path $path") - } - } - } - } - } -} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala index 743711cee7a..05958a7e4b1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala @@ -22,12 +22,13 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable -import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostColumnVector, HostMemoryBuffer, JCudfSerialization, MemoryBuffer, NvtxColor, NvtxRange, PinnedMemoryPool} +import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostColumnVector, HostMemoryBuffer, JCudfSerialization, MemoryBuffer, NvtxColor, NvtxRange} import com.nvidia.spark.rapids.Arm.{closeOnExcept, freeOnExcept, withResource} import com.nvidia.spark.rapids.SpillPriorities.{applyPriorityOffset, HOST_MEMORY_BUFFER_SPILL_OFFSET} import com.nvidia.spark.rapids.StorageTier.StorageTier import com.nvidia.spark.rapids.format.TableMeta +import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.rapids.storage.RapidsStorageUtils import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -35,29 +36,14 @@ import org.apache.spark.sql.vectorized.ColumnarBatch /** * A buffer store using host memory. * @param maxSize maximum size in bytes for all buffers in this store - * @param pageableMemoryPoolSize maximum size in bytes for the internal pageable memory pool */ class RapidsHostMemoryStore( - maxSize: Long) + maxSize: Option[Long]) extends RapidsBufferStore(StorageTier.HOST) { - override def spillableOnAdd: Boolean = false + override protected def spillableOnAdd: Boolean = false - override def getMaxSize: Option[Long] = Some(maxSize) - - private def allocateHostBuffer( - size: Long, - preferPinned: Boolean = true): HostMemoryBuffer = { - var buffer: HostMemoryBuffer = null - if (preferPinned) { - buffer = PinnedMemoryPool.tryAllocate(size) - if (buffer != null) { - return buffer - } - } - - HostMemoryBuffer.allocate(size, false) - } + override def getMaxSize: Option[Long] = maxSize def addBuffer( id: RapidsBufferId, @@ -97,47 +83,87 @@ class RapidsHostMemoryStore( } } + override protected def trySpillToMaximumSize( + buffer: RapidsBuffer, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Boolean = { + maxSize.forall { ms => + // this spillStore has a maximum size requirement (host only). We need to spill from it + // in order to make room for `buffer`. + val targetTotalSize = ms - buffer.memoryUsedBytes + if (targetTotalSize < 0) { + // lets not spill to host when the buffer we are about + // to spill is larger than our limit + false + } else { + val amountSpilled = synchronousSpill(targetTotalSize, catalog, stream) + if (amountSpilled != 0) { + logDebug(s"Spilled $amountSpilled bytes from ${name} to make room for ${buffer.id}") + TrampolineUtil.incTaskMetricsDiskBytesSpilled(amountSpilled) + } + // if after spill we can fit the new buffer, return true + buffer.memoryUsedBytes <= (ms - currentSize) + } + } + } + override protected def createBuffer( other: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { - withResource(other.getCopyIterator) { otherBufferIterator => - val isChunked = otherBufferIterator.isChunked - val totalCopySize = otherBufferIterator.getTotalCopySize - closeOnExcept(allocateHostBuffer(totalCopySize)) { hostBuffer => - withResource(new NvtxRange("spill to host", NvtxColor.BLUE)) { _ => - var hostOffset = 0L - val start = System.nanoTime() - while (otherBufferIterator.hasNext) { - val otherBuffer = otherBufferIterator.next() - withResource(otherBuffer) { _ => - otherBuffer match { - case devBuffer: DeviceMemoryBuffer => - hostBuffer.copyFromMemoryBufferAsync( - hostOffset, devBuffer, 0, otherBuffer.getLength, stream) - hostOffset += otherBuffer.getLength - case _ => - throw new IllegalStateException("copying from buffer without device memory") + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { + val wouldFit = trySpillToMaximumSize(other, catalog, stream) + if (!wouldFit) { + // skip host + logWarning(s"Buffer $other with size ${other.memoryUsedBytes} does not fit " + + s"in the host store, skipping tier.") + None + } else { + withResource(other.getCopyIterator) { otherBufferIterator => + val isChunked = otherBufferIterator.isChunked + val totalCopySize = otherBufferIterator.getTotalCopySize + closeOnExcept(HostAlloc.allocHighPriority(totalCopySize)) { hb => + hb.map { hostBuffer => + withResource(new NvtxRange("spill to host", NvtxColor.BLUE)) { _ => + var hostOffset = 0L + val start = System.nanoTime() + while (otherBufferIterator.hasNext) { + val otherBuffer = otherBufferIterator.next() + withResource(otherBuffer) { _ => + otherBuffer match { + case devBuffer: DeviceMemoryBuffer => + hostBuffer.copyFromMemoryBufferAsync( + hostOffset, devBuffer, 0, otherBuffer.getLength, stream) + hostOffset += otherBuffer.getLength + case _ => + throw new IllegalStateException("copying from buffer without device memory") + } + } } + stream.sync() + val end = System.nanoTime() + val szMB = (totalCopySize.toDouble / 1024.0 / 1024.0).toLong + val bw = (szMB.toDouble / ((end - start).toDouble / 1000000000.0)).toLong + logDebug(s"Spill to host (chunked=$isChunked) " + + s"size=$szMB MiB bandwidth=$bw MiB/sec") } + new RapidsHostMemoryBuffer( + other.id, + totalCopySize, + other.meta, + applyPriorityOffset(other.getSpillPriority, HOST_MEMORY_BUFFER_SPILL_OFFSET), + hostBuffer) + }.orElse { + // skip host + logWarning(s"Buffer $other with size ${other.memoryUsedBytes} does not fit " + + s"in the host store, skipping tier.") + None } - stream.sync() - val end = System.nanoTime() - val szMB = (totalCopySize.toDouble / 1024.0 / 1024.0).toLong - val bw = (szMB.toDouble / ((end - start).toDouble / 1000000000.0)).toLong - logDebug(s"Spill to host (chunked=$isChunked) " + - s"size=$szMB MiB bandwidth=$bw MiB/sec") } - new RapidsHostMemoryBuffer( - other.id, - totalCopySize, - other.meta, - applyPriorityOffset(other.getSpillPriority, HOST_MEMORY_BUFFER_SPILL_OFFSET), - hostBuffer) } } } - def numBytesFree: Long = maxSize - currentSize + def numBytesFree: Option[Long] = maxSize.map(_ - currentSize) class RapidsHostMemoryBuffer( id: RapidsBufferId, @@ -158,7 +184,7 @@ class RapidsHostMemoryStore( } } - override def writeToChannel(outputChannel: WritableByteChannel): Long = { + override def writeToChannel(outputChannel: WritableByteChannel, ignored: Cuda.Stream): Long = { var written: Long = 0L val iter = new HostByteBufferIterator(buffer) iter.foreach { bb => @@ -184,7 +210,7 @@ class RapidsHostMemoryStore( } /** The size of this buffer in bytes. */ - override def getMemoryUsedBytes: Long = size + override val memoryUsedBytes: Long = size // If this require triggers, we are re-adding a `HostMemoryBuffer` outside of // the catalog lock, which should not possible. The event handler is set to null @@ -304,10 +330,6 @@ class RapidsHostMemoryStore( override val storageTier: StorageTier = StorageTier.HOST - // This is the current size in batch form. It is to be used while this - // batch hasn't migrated to another store. - private val hostSizeInByes: Long = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) - // By default all columns are NOT spillable since we are not the only owners of // the columns (the caller is holding onto a ColumnarBatch that will be closed // after instantiation, triggering onClosed callbacks) @@ -329,7 +351,9 @@ class RapidsHostMemoryStore( null } - override def getMemoryUsedBytes: Long = hostSizeInByes + // This is the current size in batch form. It is to be used while this + // batch hasn't migrated to another store. + override val memoryUsedBytes: Long = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) /** * Mark a column as spillable @@ -376,7 +400,7 @@ class RapidsHostMemoryStore( "RapidsHostColumnarBatch does not support getCopyIterator") } - override def writeToChannel(outputChannel: WritableByteChannel): Long = { + override def writeToChannel(outputChannel: WritableByteChannel, ignored: Cuda.Stream): Long = { withResource(Channels.newOutputStream(outputChannel)) { outputStream => withResource(new DataOutputStream(outputStream)) { dos => val columns = RapidsHostColumnVector.extractBases(hostCb) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index df6376a7b8b..8ed35cffad4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -503,7 +503,7 @@ abstract class ScanMeta[INPUT <: Scan](scan: INPUT, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: DataFromReplacementRule) - extends RapidsMeta[INPUT, Scan, Scan](scan, conf, parent, rule) { + extends RapidsMeta[INPUT, Scan, GpuScan](scan, conf, parent, rule) { override val childPlans: Seq[SparkPlanMeta[_]] = Seq.empty override val childExprs: Seq[BaseExprMeta[_]] = Seq.empty @@ -529,7 +529,7 @@ final class RuleNotFoundScanMeta[INPUT <: Scan]( willNotWorkOnGpu(s"GPU does not currently support the operator ${scan.getClass}") } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = throw new IllegalStateException("Cannot be converted to GPU") } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 985c4efda40..1db0f868213 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -41,7 +41,8 @@ import com.nvidia.spark.rapids.RegexParser.toReadableString * - https://matt.might.net/articles/parsing-regex-with-recursive-descent/ */ class RegexParser(pattern: String) { - private val regexPunct = "!\"#$%&'()*+,-./:;<=>?@\\^_`{|}~" + // Note that [, ] and \ should be part of Punct, but they are handled separately + private val regexPunct = """!"#$%&'()*+,-./:;<=>?@^_`{|}~""" private val escapeChars = Map('n' -> '\n', 'r' -> '\r', 't' -> '\t', 'f' -> '\f', 'a' -> '\u0007', 'b' -> '\b', 'e' -> '\u001b') @@ -147,6 +148,10 @@ class RegexParser(pattern: String) { parseGroup() case '[' => parseCharacterClass() + case ']' => + RegexEscaped(']') + case '}' => + RegexEscaped('}') case '\\' => parseEscapedCharacter() case '\u0000' => @@ -471,7 +476,7 @@ class RegexParser(pattern: String) { case "Punct" => val res:ListBuffer[RegexCharacterClassComponent] = ListBuffer(regexPunct.map(RegexChar): _*) - res ++= ListBuffer(RegexEscaped('['), RegexEscaped(']')) + res ++= ListBuffer(RegexEscaped('['), RegexEscaped(']'), RegexEscaped('\\')) case "Graph" => ListBuffer(getCharacters("Alnum"), getCharacters("Punct")).flatten case "Print" => @@ -1735,14 +1740,14 @@ sealed case class RegexSequence(parts: ListBuffer[RegexAST]) extends RegexAST { sealed case class RegexGroup(capture: Boolean, term: RegexAST, val lookahead: Option[RegexLookahead]) extends RegexAST { - def this(capture: Boolean, term: RegexAST) { + def this(capture: Boolean, term: RegexAST) = { this(capture, term, None) } - def this(capture: Boolean, term: RegexAST, position: Int) { + def this(capture: Boolean, term: RegexAST, position: Int) = { this(capture, term, None) this.position = Some(position) } - def this(capture: Boolean, term: RegexAST, position: Int, lookahead: Option[RegexLookahead]) { + def this(capture: Boolean, term: RegexAST, position: Int, lookahead: Option[RegexLookahead]) = { this(capture, term, lookahead) this.position = Some(position) } @@ -1761,7 +1766,7 @@ sealed case class RegexGroup(capture: Boolean, term: RegexAST, } sealed case class RegexChoice(a: RegexAST, b: RegexAST) extends RegexAST { - def this(a: RegexAST, b: RegexAST, position: Int) { + def this(a: RegexAST, b: RegexAST, position: Int) = { this(a, b) this.position = Some(position) } @@ -1770,7 +1775,7 @@ sealed case class RegexChoice(a: RegexAST, b: RegexAST) extends RegexAST { } sealed case class RegexRepetition(a: RegexAST, quantifier: RegexQuantifier) extends RegexAST { - def this(a: RegexAST, quantifier: RegexQuantifier, position: Int) { + def this(a: RegexAST, quantifier: RegexQuantifier, position: Int) = { this(a, quantifier) this.position = Some(position) } @@ -1781,7 +1786,7 @@ sealed case class RegexRepetition(a: RegexAST, quantifier: RegexQuantifier) exte sealed trait RegexQuantifier extends RegexAST sealed case class SimpleQuantifier(ch: Char) extends RegexQuantifier { - def this(ch: Char, position: Int) { + def this(ch: Char, position: Int) = { this(ch) this.position = Some(position) } @@ -1791,7 +1796,7 @@ sealed case class SimpleQuantifier(ch: Char) extends RegexQuantifier { sealed case class QuantifierFixedLength(length: Int) extends RegexQuantifier { - def this(length: Int, position: Int) { + def this(length: Int, position: Int) = { this(length) this.position = Some(position) } @@ -1803,7 +1808,7 @@ sealed case class QuantifierFixedLength(length: Int) sealed case class QuantifierVariableLength(minLength: Int, maxLength: Option[Int]) extends RegexQuantifier{ - def this(minLength: Int, maxLength: Option[Int], position: Int) { + def this(minLength: Int, maxLength: Option[Int], position: Int) = { this(minLength, maxLength) this.position = Some(position) } @@ -1821,7 +1826,7 @@ sealed case class QuantifierVariableLength(minLength: Int, maxLength: Option[Int sealed trait RegexCharacterClassComponent extends RegexAST sealed case class RegexHexDigit(a: String) extends RegexCharacterClassComponent { - def this(a: String, position: Int) { + def this(a: String, position: Int) = { this(a) this.position = Some(position) } @@ -1838,7 +1843,7 @@ sealed case class RegexHexDigit(a: String) extends RegexCharacterClassComponent } sealed case class RegexOctalChar(a: String) extends RegexCharacterClassComponent { - def this(a: String, position: Int) { + def this(a: String, position: Int) = { this(a) this.position = Some(position) } @@ -1849,7 +1854,7 @@ sealed case class RegexOctalChar(a: String) extends RegexCharacterClassComponent } sealed case class RegexChar(ch: Char) extends RegexCharacterClassComponent { - def this(ch: Char, position: Int) { + def this(ch: Char, position: Int) = { this(ch) this.position = Some(position) } @@ -1857,8 +1862,8 @@ sealed case class RegexChar(ch: Char) extends RegexCharacterClassComponent { override def toRegexString: String = ch.toString } -sealed case class RegexEscaped(a: Char) extends RegexCharacterClassComponent{ - def this(a: Char, position: Int) { +sealed case class RegexEscaped(a: Char) extends RegexCharacterClassComponent { + def this(a: Char, position: Int) = { this(a) this.position = Some(position) } @@ -1869,7 +1874,9 @@ sealed case class RegexEscaped(a: Char) extends RegexCharacterClassComponent{ sealed case class RegexCharacterRange(start: RegexCharacterClassComponent, end: RegexCharacterClassComponent) extends RegexCharacterClassComponent{ - def this(start: RegexCharacterClassComponent, end: RegexCharacterClassComponent, position: Int) { + def this(start: RegexCharacterClassComponent, + end: RegexCharacterClassComponent, + position: Int) = { this(start, end) this.position = Some(position) } @@ -1885,7 +1892,7 @@ sealed case class RegexCharacterClass( def this ( negated: Boolean, characters: ListBuffer[RegexCharacterClassComponent], - position: Int) { + position: Int) = { this(negated, characters) this.position = Some(position) } @@ -1942,7 +1949,7 @@ sealed case class RegexCharacterClass( } sealed case class RegexBackref(num: Int, isNew: Boolean = false) extends RegexAST { - def this(num: Int, isNew: Boolean, position: Int) { + def this(num: Int, isNew: Boolean, position: Int) = { this(num, isNew) this.position = Some(position) } @@ -1952,7 +1959,7 @@ sealed case class RegexBackref(num: Int, isNew: Boolean = false) extends RegexAS sealed case class RegexReplacement(parts: ListBuffer[RegexAST], var numCaptureGroups: Int = 0) extends RegexAST { - def this(parts: ListBuffer[RegexAST], numCaptureGroups: Int, position: Int) { + def this(parts: ListBuffer[RegexAST], numCaptureGroups: Int, position: Int) = { this(parts, numCaptureGroups) this.position = Some(position) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala index 7271cfd533f..8be9f37fa55 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala @@ -20,6 +20,7 @@ import scala.annotation.tailrec import scala.collection.mutable import ai.rapids.cudf.CudfColumnSizeOverflowException +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion @@ -223,7 +224,7 @@ object RmmRapidsRetryIterator extends Logging { } /** - * withRestoreOnRetry for CheckpointRestore. This helper function calls `fn` with no input and + * withRestoreOnRetry for Retryable. This helper function calls `fn` with no input and * returns the result. In the event of an OOM Retry exception, it calls the restore() method * of the input and then throws the oom exception. This is intended to be used within the `fn` * of one of the withRetry* functions. It provides an opportunity to reset state in the case @@ -231,11 +232,11 @@ object RmmRapidsRetryIterator extends Logging { * * @param r a single item T * @param fn the work to perform. Takes no input and produces K - * @tparam T element type that must be a `CheckpointRestore` subclass + * @tparam T element type that must be a `Retryable` subclass * @tparam K `fn` result type * @return a single item of type K */ - def withRestoreOnRetry[T <: CheckpointRestore, K](r: T)(fn: => K): K = { + def withRestoreOnRetry[T <: Retryable, K](r: T)(fn: => K): K = { try { fn } catch { @@ -250,7 +251,7 @@ object RmmRapidsRetryIterator extends Logging { } /** - * withRestoreOnRetry for CheckpointRestore. This helper function calls `fn` with no input and + * withRestoreOnRetry for Retryable. This helper function calls `fn` with no input and * returns the result. In the event of an OOM Retry exception, it calls the restore() method * of the input and then throws the oom exception. This is intended to be used within the `fn` * of one of the withRetry* functions. It provides an opportunity to reset state in the case @@ -258,11 +259,11 @@ object RmmRapidsRetryIterator extends Logging { * * @param r a Seq of item T * @param fn the work to perform. Takes no input and produces K - * @tparam T element type that must be a `CheckpointRestore` subclass + * @tparam T element type that must be a `Retryable` subclass * @tparam K `fn` result type * @return a single item of type K */ - def withRestoreOnRetry[T <: CheckpointRestore, K](r: Seq[T])(fn: => K): K = { + def withRestoreOnRetry[T <: Retryable, K](r: Seq[T])(fn: => K): K = { try { fn } catch { @@ -673,18 +674,6 @@ object RmmRapidsRetryIterator extends Logging { } } -trait CheckpointRestore { - /** - * Save state so it can be restored in case of an OOM Retry. - */ - def checkpoint(): Unit - - /** - * Restore state that was checkpointed. - */ - def restore(): Unit -} - /** * This is a wrapper that turns a target size into an autocloseable to allow it to be used * in withRetry blocks. It is intended to be used to help with cases where the split calculation diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SamplingUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SamplingUtils.scala index ce3a5783414..4b4e6128ca0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SamplingUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SamplingUtils.scala @@ -271,7 +271,7 @@ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { (nextSeed & ((1L << bits) -1)).asInstanceOf[Int] } - override def setSeed(s: Long) { + override def setSeed(s: Long): Unit = { seed = XORShiftRandom.hashSeed(s) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ScalableTaskCompletion.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ScalableTaskCompletion.scala index b3fd9fb2ece..695f511493c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ScalableTaskCompletion.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ScalableTaskCompletion.scala @@ -87,11 +87,25 @@ object ScalableTaskCompletion { private class TopLevelTaskCompletion extends Function[TaskContext, Unit] { private val pending = new util.HashSet[UserTaskCompletion]() private var callbacksDone = false + private var invokingACallback: Boolean = false + + private def throwIfInCallback(): Unit = { + if (invokingACallback) { + throw new IllegalStateException( + s"Detected a task completion callback attempting " + + "to add/remove callbacks. This is not supported.") + } + } private def callAllCallbacks(tc: TaskContext): Throwable = synchronized { + throwIfInCallback() var error: Throwable = null pending.forEach { utc => try { + // this is true while we invoke the callback + // so we can throw a bette error/stack trace + // instead of a ConcurrentModificationException + invokingACallback = true if (tc == null) { utc(utc.tc) } else { @@ -104,6 +118,8 @@ object ScalableTaskCompletion { } else { error.addSuppressed(t) } + } finally { + invokingACallback = false } } pending.clear() @@ -129,6 +145,7 @@ object ScalableTaskCompletion { } def add(u: UserTaskCompletion): Unit = synchronized { + throwIfInCallback() if (callbacksDone) { // Added a callback after it was done calling them back already u(u.tc) @@ -138,6 +155,7 @@ object ScalableTaskCompletion { } def remove(u: UserTaskCompletion): Unit = synchronized { + throwIfInCallback() pending.remove(u) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala index beb5db35cbd..82efa7699ef 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala @@ -97,7 +97,7 @@ class SpillableColumnarBatchImpl ( } override lazy val sizeInBytes: Long = - withRapidsBuffer(_.getMemoryUsedBytes) + withRapidsBuffer(_.memoryUsedBytes) /** * Set a new spill priority. @@ -164,7 +164,7 @@ class SpillableHostColumnarBatchImpl ( } override lazy val sizeInBytes: Long = { - withRapidsHostBatchBuffer(_.getMemoryUsedBytes) + withRapidsHostBatchBuffer(_.memoryUsedBytes) } /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index a50853eb509..b50029ab344 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -1476,7 +1476,7 @@ class CastChecks extends ExprChecks { } } - private[this] def tagBase(meta: RapidsMeta[_, _, _], willNotWork: String => Unit): Unit = { + protected def tagBase(meta: RapidsMeta[_, _, _], willNotWork: String => Unit): Unit = { val cast = meta.wrapped.asInstanceOf[UnaryExpression] val from = cast.child.dataType val to = cast.dataType @@ -1506,6 +1506,21 @@ class CastChecks extends ExprChecks { } } +/** + * This class is just restricting the 'to' dataType to a StringType in the CastChecks class + */ +class ToPrettyStringChecks extends CastChecks { + + override protected def tagBase(meta: RapidsMeta[_, _, _], willNotWork: String => Unit): Unit = { + val cast = meta.wrapped.asInstanceOf[UnaryExpression] + val from = cast.child.dataType + val to = StringType + if (!gpuCanCast(from, to)) { + willNotWork(s"${meta.wrapped.getClass.getSimpleName} from $from to $to is not supported") + } + } +} + object ExprChecks { /** * A check for an expression that only supports project. @@ -2235,7 +2250,7 @@ object SupportedOpsForTools { } } - private def outputSupportIO() { + private def outputSupportIO(): Unit = { // Look at what we have for defaults for some configs because if the configs are off // it likely means something isn't completely compatible. val conf = new RapidsConf(Map.empty[String, String]) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index c59352473a7..9a66bf237ac 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -21,10 +21,11 @@ import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf import ai.rapids.cudf._ +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRetry, withRetryNoSplit} +import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRestoreOnRetry, withRetry, withRetryNoSplit} import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import com.nvidia.spark.rapids.jni.SplitAndRetryOOM import com.nvidia.spark.rapids.shims._ @@ -136,13 +137,15 @@ object GpuProjectExec { def projectWithRetrySingleBatch(sb: SpillableColumnarBatch, boundExprs: Seq[Expression]): ColumnarBatch = { - // First off we want to find/run all of the expressions that are non-deterministic + // First off we want to find/run all of the expressions that are not retryable, // These cannot be retried. - val (deterministicExprs, nonDeterministicExprs) = boundExprs.partition(_.deterministic) + val (retryableExprs, notRetryableExprs) = boundExprs.partition( + _.asInstanceOf[GpuExpression].retryable) + val retryables = GpuExpressionsUtils.collectRetryables(retryableExprs) - val snd = if (nonDeterministicExprs.nonEmpty) { + val snd = if (notRetryableExprs.nonEmpty) { withResource(sb.getColumnarBatch()) { cb => - Some(SpillableColumnarBatch(project(cb, nonDeterministicExprs), + Some(SpillableColumnarBatch(project(cb, notRetryableExprs), SpillPriorities.ACTIVE_ON_DECK_PRIORITY)) } } else { @@ -150,10 +153,13 @@ object GpuProjectExec { } withResource(snd) { snd => + retryables.foreach(_.checkpoint()) RmmRapidsRetryIterator.withRetryNoSplit { val deterministicResults = withResource(sb.getColumnarBatch()) { cb => - // For now we are just going to run all of these and deal with losing work... - project(cb, deterministicExprs) + withRestoreOnRetry(retryables) { + // For now we are just going to run all of these and deal with losing work... + project(cb, retryableExprs) + } } if (snd.isEmpty) { // We are done and the order should be the same so we don't need to do anything... @@ -493,14 +499,16 @@ case class GpuProjectAstExec( */ case class GpuTieredProject(exprTiers: Seq[Seq[GpuExpression]]) { /** - * Is everything deterministic. This can help with reliability in the common case. + * Is everything retryable. This can help with reliability in the common case. */ - lazy val areAllDeterministic = !exprTiers.exists { tier => + lazy val areAllRetryable = !exprTiers.exists { tier => tier.exists { expr => - !expr.deterministic + !expr.retryable } } + lazy val retryables: Seq[Retryable] = exprTiers.flatMap(GpuExpressionsUtils.collectRetryables) + lazy val outputTypes = exprTiers.last.map(_.dataType).toArray private[this] def getPassThroughIndex(tierIndex: Int, @@ -535,20 +543,22 @@ case class GpuProjectAstExec( private [this] def projectWithRetrySingleBatchInternal(sb: SpillableColumnarBatch, closeInputBatch: Boolean): ColumnarBatch = { - if (areAllDeterministic) { - // If all of the expressions are deterministic we can just run everything and retry it - // at the top level. If some things are non-deterministic we need to split them up and + if (areAllRetryable) { + // If all of the expressions are retryable we can just run everything and retry it + // at the top level. If some things are not retryable we need to split them up and // do the processing in a way that makes it so retries are more likely to succeed. - if (closeInputBatch) { - RmmRapidsRetryIterator.withRetryNoSplit(sb) { _ => - withResource(sb.getColumnarBatch()) { cb => - project(cb) - } - } + val sbToClose = if (closeInputBatch) { + Some(sb) } else { + None + } + withResource(sbToClose) { _ => + retryables.foreach(_.checkpoint()) RmmRapidsRetryIterator.withRetryNoSplit { withResource(sb.getColumnarBatch()) { cb => - project(cb) + withRestoreOnRetry(retryables) { + project(cb) + } } } } @@ -639,16 +649,16 @@ object GpuFilter { numOutputRows: GpuMetric, numOutputBatches: GpuMetric, filterTime: GpuMetric): Iterator[ColumnarBatch] = { - if (boundCondition.areAllDeterministic) { + if (boundCondition.areAllRetryable) { val sb = SpillableColumnarBatch(batch, SpillPriorities.ACTIVE_ON_DECK_PRIORITY) filterAndCloseWithRetry(sb, boundCondition, numOutputRows, numOutputBatches, filterTime) } else { - filterAndCloseNondeterministic(batch, boundCondition, numOutputRows, numOutputBatches, + filterAndCloseNoRetry(batch, boundCondition, numOutputRows, numOutputBatches, filterTime) } } - private def filterAndCloseNondeterministic(batch: ColumnarBatch, + private def filterAndCloseNoRetry(batch: ColumnarBatch, boundCondition: GpuTieredProject, numOutputRows: GpuMetric, numOutputBatches: GpuMetric, @@ -668,10 +678,13 @@ object GpuFilter { numOutputRows: GpuMetric, numOutputBatches: GpuMetric, opTime: GpuMetric): Iterator[ColumnarBatch] = { + boundCondition.retryables.foreach(_.checkpoint()) val ret = withRetry(input, splitSpillableInHalfByRows) { sb => withResource(sb.getColumnarBatch()) { cb => - withResource(new NvtxWithMetrics("filter batch", NvtxColor.YELLOW, opTime)) { _ => - GpuFilter(cb, boundCondition) + withRestoreOnRetry(boundCondition.retryables) { + withResource(new NvtxWithMetrics("filter batch", NvtxColor.YELLOW, opTime)) { _ => + GpuFilter(cb, boundCondition) + } } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProvider.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProvider.scala index 9301f992565..7e521bd4829 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProvider.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProvider.scala @@ -16,11 +16,12 @@ package com.nvidia.spark.rapids.delta -import com.nvidia.spark.rapids.{CreatableRelationProviderRule, ExecRule, RunnableCommandRule, ShimLoader} +import com.nvidia.spark.rapids.{CreatableRelationProviderRule, ExecRule, RunnableCommandRule, ShimLoader, SparkPlanMeta} import org.apache.spark.sql.Strategy -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.sources.CreatableRelationProvider /** Probe interface to determine which Delta Lake provider to use. */ @@ -39,6 +40,12 @@ trait DeltaProvider { RunnableCommandRule[_ <: RunnableCommand]] def getStrategyRules: Seq[Strategy] + + def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean + + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit + + def getReadFileFormat(format: FileFormat): FileFormat } object DeltaProvider { @@ -59,4 +66,12 @@ object NoDeltaProvider extends DeltaProvider { RunnableCommandRule[_ <: RunnableCommand]] = Map.empty override def getStrategyRules: Seq[Strategy] = Nil + + override def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = false + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = + throw new IllegalStateException("unsupported format") + + override def getReadFileFormat(format: FileFormat): FileFormat = + throw new IllegalStateException("unsupported format") } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala index b17a70d303f..e4eea2d92b8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,11 +22,7 @@ import org.apache.spark.sql.connector.read.Scan /** Interfaces to avoid accessing the optional Apache Iceberg jars directly in common code. */ trait IcebergProvider { - def isSupportedScan(scan: Scan): Boolean - def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] - - def copyScanWithInputFileTrue(scan: Scan): Scan } object IcebergProvider { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala index 830a1029a80..4a6c2b63fa7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala @@ -19,14 +19,12 @@ package com.nvidia.spark.rapids.iceberg import scala.reflect.ClassTag import scala.util.{Failure, Try} -import com.nvidia.spark.rapids.{FileFormatChecks, IcebergFormatType, RapidsConf, ReadFileOp, ScanMeta, ScanRule, ShimReflectionUtils} +import com.nvidia.spark.rapids.{FileFormatChecks, GpuScan, IcebergFormatType, RapidsConf, ReadFileOp, ScanMeta, ScanRule, ShimReflectionUtils} import com.nvidia.spark.rapids.iceberg.spark.source.GpuSparkBatchQueryScan import org.apache.spark.sql.connector.read.Scan class IcebergProviderImpl extends IcebergProvider { - override def isSupportedScan(scan: Scan): Boolean = scan.isInstanceOf[GpuSparkBatchQueryScan] - override def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = { val cpuIcebergScanClass = ShimReflectionUtils.loadClass(IcebergProvider.cpuScanClassName) Seq(new ScanRule[Scan]( @@ -60,17 +58,10 @@ class IcebergProviderImpl extends IcebergProvider { } } - override def convertToGpu(): Scan = convertedScan.get + override def convertToGpu(): GpuScan = convertedScan.get }, "Iceberg scan", ClassTag(cpuIcebergScanClass)) ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap } - - override def copyScanWithInputFileTrue(scan: Scan): Scan = scan match { - case icebergBatchScan: GpuSparkBatchQueryScan => - icebergBatchScan.copyWithInputFileTrue(); - case _ => - throw new RuntimeException(s"Unsupported scan type: ${scan.getClass.getSimpleName}") - } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala index c1d65f16980..08a1ae22f5e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala @@ -179,7 +179,7 @@ class BufferSendState( requestHandler.acquireShuffleBuffer(bufferId)) { rapidsBuffer => //these are closed later, after we synchronize streams rapidsBuffer.storageTier match { - case StorageTier.DEVICE | StorageTier.GDS => + case StorageTier.DEVICE => deviceBuffs += blockRange.rangeSize() case _ => // host/disk hostBuffs += blockRange.rangeSize() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala index 43399cb4825..e9d9d7f2d65 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala @@ -361,7 +361,7 @@ class RapidsShuffleIterator( try { sb = catalog.acquireBuffer(handle) cb = sb.getColumnarBatch(sparkTypes) - metricsUpdater.update(blockedTime, 1, sb.getMemoryUsedBytes, cb.numRows()) + metricsUpdater.update(blockedTime, 1, sb.memoryUsedBytes, cb.numRows()) } finally { nvtxRangeAfterGettingBatch.close() range.close() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTransport.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTransport.scala index 093903fec12..998f5675efe 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTransport.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTransport.scala @@ -32,7 +32,7 @@ trait TransactionCallback { } trait MemoryRegistrationCallback { - def apply(error: Option[Throwable] = None) + def apply(error: Option[Throwable] = None): Unit } /** @@ -344,7 +344,7 @@ trait RapidsShuffleTransport extends AutoCloseable { * (throttle) Adds a set of requests to be throttled as limits allowed. * @param reqs requests to add to the throttle queue */ - def queuePending(reqs: Seq[PendingTransferRequest]) + def queuePending(reqs: Seq[PendingTransferRequest]): Unit /** * Cancel requests that are waiting in the queue (not in-flight) for a specific diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index 8f1e5066c85..8ac04f19429 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -198,7 +198,7 @@ case class GpuJsonScan( dataFilters: Seq[Expression], maxReaderBatchSizeRows: Integer, maxReaderBatchSizeBytes: Long) - extends TextBasedFileScan(sparkSession, options) with ScanWithMetrics { + extends TextBasedFileScan(sparkSession, options) with GpuScan { private lazy val parsedOptions: JSONOptions = new JSONOptions( options.asScala.toMap, @@ -222,6 +222,8 @@ case class GpuJsonScan( dataSchema, readDataSchema, readPartitionSchema, parsedOptions, maxReaderBatchSizeRows, maxReaderBatchSizeBytes, metrics, options.asScala.toMap) } + + override def withInputFile(): GpuScan = this } case class GpuJsonPartitionReaderFactory( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala index 7fe59b983cc..3b85493266e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,12 +20,15 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.JSONOptionsInRead +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -66,6 +69,15 @@ class GpuReadJsonFileFormat extends JsonFileFormat with GpuReadFileFormatWithMet options) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = true + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + throw new IllegalStateException("JSON format does not support multifile reads") + } } object GpuReadJsonFileFormat { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index bc3a5b3ea9e..618ba14b792 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -573,6 +573,7 @@ object GpuMin{ abstract class GpuMin(child: Expression) extends GpuAggregateFunction with GpuBatchedRunningWindowWithFixer + with GpuUnboundToUnboundWindowWithFixer with GpuAggregateWindowFunction with GpuRunningWindowFunction with Serializable { @@ -602,6 +603,10 @@ abstract class GpuMin(child: Expression) extends GpuAggregateFunction override def newFixer(): BatchedRunningWindowFixer = new BatchedRunningWindowBinaryFixer(BinaryOp.NULL_MIN, "min") + // UNBOUNDED TO UNBOUNDED WINDOW + override def newUnboundedToUnboundedFixer: BatchedUnboundedToUnboundedWindowFixer = + new BatchedUnboundedToUnboundedBinaryFixer(BinaryOp.NULL_MIN, dataType) + override def groupByScanInputProjection(isRunningBatched: Boolean): Seq[Expression] = inputProjection @@ -752,6 +757,7 @@ object GpuMax { abstract class GpuMax(child: Expression) extends GpuAggregateFunction with GpuBatchedRunningWindowWithFixer + with GpuUnboundToUnboundWindowWithFixer with GpuAggregateWindowFunction with GpuRunningWindowFunction with Serializable { @@ -781,6 +787,10 @@ abstract class GpuMax(child: Expression) extends GpuAggregateFunction override def newFixer(): BatchedRunningWindowFixer = new BatchedRunningWindowBinaryFixer(BinaryOp.NULL_MAX, "max") + // UNBOUNDED TO UNBOUNDED WINDOW + override def newUnboundedToUnboundedFixer: BatchedUnboundedToUnboundedWindowFixer = + new BatchedUnboundedToUnboundedBinaryFixer(BinaryOp.NULL_MAX, dataType) + override def groupByScanInputProjection(isRunningBatched: Boolean): Seq[Expression] = inputProjection @@ -1576,6 +1586,13 @@ case class GpuCount(children: Seq[Expression], override def newUnboundedToUnboundedFixer: BatchedUnboundedToUnboundedWindowFixer = new CountUnboundedToUnboundedFixer(failOnError) + + // minPeriods should be 0. + // Consider the following rows: + // v = [ 0, 1, 2, 3, 4, 5 ] + // A `COUNT` window aggregation over (2, -1) should yield 0, not null, + // for the first row. + override def getMinPeriods: Int = 0 } object GpuAverage { @@ -1961,6 +1978,12 @@ case class GpuCollectList( override def windowAggregation( inputs: Seq[(ColumnVector, Int)]): RollingAggregationOnColumn = RollingAggregation.collectList().onColumn(inputs.head._2) + + // minPeriods should be 0. + // Consider the following rows: v = [ 0, 1, 2, 3, 4, 5 ] + // A `COLLECT_LIST` window aggregation over (2, -1) should yield an empty array [], + // not null, for the first row. + override def getMinPeriods: Int = 0 } /** @@ -1995,6 +2018,12 @@ case class GpuCollectSet( RollingAggregation.collectSet(NullPolicy.EXCLUDE, NullEquality.EQUAL, NaNEquality.ALL_EQUAL).onColumn(inputs.head._2) } + + // minPeriods should be 0. + // Consider the following rows: v = [ 0, 1, 2, 3, 4, 5 ] + // A `COLLECT_SET` window aggregation over (2, -1) should yield an empty array [], + // not null, for the first row. + override def getMinPeriods: Int = 0 } trait CpuToGpuAggregateBufferConverter { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala index dd913a04631..2d8dd1e2a43 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -30,26 +30,16 @@ import org.apache.spark.util.SerializableConfiguration class AvroProviderImpl 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 isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = { + format == classOf[AvroFileFormat] } def isPerFileReadEnabledForFormat(format: FileFormat, conf: RapidsConf): Boolean = { - format match { - case _: AvroFileFormat => conf.isAvroPerFileReadEnabled - case _ => false - } + conf.isAvroPerFileReadEnabled } 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}") - } + GpuReadAvroFileFormat.tagSupport(meta) } /** @@ -57,11 +47,8 @@ class AvroProviderImpl extends AvroProvider { * 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") - } + require(isSupportedFormat(format.getClass), s"unexpected format: $format") + new GpuReadAvroFileFormat } /** @@ -73,23 +60,17 @@ class AvroProviderImpl extends AvroProvider { 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.readPartitionSchema, - 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") - } + GpuAvroMultiFilePartitionReaderFactory( + fileScan.relation.sparkSession.sessionState.conf, + fileScan.rapidsConf, + broadcastedConf, + fileScan.relation.dataSchema, + fileScan.requiredSchema, + fileScan.readPartitionSchema, + new AvroOptions(fileScan.relation.options, broadcastedConf.value.value), + fileScan.allMetrics, + pushedFilters, + fileScan.queryUsesInputFile) } def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = { @@ -99,7 +80,7 @@ class AvroProviderImpl extends AvroProvider { (a, conf, p, r) => new ScanMeta[AvroScan](a, conf, p, r) { override def tagSelfForGpu(): Unit = GpuAvroScan.tagSupport(this) - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuAvroScan(a.sparkSession, a.fileIndex, a.dataSchema, @@ -113,13 +94,4 @@ class AvroProviderImpl extends AvroProvider { }) ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap } - - def isSupportedScan(scan: Scan): Boolean = scan.isInstanceOf[GpuAvroScan] - - def copyScanWithInputFileTrue(scan: Scan): Scan = scan match { - case avroScan: GpuAvroScan => - avroScan.copy(queryUsesInputFile=true) - case _ => - throw new RuntimeException(s"Unsupported scan type: ${scan.getClass.getSimpleName}") - } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala index b9631377438..fe560551aa3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala @@ -23,14 +23,13 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.delta.DeltaProvider import com.nvidia.spark.rapids.iceberg.IcebergProvider -import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.sql.connector.read.{PartitionReaderFactory, Scan} +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.sources.{CreatableRelationProvider, Filter} -import org.apache.spark.util.{SerializableConfiguration, Utils} +import org.apache.spark.sql.sources.CreatableRelationProvider +import org.apache.spark.util.Utils /** * The subclass of AvroProvider imports spark-avro classes. This file should not imports @@ -73,21 +72,24 @@ object ExternalSource extends Logging { deltaProvider.getExecRules /** If the file format is supported as an external source */ - def isSupportedFormat(format: FileFormat): Boolean = { - if (hasSparkAvroJar) { - avroProvider.isSupportedFormat(format) - } else false - } - - def isPerFileReadEnabledForFormat(format: FileFormat, conf: RapidsConf): Boolean = { - if (hasSparkAvroJar) { - avroProvider.isPerFileReadEnabledForFormat(format, conf) - } else false + def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = { + if (hasSparkAvroJar && avroProvider.isSupportedFormat(format)) { + true + } else if (deltaProvider.isSupportedFormat(format)) { + true + } else { + false + } } def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { - if (hasSparkAvroJar) { + val format = meta.wrapped.relation.fileFormat + if (hasSparkAvroJar && avroProvider.isSupportedFormat(format.getClass)) { avroProvider.tagSupportForGpuFileSourceScan(meta) + } else if (deltaProvider.isSupportedFormat(format.getClass)) { + deltaProvider.tagSupportForGpuFileSourceScan(meta) + } else { + meta.willNotWorkOnGpu(s"unsupported file format: ${format.getClass.getCanonicalName}") } } @@ -96,30 +98,15 @@ object ExternalSource extends Logging { * Better to check if the format is supported first by calling 'isSupportedFormat' */ def getReadFileFormat(format: FileFormat): FileFormat = { - if (hasSparkAvroJar) { + if (hasSparkAvroJar && avroProvider.isSupportedFormat(format.getClass)) { avroProvider.getReadFileFormat(format) + } else if (deltaProvider.isSupportedFormat(format.getClass)) { + deltaProvider.getReadFileFormat(format) } else { throw new IllegalArgumentException(s"${format.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 = { - if (hasSparkAvroJar) { - avroProvider.createMultiFileReaderFactory(format, broadcastedConf, pushedFilters, - fileScan) - } else { - throw new RuntimeException(s"File format $format is not supported yet") - } - } - def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = { var scans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Map.empty if (hasSparkAvroJar) { @@ -131,31 +118,6 @@ object ExternalSource extends Logging { scans } - /** If the scan is supported as an external source */ - def isSupportedScan(scan: Scan): Boolean = { - if (hasSparkAvroJar && avroProvider.isSupportedScan(scan)) { - true - } else if (hasIcebergJar && icebergProvider.isSupportedScan(scan)) { - true - } else { - false - } - } - - /** - * Clone the input scan with setting 'true' to the 'queryUsesInputFile'. - * Better to check if the scan is supported first by calling 'isSupportedScan'. - */ - def copyScanWithInputFileTrue(scan: Scan): Scan = { - if (hasSparkAvroJar && avroProvider.isSupportedScan(scan)) { - avroProvider.copyScanWithInputFileTrue(scan) - } else if (hasIcebergJar && icebergProvider.isSupportedScan(scan)) { - icebergProvider.copyScanWithInputFileTrue(scan) - } else { - throw new RuntimeException(s"Unsupported scan type: ${scan.getClass.getSimpleName}") - } - } - def wrapCreatableRelationProvider[INPUT <: CreatableRelationProvider]( provider: INPUT, conf: RapidsConf, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index 539cb36438c..a4c64b2b2f4 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -104,7 +104,7 @@ case class GpuAvroScan( rapidsConf: RapidsConf, partitionFilters: Seq[Expression] = Seq.empty, dataFilters: Seq[Expression] = Seq.empty, - queryUsesInputFile: Boolean = false) extends FileScan with ScanWithMetrics { + queryUsesInputFile: Boolean = false) extends FileScan with GpuScan { override def isSplitable(path: Path): Boolean = true override def createReaderFactory(): PartitionReaderFactory = { @@ -144,6 +144,8 @@ case class GpuAvroScan( override def description(): String = { super.description() + ", PushedFilters: " + seqToString(pushedFilters) } + + override def withInputFile(): GpuScan = copy(queryUsesInputFile = true) } /** Avro partition reader factory to build columnar reader */ 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 fe22478bcb3..80acf2b0f9e 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 @@ -37,7 +37,6 @@ import org.apache.spark.sql.execution.{ExecSubqueryExpression, ExplainUtils, Fil import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat -import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.rapids.shims.FilePartitionShims @@ -91,8 +90,7 @@ case class GpuFileSourceScanExec( dataOutAttrs ++ prunedPartOutAttrs }.getOrElse(originalOutput) - private[rapids] val readPartitionSchema = - requiredPartitionSchema.getOrElse(relation.partitionSchema) + val readPartitionSchema = requiredPartitionSchema.getOrElse(relation.partitionSchema) // this is set only when we either explicitly replaced a path for CONVERT_TIME // or when TASK_TIME if one of the paths will be replaced. @@ -100,14 +98,13 @@ case class GpuFileSourceScanExec( // should update this to None and read directly from s3 to get faster. private var alluxioPathReplacementMap: Option[Map[String, String]] = alluxioPathsMap - private val isPerFileReadEnabled = relation.fileFormat match { - case _: ParquetFileFormat => rapidsConf.isParquetPerFileReadEnabled - case _: OrcFileFormat => rapidsConf.isOrcPerFileReadEnabled - case ef if ExternalSource.isSupportedFormat(ef) => - ExternalSource.isPerFileReadEnabledForFormat(ef, rapidsConf) - case _ => true // For others, default to PERFILE reader + @transient private val gpuFormat = relation.fileFormat match { + case g: GpuReadFileFormatWithMetrics => g + case f => throw new IllegalStateException(s"${f.getClass} is not a GPU format with metrics") } + private val isPerFileReadEnabled = gpuFormat.isPerFileReadEnabled(rapidsConf) + override def otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) // All expressions are filter expressions used on the CPU. @@ -363,8 +360,7 @@ case class GpuFileSourceScanExec( lazy val inputRDD: RDD[InternalRow] = { val readFile: Option[(PartitionedFile) => Iterator[InternalRow]] = if (isPerFileReadEnabled) { - val fileFormat = relation.fileFormat.asInstanceOf[GpuReadFileFormatWithMetrics] - val reader = fileFormat.buildReaderWithPartitionValuesAndMetrics( + val reader = gpuFormat.buildReaderWithPartitionValuesAndMetrics( sparkSession = relation.sparkSession, dataSchema = relation.dataSchema, partitionSchema = readPartitionSchema, @@ -620,44 +616,13 @@ case class GpuFileSourceScanExec( lazy val readerFactory: PartitionReaderFactory = { // here we are making an optimization to read more then 1 file at a time on the CPU side // if they are small files before sending it down to the GPU - val sqlConf = relation.sparkSession.sessionState.conf val hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options) val broadcastedHadoopConf = relation.sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - - relation.fileFormat match { - case _: ParquetFileFormat => - GpuParquetMultiFilePartitionReaderFactory( - sqlConf, - broadcastedHadoopConf, - relation.dataSchema, - requiredSchema, - readPartitionSchema, - pushedDownFilters.toArray, - rapidsConf, - allMetrics, - queryUsesInputFile, - alluxioPathReplacementMap) - case _: OrcFileFormat => - GpuOrcMultiFilePartitionReaderFactory( - sqlConf, - broadcastedHadoopConf, - relation.dataSchema, - requiredSchema, - readPartitionSchema, - pushedDownFilters.toArray, - rapidsConf, - allMetrics, - queryUsesInputFile) - case ef if ExternalSource.isSupportedFormat(ef) => - ExternalSource.createMultiFileReaderFactory( - ef, - broadcastedHadoopConf, - pushedDownFilters.toArray, - this) - case other => - throw new IllegalArgumentException(s"${other.getClass.getCanonicalName} is not supported") - } + gpuFormat.createMultiFileReaderFactory( + broadcastedHadoopConf, + pushedDownFilters.toArray, + this) } // Filters unused DynamicPruningExpression expressions - one which has been replaced @@ -686,28 +651,36 @@ case class GpuFileSourceScanExec( object GpuFileSourceScanExec { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { - meta.wrapped.relation.fileFormat match { - case _: CSVFileFormat => GpuReadCSVFileFormat.tagSupport(meta) - case f if GpuOrcFileFormat.isSparkOrcFormat(f) => GpuReadOrcFileFormat.tagSupport(meta) - case _: ParquetFileFormat => GpuReadParquetFileFormat.tagSupport(meta) - case _: JsonFileFormat => GpuReadJsonFileFormat.tagSupport(meta) - case ef if ExternalSource.isSupportedFormat(ef) => - ExternalSource.tagSupportForGpuFileSourceScan(meta) - case other => - meta.willNotWorkOnGpu(s"unsupported file format: ${other.getClass.getCanonicalName}") + val cls = meta.wrapped.relation.fileFormat.getClass + if (cls == classOf[CSVFileFormat]) { + GpuReadCSVFileFormat.tagSupport(meta) + } else if (GpuOrcFileFormat.isSparkOrcFormat(cls)) { + GpuReadOrcFileFormat.tagSupport(meta) + } else if (cls == classOf[ParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + } else if (cls == classOf[JsonFileFormat]) { + GpuReadJsonFileFormat.tagSupport(meta) + } else if (ExternalSource.isSupportedFormat(cls)) { + ExternalSource.tagSupportForGpuFileSourceScan(meta) + } else { + meta.willNotWorkOnGpu(s"unsupported file format: ${cls.getCanonicalName}") } } def convertFileFormat(format: FileFormat): FileFormat = { - format match { - case _: CSVFileFormat => new GpuReadCSVFileFormat - case f if GpuOrcFileFormat.isSparkOrcFormat(f) => new GpuReadOrcFileFormat - case _: ParquetFileFormat => new GpuReadParquetFileFormat - case _: JsonFileFormat => new GpuReadJsonFileFormat - case ef if ExternalSource.isSupportedFormat(ef) => ExternalSource.getReadFileFormat(ef) - case other => - throw new IllegalArgumentException(s"${other.getClass.getCanonicalName} is not supported") - + val cls = format.getClass + if (cls == classOf[CSVFileFormat]) { + new GpuReadCSVFileFormat + } else if (GpuOrcFileFormat.isSparkOrcFormat(cls)) { + new GpuReadOrcFileFormat + } else if (cls == classOf[ParquetFileFormat]) { + new GpuReadParquetFileFormat + } else if (cls == classOf[JsonFileFormat]) { + new GpuReadJsonFileFormat + } else if (ExternalSource.isSupportedFormat(cls)) { + ExternalSource.getReadFileFormat(format) + } else { + throw new IllegalArgumentException(s"${cls.getCanonicalName} is not supported") } } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala index af9cfc8f717..3b5aea39bf0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala @@ -197,7 +197,7 @@ case class GpuJsonToStructs( } else { val col = rawTable.getColumn(i) // getSparkType is only used to get the from type for cast - doCast(col, getSparkType(col), dtype, false, false, false) + doCast(col, getSparkType(col), dtype) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index 354245e2a75..9e50a993537 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -37,10 +37,8 @@ object GpuOrcFileFormat extends Logging { // Spark is not always compiled with Hive support so we cannot import from Spark jars directly. private val HIVE_IMPL_CLASS = "org.apache.spark.sql.hive.orc.OrcFileFormat" - def isSparkOrcFormat(format: FileFormat): Boolean = format match { - case _: OrcFileFormat => true - case f if f.getClass.getCanonicalName.equals(HIVE_IMPL_CLASS) => true - case _ => false + def isSparkOrcFormat(cls: Class[_ <: FileFormat]): Boolean = { + cls == classOf[OrcFileFormat] || cls.getCanonicalName.equals(HIVE_IMPL_CLASS) } def tagGpuSupport(meta: RapidsMeta[_, _, _], diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala index 0386602de0a..d50b0b184ca 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,9 +20,11 @@ import com.nvidia.spark.rapids.{GpuMetric, GpuReadFileFormatWithMetrics, Partiti import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.{AvroFileFormat, AvroOptions} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.sources.Filter @@ -64,6 +66,25 @@ class GpuReadAvroFileFormat extends AvroFileFormat with GpuReadFileFormatWithMet options) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = conf.isAvroPerFileReadEnabled + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuAvroMultiFilePartitionReaderFactory( + fileScan.relation.sparkSession.sessionState.conf, + fileScan.rapidsConf, + broadcastedConf, + fileScan.relation.dataSchema, + fileScan.requiredSchema, + fileScan.readPartitionSchema, + new AvroOptions(fileScan.relation.options, broadcastedConf.value.value), + fileScan.allMetrics, + pushedFilters, + fileScan.queryUsesInputFile) + } } object GpuReadAvroFileFormat { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala index 021aecf0067..67f56cfabc3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala @@ -44,7 +44,7 @@ trait ShuffleMetricsUpdater { fetchWaitTimeInMs: Long, remoteBlocksFetched: Long, remoteBytesRead: Long, - rowsFetched: Long) + rowsFetched: Long): Unit } class RapidsCachingReader[K, C]( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala index e1cbc6b0c06..2e8d46e6f6f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -401,13 +401,16 @@ trait GpuDecimalMultiplyBase extends GpuExpression { def regularMultiply(batch: ColumnarBatch): GpuColumnVector = { val castLhs = withResource(left.columnarEval(batch)) { lhs => - GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast( + lhs.getBase, + lhs.dataType(), + intermediateLhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } val ret = withResource(castLhs) { castLhs => val castRhs = withResource(right.columnarEval(batch)) { rhs => - GpuCast.doCast(rhs.getBase, rhs.dataType(), intermediateRhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast(rhs.getBase, rhs.dataType(), intermediateRhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } withResource(castRhs) { castRhs => withResource(castLhs.mul(castRhs, @@ -436,7 +439,7 @@ trait GpuDecimalMultiplyBase extends GpuExpression { } withResource(ret) { ret => GpuColumnVector.from(GpuCast.doCast(ret, intermediateResultType, dataType, - ansiMode = failOnError, legacyCastToString = false, stringToDateAnsiModeEnabled = false), + CastOptions.getArithmeticCastOptions(failOnError)), dataType) } } @@ -851,14 +854,18 @@ trait GpuDecimalDivideBase extends GpuExpression { def regularDivide(batch: ColumnarBatch): GpuColumnVector = { val castLhs = withResource(left.columnarEval(batch)) { lhs => - GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast( + lhs.getBase, + lhs.dataType(), + intermediateLhsType, + CastOptions.getArithmeticCastOptions(failOnError)) + } val ret = withResource(castLhs) { castLhs => val castRhs = withResource(right.columnarEval(batch)) { rhs => withResource(divByZeroFixes(rhs.getBase)) { fixed => - GpuCast.doCast(fixed, rhs.dataType(), intermediateRhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast(fixed, rhs.dataType(), intermediateRhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } } withResource(castRhs) { castRhs => @@ -871,7 +878,7 @@ trait GpuDecimalDivideBase extends GpuExpression { // in the common case with us. It will also handle rounding the result to the final scale // to match what Spark does. GpuColumnVector.from(GpuCast.doCast(ret, intermediateResultType, dataType, - ansiMode = failOnError, legacyCastToString = false, stringToDateAnsiModeEnabled = false), + CastOptions.getArithmeticCastOptions(failOnError)), dataType) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index d6a0c4ad630..6675f678f6d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.rapids.catalyst.expressions import ai.rapids.cudf.{DType, HostColumnVector, NvtxColor, NvtxRange} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuLiteral} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.shims.ShimUnaryExpression @@ -27,11 +28,11 @@ import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils -import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.util.random.rapids.RapidsXORShiftRandom /** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpression - with ExpectsInputTypes with ExpressionWithRandomSeed { + with ExpectsInputTypes with ExpressionWithRandomSeed with Retryable { def this() = this(GpuLiteral(Utils.random.nextLong(), LongType)) @@ -40,12 +41,13 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres def seedExpression: Expression = child override lazy val deterministic: Boolean = false + override val selfNonDeterministic: Boolean = true /** * Record ID within each partition. By being transient, the Random Number Generator is * reset every time we serialize and deserialize and initialize it. */ - @transient protected var rng: XORShiftRandom = _ + @transient protected var rng: RapidsXORShiftRandom = _ @transient protected lazy val seed: Long = child match { case GpuLiteral(s, IntegerType) => s.asInstanceOf[Int] @@ -55,6 +57,9 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres } @transient protected var previousPartition: Int = 0 + + @transient protected var curXORShiftRandomSeed: Option[Long] = None + private def wasInitialized: Boolean = rng != null override def nullable: Boolean = false @@ -63,13 +68,23 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(IntegerType, LongType)) + private def initRandom(): Unit = { + val partId = TaskContext.getPartitionId() + if (partId != previousPartition || !wasInitialized) { + rng = new RapidsXORShiftRandom(seed + partId) + previousPartition = partId + } + } + override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { + if (curXORShiftRandomSeed.isEmpty) { + // checkpoint not called, need to init the random generator here + initRandom() + } else { + // make sure here uses the same random generator with checkpoint + assert(wasInitialized) + } withResource(new NvtxRange("GpuRand", NvtxColor.RED)) { _ => - val partId = TaskContext.getPartitionId() - if (partId != previousPartition || !wasInitialized) { - rng = new XORShiftRandom(seed + partId) - previousPartition = partId - } val numRows = batch.numRows() withResource(HostColumnVector.builder(DType.FLOAT64, numRows)) { builder => (0 until numRows).foreach(_ => builder.append(rng.nextDouble())) @@ -77,4 +92,16 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres } } } + + override def checkpoint(): Unit = { + // In a task, checkpoint is called before columnarEval, so need to try to + // init the random generator here. + initRandom() + curXORShiftRandomSeed = Some(rng.currentSeed) + } + + override def restore(): Unit = { + assert(wasInitialized && curXORShiftRandomSeed.isDefined) + rng.setHashedSeed(curXORShiftRandomSeed.get) + } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala index dfc95016363..b47de2cdac2 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala @@ -145,6 +145,7 @@ object GpuCreateMap { def createMapFromKeysValuesAsStructs(dataType: MapType, listsOfKeyValueStructs : ColumnView): GpuColumnVector = { + GpuMapUtils.assertNoNullKeys(listsOfKeyValueStructs) withResource(listsOfKeyValueStructs.dropListDuplicatesWithKeysValues()) { deduped => if (exceptionOnDupKeys) { // Compare child data row count before and after removing duplicates to determine diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index d373b2459c1..e0fe58b0857 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -460,7 +460,7 @@ case class GpuSecondsToTimestamp(child: Expression) extends GpuNumberToTimestamp } case DoubleType | FloatType => (input: GpuColumnVector) => { - GpuCast.doCast(input.getBase, input.dataType, TimestampType, false, false, false) + GpuCast.doCast(input.getBase, input.dataType, TimestampType) } case dt: DecimalType => (input: GpuColumnVector) => { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala index 195374997fe..1e12f0c6733 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala @@ -142,7 +142,7 @@ abstract class GpuBroadcastHashJoinExecBase( streamIter: Iterator[ColumnarBatch], coalesceMetricsMap: Map[String, GpuMetric]): (ColumnarBatch, Iterator[ColumnarBatch]) = { - val bufferedStreamIter = new CloseableBufferedIterator(streamIter.buffered) + val bufferedStreamIter = new CloseableBufferedIterator(streamIter) closeOnExcept(bufferedStreamIter) { _ => withResource(new NvtxRange("first stream batch", NvtxColor.RED)) { _ => if (bufferedStreamIter.hasNext) { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala index bc2cb433deb..0421802252d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala @@ -68,6 +68,7 @@ abstract class GpuPythonFunction( with UserDefinedExpression with GpuAggregateWindowFunction with Serializable { override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) + override val selfNonDeterministic: Boolean = !udfDeterministic override def toString: String = s"$name(${children.mkString(", ")})" diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index e2c55b32647..febbf75ba58 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -17,11 +17,12 @@ package org.apache.spark.sql.rapids import java.nio.charset.Charset -import java.util.Optional +import java.text.DecimalFormatSymbols +import java.util.{Locale, Optional} import scala.collection.mutable.ArrayBuffer -import ai.rapids.cudf.{BinaryOp, BinaryOperable, CaptureGroups, ColumnVector, ColumnView, DType, PadSide, RegexProgram, Scalar, Table} +import ai.rapids.cudf.{BinaryOp, BinaryOperable, CaptureGroups, ColumnVector, ColumnView, DType, PadSide, RegexProgram, RoundMode, Scalar, Table} import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ @@ -1989,8 +1990,6 @@ case class GpuStringInstr(str: Expression, substr: Expression) } } - - class GpuConvMeta( expr: Conv, conf: RapidsConf, @@ -2079,4 +2078,660 @@ case class GpuConv(num: Expression, fromBase: Expression, toBase: Expression) override def third: Expression = toBase override def dataType: DataType = StringType -} \ No newline at end of file +} + +case class GpuFormatNumber(x: Expression, d: Expression) + extends GpuBinaryExpression with ExpectsInputTypes with NullIntolerant { + + override def left: Expression = x + override def right: Expression = d + override def dataType: DataType = StringType + override def nullable: Boolean = true + override def inputTypes: Seq[AbstractDataType] = Seq(NumericType, IntegerType) + + private def removeNegSign(cv: ColumnVector): ColumnVector = { + withResource(Scalar.fromString("-")) { negativeSign => + cv.lstrip(negativeSign) + } + } + + private def getZeroCv(size: Int): ColumnVector = { + withResource(Scalar.fromString("0")) { zero => + ColumnVector.fromScalar(zero, size) + } + } + + private def handleDoublePosExp(cv: ColumnVector, intPart: ColumnVector, decPart: ColumnVector, + exp: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // handle cases when exp is positive + // append "0" * zerosNum after end of strings, zerosNum = exp - decLen + d + val expSubDecLen = withResource(decPart.getCharLengths) { decLen => + exp.sub(decLen) + } + val zerosNum = withResource(expSubDecLen) { _ => + withResource(Scalar.fromInt(d)) { dScalar => + expSubDecLen.add(dScalar) + } + } + val zeroCv = withResource(Scalar.fromString("0")) { zero => + ColumnVector.fromScalar(zero, cv.getRowCount.toInt) + } + val zeros = withResource(zerosNum) { _ => + withResource(zeroCv) { _ => + zeroCv.repeatStrings(zerosNum) + } + } + + val intAndDecParts = withResource(zeros) { _ => + ColumnVector.stringConcatenate(Array(intPart, decPart, zeros)) + } + // split intAndDecParts to intPart and decPart with substrings, start = len(intAndDecParts) - d + closeOnExcept(ArrayBuffer.empty[ColumnVector]) { resourceArray => + val (intPartPosExp, decPartPosExpTemp) = withResource(intAndDecParts) { _ => + val (start, end) = withResource(intAndDecParts.getCharLengths) { partsLength => + (withResource(Scalar.fromInt(d)) { d => + partsLength.sub(d) + }, partsLength.incRefCount()) + } + withResource(start) { _ => + withResource(end) { _ => + val zeroIntCv = withResource(Scalar.fromInt(0)) { zero => + ColumnVector.fromScalar(zero, cv.getRowCount.toInt) + } + val intPart = withResource(zeroIntCv) { _ => + intAndDecParts.substring(zeroIntCv, start) + } + val decPart = closeOnExcept(intPart) { _ => + intAndDecParts.substring(start, end) + } + (intPart, decPart) + } + } + } + resourceArray += intPartPosExp + // if decLen - exp > d, convert to float/double, round, convert back to string + // decLen's max value is 9, abs(expPart)'s min value is 7, so it is possible only when d < 2 + // because d is small, we can use double to do the rounding + val decPartPosExp = if (0 < d && d < 2) { + val pointCv = closeOnExcept(decPartPosExpTemp) { _ => + withResource(Scalar.fromString(".")) { point => + ColumnVector.fromScalar(point, cv.getRowCount.toInt) + } + } + val withPoint = withResource(decPartPosExpTemp) { _ => + withResource(pointCv) { pointCv => + ColumnVector.stringConcatenate(Array(pointCv, decPartPosExpTemp)) + } + } + val decimalTypeRounding = DType.create(DType.DTypeEnum.DECIMAL128, -9) + val withPointDecimal = withResource(withPoint) { _ => + withResource(withPoint.castTo(decimalTypeRounding)) { decimal => + decimal.round(d, RoundMode.HALF_EVEN) + } + } + val roundedString = withResource(withPointDecimal) { _ => + withPointDecimal.castTo(DType.STRING) + } + withResource(roundedString) { _ => + withResource(roundedString.stringSplit(".", 2)) { splited => + splited.getColumn(1).incRefCount() + } + } + } else { + decPartPosExpTemp + } + (intPartPosExp, decPartPosExp) + } + } + + private def handleDoubleNegExp(cv: ColumnVector, intPart: ColumnVector, decPart: ColumnVector, + exp: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // handle cases when exp is negative + // "0." + (- exp - 1) * "0" + intPart + decPart + // if -1 - d <= exp and decLen - exp > d, need to rounding + val cond1 = withResource(Scalar.fromInt(-1 - d)) { negOneSubD => + exp.greaterOrEqualTo(negOneSubD) + } + val cond2 = closeOnExcept(cond1) { _ => + val decLenSubExp = withResource(decPart.getCharLengths) { decLen => + decLen.sub(exp) + } + withResource(decLenSubExp) { _ => + withResource(Scalar.fromInt(d)) { d => + decLenSubExp.greaterThan(d) + } + } + } + val needRounding = withResource(cond1) { _ => + withResource(cond2) { _ => + cond1.and(cond2) + } + } + val anyNeedRounding = withResource(needRounding) { _ => + withResource(needRounding.any()) { any => + any.isValid && any.getBoolean + } + } + anyNeedRounding match { + case false => + // a shortcut when no need to rounding + // "0." + (- exp - 1) * "0" + intPart + decPart + withResource(getZeroCv(cv.getRowCount.toInt)) { zeroCv => + val expSubOne = withResource(Scalar.fromInt(-1)) { negOne => + negOne.sub(exp) + } + val addingZeros = withResource(expSubOne) { _ => + zeroCv.repeatStrings(expSubOne) + } + val decPartNegExp = withResource(addingZeros) { _ => + ColumnVector.stringConcatenate(Array(addingZeros, intPart, decPart)) + } + val decPartNegSubstr = withResource(decPartNegExp) { _ => + decPartNegExp.substring(0, d) + } + (zeroCv.incRefCount(), decPartNegSubstr) + } + case true => + // if -exp <= d + 1 && -exp + decLen + 1 > d, need to rounding + // dec will be round to (d + exp + 1) digits + val dExpOne = withResource(Scalar.fromInt(d + 1)) { dExpOne => + exp.add(dExpOne) + } + // To do a dataframe operation, add some zeros before + // (intPat + decPart) and round them to 10 + // zerosNumRounding = (10 - (d + exp + 1)) . max(0) + val tenSubDExpOne = withResource(dExpOne) { _ => + withResource(Scalar.fromInt(10)) { ten => + ten.sub(dExpOne) + } + } + val zerosNumRounding = withResource(tenSubDExpOne) { _ => + withResource(Scalar.fromInt(0)) { zero => + withResource(tenSubDExpOne.lessThan(zero)) { lessThanZero => + lessThanZero.ifElse(zero, tenSubDExpOne) + } + } + } + val leadingZeros = withResource(zerosNumRounding) { _ => + withResource(getZeroCv(cv.getRowCount.toInt)) { zeroCv => + zeroCv.repeatStrings(zerosNumRounding) + } + } + val numberToRoundStr = withResource(leadingZeros) { _ => + val zeroPointCv = withResource(Scalar.fromString("0.")) { point => + ColumnVector.fromScalar(point, cv.getRowCount.toInt) + } + withResource(zeroPointCv) { _ => + ColumnVector.stringConcatenate(Array(zeroPointCv, leadingZeros, intPart, decPart)) + } + } + // use a decimal type to round, set scale to -20 to keep all digits + val decimalTypeRounding = DType.create(DType.DTypeEnum.DECIMAL128, -20) + val numberToRound = withResource(numberToRoundStr) { _ => + numberToRoundStr.castTo(decimalTypeRounding) + } + // rounding 10 digits + val rounded = withResource(numberToRound) { _ => + numberToRound.round(10, RoundMode.HALF_EVEN) + } + val roundedStr = withResource(rounded) { _ => + rounded.castTo(DType.STRING) + } + // substr 2 to remove "0." + val roundedDecPart = withResource(roundedStr) { _ => + roundedStr.substring(2) + } + val decPartStriped = withResource(roundedDecPart) { _ => + withResource(Scalar.fromString("0")) { zero => + roundedDecPart.lstrip(zero) + } + } + val decPartNegExp = withResource(decPartStriped) { _ => + decPartStriped.pad(d, PadSide.LEFT, "0") + } + closeOnExcept(decPartNegExp) { _ => + (getZeroCv(cv.getRowCount.toInt), decPartNegExp) + } + } + } + + private def normalDoubleSplit(cv: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + val roundingScale = d.min(10) // cuDF will keep at most 9 digits after decimal point + val roundedStr = withResource(cv.round(roundingScale, RoundMode.HALF_EVEN)) { rounded => + rounded.castTo(DType.STRING) + } + val (intPart, decPart) = withResource(roundedStr) { _ => + withResource(roundedStr.stringSplit(".", 2)) { intAndDec => + (intAndDec.getColumn(0).incRefCount(), intAndDec.getColumn(1).incRefCount()) + } + } + val intPartNoNeg = closeOnExcept(decPart) { _ => + withResource(intPart) { _ => + removeNegSign(intPart) + } + } + val decPartPad = closeOnExcept(intPartNoNeg) { _ => + withResource(decPart) { _ => + decPart.pad(d, PadSide.RIGHT, "0") + } + } + // a workaround for cuDF float to string, e.g. 12.3 => "12.30000019" instead of "12.3" + val decPartSubstr = closeOnExcept(intPartNoNeg) { _ => + withResource(decPartPad) { _ => + decPartPad.substring(0, d) + } + } + (intPartNoNeg, decPartSubstr) + } + + private def expDoubleSplit(cv: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // handle special case: 1.234567e+7 or 1.234567e-6 + // get three parts first: + val replaceDelimToE = withResource(Scalar.fromString("e")) { e => + withResource(Scalar.fromString(".")) { p => + cv.stringReplace(e, p) + } + } + // get three parts: 1.234567e+7 -> 1, 234567, +7 + val (intPartSign, decPart, expPart) = withResource(replaceDelimToE) { _ => + withResource(replaceDelimToE.stringSplit(".", 3)) { intDecExp => + (intDecExp.getColumn(0).incRefCount(), + intDecExp.getColumn(1).incRefCount(), + intDecExp.getColumn(2).incRefCount()) + } + } + // sign will be handled later, use string-based solution instead abs to avoid overfolw + val intPart = closeOnExcept(decPart) { _ => + closeOnExcept(expPart) { _ => + withResource(intPartSign) { _ => + removeNegSign(intPartSign) + } + } + } + val exp = closeOnExcept(decPart) { _ => + closeOnExcept(intPart) { _ => + withResource(expPart) { _ => + expPart.castTo(DType.INT32) + } + } + } + // handle positive and negative exp separately + val (intPartPosExp, decPartPosExp) = closeOnExcept(intPart) { _ => + closeOnExcept(decPart) { _ => + closeOnExcept(exp) { _ => + handleDoublePosExp(cv, intPart, decPart, exp, d) + } + } + } + withResource(ArrayBuffer.empty[ColumnVector]) { resourceArray => + val (intPartNegExp, decPartNegExp) = withResource(intPart) { _ => + withResource(decPart) { _ => + closeOnExcept(exp) { _ => + handleDoubleNegExp(cv, intPart, decPart, exp, d) + } + } + } + resourceArray += intPartNegExp + resourceArray += decPartNegExp + val expPos = withResource(exp) { _ => + withResource(Scalar.fromInt(0)) { zero => + exp.greaterOrEqualTo(zero) + } + } + // combine results + withResource(expPos) { _ => + val intPartExp = withResource(intPartPosExp) { _ => + expPos.ifElse(intPartPosExp, intPartNegExp) + } + val decPartExp = closeOnExcept(intPartExp) { _ => + withResource(decPartPosExp) { _ => + expPos.ifElse(decPartPosExp, decPartNegExp) + } + } + (intPartExp, decPartExp) + } + } + } + + private def getPartsFromDouble(cv: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // handle normal case: 1234.567 + closeOnExcept(ArrayBuffer.empty[ColumnVector]) { resourceArray => + val (normalInt, normalDec) = normalDoubleSplit(cv, d) + resourceArray += normalInt + resourceArray += normalDec + // first check special case + val cvStr = withResource(cv.castTo(DType.STRING)) { cvStr => + cvStr.incRefCount() + } + val containsE = closeOnExcept(cvStr) { _ => + withResource(Scalar.fromString("e")) { e => + cvStr.stringContains(e) + } + } + withResource(containsE) { _ => + // if no special case, return normal case directly + val anyExp = closeOnExcept(cvStr) { _ => + withResource(containsE.any()) { any => + any.isValid && any.getBoolean + } + } + anyExp match { + case false => { + cvStr.safeClose() + (normalInt, normalDec) + } + case true => { + val noEReplaced = withResource(cvStr) { _ => + // replace normal case with 0e0 to avoid error + withResource(Scalar.fromString("0.0e0")) { default => + containsE.ifElse(cvStr, default) + } + } + // handle scientific notation case: + val (expInt, expDec) = withResource(noEReplaced) { _ => + expDoubleSplit(noEReplaced, d) + } + // combine results + // remove normalInt from resourceArray + resourceArray.remove(0) + val intPart = closeOnExcept(expDec) { _ => + withResource(expInt) { _ => + withResource(normalInt) { _ => + containsE.ifElse(expInt, normalInt) + } + } + } + resourceArray.clear() + resourceArray += intPart + val decPart = withResource(expDec) { _ => + withResource(normalDec) { _ => + containsE.ifElse(expDec, normalDec) + } + } + (intPart, decPart) + } + } + } + } + } + + private def getPartsFromDecimal(cv: ColumnVector, d: Int, scale: Int): + (ColumnVector, ColumnVector) = { + // prevent d too large to fit in decimalType + val roundingScale = scale.min(d) + // append zeros to the end of decPart, zerosNum = d - scale + // if d <= scale, no need to append zeros, if scale < 0, append d zeros + val appendZeroNum = (d - scale).max(0).min(d) + val (intPart, decTemp) = if (roundingScale <= 0) { + withResource(ArrayBuffer.empty[ColumnVector]) { resourceArray => + val intPart = withResource(cv.round(roundingScale, RoundMode.HALF_EVEN)) { rounded => + rounded.castTo(DType.STRING) + } + resourceArray += intPart + // if intString starts with 0, it must be "00000...", replace it with "0" + val (isZero, zeroCv) = withResource(Scalar.fromString("0")) { zero => + withResource(intPart.startsWith(zero)) { isZero => + (isZero.incRefCount(), ColumnVector.fromScalar(zero, cv.getRowCount.toInt)) + } + } + val intPartZeroHandled = withResource(isZero) { isZero => + withResource(zeroCv) { zeroCv => + isZero.ifElse(zeroCv, intPart) + } + } + resourceArray += intPartZeroHandled + // a temp decPart is empty before appending zeros + val decPart = withResource(Scalar.fromString("")) { emptyString => + ColumnVector.fromScalar(emptyString, cv.getRowCount.toInt) + } + resourceArray += decPart + (intPartZeroHandled.incRefCount(), decPart.incRefCount()) + } + } else { + withResource(cv.round(roundingScale, RoundMode.HALF_EVEN)) { rounded => + withResource(rounded.castTo(DType.STRING)) { roundedStr => + withResource(roundedStr.stringSplit(".", 2)) { intAndDec => + (intAndDec.getColumn(0).incRefCount(), intAndDec.getColumn(1).incRefCount()) + } + } + } + } + closeOnExcept(ArrayBuffer.empty[ColumnVector]) { resourceArray => + // remove negative sign from intPart, sign will be handled later + val intPartPos = closeOnExcept(decTemp) { _ => + withResource(intPart) { _ => + removeNegSign(intPart) + } + } + resourceArray += intPartPos + // append zeros + val appendZeros = "0" * appendZeroNum + val appendZerosCv = closeOnExcept(decTemp) { _ => + withResource(Scalar.fromString(appendZeros)) { zeroString => + ColumnVector.fromScalar(zeroString, cv.getRowCount.toInt) + } + } + val decPart = withResource(decTemp) { _ => + withResource(appendZerosCv) { _ => + ColumnVector.stringConcatenate(Array(decTemp, appendZerosCv)) + } + } + (intPartPos, decPart) + } + } + + private def getParts(cv: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // get int part and dec part from a column vector, int part will be set to positive + x.dataType match { + case FloatType | DoubleType => { + getPartsFromDouble(cv, d) + } + case DecimalType.Fixed(_, scale) => { + getPartsFromDecimal(cv, d, scale) + } + case IntegerType | LongType | ShortType | ByteType => { + val intPartPos = withResource(cv.castTo(DType.STRING)) { intPart => + removeNegSign(intPart) + } + // dec part is all zeros + val dzeros = "0" * d + val decPart = closeOnExcept(intPartPos) { _ => + withResource(Scalar.fromString(dzeros)) { zeroString => + ColumnVector.fromScalar(zeroString, cv.getRowCount.toInt) + } + } + (intPartPos, decPart) + } + case _ => { + throw new UnsupportedOperationException(s"format_number doesn't support type ${x.dataType}") + } + } + } + + private def negativeCheck(cv: ColumnVector): ColumnVector = { + withResource(cv.castTo(DType.STRING)) { cvStr => + withResource(Scalar.fromString("-")) { negativeSign => + cvStr.startsWith(negativeSign) + } + } + } + + private def removeExtraCommas(str: ColumnVector): ColumnVector = { + withResource(Scalar.fromString(",")) { comma => + str.rstrip(comma) + } + } + + private def addCommas(str: ColumnVector): ColumnVector = { + val maxstrlen = withResource(str.getCharLengths()) { strlen => + withResource(strlen.max()) { maxlen => + maxlen.isValid match { + case true => maxlen.getInt + case false => 0 + } + } + } + val sepCol = withResource(Scalar.fromString(",")) { sep => + ColumnVector.fromScalar(sep, str.getRowCount.toInt) + } + val substrs = closeOnExcept(sepCol) { _ => + (0 until maxstrlen by 3).safeMap { i => + str.substring(i, i + 3).asInstanceOf[ColumnView] + }.toArray + } + withResource(substrs) { _ => + withResource(sepCol) { _ => + withResource(ColumnVector.stringConcatenate(substrs, sepCol)) { res => + removeExtraCommas(res) + } + } + } + } + + private def handleInfAndNan(cv: ColumnVector, res: ColumnVector): ColumnVector = { + // replace inf and nan with infSymbol and nanSymbol in res according to cv + val symbols = DecimalFormatSymbols.getInstance(Locale.US) + val nanSymbol = symbols.getNaN + val infSymbol = symbols.getInfinity + val negInfSymbol = "-" + infSymbol + val handleNan = withResource(cv.isNan()) { isNan => + withResource(Scalar.fromString(nanSymbol)) { nan => + isNan.ifElse(nan, res) + } + } + val isInf = closeOnExcept(handleNan) { _ => + x.dataType match { + case DoubleType => { + withResource(Scalar.fromDouble(Double.PositiveInfinity)) { inf => + cv.equalTo(inf) + } + } + case FloatType => { + withResource(Scalar.fromFloat(Float.PositiveInfinity)) { inf => + cv.equalTo(inf) + } + } + } + } + val handleInf = withResource(isInf) { _ => + withResource(handleNan) { _ => + withResource(Scalar.fromString(infSymbol)) { inf => + isInf.ifElse(inf, handleNan) + } + } + } + val isNegInf = closeOnExcept(handleInf) { _ => + x.dataType match { + case DoubleType => { + withResource(Scalar.fromDouble(Double.NegativeInfinity)) { negInf => + cv.equalTo(negInf) + } + } + case FloatType => { + withResource(Scalar.fromFloat(Float.NegativeInfinity)) { negInf => + cv.equalTo(negInf) + } + } + } + } + val handleNegInf = withResource(isNegInf) { _ => + withResource(Scalar.fromString(negInfSymbol)) { negInf => + withResource(handleInf) { _ => + isNegInf.ifElse(negInf, handleInf) + } + } + } + handleNegInf + } + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { + // get int d from rhs + if (!rhs.isValid || rhs.getValue.asInstanceOf[Int] < 0) { + return GpuColumnVector.columnVectorFromNull(lhs.getRowCount.toInt, StringType) + } + val d = rhs.getValue.asInstanceOf[Int] + val (integerPart, decimalPart) = getParts(lhs.getBase, d) + // reverse integer part for adding commas + val resWithDecimalPart = withResource(decimalPart) { _ => + val reversedIntegerPart = withResource(integerPart) { intPart => + intPart.reverseStringsOrLists() + } + val reversedIntegerPartWithCommas = withResource(reversedIntegerPart) { _ => + addCommas(reversedIntegerPart) + } + // reverse result back + val reverseBack = withResource(reversedIntegerPartWithCommas) { r => + r.reverseStringsOrLists() + } + d match { + case 0 => { + // d == 0, only return integer part + reverseBack + } + case _ => { + // d > 0, append decimal part to result + withResource(reverseBack) { _ => + withResource(Scalar.fromString(".")) { point => + withResource(Scalar.fromString("")) { empty => + ColumnVector.stringConcatenate(point, empty, Array(reverseBack, decimalPart)) + } + } + } + } + } + } + // add negative sign back + val negCv = withResource(Scalar.fromString("-")) { negativeSign => + ColumnVector.fromScalar(negativeSign, lhs.getRowCount.toInt) + } + val formated = withResource(resWithDecimalPart) { _ => + val resWithNeg = withResource(negCv) { _ => + ColumnVector.stringConcatenate(Array(negCv, resWithDecimalPart)) + } + withResource(negativeCheck(lhs.getBase)) { isNegative => + withResource(resWithNeg) { _ => + isNegative.ifElse(resWithNeg, resWithDecimalPart) + } + } + } + // handle null case + val anyNull = closeOnExcept(formated) { _ => + lhs.getBase.getNullCount > 0 + } + val formatedWithNull = anyNull match { + case true => { + withResource(formated) { _ => + withResource(lhs.getBase.isNull) { isNull => + withResource(Scalar.fromNull(DType.STRING)) { nullScalar => + isNull.ifElse(nullScalar, formated) + } + } + } + } + case false => formated + } + // handle inf and nan + x.dataType match { + case FloatType | DoubleType => { + withResource(formatedWithNull) { _ => + handleInfAndNan(lhs.getBase, formatedWithNull) + } + } + case _ => formatedWithNull + } + } + + override def doColumnar(lhs: GpuScalar, rhs: GpuColumnVector): ColumnVector = { + throw new UnsupportedOperationException() + } + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuColumnVector): ColumnVector = { + throw new UnsupportedOperationException() + } + + override def doColumnar(numRows: Int, lhs: GpuScalar, rhs: GpuScalar): ColumnVector = { + withResource(GpuColumnVector.from(lhs, numRows, dataType)) { col => + doColumnar(col, rhs) + } + } +} diff --git a/sql-plugin/src/main/scala/org/apache/spark/util/random/rapids/XORShiftRandom.scala b/sql-plugin/src/main/scala/org/apache/spark/util/random/rapids/XORShiftRandom.scala new file mode 100644 index 00000000000..1f32b56a6cd --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/util/random/rapids/XORShiftRandom.scala @@ -0,0 +1,46 @@ +/* + * Copyright (c) 2023, 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.util.random.rapids + +import org.apache.spark.util.random.XORShiftRandom + +/** RAPIDS version of the Spark XORShiftRandom providing access to the internal seed. */ +class RapidsXORShiftRandom(init: Long) extends XORShiftRandom(init) { + + private var curSeed = XORShiftRandom.hashSeed(init) + + // Only override "next", since it will be called by other nextXXX. + override protected def next(bits: Int): Int = { + var nextSeed = curSeed ^ (curSeed << 21) + nextSeed ^= (nextSeed >>> 35) + nextSeed ^= (nextSeed << 4) + curSeed = nextSeed + (nextSeed & ((1L << bits) - 1)).asInstanceOf[Int] + } + + override def setSeed(s: Long): Unit = { + curSeed = XORShiftRandom.hashSeed(s) + } + + /* Set the hashed seed directly. (Not thread-safe) */ + def setHashedSeed(hashedSeed: Long): Unit = { + curSeed = hashedSeed + } + + /* Get the current internal seed. (Not thread-safe) */ + def currentSeed: Long = curSeed +} diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index a8f054e60c2..ae510d803c5 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -67,15 +67,15 @@ final class CreateDataSourceTableAsSelectCommandMeta( cmd.table.provider.get, spark.sessionState.conf) // Note that the data source V2 always fallsback to the V1 currently. // If that changes then this will start failing because we don't have a mapping. - gpuProvider = origProvider.getConstructor().newInstance() match { - case f: FileFormat if GpuOrcFileFormat.isSparkOrcFormat(f) => - GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.table.storage.properties, cmd.query.schema) - case _: ParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(this, spark, - cmd.table.storage.properties, cmd.query.schema) - case ds => - willNotWorkOnGpu(s"Data source class not supported: ${ds}") - None + gpuProvider = if (classOf[FileFormat].isAssignableFrom(origProvider) && + GpuOrcFileFormat.isSparkOrcFormat(origProvider.asInstanceOf[Class[_ <: FileFormat]])) { + GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.table.storage.properties, cmd.query.schema) + } else if (origProvider == classOf[ParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(this, spark, + cmd.table.storage.properties, cmd.query.schema) + } else { + willNotWorkOnGpu(s"Data source class not supported: $origProvider") + None } } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 82ec332f283..e7d5d452abb 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -21,7 +21,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -33,7 +33,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { + @transient scan: GpuScan) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @transient lazy val batch: Batch = scan.toBatch @transient override lazy val partitions: Seq[InputPartition] = batch.planInputPartitions() @@ -41,11 +41,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics new GpuDataSourceRDD(sparkContext, partitions, readerFactory) } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala index 3de41de76c2..ffff1d9e2c1 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala @@ -38,7 +38,7 @@ import ai.rapids.cudf.{ColumnVector, ColumnView} import com.nvidia.spark.rapids.jni.CastStrings object GpuCastShims { - def CastDecimalToString(decimalInput: ColumnView, ansiMode: Boolean): ColumnVector = { + def CastDecimalToString(decimalInput: ColumnView, usePlainString: Boolean): ColumnVector = { CastStrings.fromDecimal(decimalInput) } } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala index 3368dcd6e46..340dce31388 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -21,7 +21,6 @@ {"spark": "320"} {"spark": "321"} {"spark": "321cdh"} -{"spark": "321db"} {"spark": "322"} {"spark": "323"} {"spark": "324"} @@ -39,5 +38,5 @@ object ParquetFieldIdShims { } /** Parquet field ID support configs are not supported until Spark 3.3 */ - def getParquetIdWriteEnabled(sqlConf: SQLConf): Boolean = false + def getParquetIdWriteEnabled(conf: Configuration, sqlConf: SQLConf): Boolean = false } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index b7e8c758622..34bd8d7cc4f 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -21,9 +21,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan class RapidsOrcScanMeta( @@ -37,7 +36,7 @@ class RapidsOrcScanMeta( GpuOrcScan.tagSupport(this) } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuOrcScan(oScan.sparkSession, oScan.hadoopConf, oScan.fileIndex, diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index 7c3f57574de..42a905c9c45 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -21,9 +21,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan class RapidsParquetScanMeta( @@ -37,7 +36,7 @@ class RapidsParquetScanMeta( GpuParquetScan.tagSupport(this) } - override def convertToGpu(): Scan = { + override def convertToGpu(): GpuScan = { GpuParquetScan(pScan.sparkSession, pScan.hadoopConf, pScan.fileIndex, diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 79e3758d543..c1cac965df5 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -26,7 +26,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -42,7 +42,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression] = Seq.empty) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @transient lazy val batch: Batch = scan.toBatch @@ -97,11 +97,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala index fcac077990c..9b5c7c3fdda 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala @@ -35,9 +35,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuCSVScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuCSVScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan class RapidsCsvScanMeta( @@ -53,7 +52,7 @@ class RapidsCsvScanMeta( TagScanForRuntimeFiltering.tagScanForRuntimeFiltering(this, cScan) } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuCSVScan(cScan.sparkSession, cScan.fileIndex, cScan.dataSchema, diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index f9dfe5ef158..a713529cb83 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -25,9 +25,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan class RapidsOrcScanMeta( @@ -43,7 +42,7 @@ class RapidsOrcScanMeta( TagScanForRuntimeFiltering.tagScanForRuntimeFiltering(this, oScan) } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuOrcScan(oScan.sparkSession, oScan.hadoopConf, oScan.fileIndex, diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index 536fb420b97..5f5c8efd46d 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -25,9 +25,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan class RapidsParquetScanMeta( @@ -43,7 +42,7 @@ class RapidsParquetScanMeta( TagScanForRuntimeFiltering.tagScanForRuntimeFiltering(this, pScan) } - override def convertToGpu(): Scan = { + override def convertToGpu(): GpuScan = { GpuParquetScan(pScan.sparkSession, pScan.hadoopConf, pScan.fileIndex, diff --git a/sql-plugin/src/main/spark321db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark321db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala new file mode 100644 index 00000000000..7fffdfb4a9f --- /dev/null +++ b/sql-plugin/src/main/spark321db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -0,0 +1,44 @@ +/* + * Copyright (c) 2023, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.sql.internal.SQLConf + +object ParquetFieldIdShims { + val fieldIdOverrideKey: String = "spark.rapids.sql.parquet.writeFieldIds" + + /** Updates the Hadoop configuration with the Parquet field ID write setting from SQLConf */ + def setupParquetFieldIdWriteConfig(conf: Configuration, sqlConf: SQLConf): Unit = { + // No SQL conf for this config in Spark 3.2.x + } + + /** Get Parquet field ID write enabled configuration value */ + def getParquetIdWriteEnabled(conf: Configuration, sqlConf: SQLConf): Boolean = { + // No SQL conf for this config in Spark 3.2.x + conf.get(fieldIdOverrideKey, "false").toBoolean + } + + /** Set the Parquet field ID write enable override */ + def setWriteIdOverride(conf: Configuration, enabled: Boolean): Unit = { + conf.set(fieldIdOverrideKey, enabled.toString) + } +} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index fce9532168a..c3ab21954e6 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -24,7 +24,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -40,7 +40,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression] = Seq.empty, keyGroupedPartitioning: Option[Seq[Expression]] = None) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @@ -116,11 +116,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala index 22991261f76..1d6e06985a6 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -17,7 +17,6 @@ /*** spark-rapids-shim-json-lines {"spark": "330"} {"spark": "330cdh"} -{"spark": "330db"} {"spark": "331"} {"spark": "332"} {"spark": "332db"} @@ -41,5 +40,7 @@ object ParquetFieldIdShims { } /** Get Parquet field ID write enabled configuration value */ - def getParquetIdWriteEnabled(sqlConf: SQLConf): Boolean = sqlConf.parquetFieldIdWriteEnabled + def getParquetIdWriteEnabled(conf: Configuration, sqlConf: SQLConf): Boolean = { + sqlConf.parquetFieldIdWriteEnabled + } } diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index e78338eb72c..319df49a17a 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -28,9 +28,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan class RapidsOrcScanMeta( @@ -50,7 +49,7 @@ class RapidsOrcScanMeta( } } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuOrcScan(oScan.sparkSession, oScan.hadoopConf, oScan.fileIndex, diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index 464a4dcb73e..b9322f15b30 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -28,9 +28,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan class RapidsParquetScanMeta( @@ -51,7 +50,7 @@ class RapidsParquetScanMeta( } } - override def convertToGpu(): Scan = { + override def convertToGpu(): GpuScan = { GpuParquetScan(pScan.sparkSession, pScan.hadoopConf, pScan.fileIndex, diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index cd2b738e927..8fbb493ddb3 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -20,7 +20,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -37,7 +37,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression], keyGroupedPartitioning: Option[Seq[Expression]], ordering: Option[Seq[SortOrder]], @@ -116,11 +116,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala index 60fdd1e5e72..2493d81d9de 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala @@ -38,8 +38,8 @@ object GpuCastShims { case _ => throw new IllegalArgumentException(s"Unsupported type in cast $t") } - def CastDecimalToString(decimalInput: ColumnView, ansiMode: Boolean): ColumnVector = { - if (ansiMode) { + def CastDecimalToString(decimalInput: ColumnView, usePlainString: Boolean): ColumnVector = { + if (usePlainString) { // This is equivalent to // https://docs.oracle.com/javase/8/docs/api/java/math/BigDecimal.html#toPlainString-- // except there are a few corner cases, but they are really rare diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala new file mode 100644 index 00000000000..0b896714903 --- /dev/null +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -0,0 +1,45 @@ +/* + * Copyright (c) 2023, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "330db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.sql.internal.SQLConf + +object ParquetFieldIdShims { + private val fieldIdOverrideKey: String = "spark.rapids.sql.parquet.writeFieldIds" + + /** Updates the Hadoop configuration with the Parquet field ID write setting from SQLConf */ + def setupParquetFieldIdWriteConfig(conf: Configuration, sqlConf: SQLConf): Unit = { + conf.set( + SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED.key, + sqlConf.parquetFieldIdWriteEnabled.toString) + } + + /** Get Parquet field ID write enabled configuration value */ + def getParquetIdWriteEnabled(conf: Configuration, sqlConf: SQLConf): Boolean = { + conf.get(fieldIdOverrideKey, "false").toBoolean || sqlConf.parquetFieldIdWriteEnabled + } + + /** Set the Parquet field ID write enable override */ + def setWriteIdOverride(conf: Configuration, enabled: Boolean): Unit = { + conf.set(fieldIdOverrideKey, enabled.toString) + } +} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala index ce978508826..2a4ec2cc902 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -95,11 +95,11 @@ trait GpuAddSub extends CudfBinaryArithmetic { } else { // eval operands using the output precision val castLhs = withResource(left.columnarEval(batch)) { lhs => - GpuCast.doCast(lhs.getBase(), leftInputType, resultType, false, false, false) + GpuCast.doCast(lhs.getBase(), leftInputType, resultType) } val castRhs = closeOnExcept(castLhs){ _ => withResource(right.columnarEval(batch)) { rhs => - GpuCast.doCast(rhs.getBase(), rightInputType, resultType, false, false, false) + GpuCast.doCast(rhs.getBase(), rightInputType, resultType) } } @@ -342,14 +342,14 @@ case class GpuDecimalRemainder(left: Expression, right: Expression) private def regularRemainder(batch: ColumnarBatch): GpuColumnVector = { val castLhs = withResource(left.columnarEval(batch)) { lhs => - GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } withResource(castLhs) { castLhs => val castRhs = withResource(right.columnarEval(batch)) { rhs => withResource(divByZeroFixes(rhs.getBase)) { fixed => - GpuCast.doCast(fixed, rhs.dataType(), intermediateRhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast(fixed, rhs.dataType(), intermediateRhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } } withResource(castRhs) { castRhs => diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala index a00861b8762..ca297369328 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala @@ -131,7 +131,7 @@ case class GpuBroadcastHashJoinExec( val targetSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf) val metricsMap = allMetrics - val bufferedStreamIter = new CloseableBufferedIterator(streamIter.buffered) + val bufferedStreamIter = new CloseableBufferedIterator(streamIter) closeOnExcept(bufferedStreamIter) { _ => withResource(new NvtxRange("first stream batch", NvtxColor.RED)) { _ => if (bufferedStreamIter.hasNext) { diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index c553f0dada8..8b060743d22 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -52,15 +52,14 @@ final class CreateDataSourceTableAsSelectCommandMeta( origProvider = GpuDataSourceBase.lookupDataSourceWithFallback(cmd.table.provider.get, spark.sessionState.conf) - origProvider.getConstructor().newInstance() match { - case f: FileFormat if GpuOrcFileFormat.isSparkOrcFormat(f) => - GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.table.storage.properties, cmd.query.schema) - case _: ParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(this, spark, - cmd.table.storage.properties, cmd.query.schema) - case ds => - willNotWorkOnGpu(s"Data source class not supported: ${ds}") - None + if (classOf[FileFormat].isAssignableFrom(origProvider) && + GpuOrcFileFormat.isSparkOrcFormat(origProvider.asInstanceOf[Class[_ <: FileFormat]])) { + GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.table.storage.properties, cmd.query.schema) + } else if (origProvider == classOf[ParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(this, spark, + cmd.table.storage.properties, cmd.query.schema) + } else { + willNotWorkOnGpu(s"Data source class not supported: $origProvider") } } diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index b4dc5d8ef17..3cbdd49a47c 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -20,7 +20,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -37,7 +37,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression], keyGroupedPartitioning: Option[Seq[Expression]], ordering: Option[Seq[SortOrder]], @@ -116,11 +116,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 237d375e593..4b7a984144e 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -22,7 +22,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -40,7 +40,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression] = Seq.empty, keyGroupedPartitioning: Option[Seq[Expression]] = None, ordering: Option[Seq[SortOrder]] = None, @@ -140,10 +140,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } + scan.metrics = allMetrics val rdd = if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) @@ -205,7 +202,12 @@ case class GpuBatchScanExec( val partitionMapping = groupedPartitions.map { case (row, parts) => InternalRowComparableWrapper(row, p.expressions) -> parts }.toMap - finalPartitions = p.partitionValues.map { partValue => + + // In case `commonPartitionValues` is not defined (e.g., SPJ is not used), there + // could exist duplicated partition values, as partition grouping is not done + // at the beginning and postponed to this method. It is important to use unique + // partition values here so that grouped partitions won't get duplicated. + finalPartitions = KeyGroupedPartitioningShim.getUniquePartitions(p).map { partValue => // Use empty partition for those partition values that are not present partitionMapping.getOrElse( InternalRowComparableWrapper(partValue, p.expressions), Seq.empty) diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala new file mode 100644 index 00000000000..53644a02804 --- /dev/null +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2023, 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "340"} +{"spark": "341"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning +import org.apache.spark.sql.catalyst.util.InternalRowComparableWrapper + +object KeyGroupedPartitioningShim { + def getUniquePartitions(p: KeyGroupedPartitioning): Seq[InternalRow] = { + p.partitionValues + .map(InternalRowComparableWrapper(_, p.expressions)) + .distinct + .map(_.row) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala new file mode 100644 index 00000000000..521e2e326b8 --- /dev/null +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala @@ -0,0 +1,67 @@ +/* + * Copyright (c) 2023, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.Scalar +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.Arm._ +import com.nvidia.spark.rapids.RapidsPluginImplicits._ + +import org.apache.spark.sql.catalyst.expressions.{Expression, TimeZoneAwareExpression} +import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.vectorized.ColumnarBatch + +case class GpuToPrettyString(child: Expression, timeZoneId: Option[String] = None) + extends ShimUnaryExpression with GpuExpression with TimeZoneAwareExpression { + + override lazy val resolved: Boolean = childrenResolved + + override def dataType: DataType = StringType + + override def nullable: Boolean = false + + override def withTimeZone(timeZoneId: String): GpuToPrettyString = + copy(timeZoneId = Some(timeZoneId)) + + override def withNewChildInternal(newChild: Expression): Expression = + copy(child = newChild) + + override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { + withResource(child.columnarEval(batch)) { evaluatedCol => + withResource(GpuCast.doCast( + evaluatedCol.getBase, + evaluatedCol.dataType(), + StringType, + CastOptions.TO_PRETTY_STRING_OPTIONS)) { possibleStringResult => + if (possibleStringResult.hasNulls) { + withResource(possibleStringResult.isNull) { isNull => + val stringColWithNulls = possibleStringResult + withResource(Scalar.fromString(CastOptions.TO_PRETTY_STRING_OPTIONS.nullString)) { + nullString => + GpuColumnVector.from(isNull.ifElse(nullString, stringColWithNulls), StringType) + } + } + } else { + GpuColumnVector.from(possibleStringResult.incRefCount(), StringType) + } + } + } + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala new file mode 100644 index 00000000000..af88498cd34 --- /dev/null +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala @@ -0,0 +1,28 @@ +/* + * Copyright (c) 2023, 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "350"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning + +object KeyGroupedPartitioningShim { + def getUniquePartitions(p: KeyGroupedPartitioning): Seq[InternalRow] = { + p.uniquePartitionValues + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala index 5390f14122f..1151f8fd869 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala @@ -38,7 +38,7 @@ object PythonMapInArrowExecShims { ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), TypeSig.all), (mapPy, conf, p, r) => new GpuPythonMapInArrowExecMeta(mapPy, conf, p, r) { - override def tagPlanForGpu() { + override def tagPlanForGpu(): Unit = { super.tagPlanForGpu() if (SQLConf.get.getConf(SQLConf.ARROW_EXECUTION_USE_LARGE_VAR_TYPES)) { 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 0f4a4bf66d5..75a13143a94 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 @@ -21,13 +21,27 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ -import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDAF} +import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDAF, ToPrettyString} import org.apache.spark.sql.rapids.execution.python.GpuPythonUDAF +import org.apache.spark.sql.types.StringType object SparkShimImpl extends Spark340PlusShims { override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { val shimExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( + GpuOverrides.expr[ToPrettyString]("An internal expressions which is used to " + + "generate pretty string for all kinds of values", + new ToPrettyStringChecks(), + (toPrettyString, conf, p, r) => { + new CastExprMetaBase[ToPrettyString](toPrettyString, conf, p, r) { + + override val toType: StringType.type = StringType + + override def convertToGpu(child: Expression): GpuExpression = { + GpuToPrettyString(child) + } + } + }), GpuOverrides.expr[PythonUDAF]( "UDF run in an external python process. Does not actually run on the GPU, but " + "the transfer of data to/from it can be accelerated", diff --git a/tests/pom.xml b/tests/pom.xml index df74c595db3..dfa91fbdaa9 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -61,7 +61,7 @@ com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala index 04442a4698c..85c5806b47e 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala @@ -632,7 +632,7 @@ class AdaptiveQueryExecSuite numLocalReaders.length } - def skewJoinTest(fun: SparkSession => Unit) { + def skewJoinTest(fun: SparkSession => Unit): Unit = { val conf = new SparkConf() .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") @@ -699,7 +699,7 @@ class AdaptiveQueryExecSuite } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def testData(spark: SparkSession) { + private def testData(spark: SparkSession): Unit = { import spark.implicits._ val data: Seq[(Int, String)] = (1 to 100).map(i => (i, i.toString)) val df = data.toDF("key", "value") @@ -707,7 +707,7 @@ class AdaptiveQueryExecSuite registerAsParquetTable(spark, df, "testData") } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def testData2(spark: SparkSession) { + private def testData2(spark: SparkSession): Unit = { import spark.implicits._ val df = Seq[(Int, Int)]((1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2)) .toDF("a", "b") @@ -716,7 +716,7 @@ class AdaptiveQueryExecSuite } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def testData3(spark: SparkSession) { + private def testData3(spark: SparkSession): Unit = { import spark.implicits._ val df = Seq[(Int, Option[Int])]((1, None), (2, Some(2))) .toDF("a", "b") @@ -725,7 +725,7 @@ class AdaptiveQueryExecSuite } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def uncommonTypeTestData(spark: SparkSession) { + private def uncommonTypeTestData(spark: SparkSession): Unit = { import scala.collection.JavaConverters._ val df = spark.createDataFrame( List.tabulate(20)(i => Row(i % 3, BigDecimal(i), Array(i, i), Row(i))).asJava, @@ -740,7 +740,7 @@ class AdaptiveQueryExecSuite } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def lowerCaseData(spark: SparkSession) { + private def lowerCaseData(spark: SparkSession): Unit = { import spark.implicits._ // note that this differs from the original Spark test by generating a larger data set so that // we can trigger larger stats in the logical mode, preventing BHJ, and then our queries filter @@ -753,7 +753,7 @@ class AdaptiveQueryExecSuite registerAsParquetTable(spark, df, "lowercaseData") } - private def registerAsParquetTable(spark: SparkSession, df: Dataset[Row], name: String) { + private def registerAsParquetTable(spark: SparkSession, df: Dataset[Row], name: String): Unit = { val path = new File(TEST_FILES_ROOT, s"$name.parquet").getAbsolutePath df.write .mode(SaveMode.Overwrite) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ApproximatePercentileSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ApproximatePercentileSuite.scala index 8f3dc2305ce..7e0a4c5a3f2 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ApproximatePercentileSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ApproximatePercentileSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -102,7 +102,7 @@ class ApproximatePercentileSuite extends SparkQueryCompareTestSuite { df.sparkSession.sql("SELECT approx_percentile(salary, array(0.5)) FROM salaries") } - def sqlFallbackTest(sql: String) { + def sqlFallbackTest(sql: String): Unit = { val conf = new SparkConf() .set("spark.rapids.sql.incompatibleOps.enabled", "true") @@ -126,7 +126,7 @@ class ApproximatePercentileSuite extends SparkQueryCompareTestSuite { private def doTest( func: SparkSession => DataFrame, percentileArg: Either[Double, Array[Double]] = Right(DEFAULT_PERCENTILES), - delta: Option[Int]) { + delta: Option[Int]): Unit = { val percentiles = withCpuSparkSession { spark => calcPercentiles(spark, func, percentileArg, delta, diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala index 35d7f990798..0b531adabb7 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala @@ -31,7 +31,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) assertResult(true)(handler.onAllocFailure(1024, 0)) } @@ -45,7 +44,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) assertResult(true)(handler.onAllocFailure(1024, 0)) // sync assertResult(true)(handler.onAllocFailure(1024, 1)) // sync 2 @@ -61,7 +59,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) // with this call we sync, and we mark our attempts at 1, we store 0 as the last count assertResult(true)(handler.onAllocFailure(1024, 0)) @@ -80,7 +77,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) assertThrows[IllegalArgumentException](handler.onAllocFailure(-1, 0)) } @@ -94,7 +90,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) assertThrows[IllegalArgumentException](handler.onAllocFailure(1024, -1)) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala index 2f26d7efa50..2080d79b3ca 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala @@ -41,25 +41,31 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite } } + private def getAndResetNumRetryThrowCurrentTask: Int = { + // taskId 1 was associated with the current thread in RmmSparkRetrySuiteBase + RmmSpark.getAndResetNumRetryThrow(/*taskId*/ 1) + } + test("a retry when copying to device is handled") { val batch = buildBatch() - val ctriter = new ColumnarToRowIterator( - Seq(batch).iterator, - NoopMetric, NoopMetric, NoopMetric, NoopMetric) - val schema = Array(AttributeReference("longcol", LongType)().toAttribute) - val myIter = GeneratedInternalRowToCudfRowIterator( - ctriter, schema, TargetSize(Int.MaxValue), - NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric) - // this forces a retry on the copy of the host column to a device column - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId) - withResource(myIter.next()) { devBatch => - withResource(buildBatch()) { expected => - TestUtils.compareBatches(expected, devBatch) + val batchIter = Seq(batch).iterator + withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, + NoopMetric)) { ctriter => + val schema = Array(AttributeReference("longcol", LongType)().toAttribute) + val myIter = GeneratedInternalRowToCudfRowIterator( + ctriter, schema, TargetSize(Int.MaxValue), + NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric) + // this forces a retry on the copy of the host column to a device column + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId) + withResource(myIter.next()) { devBatch => + withResource(buildBatch()) { expected => + TestUtils.compareBatches(expected, devBatch) + } } + assert(!GpuColumnVector.extractBases(batch).exists(_.getRefCount > 0)) + assert(!myIter.hasNext) + assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) } - assert(!GpuColumnVector.extractBases(batch).exists(_.getRefCount > 0)) - assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) } test("a retry when converting to a table is handled") { @@ -78,31 +84,29 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite }).when(deviceStorage) .addTable(any(), any(), any(), any()) - val ctriter = new ColumnarToRowIterator(batchIter, - NoopMetric, NoopMetric, NoopMetric, NoopMetric) - val schema = Array(AttributeReference("longcol", LongType)().toAttribute) - val myIter = spy(GeneratedInternalRowToCudfRowIterator( - ctriter, schema, TargetSize(Int.MaxValue), - NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) - assertResult(0)(RmmSpark.getAndResetNumRetryThrow(RmmSpark.getCurrentThreadId)) - withResource(myIter.next()) { devBatch => - withResource(buildBatch()) { expected => - TestUtils.compareBatches(expected, devBatch) + withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, + NoopMetric)) { ctriter => + val schema = Array(AttributeReference("longcol", LongType)().toAttribute) + val myIter = spy(GeneratedInternalRowToCudfRowIterator( + ctriter, schema, TargetSize(Int.MaxValue), + NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) + assertResult(0)(getAndResetNumRetryThrowCurrentTask) + withResource(myIter.next()) { devBatch => + withResource(buildBatch()) { expected => + TestUtils.compareBatches(expected, devBatch) + } } + assertResult(5)(getAndResetNumRetryThrowCurrentTask) + assert(!myIter.hasNext) + assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) + // This is my wrap around of checking that we did retry the last part + // where we are converting the device column of rows into an actual column. + // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize + // a batch. + verify(rapidsBufferSpy, times(4)) + .getColumnarBatch(any()) } - // TODO: enable this assert, for some reason this is returning 0, but I verified - // via the debugger and printfs that we are retrying 2 times total in the first block, - // and 3 times in the second block that I have added retries to. - // assertResult(5)(RmmSpark.getAndResetNumRetryThrow(RmmSpark.getCurrentThreadId)) - assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) - // This is my wrap around of checking that we did retry the last part - // where we are converting the device column of rows into an actual column. - // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize - // a batch. - verify(rapidsBufferSpy, times(4)) - .getColumnarBatch(any()) } test("spilling the device column of rows works") { @@ -124,47 +128,46 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite }).when(deviceStorage) .addTable(any(), any(), any(), any()) - val ctriter = new ColumnarToRowIterator(batchIter, - NoopMetric, NoopMetric, NoopMetric, NoopMetric) - val schema = Array(AttributeReference("longcol", LongType)().toAttribute) - val myIter = spy(GeneratedInternalRowToCudfRowIterator( - ctriter, schema, TargetSize(Int.MaxValue), - NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) - assertResult(0)(RmmSpark.getAndResetNumRetryThrow(RmmSpark.getCurrentThreadId)) - withResource(myIter.next()) { devBatch => - withResource(buildBatch()) { expected => - TestUtils.compareBatches(expected, devBatch) + withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, + NoopMetric)) { ctriter => + val schema = Array(AttributeReference("longcol", LongType)().toAttribute) + val myIter = spy(GeneratedInternalRowToCudfRowIterator( + ctriter, schema, TargetSize(Int.MaxValue), + NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) + assertResult(0)(getAndResetNumRetryThrowCurrentTask) + withResource(myIter.next()) { devBatch => + withResource(buildBatch()) { expected => + TestUtils.compareBatches(expected, devBatch) + } } + assertResult(5)(getAndResetNumRetryThrowCurrentTask) + assert(!myIter.hasNext) + assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) + // This is my wrap around of checking that we did retry the last part + // where we are converting the device column of rows into an actual column. + // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize + // a batch. + verify(rapidsBufferSpy, times(4)) + .getColumnarBatch(any()) } - // TODO: enable this assert, for some reason this is returning 0, but I verified - // via the debugger and printfs that we are retrying 2 times total in the first block, - // and 3 times in the second block that I have added retries to. - // assertResult(5)(RmmSpark.getAndResetNumRetryThrow(RmmSpark.getCurrentThreadId)) - assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) - // This is my wrap around of checking that we did retry the last part - // where we are converting the device column of rows into an actual column. - // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize - // a batch. - verify(rapidsBufferSpy, times(4)) - .getColumnarBatch(any()) } test("a split and retry when copying to device is not handled, and we throw") { val batch = buildBatch() val batchIter = Seq(batch).iterator - val ctriter = new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, - NoopMetric) - val schema = Array(AttributeReference("longcol", LongType)().toAttribute) - val myIter = GeneratedInternalRowToCudfRowIterator( - ctriter, schema, TargetSize(1), - NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric) - RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId) - assertThrows[SplitAndRetryOOM] { - myIter.next() + withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, + NoopMetric)) { ctriter => + val schema = Array(AttributeReference("longcol", LongType)().toAttribute) + val myIter = GeneratedInternalRowToCudfRowIterator( + ctriter, schema, TargetSize(1), + NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric) + RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId) + assertThrows[SplitAndRetryOOM] { + myIter.next() + } + assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) } - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) } } \ No newline at end of file diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala index 99f0f406d91..87b73e1cd10 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.funsuite.AnyFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericRow} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -158,7 +159,7 @@ class GpuBatchUtilsSuite extends AnyFunSuite { assert(GpuBatchUtils.estimateRowCount(200, 0, 1) == 1) } - private def compareEstimateWithActual(schema: StructType, rowCount: Int) { + private def compareEstimateWithActual(schema: StructType, rowCount: Int): Unit = { val rows = GpuBatchUtilsSuite.createRows(schema, rowCount) val estimate = GpuBatchUtils.estimateGpuMemory(schema, rows.length) val actual = calculateGpuMemory(schema, rows) @@ -207,7 +208,12 @@ object GpuBatchUtilsSuite { externalRows.toArray } - private def createValueForType(i: Int, r: Random, dt: DataType, nullable: Boolean): Any = { + private def createValueForType( + i: Int, + r: Random, + dt: DataType, + nullable: Boolean, + isInternal: Boolean = true): Any = { dt match { case DataTypes.BooleanType => maybeNull(nullable, i, r.nextBoolean()) case DataTypes.ByteType => maybeNull(nullable, i, r.nextInt().toByte) @@ -225,30 +231,43 @@ object GpuBatchUtilsSuite { case dataType: DecimalType => val upperBound = (0 until dataType.precision).foldLeft(1L)((x, _) => x * 10) val unScaledValue = r.nextLong() % upperBound - maybeNull(nullable, i, Decimal(unScaledValue, dataType.precision, dataType.scale)) + val d = maybeNull(nullable, i, Decimal(unScaledValue, dataType.precision, dataType.scale)) + if (d != null && !isInternal) { + d.asInstanceOf[Decimal].toJavaBigDecimal + } else { + d + } case dataType@DataTypes.StringType => + val length = if (nullable) dataType.defaultSize * 2 else dataType.defaultSize + val utf8String = createUTF8String(length) + val string = if (!isInternal) utf8String.toString() else utf8String + if (nullable) { // since we want a deterministic test that compares the estimate with actual // usage we need to make sure the average length of strings is `dataType.defaultSize` if (i % 2 == 0) { null } else { - createUTF8String(dataType.defaultSize * 2) + string } } else { - createUTF8String(dataType.defaultSize) + string } case dataType@DataTypes.BinaryType => + val length = if (nullable) dataType.defaultSize * 2 else dataType.defaultSize + val bytes = r.nextString(length).getBytes + val binary = if (!isInternal) bytes.toSeq else bytes + if (nullable) { // since we want a deterministic test that compares the estimate with actual usage we // need to make sure the average length of binary values is `dataType.defaultSize` if (i % 2 == 0) { null } else { - r.nextString(dataType.defaultSize * 2).getBytes + binary } } else { - r.nextString(dataType.defaultSize).getBytes + binary } case ArrayType(elementType, containsNull) => if (nullable && i % 2 == 0) { @@ -256,33 +275,48 @@ object GpuBatchUtilsSuite { } else { val arrayValues = new mutable.ArrayBuffer[Any]() for (_ <- 0 to r.nextInt(10)) { - arrayValues.append(createValueForType(i, r, elementType, containsNull)) + arrayValues.append(createValueForType(i, r, elementType, containsNull, isInternal)) + } + val array = ArrayData.toArrayData(arrayValues) + if (!isInternal && array != null) { + array.toSeq(elementType) + } else { + array } - arrayValues.toArray.toSeq } - case MapType(_, _, valueContainsNull) => + case MapType(keyType, valueType, valueContainsNull) => if (nullable && i % 2 == 0) { null } else { - // TODO: add other types - val map = mutable.Map[String, String]() - for ( j <- 0 until 10) { + val map = mutable.Map[Any, Any]() + for (j <- 0 until 10) { if (valueContainsNull && j % 2 == 0) { - map += (createUTF8String(10).toString -> null) + map += (createValueForType(i, r, keyType, nullable = false, isInternal) -> null) } else { - map += (createUTF8String(10).toString -> createUTF8String(10).toString) + map += (createValueForType(i, r, keyType, nullable = false, isInternal) -> + createValueForType(i, r, valueType, nullable = false, isInternal)) } } - map + val mapData = ArrayBasedMapData(map) + if (mapData != null && !isInternal) { + ArrayBasedMapData.toScalaMap(mapData) + } else { + mapData + } } case StructType(fields) => - new GenericRow(fields.map(f => createValueForType(i, r, f.dataType, nullable))) - case unknown => throw new UnsupportedOperationException( + if (!isInternal) { + new GenericRow(fields.map(f => + createValueForType(i, r, f.dataType, nullable = f.nullable, isInternal = false))) + } else { + InternalRow(fields.map(f => createValueForType(i, r, f.dataType, nullable)): _*) + } + + case unknown => throw new UnsupportedOperationException( s"Type $unknown not supported") } } - private def createRowValues(i: Int, r: Random, fields: Array[StructField]) = { val values: Array[Any] = fields.map(field => { createValueForType(i, r, field.dataType, field.nullable) @@ -291,34 +325,9 @@ object GpuBatchUtilsSuite { } private def createExternalRowValues(i: Int, r: Random, fields: Array[StructField]): Array[Any] = { - val values: Array[Any] = fields.map(field => { - field.dataType match { - // Since it's using the createUTF8String method for InternalRow case, need to convert to - // String for Row case. - case StringType => - val utf8StringOrNull = createValueForType(i, r, field.dataType, field.nullable) - if (utf8StringOrNull != null) { - utf8StringOrNull.asInstanceOf[UTF8String].toString - } else { - utf8StringOrNull - } - case BinaryType => - val b = createValueForType(i, r, field.dataType, field.nullable) - if (b != null) { - b.asInstanceOf[Array[Byte]].toSeq - } else { - b - } - case DecimalType() => - val d = createValueForType(i, r, field.dataType, field.nullable) - if (d != null) { - d.asInstanceOf[Decimal].toJavaBigDecimal - } else { - d - } - case _ => createValueForType(i, r, field.dataType, field.nullable) - } - }) + val values: Array[Any] = fields.map { field => + createValueForType(i, r, field.dataType, field.nullable, isInternal = false) + } values } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala index 16236810181..20919d171d4 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala @@ -487,7 +487,7 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { }, conf) } - def testCompressedBatches(maxCompressedBatchMemoryLimit: Long) { + def testCompressedBatches(maxCompressedBatchMemoryLimit: Long): Unit = { val coalesceTargetBytes = 8000 val stop = 10000 var start = 0 diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala index a9c93039cbf..b4f58a74015 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala @@ -55,9 +55,7 @@ class GpuShuffledHashJoinExecSuite extends AnyFunSuite with MockitoSugar { optimalCase: Boolean = false) (verifyBuiltData: Either[ColumnarBatch, Iterator[ColumnarBatch]] => Unit): Unit = { val mockStreamIter = mock[Iterator[ColumnarBatch]] - val mockBufferedStreamIterator = mock[BufferedIterator[ColumnarBatch]] - when(mockStreamIter.buffered).thenReturn(mockBufferedStreamIterator) - when(mockBufferedStreamIterator.hasNext).thenReturn(true) + when(mockStreamIter.hasNext).thenReturn(true) val (builtData, _) = GpuShuffledHashJoinExec.prepareBuildBatchesForJoin( buildIter, mockStreamIter, @@ -68,11 +66,12 @@ class GpuShuffledHashJoinExecSuite extends AnyFunSuite with MockitoSugar { verifyBuiltData(builtData) // build iterator should be drained assertResult(expected = false)(buildIter.hasNext) - verify(mockStreamIter, times(0)).hasNext if (optimalCase) { - verify(mockStreamIter, times(1)).buffered - verify(mockBufferedStreamIterator, times(1)).hasNext - verify(mockBufferedStreamIterator, times(1)).head + verify(mockStreamIter, times(1)).hasNext + verify(mockStreamIter, times(1)).next + } else { + verify(mockStreamIter, times(0)).hasNext + verify(mockStreamIter, times(0)).next } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala index b6cf36ac51c..0a2c6a654e9 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -61,7 +61,7 @@ class HashAggregatesSuite extends SparkQueryCompareTestSuite { batchSize: Int = 0, repart: Int = 1, maxFloatDiff: Double = 0.0) - (fn: DataFrame => DataFrame) { + (fn: DataFrame => DataFrame): Unit = { if (batchSize > 0) { makeBatchedBytes(batchSize, conf) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala new file mode 100644 index 00000000000..b0cecd80c87 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala @@ -0,0 +1,151 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids + +import ai.rapids.cudf.{ColumnVector, Table} +import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq +import com.nvidia.spark.rapids.jni.RmmSpark + +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId} +import org.apache.spark.sql.rapids.GpuGreaterThan +import org.apache.spark.sql.rapids.catalyst.expressions.GpuRand +import org.apache.spark.sql.types.{DoubleType, IntegerType} +import org.apache.spark.sql.vectorized.ColumnarBatch + +class NonDeterministicRetrySuite extends RmmSparkRetrySuiteBase { + private val NUM_ROWS = 500 + private val RAND_SEED = 10 + private val batchAttrs = Seq(AttributeReference("int", IntegerType)(ExprId(10))) + + private def buildBatch(ints: Seq[Int] = 0 until NUM_ROWS): ColumnarBatch = { + new ColumnarBatch( + Array(GpuColumnVector.from(ColumnVector.fromInts(ints: _*), IntegerType)), ints.length) + } + + test("GPU rand outputs the same sequence with checkpoint and restore") { + val gpuRand = GpuRand(GpuLiteral(RAND_SEED, IntegerType)) + withResource(buildBatch()) { inputCB => + // checkpoint the state + gpuRand.checkpoint() + val randHCol1 = withResource(gpuRand.columnarEval(inputCB)) { randCol1 => + randCol1.copyToHost() + } + withResource(randHCol1) { _ => + assert(randHCol1.getRowCount.toInt == NUM_ROWS) + // Restore the state, and generate data again + gpuRand.restore() + val randHCol2 = withResource(gpuRand.columnarEval(inputCB)) { randCol2 => + randCol2.copyToHost() + } + withResource(randHCol2) { _ => + // check the two random columns are equal. + assert(randHCol1.getRowCount == randHCol2.getRowCount) + (0 until randHCol1.getRowCount.toInt).foreach { pos => + assert(randHCol1.getDouble(pos) == randHCol2.getDouble(pos)) + } + } + } + } + } + + test("GPU project retry with GPU rand") { + def projectRand(): Seq[GpuExpression] = Seq( + GpuAlias(GpuRand(GpuLiteral(RAND_SEED)), "rand")()) + + Seq(true, false).foreach { useTieredProject => + // expression should be retryable + val boundProjectRand = GpuBindReferences.bindGpuReferencesTiered(projectRand(), + batchAttrs, useTieredProject) + assert(boundProjectRand.areAllRetryable) + // project with and without retry + val batches = Seq(true, false).safeMap { forceRetry => + val boundProjectList = GpuBindReferences.bindGpuReferencesTiered( + projectRand() ++ batchAttrs, batchAttrs, useTieredProject) + assert(boundProjectList.areAllRetryable) + + val sb = closeOnExcept(buildBatch()) { cb => + SpillableColumnarBatch(cb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY) + } + closeOnExcept(sb) { _ => + if (forceRetry) { + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId) + } + } + boundProjectList.projectAndCloseWithRetrySingleBatch(sb) + } + // check the random columns + val randCols = withResource(batches) { case Seq(retriedBatch, batch) => + assert(retriedBatch.numRows() == batch.numRows()) + assert(retriedBatch.numCols() == batch.numCols()) + batches.safeMap(_.column(0).asInstanceOf[GpuColumnVector].copyToHost()) + } + withResource(randCols) { case Seq(retriedRand, rand) => + (0 until rand.getRowCount.toInt).foreach { pos => + assert(retriedRand.getDouble(pos) == rand.getDouble(pos)) + } + } + } + } + + test("GPU filter retry with GPU rand") { + def filterRand(): Seq[GpuExpression] = Seq( + GpuGreaterThan( + GpuRand(GpuLiteral.create(RAND_SEED, IntegerType)), + GpuLiteral.create(0.1d, DoubleType))) + + Seq(true, false).foreach { useTieredProject => + // filter with and without retry + val tables = Seq(true, false).safeMap { forceRetry => + val boundCondition = GpuBindReferences.bindGpuReferencesTiered(filterRand(), + batchAttrs, useTieredProject) + assert(boundCondition.areAllRetryable) + + val cb = buildBatch() + if (forceRetry) { + RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId) + } + val batchSeq = GpuFilter.filterAndClose(cb, boundCondition, + NoopMetric, NoopMetric, NoopMetric).toSeq + withResource(batchSeq) { _ => + val tables = batchSeq.safeMap(GpuColumnVector.from) + if (tables.size == 1) { + tables.head + } else { + withResource(tables) { _ => + assert(tables.size > 1) + Table.concatenate(tables: _*) + } + } + } + } + + // check the outputs + val cols = withResource(tables) { case Seq(retriedTable, table) => + assert(retriedTable.getRowCount == table.getRowCount) + assert(retriedTable.getNumberOfColumns == table.getNumberOfColumns) + tables.safeMap(_.getColumn(0).copyToHost()) + } + withResource(cols) { case Seq(retriedInts, ints) => + (0 until ints.getRowCount.toInt).foreach { pos => + assert(retriedInts.getInt(pos) == ints.getInt(pos)) + } + } + } + } + +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala new file mode 100644 index 00000000000..e8849b35941 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala @@ -0,0 +1,646 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. All rights reserved. + * + * 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 com.nvidia.spark.rapids + +import java.io.File + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.parquet.format.converter.ParquetMetadataConverter +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.schema.PrimitiveType + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.tests.datagen.{DBGen, TableGen} +import org.apache.spark.sql.types._ + +class ParquetScaleTestSuite extends SparkQueryCompareTestSuite with Logging { + private val sparkConf = new SparkConf() + // for date time + .set("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "CORRECTED") // for Spark 31x + .set("spark.sql.parquet.datetimeRebaseModeInWrite", "CORRECTED") // for Spark 32x, 33x and ... + .set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "CORRECTED") // for Spark 31x + .set("spark.sql.parquet.datetimeRebaseModeInRead", "CORRECTED") // for Spark 32x, 33x and ... + + // for int96 + .set("spark.sql.legacy.parquet.int96RebaseModeInWrite", "CORRECTED") // for Spark 31x + .set("spark.sql.parquet.int96RebaseModeInWrite", "CORRECTED") // for Spark 32x, 33x and ... + .set("spark.sql.legacy.parquet.int96RebaseModeInRead", "CORRECTED") // for Spark 31x + .set("spark.sql.parquet.int96RebaseModeInRead", "CORRECTED") // for Spark 32x, 33x and ... + .set("spark.rapids.sql.explain", "ALL") + + /** + * By default cuDF splits row groups by 1,000,000 rows, we expect one row group. + * Refer to cuDF parquet.hpp + * default_row_group_size_bytes = 128 * 1024 * 1024; ///< 128MB per row group + * default_row_group_size_rows = 1000000; ///< 1 million rows per row group + */ + private val rowsNum: Long = 1000L * 1000L + + case class ParquetStat( + schema: Seq[String], + rowGroupStats: Seq[RowGroupStat]) + + case class RowGroupStat(rowCount: Long, columnStats: Seq[ColumnChunkStat]) + + case class ColumnChunkStat( + primitiveType: PrimitiveType, + min: Comparable[_], + max: Comparable[_], + hasNonNullValue: Boolean, + isNumNullsSet: Boolean, + numNulls: Long + ) + + def writeScaleTestDataOnCpu(testDataPath: File, gen: SparkSession => DataFrame): Unit = { + withCpuSparkSession( + spark => { + // define table + val path = testDataPath.getAbsolutePath + // write to a file on CPU + gen(spark).coalesce(1).write.mode("overwrite").parquet(path) + }, + sparkConf) + } + + // write test data on CPU or GPU, then read the stats + def getStats(filePath: File): SparkSession => ParquetStat = { spark => + withTempPath { writePath => + // Read from the testing Parquet file and then write to a Parquet file + spark.read.parquet(filePath.getAbsolutePath).coalesce(1) + .write.mode("overwrite").parquet(writePath.getAbsolutePath) + + // get Stats + getStatsFromFile(writePath) + } + } + + /** + * Find a parquet file in parquetDir and get the stats. It's similar to output of + * `Parquet-cli meta file`. Parquet-cli:https://github + * .com/apache/parquet-mr/tree/master/parquet-cli + * + * @param parquetDir parquet file directory + * @return Parquet statistics + */ + @scala.annotation.nowarn("msg=method readFooter in class ParquetFileReader is deprecated") + private def getStatsFromFile(parquetDir: File): ParquetStat = { + val parquetFile = parquetDir.listFiles(f => f.getName.endsWith(".parquet"))(0) + val p = new Path(parquetFile.getCanonicalPath) + val footer = ParquetFileReader.readFooter(new Configuration(), p, + ParquetMetadataConverter.NO_FILTER) + val columnTypes = footer.getFileMetaData.getSchema.getColumns.asScala.toArray + .map(c => c.toString) + val groupStats = footer.getBlocks.asScala.toArray.map { g => + val rowCount = g.getRowCount + val columnChunkStats = g.getColumns.asScala.toArray.map { col => + ColumnChunkStat( + col.getPrimitiveType, + col.getStatistics.genericGetMin(), + col.getStatistics.genericGetMax(), + col.getStatistics.hasNonNullValue, + col.getStatistics.isNumNullsSet, + col.getStatistics.getNumNulls) + } + RowGroupStat(rowCount, columnChunkStats) + } + + ParquetStat(columnTypes, groupStats) + } + + private def checkStats( + genDf: SparkSession => DataFrame, + skipCheckSchema: Boolean = false): (ParquetStat, ParquetStat) = { + withTempPath { testDataFile => + // Write test data to a file on CPU + writeScaleTestDataOnCpu(testDataFile, genDf) + + // write data and get stats on CPU + val cpuStats = withCpuSparkSession(getStats(testDataFile), sparkConf) + val cpuFileSize = testDataFile.listFiles(f => f.getName.endsWith(".parquet"))(0).length() + + // write data and get stats on GPU + val gpuStats = withGpuSparkSession(getStats(testDataFile), sparkConf) + val gpuFileSize = testDataFile.listFiles(f => f.getName.endsWith(".parquet"))(0).length() + + // compare schema + if (!skipCheckSchema) { + assertResult(cpuStats.schema)(gpuStats.schema) + } + + // compare stats + assert(cpuStats.rowGroupStats.length == gpuStats.rowGroupStats.length) + cpuStats.rowGroupStats.zip(gpuStats.rowGroupStats).foreach { + case (cpuRowGroup, gpuRowGroup) => { + assert(cpuRowGroup.rowCount == gpuRowGroup.rowCount) + assert(cpuRowGroup.columnStats.length == gpuRowGroup.columnStats.length) + cpuRowGroup.columnStats.zip(gpuRowGroup.columnStats).foreach { + case (cpuColumnStat, gpuColumnStat) => { + assert(cpuColumnStat.hasNonNullValue == gpuColumnStat.hasNonNullValue) + if (cpuColumnStat.hasNonNullValue) { + // compare all the attributes + assertResult(cpuColumnStat)(gpuColumnStat) + } else { + // hasNonNullValue is false, which means stats are invalid, no need to compare + // other attributes. + /** + * hasNonNullValue means: + * + * Returns whether there have been non-null values added to this statistics + * + * @return true if the values contained at least one non-null value + * + * Refer to link: https://github.com/apache/parquet-mr/blob/apache-parquet-1.10.1 + * /parquet-column/src/main/java/org/apache/parquet/column/statistics + * /Statistics.java#L504-L506 + * + * e.g.: Spark 31x, for timestamp type + * CPU: hasNonNullValue: false, isNumNullsSet: false, getNumNulls: -1 + * GPU: hasNonNullValue: false, isNumNullsSet: true, getNumNulls: 0 + * + * Above are expected differences. + */ + assertResult(cpuColumnStat.primitiveType)(gpuColumnStat.primitiveType) + } + } + } + } + } + + // Check the Gpu file size is not too large. + assert(gpuFileSize < 2 * cpuFileSize) + + (cpuStats, gpuStats) + } + } + + private val basicTypes = Seq( + // "float", "Blocked by https://github.com/rapidsai/cudf/issues/13948" + // "double", "Blocked by https://github.com/rapidsai/cudf/issues/13948" + "boolean", + "byte", + "short", + "int", + "long", + "decimal", + "string", + "date", + "timestamp") + + test("Statistics tests for Parquet files written by GPU, float/double, with NaN") { + assume(false, "Blocked by https://github.com/rapidsai/cudf/issues/13948") + assume(false, "Move to scale test") + + val schema = StructType(Seq( + StructField("c01", FloatType), + StructField("c02", DoubleType) + )) + // 2 rows with NaN + val data = Seq(Row(1.1f, Double.NaN), Row(Float.NaN, 2.2d)) + + def genDf(schema: StructType, data: Seq[Row]): SparkSession => DataFrame = spark => + spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + + checkStats(genDf(schema, data)) + + // After https://github.com/rapidsai/cudf/issues/13948 is fixed + // Change this test to the following code: + // val schemaStr = + // """ + // struct< + // c01: float, + // c02: double + // > + // """ + // val gen = DBGen() + // val tab = gen.addTable("tab", schemaStr, rowsNum) + // tab("c01").setNullProbability(0.5) + // checkStats(tab) + } + + test("Statistics tests for Parquet files written by GPU, float/double, without NaN") { + assume(true, "Move to scale test") + val schemaStr = + """ + struct< + c01: float, + c02: double + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schemaStr, rowsNum) + tab("c01").setValueGen(NonNaNFloatGenFunc()) + tab("c02").setValueGen(NonNaNDoubleGenFunc()) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + withTempPath { testDataFile => + // Write test data to a file on CPU + writeScaleTestDataOnCpu(testDataFile, genDf(tab)) + + // write data and get stats on CPU + val cpuStats = withCpuSparkSession(getStats(testDataFile), sparkConf) + val cpuFileSize = testDataFile.listFiles(f => f.getName.endsWith(".parquet"))(0).length() + + // write data and get stats on GPU + val gpuStats = withGpuSparkSession(getStats(testDataFile), sparkConf) + val gpuFileSize = testDataFile.listFiles(f => f.getName.endsWith(".parquet"))(0).length() + + // compare schema + assertResult(cpuStats.schema)(gpuStats.schema) + + // Check the Gpu file size is not too large. + assert(gpuFileSize < 2 * cpuFileSize) + + /** + * + * CPU stat: + * + * ParquetStat(WrappedArray([c01] optional float c01, [c02] optional double c02), + * WrappedArray(RowGroupStat(1000000, + * WrappedArray( + * ColumnChunkStat(optional float c01, + * min = 0.0, max = 0.0, + * hasNonNullValue = false, + * isNumNullsSet = true, + * numNulls = 500532), + * ColumnChunkStat(optional double c02, + * min = 0.0, + * max = 0.0, + * hasNonNullValue = false, + * isNumNullsSet = true, + * numNulls = 498986))))) + * + * GPU stat: + * + * ParquetStat(WrappedArray([c01] optional float c01, [c02] optional double c02), + * WrappedArray(RowGroupStat(1000000, + * WrappedArray( + * ColumnChunkStat(optional float c01, + * min = -3.4026107E38, + * max = 3.4015179E38, + * hasNonNullValue = true, + * isNumNullsSet = true, + * numNulls = 500532), + * ColumnChunkStat(optional double c02, + * min = -1.7xxxxxE308, + * max = 1.7xxxxE308, + * hasNonNullValue = true, + * isNumNullsSet = true, + * numNulls = 498986))))) + * + * There are differences between CPU and GPU: + * CPU hasNonNullValue is false, CPU min/max is 0.0 + */ + assert(cpuStats.rowGroupStats.length == gpuStats.rowGroupStats.length) + assert(cpuStats.rowGroupStats(0).columnStats(0).isNumNullsSet == + gpuStats.rowGroupStats(0).columnStats(0).isNumNullsSet) + assert(cpuStats.rowGroupStats(0).columnStats(1).isNumNullsSet == + gpuStats.rowGroupStats(0).columnStats(1).isNumNullsSet) + assert(cpuStats.rowGroupStats(0).columnStats(0).numNulls == + gpuStats.rowGroupStats(0).columnStats(0).numNulls) + assert(cpuStats.rowGroupStats(0).columnStats(1).numNulls == + gpuStats.rowGroupStats(0).columnStats(1).numNulls) + + // write by GPU, read min/max by CPU + val (floatMin, floatMax, doubleMin, doubleMax) = withTempPath { gpuFile => + withGpuSparkSession(spark => { + // Read from the testing Parquet file and then write to a Parquet file + spark.read.parquet(testDataFile.getAbsolutePath).coalesce(1) + .write.mode("overwrite").parquet(gpuFile.getAbsolutePath) + }) + + val rowArray = withCpuSparkSession(spark => { + // Read from the testing Parquet file and then write to a Parquet file + spark.read.parquet(gpuFile.getAbsolutePath) + .selectExpr("min(c01)", "max(c01)", "min(c02)", "max(c02)").collect() + }) + + (rowArray(0)(0), rowArray(0)(1), rowArray(0)(2), rowArray(0)(3)) + } + + assertResult(floatMin)(gpuStats.rowGroupStats(0).columnStats(0).min) + assertResult(floatMax)(gpuStats.rowGroupStats(0).columnStats(0).max) + assertResult(doubleMin)(gpuStats.rowGroupStats(0).columnStats(1).min) + assertResult(doubleMax)(gpuStats.rowGroupStats(0).columnStats(1).max) + } + } + + test("Statistics tests for Parquet files written by GPU, basic types") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c01: boolean, + c02: byte, + c03: short, + c04: int, + c05: long, + c06: decimal, + c07: string, + c08: date, + c09: timestamp + > + """ + // "float", "Blocked by https://github.com/rapidsai/cudf/issues/13948" + // "double", "Blocked by https://github.com/rapidsai/cudf/issues/13948" + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c01").setNullProbability(0.5) + tab("c06").setNullProbability(0.5) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, array") { + assume(false, "Move to scale test") + basicTypes.foreach { t => + val schema = s"struct>" + val nullProbabilities = Seq(0d, 0.5d) + nullProbabilities.foreach { nullProbability => + try { + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c01").setNullProbability(nullProbability) + tab("c01").setLength(5) // avoid row group exceeds 128M, we expect one row group + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } catch { + case e: Exception => + logError(s"check $schema, $nullProbability failed", e) + throw e + } + } + } + } + + test("Statistics tests for Parquet files written by GPU, map") { + assume(false, "Move to scale test") + basicTypes.foreach { t => + val nullProbabilities = Seq(0d, 0.5d) + nullProbabilities.foreach { nullProbability => + val schema = s"struct>" + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c01").setNullProbability(nullProbability) + tab("c01").setLength(3) // avoid row group exceeds 128M, we expect one row group + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + } + } + + test("Statistics tests for Parquet files written by GPU, struct") { + assume(false, "Move to scale test") + val schema = basicTypes.zipWithIndex.map { case (t, index) => + s"c0$index: $t" + }.mkString("struct<\nc1: struct<", ", \n", ">>") + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.5) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, struct(array, map, struct)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: struct< + c01: array + >, + c2: struct< + c01: map + >, + c3: struct< + c301: struct< + c30101: int, + c30102: byte + > + > + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.9) + tab("c2").setNullProbability(0.9) + tab("c3").setNullProbability(0.9) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, array(struct, array)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: array< + struct< + c201: long, + c202: int + > + >, + c2: array> + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c2").setNullProbability(0.8) + tab("c1").setLength(3) + tab("c2").setLength(3) + tab("c2")("child").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, map(array)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: map< + array, + array + > + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c1").setLength(3) + tab("c1")("key").setLength(3) + tab("c1")("value").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, map(struct)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: map< + struct< + c101: long, + c102: int + >, + struct< + c101: long, + c102: string + > + > + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c1").setLength(3) + tab("c1")("key").setLength(3) + tab("c1")("value").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, array(map)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: array> + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c1")("child").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + /** + * Note: There are discrepancies between CPU and GPU file schemas, + * but the Spark can read both them correctly, so it's not an issue. + * + * Details: + * + * CPU Parquet file schema is: + * message spark_schema { + * optional group c1 (LIST) { + * repeated group list { + * optional group element (MAP) { + * repeated group key_value { + * required binary key (STRING); + * optional int64 value; + * } + * } + * } + * } + * } + * + * GPU Parquet file schema is: + * message schema { + * optional group c1 (LIST) { + * repeated group list { + * optional group c1 (MAP) { + * repeated group key_value { + * required binary key (STRING); + * optional int64 value; + * } + * } + * } + * } + * } + * + * Spark reads both of them as: + * + * df.printSchema() + * root + * |-- c1: array (nullable = true) + * | |-- element: map (containsNull = true) + * | | |-- key: string + * | | |-- value: long (valueContainsNull = true) + * + */ + // skip check the schema + val (cpuStat, gpuStat) = checkStats(genDf(tab), skipCheckSchema = true) + + val expectedCpuSchemaForSpark31x = Seq( + "[c1, list, element, key_value, key] required binary key (UTF8)", + "[c1, list, element, key_value, value] optional int64 value") + + val expectedCpuSchemaForSpark320Plus = Seq( + "[c1, list, element, key_value, key] required binary key (STRING)", + "[c1, list, element, key_value, value] optional int64 value") + + val expectedGpuSchemaForSpark31x = Seq( + "[c1, list, c1, key_value, key] required binary key (UTF8)", + "[c1, list, c1, key_value, value] optional int64 value") + + val expectedGpuSchemaForSpark320Plus = Seq( + "[c1, list, c1, key_value, key] required binary key (STRING)", + "[c1, list, c1, key_value, value] optional int64 value") + + assert(cpuStat.schema == expectedCpuSchemaForSpark31x || + cpuStat.schema == expectedCpuSchemaForSpark320Plus) + + assert(gpuStat.schema == expectedGpuSchemaForSpark31x || + gpuStat.schema == expectedGpuSchemaForSpark320Plus) + } + + test("Statistics tests for Parquet files written by GPU, map(map)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: map< + map, + map + > + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c1").setLength(3) + tab("c1")("key").setLength(3) + tab("c1")("value").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestUtils.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestUtils.scala new file mode 100644 index 00000000000..fa6f6bb5516 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestUtils.scala @@ -0,0 +1,53 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. All rights reserved. + * + * 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 com.nvidia.spark.rapids + +import org.apache.spark.sql.tests.datagen.{DataGen, GeneratorFunction, LocationToSeedMapping, RowLocation} + +case class NonNaNFloatGenFunc(mapping: LocationToSeedMapping = null) extends GeneratorFunction { + override def apply(rowLoc: RowLocation): Any = { + val v = java.lang.Float.intBitsToFloat(DataGen.getRandomFor(mapping(rowLoc)).nextInt()) + if (v.isNaN) { + 1.toFloat // just use 1.0 + } else { + v + } + } + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = + NonNaNFloatGenFunc(mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalStateException("value ranges are not supported for Float yet") +} + +case class NonNaNDoubleGenFunc(mapping: LocationToSeedMapping = null) extends GeneratorFunction { + override def apply(rowLoc: RowLocation): Any = { + val v = java.lang.Double.longBitsToDouble(DataGen.nextLong(rowLoc, mapping)) + if (v.isNaN) { + 1.toDouble // just use 1.0 + } else { + v + } + } + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = + NonNaNDoubleGenFunc(mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalStateException("value ranges are not supported for Double yet") +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala index a1e7f16c17d..702928225a7 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala @@ -39,11 +39,11 @@ class ParseDateTimeSuite extends SparkQueryCompareTestSuite with BeforeAndAfterE .set(SQLConf.LEGACY_TIME_PARSER_POLICY.key, "LEGACY") .set(RapidsConf.INCOMPATIBLE_DATE_FORMATS.key, "true") - override def beforeEach() { + override def beforeEach(): Unit = { GpuOverrides.removeAllListeners() } - override def afterEach() { + override def afterEach(): Unit = { GpuOverrides.removeAllListeners() } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala index 54827e12878..61940ffd463 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala @@ -215,7 +215,7 @@ class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { withResource(spy(new RapidsDeviceMemoryStore)) { deviceStore => val mockStore = mock[RapidsBufferStore] withResource( - new RapidsHostMemoryStore(10000)) { hostStore => + new RapidsHostMemoryStore(Some(10000))) { hostStore => deviceStore.setSpillStore(hostStore) hostStore.setSpillStore(mockStore) val catalog = new RapidsBufferCatalog(deviceStore) @@ -243,7 +243,7 @@ class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { assertResult(unspilled)(unspilledSame) } // verify that we invoked the copy function exactly once - verify(deviceStore, times(1)).copyBuffer(any(), any()) + verify(deviceStore, times(1)).copyBuffer(any(), any(), any()) unspilled } val unspilledSame = catalog.unspillBufferToDeviceStore( @@ -253,7 +253,7 @@ class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { assertResult(unspilled)(unspilledSame) } // verify that we invoked the copy function exactly once - verify(deviceStore, times(1)).copyBuffer(any(), any()) + verify(deviceStore, times(1)).copyBuffer(any(), any(), any()) } } } @@ -330,7 +330,7 @@ class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { var _acquireAttempts: Int = acquireAttempts var currentPriority: Long = initialPriority override val id: RapidsBufferId = bufferId - override def getMemoryUsedBytes: Long = 0 + override val memoryUsedBytes: Long = 0 override def meta: TableMeta = tableMeta override val storageTier: StorageTier = tier override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = null diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala index 1e4bde7a65d..45d96be4cb6 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala @@ -466,9 +466,11 @@ class RapidsDeviceMemoryStoreSuite extends AnyFunSuite with MockitoSugar { override protected def createBuffer( b: RapidsBuffer, - s: Cuda.Stream): RapidsBufferBase = { + c: RapidsBufferCatalog, + s: Cuda.Stream): Option[RapidsBufferBase] = { spilledBuffers += b.id - new MockRapidsBuffer(b.id, b.getPackedSizeBytes, b.meta, b.getSpillPriority) + Some(new MockRapidsBuffer( + b.id, b.getPackedSizeBytes, b.meta, b.getSpillPriority)) } class MockRapidsBuffer(id: RapidsBufferId, size: Long, meta: TableMeta, spillPriority: Long) @@ -481,7 +483,7 @@ class RapidsDeviceMemoryStoreSuite extends AnyFunSuite with MockitoSugar { throw new UnsupportedOperationException /** The size of this buffer in bytes. */ - override def getMemoryUsedBytes: Long = size + override val memoryUsedBytes: Long = size } } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala index e8878e2cc6c..b911bccb286 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids import java.io.File import java.math.RoundingMode -import ai.rapids.cudf.{ContiguousTable, DeviceMemoryBuffer, HostMemoryBuffer, Table} +import ai.rapids.cudf.{ColumnVector, ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, Table} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import org.mockito.ArgumentMatchers import org.mockito.Mockito.{spy, times, verify} @@ -31,30 +31,45 @@ import org.apache.spark.sql.types.{DataType, DecimalType, DoubleType, IntegerTyp class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { private def buildContiguousTable(): ContiguousTable = { - withResource(new Table.TestBuilder() + withResource(buildTable()) { table => + table.contiguousSplit()(0) + } + } + + private def buildTable(): Table = { + new Table.TestBuilder() .column(5, null.asInstanceOf[java.lang.Integer], 3, 1) .column("five", "two", null, null) .column(5.0, 2.0, 3.0, 1.0) .decimal64Column(-5, RoundingMode.UNNECESSARY, 0, null, -1.4, 10.123) - .build()) { table => - table.contiguousSplit()(0) + .build() + } + + private def buildEmptyTable(): Table = { + withResource(buildTable()) { tbl => + withResource(ColumnVector.fromBooleans(false, false, false, false)) { mask => + tbl.filter(mask) // filter all out + } } } + private val mockTableDataTypes: Array[DataType] = + Array(IntegerType, StringType, DoubleType, DecimalType(10, 5)) + test("spill updates catalog") { val bufferId = MockRapidsBufferId(7, canShareDiskPaths = false) val spillPriority = -7 val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => assertResult(0)(diskStore.currentSize) hostStore.setSpillStore(diskStore) val (bufferSize, handle) = - addTableToCatalog(catalog, bufferId, spillPriority) + addContiguousTableToCatalog(catalog, bufferId, spillPriority) val path = handle.id.getDiskPath(null) assert(!path.exists()) catalog.synchronousSpill(devStore, 0) @@ -68,7 +83,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { ArgumentMatchers.eq(handle.id), ArgumentMatchers.eq(StorageTier.DEVICE)) withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.DISK)(buffer.storageTier) - assertResult(bufferSize)(buffer.getMemoryUsedBytes) + assertResult(bufferSize)(buffer.memoryUsedBytes) assertResult(handle.id)(buffer.id) assertResult(spillPriority)(buffer.getSpillPriority) } @@ -87,13 +102,13 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => hostStore.setSpillStore(diskStore) - val (_, handle) = addTableToCatalog(catalog, bufferId, spillPriority) + val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) assert(!handle.id.getDiskPath(null).exists()) val expectedTable = withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.DEVICE)(buffer.storageTier) @@ -129,12 +144,12 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => hostStore.setSpillStore(diskStore) - val (_, handle) = addTableToCatalog(catalog, bufferId, spillPriority) + val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) assert(!handle.id.getDiskPath(null).exists()) val expectedBuffer = withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.DEVICE)(buffer.storageTier) @@ -162,6 +177,161 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { } } + test("skip host: spill device memory buffer to disk") { + val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) + val bufferPath = bufferId.getDiskPath(null) + assert(!bufferPath.exists) + val spillPriority = -7 + withResource(new RapidsDeviceMemoryStore) { devStore => + val catalog = new RapidsBufferCatalog(devStore) + withResource(new AlwaysFailingRapidsHostMemoryStore) { + hostStore => + devStore.setSpillStore(hostStore) + withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => + hostStore.setSpillStore(diskStore) + val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) + assert(!handle.id.getDiskPath(null).exists()) + val expectedBuffer = withResource(catalog.acquireBuffer(handle)) { buffer => + assertResult(StorageTier.DEVICE)(buffer.storageTier) + withResource(buffer.getMemoryBuffer) { devbuf => + closeOnExcept(HostMemoryBuffer.allocate(devbuf.getLength)) { hostbuf => + hostbuf.copyFromDeviceBuffer(devbuf.asInstanceOf[DeviceMemoryBuffer]) + hostbuf + } + } + } + withResource(expectedBuffer) { expectedBuffer => + catalog.synchronousSpill(devStore, 0) + withResource(catalog.acquireBuffer(handle)) { buffer => + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getMemoryBuffer) { actualBuffer => + assert(actualBuffer.isInstanceOf[HostMemoryBuffer]) + val actualHostBuffer = actualBuffer.asInstanceOf[HostMemoryBuffer] + assertResult(expectedBuffer.asByteBuffer)(actualHostBuffer.asByteBuffer) + } + } + } + } + } + } + } + + test("skip host: spill table to disk") { + val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) + val bufferPath = bufferId.getDiskPath(null) + assert(!bufferPath.exists) + val spillPriority = -7 + withResource(new RapidsDeviceMemoryStore) { devStore => + val catalog = new RapidsBufferCatalog(devStore) + withResource(new AlwaysFailingRapidsHostMemoryStore) { + hostStore => + devStore.setSpillStore(hostStore) + withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => + hostStore.setSpillStore(diskStore) + val handle = addTableToCatalog(catalog, bufferId, spillPriority) + withResource(buildTable()) { expectedTable => + withResource( + GpuColumnVector.from(expectedTable, mockTableDataTypes)) { expectedBatch => + catalog.synchronousSpill(devStore, 0) + withResource(catalog.acquireBuffer(handle)) { buffer => + assert(handle.id.getDiskPath(null).exists()) + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getColumnarBatch(mockTableDataTypes)) { fromDiskBatch => + TestUtils.compareBatches(expectedBatch, fromDiskBatch) + } + } + } + } + } + } + } + } + + test("skip host: spill table to disk with small host bounce buffer") { + val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) + val bufferPath = bufferId.getDiskPath(null) + assert(!bufferPath.exists) + val spillPriority = -7 + withResource(new RapidsDeviceMemoryStore(1L*1024*1024, 10)) { devStore => + val catalog = new RapidsBufferCatalog(devStore) + withResource(new AlwaysFailingRapidsHostMemoryStore) { + hostStore => + devStore.setSpillStore(hostStore) + withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => + hostStore.setSpillStore(diskStore) + val handle = addTableToCatalog(catalog, bufferId, spillPriority) + withResource(buildTable()) { expectedTable => + withResource( + GpuColumnVector.from(expectedTable, mockTableDataTypes)) { expectedBatch => + catalog.synchronousSpill(devStore, 0) + withResource(catalog.acquireBuffer(handle)) { buffer => + assert(handle.id.getDiskPath(null).exists()) + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getColumnarBatch(mockTableDataTypes)) { fromDiskBatch => + TestUtils.compareBatches(expectedBatch, fromDiskBatch) + } + } + } + } + } + } + } + } + + + test("0-byte table is never spillable as we would fail to mmap") { + val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) + val bufferPath = bufferId.getDiskPath(null) + val bufferId2 = MockRapidsBufferId(2, canShareDiskPaths = false) + assert(!bufferPath.exists) + val spillPriority = -7 + val hostStoreMaxSize = 1L * 1024 * 1024 + withResource(new RapidsDeviceMemoryStore) { devStore => + val catalog = new RapidsBufferCatalog(devStore) + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => + devStore.setSpillStore(hostStore) + withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => + hostStore.setSpillStore(diskStore) + val handle = addZeroRowsTableToCatalog(catalog, bufferId, spillPriority - 1) + val handle2 = addTableToCatalog(catalog, bufferId2, spillPriority) + withResource(handle2) { _ => + assert(!handle.id.getDiskPath(null).exists()) + withResource(buildTable()) { expectedTable => + withResource(buildEmptyTable()) { expectedEmptyTable => + withResource( + GpuColumnVector.from( + expectedTable, mockTableDataTypes)) { expectedCb => + withResource( + GpuColumnVector.from( + expectedEmptyTable, mockTableDataTypes)) { expectedEmptyCb => + catalog.synchronousSpill(devStore, 0) + catalog.synchronousSpill(hostStore, 0) + withResource(catalog.acquireBuffer(handle2)) { buffer => + withResource(catalog.acquireBuffer(handle)) { emptyBuffer => + // the 0-byte table never moved from device. It is not spillable + assertResult(StorageTier.DEVICE)(emptyBuffer.storageTier) + withResource(emptyBuffer.getColumnarBatch(mockTableDataTypes)) { cb => + TestUtils.compareBatches(expectedEmptyCb, cb) + } + // the second table (with rows) did spill + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getColumnarBatch(mockTableDataTypes)) { cb => + TestUtils.compareBatches(expectedCb, cb) + } + } + } + assertResult(0)(devStore.currentSize) + assertResult(0)(hostStore.currentSize) + } + } + } + } + } + } + } + } + } + test("exclusive spill files are deleted when buffer deleted") { testBufferFileDeletion(canShareDiskPaths = false) } @@ -170,6 +340,15 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { testBufferFileDeletion(canShareDiskPaths = true) } + class AlwaysFailingRapidsHostMemoryStore extends RapidsHostMemoryStore(Some(0L)){ + override def createBuffer( + other: RapidsBuffer, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { + None + } + } + private def testBufferFileDeletion(canShareDiskPaths: Boolean): Unit = { val bufferId = MockRapidsBufferId(1, canShareDiskPaths) val bufferPath = bufferId.getDiskPath(null) @@ -178,12 +357,12 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => hostStore.setSpillStore(diskStore) - val (_, handle) = addTableToCatalog(catalog, bufferId, spillPriority) + val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) val bufferPath = handle.id.getDiskPath(null) assert(!bufferPath.exists()) catalog.synchronousSpill(devStore, 0) @@ -200,7 +379,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { } } - private def addTableToCatalog( + private def addContiguousTableToCatalog( catalog: RapidsBufferCatalog, bufferId: RapidsBufferId, spillPriority: Long): (Long, RapidsBufferHandle) = { @@ -216,6 +395,31 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { } } + private def addTableToCatalog( + catalog: RapidsBufferCatalog, + bufferId: RapidsBufferId, + spillPriority: Long): RapidsBufferHandle = { + // store takes ownership of the table + catalog.addTable( + bufferId, + buildTable(), + spillPriority, + false) + } + + private def addZeroRowsTableToCatalog( + catalog: RapidsBufferCatalog, + bufferId: RapidsBufferId, + spillPriority: Long): RapidsBufferHandle = { + val table = buildEmptyTable() + // store takes ownership of the table + catalog.addTable( + bufferId, + table, + spillPriority, + false) + } + case class MockRapidsBufferId( tableId: Int, override val canShareDiskPaths: Boolean) extends RapidsBufferId { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala deleted file mode 100644 index 275d1d99fb2..00000000000 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Copyright (c) 2021-2023, 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 com.nvidia.spark.rapids - -import java.io.File - -import ai.rapids.cudf.{ContiguousTable, CuFile, Table} -import com.nvidia.spark.rapids.Arm.withResource -import org.mockito.ArgumentMatchers -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{spy, times, verify, when} -import org.scalatest.Tag -import org.scalatest.compatible.Assertion -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.sql.rapids.RapidsDiskBlockManager -import org.apache.spark.storage.BlockId - -object GdsTest extends Tag("GdsTest") - -class RapidsGdsStoreSuite extends FunSuiteWithTempDir with MockitoSugar { - - test("single shot spill with shared path", GdsTest) { - println("Trying to load CuFile") - assume(CuFile.libraryLoaded()) - println("DID LOAD") - verifySingleShotSpill(canShareDiskPaths = true) - } - - test("single shot spill with exclusive path", GdsTest) { - assume(CuFile.libraryLoaded()) - verifySingleShotSpill(canShareDiskPaths = false) - } - - test("batch spill", GdsTest) { - assume(CuFile.libraryLoaded()) - - val bufferIds = Array(MockRapidsBufferId(7), MockRapidsBufferId(8), MockRapidsBufferId(9)) - val diskBlockManager = mock[RapidsDiskBlockManager] - val paths = Array( - new File(TEST_FILES_ROOT, s"gdsbuffer-0"), new File(TEST_FILES_ROOT, s"gdsbuffer-1")) - when(diskBlockManager.getFile(any[BlockId]())) - .thenReturn(paths(0)) - .thenReturn(paths(1)) - paths.foreach(f => assert(!f.exists)) - val spillPriority = -7 - val batchWriteBufferSize = 16384 // Holds 2 buffers. - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsGdsStore( - diskBlockManager, batchWriteBufferSize)) { gdsStore => - - devStore.setSpillStore(gdsStore) - assertResult(0)(gdsStore.currentSize) - - val bufferSizes = new Array[Long](bufferIds.length) - val bufferHandles = new Array[RapidsBufferHandle](bufferIds.length) - - bufferIds.zipWithIndex.foreach { case(id, ix) => - val (size, handle) = addTableToCatalog(catalog, id, spillPriority) - catalog.synchronousSpill(devStore, 0) - bufferSizes(ix) = size - bufferHandles(ix) = handle - } - - val totalSize = bufferSizes.sum - assertResult(totalSize)(gdsStore.currentSize) - - assert(paths(0).exists) - assert(!paths(1).exists) - val alignedSize = Math.ceil((bufferSizes(0) + bufferSizes(1)) / 4096d).toLong * 4096 - assertResult(alignedSize)(paths(0).length) - - verify(catalog, times(6)).registerNewBuffer(ArgumentMatchers.any[RapidsBuffer]) - (bufferIds, bufferSizes, bufferHandles).zipped.foreach { (id, size, handle) => - verify(catalog).removeBufferTier( - ArgumentMatchers.eq(id), ArgumentMatchers.eq(StorageTier.DEVICE)) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.GDS)(buffer.storageTier) - assertResult(id)(buffer.id) - assertResult(size)(buffer.getMemoryUsedBytes) - assertResult(spillPriority)(buffer.getSpillPriority) - } - } - - bufferHandles(0).close() - assert(paths(0).exists) - bufferHandles(1).close() - assert(!paths(0).exists) - } - } - } - - private def verifySingleShotSpill(canShareDiskPaths: Boolean): Assertion = { - val bufferId = MockRapidsBufferId(7, canShareDiskPaths) - val path = bufferId.getDiskPath(null) - assert(!path.exists) - val spillPriority = -7 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsGdsStore(mock[RapidsDiskBlockManager], 4096)) { - gdsStore => - devStore.setSpillStore(gdsStore) - assertResult(0)(gdsStore.currentSize) - val (bufferSize, handle) = addTableToCatalog(catalog, bufferId, spillPriority) - catalog.synchronousSpill(devStore, 0) - assertResult(bufferSize)(gdsStore.currentSize) - assert(path.exists) - assertResult(bufferSize)(path.length) - verify(catalog, times(2)).registerNewBuffer(ArgumentMatchers.any[RapidsBuffer]) - verify(catalog).removeBufferTier( - ArgumentMatchers.eq(bufferId), ArgumentMatchers.eq(StorageTier.DEVICE)) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.GDS)(buffer.storageTier) - assertResult(bufferSize)(buffer.getMemoryUsedBytes) - assertResult(bufferId)(buffer.id) - assertResult(spillPriority)(buffer.getSpillPriority) - } - - handle.close() - if (canShareDiskPaths) { - assert(path.exists()) - } else { - assert(!path.exists) - } - } - } - } - - private def addTableToCatalog( - catalog: RapidsBufferCatalog, - bufferId: RapidsBufferId, - spillPriority: Long): (Long, RapidsBufferHandle) = { - withResource(buildContiguousTable()) { ct => - val bufferSize = ct.getBuffer.getLength - // store takes ownership of the table - val handle = catalog.addContiguousTable(bufferId, ct, spillPriority, false) - (bufferSize, handle) - } - } - - /** Build a table of size 7808 bytes. */ - private def buildContiguousTable(): ContiguousTable = { - withResource(new Table.TestBuilder() - .column(Array.fill[String](512)("Lorem Ipsum")) - .build()) { table => - table.contiguousSplit()(0) - } - } - - case class MockRapidsBufferId( - tableId: Int, - override val canShareDiskPaths: Boolean = false) extends RapidsBufferId { - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = - new File(TEST_FILES_ROOT, s"gdsbuffer-$tableId") - } -} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala index 1354cc32bba..153b8da6556 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala @@ -22,8 +22,7 @@ import java.math.RoundingMode import ai.rapids.cudf.{ContiguousTable, Cuda, HostColumnVector, HostMemoryBuffer, Table} import com.nvidia.spark.rapids.Arm._ import org.mockito.{ArgumentCaptor, ArgumentMatchers} -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{never, spy, times, verify, when} +import org.mockito.Mockito.{spy, times, verify} import org.scalatest.funsuite.AnyFunSuite import org.scalatestplus.mockito.MockitoSugar @@ -92,10 +91,10 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val mockStore = mock[RapidsHostMemoryStore] withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => assertResult(0)(hostStore.currentSize) - assertResult(hostStoreMaxSize)(hostStore.numBytesFree) + assertResult(hostStoreMaxSize)(hostStore.numBytesFree.get) devStore.setSpillStore(hostStore) hostStore.setSpillStore(mockStore) @@ -110,13 +109,13 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { catalog.synchronousSpill(devStore, 0) assertResult(bufferSize)(hostStore.currentSize) - assertResult(hostStoreMaxSize - bufferSize)(hostStore.numBytesFree) + assertResult(hostStoreMaxSize - bufferSize)(hostStore.numBytesFree.get) verify(catalog, times(2)).registerNewBuffer(ArgumentMatchers.any[RapidsBuffer]) verify(catalog).removeBufferTier( ArgumentMatchers.eq(handle.id), ArgumentMatchers.eq(StorageTier.DEVICE)) withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.HOST)(buffer.storageTier) - assertResult(bufferSize)(buffer.getMemoryUsedBytes) + assertResult(bufferSize)(buffer.memoryUsedBytes) assertResult(handle.id)(buffer.id) assertResult(spillPriority)(buffer.getSpillPriority) } @@ -130,7 +129,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val mockStore = mock[RapidsHostMemoryStore] withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) hostStore.setSpillStore(mockStore) @@ -165,7 +164,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val mockStore = mock[RapidsHostMemoryStore] withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) hostStore.setSpillStore(mockStore) @@ -204,7 +203,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val (catalog, devStore, hostStore, diskStore) = closeOnExcept(new RapidsDiskStore(bm)) { diskStore => closeOnExcept(new RapidsDeviceMemoryStore()) { devStore => - closeOnExcept(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + closeOnExcept(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) hostStore.setSpillStore(diskStore) val catalog = closeOnExcept( @@ -257,7 +256,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val spillPriority = -10 val hostStoreMaxSize = 1L * 1024 * 1024 val mockStore = mock[RapidsDiskStore] - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -283,7 +282,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -314,7 +313,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -347,7 +346,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -395,7 +394,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -437,7 +436,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -480,7 +479,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) val hostBatch = buildHostBatch() @@ -514,7 +513,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) val hostBatch = buildHostBatchWithDuplicate() @@ -549,58 +548,46 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 256 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - val mockStore = mock[RapidsBufferStore] - val mockBuff = mock[mockStore.RapidsBufferBase] - when(mockBuff.id).thenReturn(new RapidsBufferId { - override val tableId: Int = 0 - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = null - }) - when(mockStore.getMaxSize).thenAnswer(_ => None) - when(mockStore.copyBuffer(any(), any())).thenReturn(mockBuff) - when(mockStore.tier) thenReturn (StorageTier.DISK) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + val spyStore = spy(new RapidsDiskStore(new RapidsDiskBlockManager(new SparkConf()))) + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) - hostStore.setSpillStore(mockStore) + hostStore.setSpillStore(spyStore) var bigHandle: RapidsBufferHandle = null var bigTable = buildContiguousTable(1024 * 1024) - var smallTable = buildContiguousTable(1) closeOnExcept(bigTable) { _ => - closeOnExcept(smallTable) { _ => - // make a copy of the table so we can compare it later to the - // one reconstituted after the spill - val expectedBatch = - withResource(bigTable.getTable.contiguousSplit()) { expectedTable => - GpuColumnVector.from(expectedTable(0).getTable, sparkTypes) - } - withResource(expectedBatch) { _ => - bigHandle = withResource(bigTable) { _ => - catalog.addContiguousTable( - bigTable, - spillPriority) - } // close the bigTable so it can be spilled - bigTable = null - catalog.synchronousSpill(devStore, 0) - verify(mockStore, never()).copyBuffer(ArgumentMatchers.any[RapidsBuffer], - ArgumentMatchers.any[Cuda.Stream]) - withResource(catalog.acquireBuffer(bigHandle)) { buffer => - assertResult(StorageTier.HOST)(buffer.storageTier) - withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => - TestUtils.compareBatches(expectedBatch, actualBatch) - } - } + // make a copy of the table so we can compare it later to the + // one reconstituted after the spill + val expectedBatch = + withResource(bigTable.getTable.contiguousSplit()) { expectedTable => + GpuColumnVector.from(expectedTable(0).getTable, sparkTypes) } - withResource(smallTable) { _ => + withResource(expectedBatch) { _ => + bigHandle = withResource(bigTable) { _ => catalog.addContiguousTable( - smallTable, spillPriority, - false) - } // close the smallTable so it can be spilled - smallTable = null + bigTable, + spillPriority) + } // close the bigTable so it can be spilled + bigTable = null + withResource(catalog.acquireBuffer(bigHandle)) { buffer => + assertResult(StorageTier.DEVICE)(buffer.storageTier) + withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } catalog.synchronousSpill(devStore, 0) val rapidsBufferCaptor: ArgumentCaptor[RapidsBuffer] = ArgumentCaptor.forClass(classOf[RapidsBuffer]) - verify(mockStore).copyBuffer(rapidsBufferCaptor.capture(), + verify(spyStore).copyBuffer( + rapidsBufferCaptor.capture(), + ArgumentMatchers.any[RapidsBufferCatalog], ArgumentMatchers.any[Cuda.Stream]) assertResult(bigHandle.id)(rapidsBufferCaptor.getValue.id) + withResource(catalog.acquireBuffer(bigHandle)) { buffer => + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } } } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala index 3c4f91d0816..00465308e6f 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala @@ -52,7 +52,7 @@ class RegularExpressionParserSuite extends AnyFunSuite { test("not a quantifier") { assert(parse("{1}") === RegexSequence(ListBuffer( - RegexChar('{'), RegexChar('1'),RegexChar('}')))) + RegexChar('{'), RegexChar('1'),RegexEscaped('}')))) } test("nested repetition") { @@ -109,7 +109,7 @@ class RegularExpressionParserSuite extends AnyFunSuite { assert(parse("[a]]") === RegexSequence(ListBuffer( RegexCharacterClass(negated = false, - ListBuffer(RegexChar('a'))), RegexChar(']')))) + ListBuffer(RegexChar('a'))), RegexEscaped(']')))) } test("escaped brackets") { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala index b98c1ef4482..e63c1cce8de 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala @@ -484,12 +484,24 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { doTranspileTest("a\\Z{1,}", expected) } - test("transpile predefined character classes") { + test("transpile predefined character classes Lower") { doTranspileTest("\\p{Lower}", "[a-z]") + } + + test("transpile predefined character classes Alpha") { doTranspileTest("\\p{Alpha}", "[a-zA-Z]") + } + + test("transpile predefined character classes Alnum") { doTranspileTest("\\p{Alnum}", "[a-zA-Z0-9]") - doTranspileTest("\\p{Punct}", "[!\"#$%&'()*+,\\-./:;<=>?@\\^_`{|}~\\[\\]]") - doTranspileTest("\\p{Print}", "[a-zA-Z0-9!\"#$%&'()*+,\\-./:;<=>?@\\^_`{|}~\\[\\]\u0020]") + } + + test("transpile predefined character classes Punct") { + doTranspileTest("\\p{Punct}", "[!\"#$%&'()*+,\\-./:;<=>?@^_`{|}~\\[\\]\\\\]") + } + + test("transpile predefined character classes Print") { + doTranspileTest("\\p{Print}", "[a-zA-Z0-9!\"#$%&'()*+,\\-./:;<=>?@^_`{|}~\\[\\]\\\\\u0020]") } test("transpile with group index to extract") { @@ -672,7 +684,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { doFuzzTest(None, RegexFindMode) } - private def doFuzzTest(validChars: Option[String], mode: RegexMode) { + private def doFuzzTest(validChars: Option[String], mode: RegexMode): Unit = { val r = new EnhancedRandom(new Random(seed = 0L), options = FuzzerOptions(validChars, maxStringLen = 12)) @@ -826,7 +838,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { doStringSplitTest(patterns, data, -1) } - def assertTranspileToSplittableString(patterns: Set[String]) { + def assertTranspileToSplittableString(patterns: Set[String]): Unit = { for (pattern <- patterns) { val transpiler = new CudfRegexTranspiler(RegexSplitMode) transpiler.transpileToSplittableString(pattern) match { @@ -839,7 +851,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { } } - def assertNoTranspileToSplittableString(patterns: Set[String]) { + def assertNoTranspileToSplittableString(patterns: Set[String]): Unit = { for (pattern <- patterns) { val transpiler = new CudfRegexTranspiler(RegexSplitMode) transpiler.transpileToSplittableString(pattern) match { @@ -853,7 +865,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { } } - def doStringSplitTest(patterns: Set[String], data: Seq[String], limit: Int) { + def doStringSplitTest(patterns: Set[String], data: Seq[String], limit: Int): Unit = { for (pattern <- patterns) { val cpu = cpuSplit(pattern, data, limit) val transpiler = new CudfRegexTranspiler(RegexSplitMode) @@ -883,7 +895,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { } private def doAstFuzzTest(validDataChars: Option[String], validPatternChars: String, - mode: RegexMode) { + mode: RegexMode): Unit = { val (data, patterns) = generateDataAndPatterns(validDataChars, validPatternChars, mode) if (mode == RegexReplaceMode) { assertCpuGpuMatchesRegexpReplace(patterns.toSeq, data) @@ -922,7 +934,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { (data, patterns.toSet) } - private def assertCpuGpuMatchesRegexpFind(javaPatterns: Seq[String], input: Seq[String]) = { + private def assertCpuGpuMatchesRegexpFind(javaPatterns: Seq[String], input: Seq[String]): Unit = { for ((javaPattern, patternIndex) <- javaPatterns.zipWithIndex) { val cpu = cpuContains(javaPattern, input) val (cudfPattern, _) = @@ -946,7 +958,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { private def assertCpuGpuMatchesRegexpReplace( javaPatterns: Seq[String], - input: Seq[String]) = { + input: Seq[String]): Unit = { for ((javaPattern, patternIndex) <- javaPatterns.zipWithIndex) { val cpu = cpuReplace(javaPattern, input) val (cudfPattern, replaceString) = @@ -1048,12 +1060,12 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { } } - private def doTranspileTest(pattern: String, expected: String) { + private def doTranspileTest(pattern: String, expected: String): Unit = { val transpiled: String = transpile(pattern, RegexFindMode) assert(toReadableString(transpiled) === toReadableString(expected)) } - private def doTranspileTest(pattern: String, expected: String, groupIdx: Int) { + private def doTranspileTest(pattern: String, expected: String, groupIdx: Int): Unit = { val transpiled: String = transpile(pattern, groupIdx) assert(toReadableString(transpiled) === toReadableString(expected)) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala index d0db74d2e9d..a8e0ad550ea 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala @@ -37,7 +37,7 @@ class RmmSparkRetrySuiteBase extends AnyFunSuite with BeforeAndAfterEach { Rmm.initialize(RmmAllocationMode.CUDA_DEFAULT, null, 512 * 1024 * 1024) } deviceStorage = spy(new RapidsDeviceMemoryStore()) - val hostStore = new RapidsHostMemoryStore(1L * 1024 * 1024) + val hostStore = new RapidsHostMemoryStore(Some(1L * 1024 * 1024)) deviceStorage.setSpillStore(hostStore) val catalog = new RapidsBufferCatalog(deviceStorage, hostStore) // set these against the singleton so we close them later diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ScalableTaskCompletionSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ScalableTaskCompletionSuite.scala new file mode 100644 index 00000000000..cf1dc7bb085 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ScalableTaskCompletionSuite.scala @@ -0,0 +1,65 @@ +/* + * Copyright (c) 2023, 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 com.nvidia.spark.rapids + +import org.scalatest.funsuite.AnyFunSuite +import org.scalatestplus.mockito.MockitoSugar.mock + +import org.apache.spark.TaskContext + +class ScalableTaskCompletionSuite extends AnyFunSuite { + test("ScalableTaskCompletion prevents registering callbacks from a running callback") { + val tc = mock[TaskContext] + var doFail = true + ScalableTaskCompletion.onTaskCompletion(tc, _ => { + if (doFail) { + ScalableTaskCompletion.onTaskCompletion(tc, _ => { + }) + } + }) + assertThrows[IllegalStateException] { + ScalableTaskCompletion.reset() + } + doFail = false + ScalableTaskCompletion.reset() + } + + test("ScalableTaskCompletion prevents calling callbacks from a running callback") { + val tc = mock[TaskContext] + var doFail = true + ScalableTaskCompletion.onTaskCompletion(tc, _ => { + if (doFail) { + ScalableTaskCompletion.reset() + } + }) + assertThrows[IllegalStateException] { + ScalableTaskCompletion.reset() + } + doFail = false + ScalableTaskCompletion.reset() + } + + test("ScalableTaskCompletion succeeds in the good case") { + val tc = mock[TaskContext] + var called = false + ScalableTaskCompletion.onTaskCompletion(tc, _ => { + called = true + }) + ScalableTaskCompletion.reset() + assert(called) + } +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/SparkQueryCompareTestSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/SparkQueryCompareTestSuite.scala index fb8b8e11dcb..798384ae85e 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/SparkQueryCompareTestSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/SparkQueryCompareTestSuite.scala @@ -202,8 +202,6 @@ trait SparkQueryCompareTestSuite extends AnyFunSuite with BeforeAndAfterAll { def withCpuSparkSession[U](f: SparkSession => U, conf: SparkConf = new SparkConf()): U = { val c = conf.clone() .set(RapidsConf.SQL_ENABLED.key, "false") // Just to be sure - // temp work around to unsupported timestamp type - .set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MICROS") withSparkSession(c, f) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/StringFunctionSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/StringFunctionSuite.scala index 3c3933946c5..4f64839acfd 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/StringFunctionSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/StringFunctionSuite.scala @@ -220,6 +220,35 @@ class RegExpUtilsSuite extends AnyFunSuite { } } +class FormatNumberSuite extends SparkQueryCompareTestSuite { + def testFormatNumberDf(session: SparkSession): DataFrame = { + import session.sqlContext.implicits._ + Seq[java.lang.Float]( + -0.0f, + 0.0f, + Float.PositiveInfinity, + Float.NegativeInfinity, + Float.NaN, + 1.0f, + 1.2345f, + 123456789.0f, + 123456789.123456789f, + 0.00123456789f, + 0.0000000123456789f, + 1.0000000123456789f + ).toDF("doubles") + } + + testSparkResultsAreEqual("Test format_number float", + testFormatNumberDf, + conf = new SparkConf().set("spark.rapids.sql.formatNumberFloat.enabled", "true")) { + frame => frame.selectExpr("format_number(doubles, -1)", + "format_number(doubles, 0)", + "format_number(doubles, 1)", + "format_number(doubles, 5)") + } +} + /* * This isn't actually a test. It's just useful to help visualize what's going on when there are * differences present. diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala index 2354544606a..c808a36af85 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{Rmm, RmmAllocationMode, RmmEventHandler, Table} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitTargetSizeInHalf, withRestoreOnRetry, withRetry, withRetryNoSplit} import com.nvidia.spark.rapids.jni.{RetryOOM, RmmSpark, SplitAndRetryOOM} @@ -297,7 +298,7 @@ class WithRetrySuite false } } - private class SimpleCheckpointRestore(var value:Int) extends CheckpointRestore { + private class SimpleCheckpointRestore(var value:Int) extends Retryable { private var lastValue:Int = value def setValue(newVal: Int) = { value = newVal diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala index afc17081afb..4e0325f9048 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala @@ -227,7 +227,7 @@ class RapidsShuffleIteratorSuite extends RapidsShuffleTestHelper { assert(cl.hasNext) assertResult(cb)(cl.next()) assertResult(1)(testMetricsUpdater.totalRemoteBlocksFetched) - assertResult(mockBuffer.getMemoryUsedBytes)(testMetricsUpdater.totalRemoteBytesRead) + assertResult(mockBuffer.memoryUsedBytes)(testMetricsUpdater.totalRemoteBytesRead) assertResult(10)(testMetricsUpdater.totalRowsFetched) } finally { RmmSpark.taskDone(taskId) diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala index a5209e9bd0e..c4a531a8d7d 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala @@ -47,7 +47,7 @@ class SpillableColumnarBatchSuite extends AnyFunSuite { } class MockBuffer(override val id: RapidsBufferId) extends RapidsBuffer { - override def getMemoryUsedBytes: Long = 123 + override val memoryUsedBytes: Long = 123 override def meta: TableMeta = null override val storageTier: StorageTier = StorageTier.DEVICE override def getMemoryBuffer: MemoryBuffer = null diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala index fa23e2f5056..97cd4b02277 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala @@ -16,16 +16,15 @@ package org.apache.spark.sql.rapids.filecache -import com.nvidia.spark.rapids.{RapidsBufferCatalog, RapidsDeviceMemoryStore, SparkQueryCompareTestSuite} +import com.nvidia.spark.rapids.SparkQueryCompareTestSuite import com.nvidia.spark.rapids.shims.GpuBatchScanExec -import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkConf import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.rapids.GpuFileSourceScanExec -class FileCacheIntegrationSuite extends SparkQueryCompareTestSuite with BeforeAndAfterEach { +class FileCacheIntegrationSuite extends SparkQueryCompareTestSuite { import com.nvidia.spark.rapids.GpuMetric._ private val FILE_SPLITS_PARQUET = "file-splits.parquet" @@ -33,17 +32,6 @@ class FileCacheIntegrationSuite extends SparkQueryCompareTestSuite with BeforeAn private val MAP_OF_STRINGS_PARQUET = "map_of_strings.snappy.parquet" private val SCHEMA_CANT_PRUNE_ORC = "schema-cant-prune.orc" - override def beforeEach(): Unit = { - val deviceStorage = new RapidsDeviceMemoryStore() - val catalog = new RapidsBufferCatalog(deviceStorage) - RapidsBufferCatalog.setDeviceStorage(deviceStorage) - RapidsBufferCatalog.setCatalog(catalog) - } - - override def afterEach(): Unit = { - RapidsBufferCatalog.close() - } - def isFileCacheEnabled(conf: SparkConf): Boolean = { // File cache only supported on Spark 3.2+ assumeSpark320orLater @@ -187,29 +175,21 @@ class FileCacheIntegrationSuite extends SparkQueryCompareTestSuite with BeforeAn } private def checkMetricsFullHit(func: () => Option[SparkPlan]): Unit = { - var expectedFooterMisses = 0L - var expectedDataRangeMisses = 0L var metrics: Map[String, SQLMetric] = func().get.metrics var attempts = 0 - while (attempts < 10 && metrics(FILECACHE_DATA_RANGE_MISSES).value > expectedDataRangeMisses) { - expectedDataRangeMisses = metrics(FILECACHE_DATA_RANGE_MISSES).value - expectedFooterMisses = metrics(FILECACHE_FOOTER_MISSES).value - Thread.sleep(100) + while (attempts < 10 && metrics(FILECACHE_DATA_RANGE_MISSES).value > 0) { + Thread.sleep(1000) metrics = func().get.metrics attempts += 1 } assert(metrics(FILECACHE_FOOTER_HITS).value > 0) assert(metrics(FILECACHE_FOOTER_HITS_SIZE).value > 0) - assertResult(expectedFooterMisses)(metrics(FILECACHE_FOOTER_MISSES).value) - if (expectedFooterMisses == 0) { - assertResult(0)(metrics(FILECACHE_FOOTER_MISSES_SIZE).value) - } + assertResult(0)(metrics(FILECACHE_FOOTER_MISSES).value) + assertResult(0)(metrics(FILECACHE_FOOTER_MISSES_SIZE).value) assert(metrics(FILECACHE_DATA_RANGE_HITS).value > 0) assert(metrics(FILECACHE_DATA_RANGE_HITS_SIZE).value > 0) - assertResult(expectedDataRangeMisses)(metrics(FILECACHE_DATA_RANGE_MISSES).value) - if (expectedDataRangeMisses == 0) { - assertResult(0)(metrics(FILECACHE_DATA_RANGE_MISSES_SIZE).value) - } + assertResult(0)(metrics(FILECACHE_DATA_RANGE_MISSES).value) + assertResult(0)(metrics(FILECACHE_DATA_RANGE_MISSES_SIZE).value) assert(metrics.contains(FILECACHE_FOOTER_READ_TIME)) assert(metrics.contains(FILECACHE_DATA_RANGE_READ_TIME)) } diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala index f33c145c622..9a53648f80c 100644 --- a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala +++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala @@ -190,7 +190,7 @@ class DynamicPruningSuite factData(spark) } - private def dimData(spark: SparkSession) { + private def dimData(spark: SparkSession): Unit = { val schema = StructType(Seq( StructField("key", DataTypes.IntegerType, false), StructField("skey", DataTypes.IntegerType, false), @@ -204,7 +204,7 @@ class DynamicPruningSuite )) registerAsParquetTable(spark, df, "dim", None) } - private def factData(spark: SparkSession) { + private def factData(spark: SparkSession): Unit = { val schema = StructType(Seq( StructField("key", DataTypes.IntegerType, false), StructField("skey", DataTypes.IntegerType, false), @@ -218,7 +218,7 @@ class DynamicPruningSuite registerAsParquetTable(spark, df, "fact", Some(List("key", "skey"))) } private def registerAsParquetTable(spark: SparkSession, df: Dataset[Row], name: String, - partitionBy: Option[Seq[String]]) { + partitionBy: Option[Seq[String]]): Unit = { val path = new File(TEST_FILES_ROOT, s"$name.parquet").getAbsolutePath partitionBy match { diff --git a/tests/src/test/spark350/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala b/tests/src/test/spark350/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala new file mode 100644 index 00000000000..b13ebaa8297 --- /dev/null +++ b/tests/src/test/spark350/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala @@ -0,0 +1,116 @@ +/* + * Copyright (c) 2023, 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids + +import ai.rapids.cudf.ColumnVector +import com.nvidia.spark.rapids.Arm._ +import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder +import com.nvidia.spark.rapids.shims.GpuToPrettyString +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.sql.catalyst.expressions.{BoundReference, NamedExpression, ToPrettyString} +import org.apache.spark.sql.types.{ArrayType, DataType, DataTypes, DecimalType, MapType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String + +class ToPrettyStringSuite extends GpuUnitTests { + + private def testDataType(dataType: DataType): Unit = { + val schema = (new StructType) + .add(StructField("a", dataType, true)) + val numRows = 100 + val inputRows = GpuBatchUtilsSuite.createRows(schema, numRows) + val cpuOutput: Array[String] = inputRows.map { + input => + ToPrettyString(BoundReference(0, dataType, true), Some("UTC")) + .eval(input).asInstanceOf[UTF8String].toString() + } + val child = GpuBoundReference(0, dataType, true)(NamedExpression.newExprId, "arg") + val gpuToPrettyStr = GpuToPrettyString(child, Some("UTC")) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val r2cConverter = new GpuRowToColumnConverter(schema) + inputRows.foreach(r2cConverter.convert(_, batchBuilder)) + withResource(batchBuilder.build(numRows)) { columnarBatch => + withResource(GpuColumnVector.from(ColumnVector.fromStrings(cpuOutput: _*), + DataTypes.StringType)) { expected => + checkEvaluation(gpuToPrettyStr, expected, columnarBatch) + } + } + } + } + + test("test show() on booleans") { + testDataType(DataTypes.BooleanType) + } + + test("test show() on bytes") { + testDataType(DataTypes.ByteType) + } + + test("test show() on shorts") { + testDataType(DataTypes.ShortType) + } + + test("test show() on ints") { + testDataType(DataTypes.IntegerType) + } + + test("test show() on longs") { + testDataType(DataTypes.LongType) + } + + test("test show() on floats") { + // This test is expected to fail until https://github.com/NVIDIA/spark-rapids/issues/4204 + // is resolved + assertThrows[TestFailedException](testDataType(DataTypes.FloatType)) + } + + test("test show() on doubles") { + // This test is expected to fail until https://github.com/NVIDIA/spark-rapids/issues/4204 + // is resolved + assertThrows[TestFailedException](testDataType(DataTypes.DoubleType)) + } + + test("test show() on strings") { + testDataType(DataTypes.StringType) + } + + test("test show() on decimals") { + testDataType(DecimalType(8,2)) + } + + test("test show() on binary") { + testDataType(DataTypes.BinaryType) + } + + test("test show() on array") { + testDataType(ArrayType(DataTypes.IntegerType)) + } + + test("test show() on map") { + testDataType(MapType(DataTypes.IntegerType, DataTypes.IntegerType)) + } + + test("test show() on struct") { + testDataType(StructType(Seq(StructField("a", DataTypes.IntegerType), + StructField("b", DataTypes.IntegerType), + StructField("c", DataTypes.IntegerType)))) + } +} diff --git a/tools/generated_files/operatorsScore.csv b/tools/generated_files/operatorsScore.csv index 9582404b8e0..0c583853490 100644 --- a/tools/generated_files/operatorsScore.csv +++ b/tools/generated_files/operatorsScore.csv @@ -110,6 +110,7 @@ Expm1,4 First,4 Flatten,4 Floor,4 +FormatNumber,4 FromUTCTimestamp,4 FromUnixTime,4 GetArrayItem,4 diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv index f5a99f7197e..5b16293d07a 100644 --- a/tools/generated_files/supportedExprs.csv +++ b/tools/generated_files/supportedExprs.csv @@ -208,6 +208,9 @@ Flatten,S,`flatten`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Flatten,S,`flatten`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA Floor,S,`floor`,None,project,input,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA Floor,S,`floor`,None,project,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,x,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,d,NA,NA,NA,PS,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timezone,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA FromUTCTimestamp,S,`from_utc_timestamp`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/udf-compiler/pom.xml b/udf-compiler/pom.xml index 1ff68981472..64021094c7d 100644 --- a/udf-compiler/pom.xml +++ b/udf-compiler/pom.xml @@ -39,7 +39,7 @@ com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} org.scala-lang