Skip to content

Commit

Permalink
Support bucketing write for GPU (#10957)
Browse files Browse the repository at this point in the history
This PR adds the GPU support for the bucketing write.

- React the code of the dynamic partition single writer and concurrent writer to try to reuse the code
   as much as possible, and then add in the bucketing write logic for both of them.
- Update the bucket check during the plan overriding for the write commands, including 
   InsertIntoHadoopFsRelationCommand, CreateDataSourceTableAsSelectCommand,
   InsertIntoHiveTable, CreateHiveTableAsSelectCommand.
- From 330, Spark also supports HiveHash to generate the bucket IDs, in addition to Murmur3Hash.
   So the shim object GpuBucketingUtils is introduced to handle the shim things.
- This change also adds two functions (tagForHiveBucketingWrite and tagForBucketing) to do the
   overriding check for the two hashing functions separately. And the Hive write nodes will fall back 
   to CPU when HiveHash is chosen, because HiveHash is not supported on GPU.


---------

Signed-off-by: Firestarman <firestarmanllc@gmail.com>
  • Loading branch information
firestarman authored Jun 24, 2024
1 parent 7bac3a6 commit 4b44903
Show file tree
Hide file tree
Showing 19 changed files with 896 additions and 749 deletions.
6 changes: 5 additions & 1 deletion integration_tests/src/main/python/asserts.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# Copyright (c) 2020-2023, NVIDIA CORPORATION.
# Copyright (c) 2020-2024, NVIDIA CORPORATION.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
Expand Down Expand Up @@ -101,6 +101,10 @@ def _assert_equal(cpu, gpu, float_check, path):
else:
assert False, "Found unexpected type {} at {}".format(t, path)

def assert_equal_with_local_sort(cpu, gpu):
_sort_locally(cpu, gpu)
assert_equal(cpu, gpu)

def assert_equal(cpu, gpu):
"""Verify that the result from the CPU and the GPU are equal"""
try:
Expand Down
48 changes: 38 additions & 10 deletions integration_tests/src/main/python/orc_write_test.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# Copyright (c) 2020-2023, NVIDIA CORPORATION.
# Copyright (c) 2020-2024, NVIDIA CORPORATION.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
Expand Down Expand Up @@ -209,7 +209,7 @@ def test_write_sql_save_table(spark_tmp_path, orc_gens, ts_type, orc_impl, spark
@pytest.mark.parametrize('codec', ['zlib', 'lzo'])
def test_orc_write_compression_fallback(spark_tmp_path, codec, spark_tmp_table_factory):
gen = TimestampGen()
data_path = spark_tmp_path + '/PARQUET_DATA'
data_path = spark_tmp_path + '/ORC_DATA'
all_confs={'spark.sql.orc.compression.codec': codec, 'spark.rapids.sql.format.orc.write.enabled': True}
assert_gpu_fallback_write(
lambda spark, path: unary_op_df(spark, gen).coalesce(1).write.format("orc").mode('overwrite').option("path", path).saveAsTable(spark_tmp_table_factory.get()),
Expand All @@ -218,17 +218,45 @@ def test_orc_write_compression_fallback(spark_tmp_path, codec, spark_tmp_table_f
'DataWritingCommandExec',
conf=all_confs)

@ignore_order
@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec')
def test_buckets_write_fallback(spark_tmp_path, spark_tmp_table_factory):
@ignore_order(local=True)
def test_buckets_write_round_trip(spark_tmp_path, spark_tmp_table_factory):
data_path = spark_tmp_path + '/ORC_DATA'
gen_list = [["id", int_gen], ["data", long_gen]]
assert_gpu_and_cpu_writes_are_equal_collect(
lambda spark, path: gen_df(spark, gen_list).selectExpr("id % 100 as b_id", "data").write
.bucketBy(4, "b_id").format('orc').mode('overwrite').option("path", path)
.saveAsTable(spark_tmp_table_factory.get()),
lambda spark, path: spark.read.orc(path),
data_path,
conf={'spark.rapids.sql.format.orc.write.enabled': True})

@ignore_order(local=True)
@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec, SortExec')
def test_buckets_write_fallback_unsupported_types(spark_tmp_path, spark_tmp_table_factory):
data_path = spark_tmp_path + '/ORC_DATA'
gen_list = [["id", binary_gen], ["data", long_gen]]
assert_gpu_fallback_write(
lambda spark, path: spark.range(10e4).write.bucketBy(4, "id").sortBy("id").format('orc').mode('overwrite').option("path", path).saveAsTable(spark_tmp_table_factory.get()),
lambda spark, path: spark.read.orc(path),
data_path,
'DataWritingCommandExec',
conf = {'spark.rapids.sql.format.orc.write.enabled': True})
lambda spark, path: gen_df(spark, gen_list).selectExpr("id as b_id", "data").write
.bucketBy(4, "b_id").format('orc').mode('overwrite').option("path", path)
.saveAsTable(spark_tmp_table_factory.get()),
lambda spark, path: spark.read.orc(path),
data_path,
'DataWritingCommandExec',
conf={'spark.rapids.sql.format.orc.write.enabled': True})

@ignore_order(local=True)
def test_partitions_and_buckets_write_round_trip(spark_tmp_path, spark_tmp_table_factory):
data_path = spark_tmp_path + '/ORC_DATA'
gen_list = [["id", int_gen], ["data", long_gen]]
assert_gpu_and_cpu_writes_are_equal_collect(
lambda spark, path: gen_df(spark, gen_list)
.selectExpr("id % 5 as b_id", "id % 10 as p_id", "data").write
.partitionBy("p_id")
.bucketBy(4, "b_id").format('orc').mode('overwrite').option("path", path)
.saveAsTable(spark_tmp_table_factory.get()),
lambda spark, path: spark.read.orc(path),
data_path,
conf={'spark.rapids.sql.format.orc.write.enabled': True})

@ignore_order
@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec')
Expand Down
79 changes: 72 additions & 7 deletions integration_tests/src/main/python/parquet_write_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -409,16 +409,81 @@ def test_parquet_writeLegacyFormat_fallback(spark_tmp_path, spark_tmp_table_fact
'DataWritingCommandExec',
conf=all_confs)

@ignore_order
@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec')
def test_buckets_write_fallback(spark_tmp_path, spark_tmp_table_factory):
@ignore_order(local=True)
def test_buckets_write_round_trip(spark_tmp_path, spark_tmp_table_factory):
data_path = spark_tmp_path + '/PARQUET_DATA'
gen_list = [["id", int_gen], ["data", long_gen]]
assert_gpu_and_cpu_writes_are_equal_collect(
lambda spark, path: gen_df(spark, gen_list).selectExpr("id % 100 as b_id", "data").write
.bucketBy(4, "b_id").format('parquet').mode('overwrite').option("path", path)
.saveAsTable(spark_tmp_table_factory.get()),
lambda spark, path: spark.read.parquet(path),
data_path,
conf=writer_confs)


def test_buckets_write_correctness(spark_tmp_path, spark_tmp_table_factory):
cpu_path = spark_tmp_path + '/PARQUET_DATA/CPU'
gpu_path = spark_tmp_path + '/PARQUET_DATA/GPU'
gen_list = [["id", int_gen], ["data", long_gen]]
num_buckets = 4

def do_bucketing_write(spark, path):
df = gen_df(spark, gen_list).selectExpr("id % 100 as b_id", "data")
df.write.bucketBy(num_buckets, "b_id").format('parquet').mode('overwrite') \
.option("path", path).saveAsTable(spark_tmp_table_factory.get())

def read_single_bucket(path, bucket_id):
# Bucket Id string format: f"_$id%05d" + ".c$fileCounter%03d".
# fileCounter is always 0 in this test. For example '_00002.c000' is for
# bucket id being 2.
# We leverage this bucket segment in the file path to filter rows belong
# to a bucket.
bucket_segment = '_' + "{}".format(bucket_id).rjust(5, '0') + '.c000'
return with_cpu_session(
lambda spark: spark.read.parquet(path)
.withColumn('file_name', f.input_file_name())
.filter(f.col('file_name').contains(bucket_segment))
.selectExpr('b_id', 'data') # need to drop the file_name column for comparison.
.collect())

with_cpu_session(lambda spark: do_bucketing_write(spark, cpu_path), writer_confs)
with_gpu_session(lambda spark: do_bucketing_write(spark, gpu_path), writer_confs)
cur_bucket_id = 0
while cur_bucket_id < num_buckets:
# Verify the result bucket by bucket
ret_cpu = read_single_bucket(cpu_path, cur_bucket_id)
ret_gpu = read_single_bucket(gpu_path, cur_bucket_id)
assert_equal_with_local_sort(ret_cpu, ret_gpu)
cur_bucket_id += 1

@ignore_order(local=True)
@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec, SortExec')
def test_buckets_write_fallback_unsupported_types(spark_tmp_path, spark_tmp_table_factory):
data_path = spark_tmp_path + '/PARQUET_DATA'
gen_list = [["id", binary_gen], ["data", long_gen]]
assert_gpu_fallback_write(
lambda spark, path: spark.range(10e4).write.bucketBy(4, "id").sortBy("id").format('parquet').mode('overwrite').option("path", path).saveAsTable(spark_tmp_table_factory.get()),
lambda spark, path: spark.read.parquet(path),
data_path,
'DataWritingCommandExec')
lambda spark, path: gen_df(spark, gen_list).selectExpr("id as b_id", "data").write
.bucketBy(4, "b_id").format('parquet').mode('overwrite').option("path", path)
.saveAsTable(spark_tmp_table_factory.get()),
lambda spark, path: spark.read.parquet(path),
data_path,
'DataWritingCommandExec',
conf=writer_confs)

@ignore_order(local=True)
def test_partitions_and_buckets_write_round_trip(spark_tmp_path, spark_tmp_table_factory):
data_path = spark_tmp_path + '/PARQUET_DATA'
gen_list = [["id", int_gen], ["data", long_gen]]
assert_gpu_and_cpu_writes_are_equal_collect(
lambda spark, path: gen_df(spark, gen_list)
.selectExpr("id % 5 as b_id", "id % 10 as p_id", "data").write
.partitionBy("p_id")
.bucketBy(4, "b_id").format('parquet').mode('overwrite').option("path", path)
.saveAsTable(spark_tmp_table_factory.get()),
lambda spark, path: spark.read.parquet(path),
data_path,
conf=writer_confs)

@ignore_order
@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec')
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
/*
* Copyright (c) 2020-2023, NVIDIA CORPORATION.
* Copyright (c) 2020-2024, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
Expand All @@ -21,7 +21,7 @@ import com.nvidia.spark.rapids.Arm.withResource
import com.nvidia.spark.rapids.shims.ShimExpression

import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.rapids.GpuMurmur3Hash
import org.apache.spark.sql.rapids.{GpuMurmur3Hash, GpuPmod}
import org.apache.spark.sql.types.{DataType, IntegerType}
import org.apache.spark.sql.vectorized.ColumnarBatch

Expand Down Expand Up @@ -59,6 +59,10 @@ abstract class GpuHashPartitioningBase(expressions: Seq[Expression], numPartitio
sliceInternalGpuOrCpuAndClose(numRows, partitionIndexes, partitionColumns)
}
}

def partitionIdExpression: GpuExpression = GpuPmod(
GpuMurmur3Hash(expressions, GpuHashPartitioningBase.DEFAULT_HASH_SEED),
GpuLiteral(numPartitions))
}

object GpuHashPartitioningBase {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -322,10 +322,11 @@ final class InsertIntoHadoopFsRelationCommandMeta(
private var fileFormat: Option[ColumnarFileFormat] = None

override def tagSelfForGpuInternal(): Unit = {
if (cmd.bucketSpec.isDefined) {
willNotWorkOnGpu("bucketing is not supported")
if (GpuBucketingUtils.isHiveHashBucketing(cmd.options)) {
GpuBucketingUtils.tagForHiveBucketingWrite(this, cmd.bucketSpec, cmd.outputColumns, false)
} else {
BucketIdMetaUtils.tagForBucketingWrite(this, cmd.bucketSpec, cmd.outputColumns)
}

val spark = SparkSession.active
val formatCls = cmd.fileFormat.getClass
fileFormat = if (formatCls == classOf[CSVFileFormat]) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import com.google.common.base.Charsets
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.Arm.withResource
import com.nvidia.spark.rapids.jni.CastStrings
import com.nvidia.spark.rapids.shims.GpuBucketingUtils
import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}

import org.apache.spark.internal.Logging
Expand All @@ -43,9 +44,8 @@ object GpuHiveFileFormat extends Logging {
def tagGpuSupport(meta: GpuInsertIntoHiveTableMeta): Option[ColumnarFileFormat] = {
val insertCmd = meta.wrapped
// Bucketing write
if (insertCmd.table.bucketSpec.isDefined) {
meta.willNotWorkOnGpu("bucketed tables are not supported yet")
}
GpuBucketingUtils.tagForHiveBucketingWrite(meta, insertCmd.table.bucketSpec,
insertCmd.outputColumns, false)

// Infer the file format from the serde string, similar as what Spark does in
// RelationConversions for Hive.
Expand Down
Loading

0 comments on commit 4b44903

Please sign in to comment.