diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md
index 64ad8723a7b..8c0a4682e74 100644
--- a/docs/additional-functionality/advanced_configs.md
+++ b/docs/additional-functionality/advanced_configs.md
@@ -123,6 +123,9 @@ Name | Description | Default Value | Applicable at
spark.rapids.sql.format.parquet.reader.type|Sets the Parquet reader type. We support different types that are optimized for different environments. The original Spark style reader can be selected by setting this to PERFILE which individually reads and copies files to the GPU. Loading many small files individually has high overhead, and using either COALESCING or MULTITHREADED is recommended instead. The COALESCING reader is good when using a local file system where the executors are on the same nodes or close to the nodes the data is being read on. This reader coalesces all the files assigned to a task into a single host buffer before sending it down to the GPU. It copies blocks from a single file into a host buffer in separate threads in parallel, see spark.rapids.sql.multiThreadedRead.numThreads. MULTITHREADED is good for cloud environments where you are reading from a blobstore that is totally separate and likely has a higher I/O read cost. Many times the cloud environments also get better throughput when you have multiple readers in parallel. This reader uses multiple threads to read each file in parallel and each file is sent to the GPU separately. This allows the CPU to keep reading while GPU is also doing work. See spark.rapids.sql.multiThreadedRead.numThreads and spark.rapids.sql.format.parquet.multiThreadedRead.maxNumFilesParallel to control the number of threads and amount of memory used. By default this is set to AUTO so we select the reader we think is best. This will either be the COALESCING or the MULTITHREADED based on whether we think the file is in the cloud. See spark.rapids.cloudSchemes.|AUTO|Runtime
spark.rapids.sql.format.parquet.write.enabled|When set to false disables parquet output acceleration|true|Runtime
spark.rapids.sql.format.parquet.writer.int96.enabled|When set to false, disables accelerated parquet write if the spark.sql.parquet.outputTimestampType is set to INT96|true|Runtime
+spark.rapids.sql.format.sequencefile.multiThreadedRead.maxNumFilesParallel|A limit on the maximum number of files per task processed in parallel on the CPU side before the file is sent to the GPU. This affects the amount of host memory used when reading the files in parallel. Used with MULTITHREADED reader, see spark.rapids.sql.format.sequencefile.reader.type.|2147483647|Runtime
+spark.rapids.sql.format.sequencefile.rddScan.physicalReplace.enabled|Enable physical-plan replacement for SequenceFile RDD scans (RDDScanExec) when the lineage can be safely identified as a simple SequenceFile scan with BinaryType key/value output. Unsupported or risky cases automatically remain on CPU.|true|Runtime
+spark.rapids.sql.format.sequencefile.reader.type|Sets the SequenceFile reader type. Since SequenceFile decoding happens on the CPU (using Hadoop's SequenceFile.Reader), COALESCING mode is not supported and will throw an exception. MULTITHREADED uses multiple threads to read files in parallel, utilizing multiple CPU cores for I/O and decoding. MULTITHREADED is recommended when reading many files as it allows the CPU to keep reading while GPU is also doing work. See spark.rapids.sql.multiThreadedRead.numThreads and spark.rapids.sql.format.sequencefile.multiThreadedRead.maxNumFilesParallel to control the number of threads and amount of memory used. AUTO is kept for compatibility, but MULTITHREADED is the default for SequenceFile.|MULTITHREADED|Runtime
spark.rapids.sql.formatNumberFloat.enabled|format_number with floating point types on the GPU returns results that have a different precision than the default results of Spark.|true|Runtime
spark.rapids.sql.hasExtendedYearValues|Spark 3.2.0+ extended parsing of years in dates and timestamps to support the full range of possible values. Prior to this it was limited to a positive 4 digit year. The Accelerator does not support the extended range yet. This config indicates if your data includes this extended range or not, or if you don't care about getting the correct values on values with the extended range.|true|Runtime
spark.rapids.sql.hashOptimizeSort.enabled|Whether sorts should be inserted after some hashed operations to improve output ordering. This can improve output file sizes when saving to columnar formats.|false|Runtime
@@ -473,6 +476,7 @@ Name | Description | Default Value | Notes
spark.rapids.sql.exec.ProjectExec|The backend for most select, withColumn and dropColumn statements|true|None|
spark.rapids.sql.exec.RangeExec|The backend for range operator|true|None|
spark.rapids.sql.exec.SampleExec|The backend for the sample operator|true|None|
+spark.rapids.sql.exec.SerializeFromObjectExec|Serialize object rows to binary columns for SequenceFile RDD scans|true|None|
spark.rapids.sql.exec.SortExec|The backend for the sort operator|true|None|
spark.rapids.sql.exec.SubqueryBroadcastExec|Plan to collect and transform the broadcast key values|true|None|
spark.rapids.sql.exec.TakeOrderedAndProjectExec|Take the first limit elements as defined by the sortOrder, and do projection if needed|true|None|
diff --git a/docs/supported_ops.md b/docs/supported_ops.md
index 888f4fa807c..975f807d2be 100644
--- a/docs/supported_ops.md
+++ b/docs/supported_ops.md
@@ -421,6 +421,32 @@ Accelerator supports are described below.
S |
+| SerializeFromObjectExec |
+Serialize object rows to binary columns for SequenceFile RDD scans |
+None |
+Input/Output |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+S |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+
+
| SortExec |
The backend for the sort operator |
None |
@@ -499,32 +525,6 @@ Accelerator supports are described below.
NS |
-| UnionExec |
-The backend for the union operator |
-None |
-Input/Output |
-S |
-S |
-S |
-S |
-S |
-S |
-S |
-S |
-PS UTC is only supported TZ for TIMESTAMP |
-S |
-S |
-S |
-NS |
-NS |
-PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
-PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
-PS unionByName will not optionally impute nulls for missing struct fields when the column is a struct and there are non-overlapping fields; UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
-NS |
-NS |
-NS |
-
-
| Executor |
Description |
Notes |
@@ -551,6 +551,32 @@ Accelerator supports are described below.
YEARMONTH |
+| UnionExec |
+The backend for the union operator |
+None |
+Input/Output |
+S |
+S |
+S |
+S |
+S |
+S |
+S |
+S |
+PS UTC is only supported TZ for TIMESTAMP |
+S |
+S |
+S |
+NS |
+NS |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+PS unionByName will not optionally impute nulls for missing struct fields when the column is a struct and there are non-overlapping fields; UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+NS |
+NS |
+NS |
+
+
| AQEShuffleReadExec |
A wrapper of shuffle query stage |
None |
@@ -915,6 +941,32 @@ Accelerator supports are described below.
S |
+| Executor |
+Description |
+Notes |
+Param(s) |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
| BroadcastHashJoinExec |
Implementation of join using broadcast data |
None |
@@ -1010,32 +1062,6 @@ Accelerator supports are described below.
NS |
-| Executor |
-Description |
-Notes |
-Param(s) |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
| BroadcastNestedLoopJoinExec |
Implementation of join using brute force. Full outer joins and joins where the broadcast side matches the join side (e.g.: LeftOuter with left broadcast) are not supported |
None |
@@ -1301,6 +1327,32 @@ Accelerator supports are described below.
NS |
+| Executor |
+Description |
+Notes |
+Param(s) |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
| AggregateInPandasExec |
The backend for an Aggregation Pandas UDF. This accelerates the data transfer between the Java process and the Python process. It also supports scheduling GPU resources for the Python process when enabled. |
None |
@@ -1405,32 +1457,6 @@ Accelerator supports are described below.
NS |
-| Executor |
-Description |
-Notes |
-Param(s) |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
| MapInPandasExec |
The backend for Map Pandas Iterator UDF. Accelerates the data transfer between the Java process and the Python process. It also supports scheduling GPU resources for the Python process when enabled. |
None |
@@ -8267,7 +8293,7 @@ are limited.
|
|
|
-PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types CALENDAR, UDT, DAYTIME, YEARMONTH |
|
|
|
@@ -8290,7 +8316,7 @@ are limited.
|
|
|
-PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types CALENDAR, UDT, DAYTIME, YEARMONTH |
|
|
|
diff --git a/integration_tests/src/main/python/sequencefile_test.py b/integration_tests/src/main/python/sequencefile_test.py
new file mode 100644
index 00000000000..31a968c7ee4
--- /dev/null
+++ b/integration_tests/src/main/python/sequencefile_test.py
@@ -0,0 +1,342 @@
+# Copyright (c) 2026, 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.
+
+"""
+Integration tests for SequenceFile RDD reads with RAPIDS plugin enabled.
+"""
+
+import pytest
+import os
+import struct
+
+from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error
+from data_gen import *
+from marks import *
+from pyspark.sql.types import *
+from spark_session import with_cpu_session, is_databricks_runtime
+
+# Reader types supported by SequenceFile (COALESCING is not supported)
+# AUTO is accepted for compatibility and resolves to MULTITHREADED.
+sequencefile_reader_types = ['AUTO', 'MULTITHREADED']
+
+
+def write_sequencefile_with_rdd(spark, data_path, payloads):
+ """
+ Write an uncompressed SequenceFile using Spark's RDD saveAsNewAPIHadoopFile method.
+ payloads: list of byte arrays to be written as values (keys will be incrementing integers).
+
+ This writes actual BytesWritable key/value pairs.
+ """
+ sc = spark.sparkContext
+
+ # Create (key, value) pairs where key is 4-byte big-endian integer
+ # Convert to bytearray for proper BytesWritable serialization
+ records = [(bytearray(struct.pack('>I', idx)), bytearray(payload))
+ for idx, payload in enumerate(payloads)]
+
+ # Create RDD and save as SequenceFile using Hadoop API
+ rdd = sc.parallelize(records, 1)
+
+ # Use saveAsNewAPIHadoopFile with BytesWritable key/value classes
+ rdd.saveAsNewAPIHadoopFile(
+ data_path,
+ "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
+ "org.apache.hadoop.io.BytesWritable",
+ "org.apache.hadoop.io.BytesWritable"
+ )
+
+
+def read_sequencefile_via_rdd(spark, data_path):
+ """
+ Read a SequenceFile using the RDD path.
+ Reads data through the RDD SequenceFile path.
+ """
+ sc = spark.sparkContext
+ rdd = sc.newAPIHadoopFile(
+ data_path,
+ "org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat",
+ "org.apache.hadoop.io.BytesWritable",
+ "org.apache.hadoop.io.BytesWritable"
+ )
+
+ # Map to extract raw bytes (BytesWritable has length prefix)
+ def extract_bytes(kv):
+ k, v = kv
+ # BytesWritable stores data after a 4-byte length prefix
+ return (bytes(k[4:]) if len(k) > 4 else bytes(k),
+ bytes(v[4:]) if len(v) > 4 else bytes(v))
+
+ mapped_rdd = rdd.map(extract_bytes)
+ # Use explicit schema to avoid schema inference failure on empty RDD
+ schema = StructType([
+ StructField("key", BinaryType(), True),
+ StructField("value", BinaryType(), True)
+ ])
+ return spark.createDataFrame(mapped_rdd, schema)
+
+
+def read_sequencefile_value_only(spark, data_path):
+ """
+ Read only the value column from a SequenceFile (common pattern for protobuf payloads).
+ """
+ sc = spark.sparkContext
+ rdd = sc.newAPIHadoopFile(
+ data_path,
+ "org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat",
+ "org.apache.hadoop.io.BytesWritable",
+ "org.apache.hadoop.io.BytesWritable"
+ )
+
+ def extract_value(kv):
+ _, v = kv
+ # BytesWritable stores data after a 4-byte length prefix
+ return (bytes(v[4:]) if len(v) > 4 else bytes(v),)
+
+ mapped_rdd = rdd.map(extract_value)
+ # Use explicit schema to avoid schema inference failure on empty RDD
+ schema = StructType([
+ StructField("value", BinaryType(), True)
+ ])
+ return spark.createDataFrame(mapped_rdd, schema)
+
+
+def write_corrupt_file(path, payload=b'not-a-sequence-file'):
+ """Write a non-SequenceFile payload to simulate a corrupt input file."""
+ os.makedirs(os.path.dirname(path), exist_ok=True)
+ with open(path, "wb") as f:
+ f.write(payload)
+
+
+# ============================================================================
+# Basic Read Tests
+# ============================================================================
+
+@pytest.mark.parametrize('reader_type', sequencefile_reader_types)
+def test_basic_read(spark_tmp_path, reader_type):
+ """Test basic SequenceFile reading via RDD conversion."""
+ data_path = spark_tmp_path + '/SEQFILE_DATA'
+
+ # Write test data using CPU
+ payloads = [
+ b'\x01\x02\x03',
+ b'hello world',
+ b'\xff' * 10
+ ]
+ with_cpu_session(lambda spark: write_sequencefile_with_rdd(spark, data_path, payloads))
+
+ all_confs = {
+ 'spark.rapids.sql.format.sequencefile.reader.type': reader_type
+ }
+
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark: read_sequencefile_via_rdd(spark, data_path),
+ conf=all_confs)
+
+
+@pytest.mark.parametrize('reader_type', sequencefile_reader_types)
+def test_read_value_only(spark_tmp_path, reader_type):
+ """Test reading only the value column."""
+ data_path = spark_tmp_path + '/SEQFILE_DATA'
+
+ payloads = [b'value1', b'value2', b'value3']
+ with_cpu_session(lambda spark: write_sequencefile_with_rdd(spark, data_path, payloads))
+
+ all_confs = {
+ 'spark.rapids.sql.format.sequencefile.reader.type': reader_type
+ }
+
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark: read_sequencefile_value_only(spark, data_path),
+ conf=all_confs)
+
+
+# ============================================================================
+# Empty File Tests
+# ============================================================================
+
+@pytest.mark.parametrize('reader_type', sequencefile_reader_types)
+def test_empty_file(spark_tmp_path, reader_type):
+ """Test reading an empty SequenceFile."""
+ data_path = spark_tmp_path + '/SEQFILE_DATA'
+
+ # Write empty file
+ with_cpu_session(lambda spark: write_sequencefile_with_rdd(spark, data_path, []))
+
+ all_confs = {
+ 'spark.rapids.sql.format.sequencefile.reader.type': reader_type
+ }
+
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark: read_sequencefile_via_rdd(spark, data_path),
+ conf=all_confs)
+
+
+# ============================================================================
+# Large Data Tests
+# ============================================================================
+
+@pytest.mark.parametrize('reader_type', sequencefile_reader_types)
+def test_large_batch(spark_tmp_path, reader_type):
+ """Test reading many records to verify batch handling."""
+ data_path = spark_tmp_path + '/SEQFILE_DATA'
+
+ # Create many records
+ num_records = 1000
+ payloads = [f'record_{i}_with_some_data'.encode() for i in range(num_records)]
+ with_cpu_session(lambda spark: write_sequencefile_with_rdd(spark, data_path, payloads))
+
+ all_confs = {
+ 'spark.rapids.sql.format.sequencefile.reader.type': reader_type
+ }
+
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark: read_sequencefile_via_rdd(spark, data_path),
+ conf=all_confs)
+
+
+@pytest.mark.parametrize('reader_type', sequencefile_reader_types)
+def test_large_records(spark_tmp_path, reader_type):
+ """Test reading records with large values."""
+ data_path = spark_tmp_path + '/SEQFILE_DATA'
+
+ # Create records with varying sizes, including some large ones
+ payloads = [b'x' * (1024 * i) for i in range(1, 11)] # 1KB to 10KB
+ with_cpu_session(lambda spark: write_sequencefile_with_rdd(spark, data_path, payloads))
+
+ all_confs = {
+ 'spark.rapids.sql.format.sequencefile.reader.type': reader_type
+ }
+
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark: read_sequencefile_via_rdd(spark, data_path),
+ conf=all_confs)
+
+
+def test_multithreaded_reader_combine_mode_correctness(spark_tmp_path):
+ """Test MULTITHREADED reader combine mode with many small files."""
+ base_path = spark_tmp_path + '/SEQFILE_COMBINE_DATA'
+ payload_sets = [
+ [b'a1', b'a2', b'a3'],
+ [b'b1', b'b2'],
+ [b'c1', b'c2', b'c3', b'c4']
+ ]
+
+ def write_all_files(spark):
+ for idx, payloads in enumerate(payload_sets):
+ write_sequencefile_with_rdd(spark, f'{base_path}/part_{idx}', payloads)
+
+ with_cpu_session(write_all_files)
+
+ all_confs = {
+ 'spark.rapids.sql.format.sequencefile.reader.type': 'MULTITHREADED',
+ # Force combine behavior in multithreaded reader.
+ 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes': '1',
+ 'spark.rapids.sql.reader.multithreaded.combine.waitTime': '1',
+ 'spark.rapids.sql.files.maxPartitionBytes': str(1 << 20),
+ }
+
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark: read_sequencefile_value_only(spark, base_path + '/*'),
+ conf=all_confs)
+
+
+# ============================================================================
+# Configuration Tests
+# ============================================================================
+
+def test_conversion_disabled_by_default(spark_tmp_path):
+ """Test that RDD conversion is disabled by default."""
+ data_path = spark_tmp_path + '/SEQFILE_DATA'
+
+ payloads = [b'test']
+ with_cpu_session(lambda spark: write_sequencefile_with_rdd(spark, data_path, payloads))
+
+ # Without enabling conversion, this should still work via the original RDD path
+ # (no conversion happens, just regular RDD execution)
+ all_confs = {}
+
+ # This should work - the RDD path still functions, just without conversion
+ assert_gpu_and_cpu_are_equal_collect(
+ lambda spark: read_sequencefile_via_rdd(spark, data_path),
+ conf=all_confs)
+
+
+# ============================================================================
+# Binary Data Tests
+# ============================================================================
+
+@pytest.mark.parametrize('reader_type', sequencefile_reader_types)
+def test_binary_data(spark_tmp_path, reader_type):
+ """Test reading various binary data patterns."""
+ data_path = spark_tmp_path + '/SEQFILE_DATA'
+
+ payloads = [
+ bytes(range(256)), # All byte values
+ b'\x00' * 100, # Nulls
+ b'\xff' * 100, # All 1s
+ b''.join(struct.pack(' new GpuMapInPandasExecMeta(mapPy, conf, p, r)),
+ exec[SerializeFromObjectExec](
+ "Serialize object rows to binary columns for SequenceFile RDD scans",
+ ExecChecks(TypeSig.all, TypeSig.BINARY),
+ (sfo, conf, p, r) => new GpuSequenceFileSerializeFromObjectExecMeta(sfo, conf, p, r)),
exec[InMemoryTableScanExec](
"Implementation of InMemoryTableScanExec to use GPU accelerated caching",
ExecChecks((TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + TypeSig.ARRAY +
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPostHocResolutionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPostHocResolutionOverrides.scala
index eda79ca3e96..a3c7dfd015c 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPostHocResolutionOverrides.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuPostHocResolutionOverrides.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2025, NVIDIA CORPORATION.
+ * Copyright (c) 2025-2026, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSequenceFileSerializeFromObjectExecMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSequenceFileSerializeFromObjectExecMeta.scala
new file mode 100644
index 00000000000..e363ded2cfb
--- /dev/null
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSequenceFileSerializeFromObjectExecMeta.scala
@@ -0,0 +1,290 @@
+/*
+ * Copyright (c) 2026, 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 scala.annotation.tailrec
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileInputFormat => OldFileInputFormat}
+import org.apache.hadoop.mapreduce.lib.input.{
+ FileInputFormat => NewFileInputFormat}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.{HadoopRDD, NewHadoopRDD, RDD}
+import org.apache.spark.sql.execution.{ExternalRDDScanExec, SerializeFromObjectExec, SparkPlan}
+import org.apache.spark.sql.rapids.GpuSequenceFileSerializeFromObjectExec
+import org.apache.spark.sql.types.BinaryType
+
+class GpuSequenceFileSerializeFromObjectExecMeta(
+ plan: SerializeFromObjectExec,
+ conf: RapidsConf,
+ parent: Option[RapidsMeta[_, _, _]],
+ rule: DataFromReplacementRule)
+ extends SparkPlanMeta[SerializeFromObjectExec](plan, conf, parent, rule) with Logging {
+
+ // Override childExprs to empty: we replace the entire SerializeFromObjectExec including its
+ // serializer expressions, so we don't need them to be individually GPU-compatible.
+ // Without this, the framework's canExprTreeBeReplaced check rejects us because the
+ // serializer contains object-related expressions (Invoke, StaticInvoke, etc.) that are
+ // not registered as GPU expressions.
+ override val childExprs: Seq[BaseExprMeta[_]] = Seq.empty
+
+ // Similarly, the child ExternalRDDScanExec is not a registered GPU exec, so we skip
+ // wrapping child plans to avoid "not all children can be replaced" cascading failures.
+ override val childPlans: Seq[SparkPlanMeta[SparkPlan]] = Seq.empty
+
+ private var scanAnalysis: Option[GpuSequenceFileSerializeFromObjectExecMeta.SequenceFileScanAnalysis] =
+ None
+
+ override def tagPlanForGpu(): Unit = {
+ if (!conf.isSequenceFileRDDPhysicalReplaceEnabled) {
+ willNotWorkOnGpu("SequenceFile RDD physical replacement is disabled")
+ return
+ }
+ val outOk = wrapped.output.nonEmpty && wrapped.output.forall { a =>
+ val isKeyOrValue = a.name.equalsIgnoreCase("key") ||
+ a.name.equalsIgnoreCase("value")
+ isKeyOrValue && a.dataType == BinaryType
+ }
+ if (!outOk) {
+ willNotWorkOnGpu("SequenceFile object replacement only supports BinaryType key/value output")
+ return
+ }
+ wrapped.child match {
+ case e: ExternalRDDScanExec[_] =>
+ if (!GpuSequenceFileSerializeFromObjectExecMeta.isSimpleSequenceFileRDD(e.rdd)) {
+ willNotWorkOnGpu("RDD lineage is not a simple SequenceFile scan")
+ return
+ }
+ val analysis = GpuSequenceFileSerializeFromObjectExecMeta.analyzeSequenceFileScan(
+ e, e.rdd.context.hadoopConfiguration)
+ scanAnalysis = Some(analysis)
+ if (analysis.hasCompressedInput) {
+ willNotWorkOnGpu("Compressed SequenceFile input falls back to CPU")
+ }
+ case _ =>
+ willNotWorkOnGpu("SerializeFromObject child is not ExternalRDDScanExec")
+ return
+ }
+ }
+
+ override def convertToGpu(): GpuExec = {
+ val analysis = scanAnalysis.getOrElse {
+ val sourceScan = wrapped.child.asInstanceOf[ExternalRDDScanExec[_]]
+ GpuSequenceFileSerializeFromObjectExecMeta.analyzeSequenceFileScan(
+ sourceScan, sourceScan.rdd.context.hadoopConfiguration)
+ }
+ GpuSequenceFileSerializeFromObjectExec(
+ wrapped.output,
+ wrapped.child,
+ TargetSize(conf.gpuTargetBatchSizeBytes),
+ analysis.inputPaths)(conf)
+ }
+
+ override def convertToCpu(): SparkPlan = wrapped
+}
+
+/**
+ * Utilities for identifying simple SequenceFile RDD scans and extracting their input paths.
+ *
+ * This code uses reflection against `NewHadoopRDD`/`HadoopRDD` internals because the generic RDD
+ * lineage API does not expose enough structured metadata for safe physical replacement.
+ * Those internal fields/methods may change across Spark/Hadoop versions and can also be
+ * restricted by JDK module access settings. Any reflection failure here intentionally falls back
+ * to the CPU path by returning conservative defaults.
+ */
+object GpuSequenceFileSerializeFromObjectExecMeta extends Logging {
+ private val SequenceFileBlockCompressionVersion = 5
+
+ private case class SequenceFileScanAnalysis(
+ sourceScan: ExternalRDDScanExec[_],
+ inputPaths: Seq[String],
+ hasCompressedInput: Boolean)
+
+ private def analyzeSequenceFileScan(
+ sourceScan: ExternalRDDScanExec[_],
+ conf: org.apache.hadoop.conf.Configuration): SequenceFileScanAnalysis = {
+ val inputPaths = collectInputPaths(sourceScan.rdd)
+ SequenceFileScanAnalysis(
+ sourceScan = sourceScan,
+ inputPaths = inputPaths,
+ hasCompressedInput = hasCompressedInput(inputPaths, conf))
+ }
+
+ private def isNewApiSequenceFileRDD(rdd: NewHadoopRDD[_, _]): Boolean = {
+ try {
+ val cls = classOf[NewHadoopRDD[_, _]]
+ cls.getDeclaredFields.filter(_.getName.contains("inputFormatClass")).exists { f =>
+ f.setAccessible(true)
+ val v = f.get(rdd)
+ val c = v match {
+ case c: Class[_] => c
+ case other =>
+ try {
+ val vf = other.getClass.getDeclaredField("value")
+ vf.setAccessible(true)
+ vf.get(other).asInstanceOf[Class[_]]
+ } catch {
+ case NonFatal(_) => null
+ }
+ }
+ c != null && c.getName.contains("SequenceFile")
+ }
+ } catch {
+ case NonFatal(e) =>
+ logDebug(s"Failed to inspect NewHadoopRDD input format via reflection: ${e.getMessage}", e)
+ false
+ }
+ }
+
+ private def isOldApiSequenceFileRDD(rdd: HadoopRDD[_, _]): Boolean = {
+ try {
+ val m = rdd.getClass.getMethod("getJobConf")
+ val jc = m.invoke(rdd).asInstanceOf[org.apache.hadoop.mapred.JobConf]
+ val ifc = jc.get("mapred.input.format.class")
+ ifc != null && ifc.contains("SequenceFile")
+ } catch {
+ case NonFatal(e) =>
+ logDebug(s"Failed to inspect HadoopRDD input format via reflection: ${e.getMessage}", e)
+ false
+ }
+ }
+
+ def isSimpleSequenceFileRDD(
+ rdd: RDD[_]): Boolean = {
+ @tailrec
+ def recurse(current: RDD[_], seen: Set[RDD[_]]): Boolean = {
+ if (seen.contains(current)) {
+ false
+ } else {
+ current match {
+ case n: NewHadoopRDD[_, _] => isNewApiSequenceFileRDD(n)
+ case h: HadoopRDD[_, _] => isOldApiSequenceFileRDD(h)
+ case other =>
+ if (other.dependencies.size != 1) false
+ else recurse(other.dependencies.head.rdd, seen + current)
+ }
+ }
+ }
+
+ recurse(rdd, Set.empty)
+ }
+
+ private[rapids] def collectInputPaths(rdd: RDD[_]): Seq[String] = {
+ rdd match {
+ case n: NewHadoopRDD[_, _] =>
+ try {
+ val cls = classOf[NewHadoopRDD[_, _]]
+ cls.getDeclaredFields
+ .filter(f => f.getName == "_conf" || f.getName.contains("_conf"))
+ .flatMap { f =>
+ f.setAccessible(true)
+ val cv = f.get(n)
+ val conf = cv match {
+ case c: org.apache.hadoop.conf.Configuration => c
+ case other =>
+ try {
+ val vf = other.getClass.getDeclaredField("value")
+ vf.setAccessible(true)
+ vf.get(other).asInstanceOf[org.apache.hadoop.conf.Configuration]
+ } catch {
+ case NonFatal(_) => null
+ }
+ }
+ val p = if (conf != null) conf.get(NewFileInputFormat.INPUT_DIR) else null
+ Option(p).toSeq
+ }.flatMap(_.split(",").map(_.trim)).filter(_.nonEmpty)
+ } catch {
+ case NonFatal(e) =>
+ logDebug(s"Failed to collect input paths from NewHadoopRDD: ${e.getMessage}", e)
+ Seq.empty
+ }
+ case h: HadoopRDD[_, _] =>
+ try {
+ val m = h.getClass.getMethod("getJobConf")
+ val jc = m.invoke(h).asInstanceOf[org.apache.hadoop.mapred.JobConf]
+ val paths = OldFileInputFormat.getInputPaths(jc)
+ if (paths == null) Seq.empty else paths.map(_.toString).toSeq
+ } catch {
+ case NonFatal(e) =>
+ logDebug(s"Failed to collect input paths from HadoopRDD: ${e.getMessage}", e)
+ Seq.empty
+ }
+ case other if other.dependencies.size == 1 =>
+ collectInputPaths(other.dependencies.head.rdd)
+ case _ => Seq.empty
+ }
+ }
+
+ private def findAnyFile(path: Path, conf: org.apache.hadoop.conf.Configuration): Option[Path] = {
+ val fs = path.getFileSystem(conf)
+ val statuses = fs.globStatus(path)
+ if (statuses == null || statuses.isEmpty) {
+ None
+ } else {
+ val first = statuses.head
+ if (first.isFile) Some(first.getPath)
+ else {
+ val it = fs.listFiles(first.getPath, true)
+ if (it.hasNext) Some(it.next().getPath) else None
+ }
+ }
+ }
+
+ private def isCompressedSequenceFile(
+ file: Path,
+ conf: org.apache.hadoop.conf.Configuration): Boolean = {
+ var in: java.io.DataInputStream = null
+ try {
+ in = new java.io.DataInputStream(file.getFileSystem(conf).open(file))
+ val magic = new Array[Byte](4)
+ in.readFully(magic)
+ if (!(magic(0) == 'S' && magic(1) == 'E' && magic(2) == 'Q')) {
+ false
+ } else {
+ val version = magic(3) & 0xFF
+ org.apache.hadoop.io.Text.readString(in)
+ org.apache.hadoop.io.Text.readString(in)
+ val isCompressed = in.readBoolean()
+ val isBlockCompressed = if (version >= SequenceFileBlockCompressionVersion) {
+ in.readBoolean()
+ } else {
+ false
+ }
+ isCompressed || isBlockCompressed
+ }
+ } catch {
+ case NonFatal(_) => false
+ } finally {
+ if (in != null) in.close()
+ }
+ }
+
+ private def hasCompressedInput(
+ inputPaths: Seq[String],
+ conf: org.apache.hadoop.conf.Configuration): Boolean = {
+ inputPaths.exists { p =>
+ try {
+ findAnyFile(new Path(p), conf).exists(f => isCompressedSequenceFile(f, conf))
+ } catch {
+ case NonFatal(_) => false
+ }
+ }
+ }
+}
+
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 5a507e7dab7..e5c8399f343 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
@@ -1774,6 +1774,39 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern")
.checkValue(v => v > 0, "The maximum number of files must be greater than 0.")
.createWithDefault(Integer.MAX_VALUE)
+ val SEQUENCEFILE_READER_TYPE = conf("spark.rapids.sql.format.sequencefile.reader.type")
+ .doc("Sets the SequenceFile reader type. Since SequenceFile decoding happens on the CPU " +
+ "(using Hadoop's SequenceFile.Reader), COALESCING mode is not supported and will throw " +
+ "an exception. MULTITHREADED uses multiple threads to read files in parallel, utilizing " +
+ "multiple CPU cores for I/O and decoding. MULTITHREADED is recommended when reading " +
+ "many files as it allows the CPU to keep reading while GPU is also doing work. " +
+ s"See $MULTITHREAD_READ_NUM_THREADS and " +
+ "spark.rapids.sql.format.sequencefile.multiThreadedRead.maxNumFilesParallel to control " +
+ "the number of threads and amount of memory used. " +
+ "AUTO is kept for compatibility, but MULTITHREADED is the default for SequenceFile.")
+ .stringConf
+ .transform(_.toUpperCase(java.util.Locale.ROOT))
+ .checkValues(RapidsReaderType.values.map(_.toString))
+ .createWithDefault(RapidsReaderType.MULTITHREADED.toString)
+
+ val SEQUENCEFILE_MULTITHREAD_READ_MAX_NUM_FILES_PARALLEL =
+ conf("spark.rapids.sql.format.sequencefile.multiThreadedRead.maxNumFilesParallel")
+ .doc("A limit on the maximum number of files per task processed in parallel on the CPU " +
+ "side before the file is sent to the GPU. This affects the amount of host memory used " +
+ "when reading the files in parallel. Used with MULTITHREADED reader, see " +
+ s"$SEQUENCEFILE_READER_TYPE.")
+ .integerConf
+ .checkValue(v => v > 0, "The maximum number of files must be greater than 0.")
+ .createWithDefault(Integer.MAX_VALUE)
+
+ val SEQUENCEFILE_RDD_PHYSICAL_REPLACE_ENABLED =
+ conf("spark.rapids.sql.format.sequencefile.rddScan.physicalReplace.enabled")
+ .doc("Enable physical-plan replacement for SequenceFile RDD scans (RDDScanExec) when " +
+ "the lineage can be safely identified as a simple SequenceFile scan with BinaryType " +
+ "key/value output. Unsupported or risky cases automatically remain on CPU.")
+ .booleanConf
+ .createWithDefault(true)
+
val ENABLE_DELTA_WRITE = conf("spark.rapids.sql.format.delta.write.enabled")
.doc("When set to false disables Delta Lake output acceleration.")
.booleanConf
@@ -3684,6 +3717,29 @@ class RapidsConf(conf: Map[String, String]) extends Logging {
lazy val maxNumAvroFilesParallel: Int = get(AVRO_MULTITHREAD_READ_MAX_NUM_FILES_PARALLEL)
+ lazy val isSequenceFileMultiThreadReadEnabled: Boolean = {
+ val readerType = RapidsReaderType.withName(get(SEQUENCEFILE_READER_TYPE))
+ readerType match {
+ case RapidsReaderType.COALESCING =>
+ throw new IllegalArgumentException(
+ s"COALESCING reader type is not supported for SequenceFile. " +
+ s"SequenceFile decoding happens on CPU, so coalescing provides no benefit. " +
+ s"Use MULTITHREADED or AUTO instead.")
+ case RapidsReaderType.PERFILE =>
+ logWarning("SequenceFile PERFILE reader has been removed; using MULTITHREADED instead.")
+ true
+ case _ =>
+ // AUTO and MULTITHREADED both use the multithreaded reader implementation.
+ // SequenceFile has no separate per-file reader implementation.
+ true
+ }
+ }
+
+ lazy val maxNumSequenceFilesParallel: Int = get(
+ SEQUENCEFILE_MULTITHREAD_READ_MAX_NUM_FILES_PARALLEL)
+ lazy val isSequenceFileRDDPhysicalReplaceEnabled: Boolean =
+ get(SEQUENCEFILE_RDD_PHYSICAL_REPLACE_ENABLED)
+
lazy val isDeltaWriteEnabled: Boolean = get(ENABLE_DELTA_WRITE)
lazy val isIcebergEnabled: Boolean = get(ENABLE_ICEBERG)
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/sequencefile/GpuSequenceFileReaders.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/sequencefile/GpuSequenceFileReaders.scala
new file mode 100644
index 00000000000..f3487814c75
--- /dev/null
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/sequencefile/GpuSequenceFileReaders.scala
@@ -0,0 +1,849 @@
+/*
+ * Copyright (c) 2026, 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.sequencefile
+
+import java.io.{FileNotFoundException, IOException}
+import java.net.URI
+import java.util
+import java.util.Optional
+
+import scala.collection.mutable.ArrayBuffer
+
+import ai.rapids.cudf._
+import com.nvidia.spark.rapids._
+import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource}
+import com.nvidia.spark.rapids.io.async.{AsyncRunner, UnboundedAsyncRunner}
+import com.nvidia.spark.rapids.jni.RmmSpark
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.{DataOutputBuffer, SequenceFile}
+
+import org.apache.spark.TaskContext
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.connector.read.PartitionReader
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.rapids.execution.TrampolineUtil
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{BinaryType, StructType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector => SparkVector}
+import org.apache.spark.util.SerializableConfiguration
+
+private[sequencefile] object GpuSequenceFileReaders {
+ final val KEY_FIELD: String = "key"
+ final val VALUE_FIELD: String = "value"
+
+ def addBytesWritablePayload(
+ bufferer: HostBinaryListBufferer,
+ bytes: Array[Byte],
+ offset: Int,
+ totalLen: Int): Unit = {
+ if (totalLen < 4) {
+ bufferer.addBytes(bytes, offset, 0)
+ } else {
+ val payloadLen = ((bytes(offset) & 0xFF) << 24) |
+ ((bytes(offset + 1) & 0xFF) << 16) |
+ ((bytes(offset + 2) & 0xFF) << 8) |
+ (bytes(offset + 3) & 0xFF)
+ if (payloadLen > 0 && payloadLen <= totalLen - 4) {
+ bufferer.addBytes(bytes, offset + 4, payloadLen)
+ } else {
+ bufferer.addBytes(bytes, offset, 0)
+ }
+ }
+ }
+}
+
+private[sequencefile] final class UnsupportedSequenceFileCompressionException(msg: String)
+ extends Exception(msg)
+
+/**
+ * Buffers binary values into one contiguous bytes buffer with an INT32 offsets buffer, and then
+ * materializes a cuDF LIST device column using `makeListFromOffsets`.
+ *
+ * This class uses pinned memory (via HostAlloc) when available for better H2D transfer
+ * performance. Pinned memory allows for faster and potentially asynchronous copies to the GPU.
+ */
+private[sequencefile] final class HostBinaryListBufferer(
+ initialSizeBytes: Long,
+ initialRows: Int) extends AutoCloseable with Logging {
+ // Use HostAlloc which prefers pinned memory for better H2D transfer performance
+ private var dataBuffer: HostMemoryBuffer =
+ HostAlloc.alloc(math.max(initialSizeBytes, 1L), preferPinned = true)
+ private var dataLocation: Long = 0L
+
+ private var rowsAllocated: Int = math.max(initialRows, 1)
+ private var offsetsBuffer: HostMemoryBuffer =
+ HostAlloc.alloc((rowsAllocated.toLong + 1L) * DType.INT32.getSizeInBytes, preferPinned = true)
+ private var numRows: Int = 0
+
+ logDebug(s"HostBinaryListBufferer allocated: data=${dataBuffer.getLength} bytes, " +
+ s"offsets=${offsetsBuffer.getLength} bytes")
+
+ def rows: Int = numRows
+
+ def usedBytes: Long = dataLocation
+
+ private def growOffsetsIfNeeded(): Unit = {
+ if (numRows + 1 > rowsAllocated) {
+ // Use Int.MaxValue - 2 to ensure (rowsAllocated + 1) * 4 doesn't overflow
+ val newRowsAllocated = math.min(rowsAllocated.toLong * 2, Int.MaxValue.toLong - 2L).toInt
+ val newSize = (newRowsAllocated.toLong + 1L) * DType.INT32.getSizeInBytes
+ // Use HostAlloc for pinned memory preference
+ closeOnExcept(HostAlloc.alloc(newSize, preferPinned = true)) { tmpBuffer =>
+ tmpBuffer.copyFromHostBuffer(0, offsetsBuffer, 0, offsetsBuffer.getLength)
+ offsetsBuffer.close()
+ offsetsBuffer = tmpBuffer
+ rowsAllocated = newRowsAllocated
+ logDebug(s"HostBinaryListBufferer grew offsets buffer to $newSize bytes")
+ }
+ }
+ }
+
+ private def growDataIfNeeded(requiredEnd: Long): Unit = {
+ if (requiredEnd > dataBuffer.getLength) {
+ val newSize = math.max(dataBuffer.getLength * 2, requiredEnd)
+ // Use HostAlloc for pinned memory preference
+ closeOnExcept(HostAlloc.alloc(newSize, preferPinned = true)) { newBuff =>
+ newBuff.copyFromHostBuffer(0, dataBuffer, 0, dataLocation)
+ dataBuffer.close()
+ dataBuffer = newBuff
+ logDebug(s"HostBinaryListBufferer grew data buffer to $newSize bytes")
+ }
+ }
+ }
+
+ def addBytes(bytes: Array[Byte], offset: Int, len: Int): Unit = {
+ val newEnd = dataLocation + len
+ if (newEnd > Int.MaxValue) {
+ throw new IllegalStateException(
+ s"Binary column child size $newEnd would exceed INT32 offset limit")
+ }
+ growOffsetsIfNeeded()
+ growDataIfNeeded(newEnd)
+ val offsetPosition = numRows.toLong * DType.INT32.getSizeInBytes
+ val startDataLocation = dataLocation
+ dataBuffer.setBytes(dataLocation, bytes, offset, len)
+ dataLocation = newEnd
+ // Write offset only after successful data write
+ offsetsBuffer.setInt(offsetPosition, startDataLocation.toInt)
+ numRows += 1
+ }
+
+ /**
+ * Returns the host memory buffers (data and offsets) and releases ownership.
+ * The caller is responsible for closing the returned buffers.
+ * This is used by the multi-file reader which needs host buffers for later GPU transfer.
+ *
+ * IMPORTANT: This method returns buffers sized exactly to the actual data, not the allocated
+ * size. This is critical because HostAlloc.alloc doesn't zero-initialize memory, and passing
+ * oversized buffers to cuDF can result in garbage data being included in the output.
+ *
+ * @return a tuple of (Some(dataBuffer), Some(offsetsBuffer)) if there is data,
+ * or (None, None) if empty
+ */
+ def getHostBuffersAndRelease(): (Option[HostMemoryBuffer], Option[HostMemoryBuffer]) = {
+ if (numRows == 0) {
+ return (None, None)
+ }
+
+ if (dataLocation > Int.MaxValue) {
+ throw new IllegalStateException(
+ s"Binary column child size $dataLocation exceeds INT32 offset limit")
+ }
+ // Write the final offset
+ offsetsBuffer.setInt(numRows.toLong * DType.INT32.getSizeInBytes, dataLocation.toInt)
+
+ // Calculate exact sizes needed
+ val exactDataSize = dataLocation
+ val exactOffsetsSize = (numRows + 1).toLong * DType.INT32.getSizeInBytes
+
+ // Copy to exactly-sized buffers only if the over-allocation is significant.
+ // cuDF's HostColumnVector.copyToDevice() for flat types (like the UINT8 child) uses
+ // numRows * dtype.getSizeInBytes() to determine the copy size, not the buffer's
+ // allocated length. So small over-allocation is safe. However, large over-allocation
+ // wastes pinned memory and H2D bandwidth, so we copy when the buffer is >25% oversized.
+ val exactDataBuffer = if (exactDataSize > 0) {
+ if (dataBuffer.getLength <= exactDataSize * 5 / 4) {
+ // Buffer is close to the exact size - reuse directly (no copy)
+ val buf = dataBuffer
+ dataBuffer = null
+ buf
+ } else {
+ // Buffer is significantly over-allocated - copy to exact size
+ closeOnExcept(dataBuffer) { _ =>
+ val newBuf = HostAlloc.alloc(exactDataSize, preferPinned = true)
+ newBuf.copyFromHostBuffer(0, dataBuffer, 0, exactDataSize)
+ dataBuffer.close()
+ dataBuffer = null
+ newBuf
+ }
+ }
+ } else {
+ // For empty data, still need a valid (but minimal) buffer
+ dataBuffer.close()
+ dataBuffer = null
+ HostAlloc.alloc(1L, preferPinned = true)
+ }
+
+ val exactOffsetsBuffer = closeOnExcept(exactDataBuffer) { _ =>
+ if (offsetsBuffer.getLength <= exactOffsetsSize * 5 / 4) {
+ // Buffer is close to the exact size - reuse directly
+ val buf = offsetsBuffer
+ offsetsBuffer = null
+ buf
+ } else {
+ closeOnExcept(offsetsBuffer) { _ =>
+ val newBuf = HostAlloc.alloc(exactOffsetsSize, preferPinned = true)
+ newBuf.copyFromHostBuffer(0, offsetsBuffer, 0, exactOffsetsSize)
+ offsetsBuffer.close()
+ offsetsBuffer = null
+ newBuf
+ }
+ }
+ }
+
+ dataBuffer = null
+ offsetsBuffer = null
+
+ (Some(exactDataBuffer), Some(exactOffsetsBuffer))
+ }
+
+ override def close(): Unit = {
+ if (dataBuffer != null) {
+ dataBuffer.close()
+ dataBuffer = null
+ }
+ if (offsetsBuffer != null) {
+ offsetsBuffer.close()
+ offsetsBuffer = null
+ }
+ }
+}
+
+/**
+ * Represents a single chunk of SequenceFile binary data with its offsets.
+ * Used for GPU concat optimization - each file becomes one chunk.
+ *
+ * @param dataBuffer host memory buffer containing binary data
+ * @param offsetsBuffer host memory buffer containing INT32 offsets
+ * @param numRows number of rows in this chunk
+ */
+private[sequencefile] case class SequenceFileChunk(
+ dataBuffer: HostMemoryBuffer,
+ offsetsBuffer: HostMemoryBuffer,
+ numRows: Int) extends AutoCloseable {
+ override def close(): Unit = {
+ dataBuffer.close()
+ offsetsBuffer.close()
+ }
+}
+
+/**
+ * Host memory buffer metadata for SequenceFile multi-thread reader.
+ *
+ * Supports two modes:
+ * 1. Single file mode: keyChunks/valueChunks have one element
+ * 2. Combined mode (GPU concat): keyChunks/valueChunks have multiple elements,
+ * which will be concatenated on GPU for better performance (zero CPU copy)
+ *
+ * @param partitionedFile the partitioned file info
+ * @param memBuffersAndSizes array of buffer metadata
+ * @param bytesRead total bytes read from the file
+ * @param keyChunks array of key data chunks (one per file when combined)
+ * @param valueChunks array of value data chunks (one per file when combined)
+ * @param totalRows total number of rows across all chunks
+ * @param wantsKey whether the key column is requested
+ * @param wantsValue whether the value column is requested
+ * @param allPartValues optional array of (rowCount, partitionValues) when combining
+ */
+private[sequencefile] case class SequenceFileHostBuffersWithMetaData(
+ override val partitionedFile: PartitionedFile,
+ override val memBuffersAndSizes: Array[SingleHMBAndMeta],
+ override val bytesRead: Long,
+ keyChunks: Array[SequenceFileChunk],
+ valueChunks: Array[SequenceFileChunk],
+ totalRows: Int,
+ wantsKey: Boolean,
+ wantsValue: Boolean,
+ override val allPartValues: Option[Array[(Long, InternalRow)]] = None)
+ extends HostMemoryBuffersWithMetaDataBase {
+
+ override def close(): Unit = {
+ keyChunks.foreach(_.close())
+ valueChunks.foreach(_.close())
+ super.close()
+ }
+}
+
+/**
+ * Empty metadata returned when a file has no records.
+ *
+ * @param partitionedFile the partitioned file info
+ * @param bytesRead total bytes read from the file
+ * @param numRows number of rows (usually 0 for empty files, but may be > 0 when combining)
+ * @param allPartValues optional array of (rowCount, partitionValues) when combining multiple files
+ */
+private[sequencefile] case class SequenceFileEmptyMetaData(
+ override val partitionedFile: PartitionedFile,
+ override val bytesRead: Long,
+ numRows: Long = 0,
+ override val allPartValues: Option[Array[(Long, InternalRow)]] = None)
+ extends HostMemoryBuffersWithMetaDataBase {
+ override def memBuffersAndSizes: Array[SingleHMBAndMeta] = Array(SingleHMBAndMeta.empty())
+}
+
+/**
+ * Multi-threaded cloud reader for SequenceFile format.
+ * Reads multiple files in parallel using a thread pool.
+ * Supports combining small files into larger batches for better GPU efficiency.
+ */
+class MultiFileCloudSequenceFilePartitionReader(
+ conf: Configuration,
+ files: Array[PartitionedFile],
+ requiredSchema: StructType,
+ partitionSchema: StructType,
+ maxReadBatchSizeRows: Int,
+ maxReadBatchSizeBytes: Long,
+ maxGpuColumnSizeBytes: Long,
+ poolConf: ThreadPoolConf,
+ maxNumFileProcessed: Int,
+ execMetrics: Map[String, GpuMetric],
+ ignoreMissingFiles: Boolean,
+ ignoreCorruptFiles: Boolean,
+ queryUsesInputFile: Boolean,
+ combineConf: CombineConf = CombineConf(-1, -1))
+ extends MultiFileCloudPartitionReaderBase(conf, files, poolConf, maxNumFileProcessed,
+ Array.empty[Filter], execMetrics, maxReadBatchSizeRows, maxReadBatchSizeBytes,
+ ignoreCorruptFiles, combineConf = combineConf) with MultiFileReaderFunctions with Logging {
+
+ private val wantsKey = requiredSchema.fieldNames.exists(
+ _.equalsIgnoreCase(GpuSequenceFileReaders.KEY_FIELD))
+ private val wantsValue = requiredSchema.fieldNames.exists(
+ _.equalsIgnoreCase(GpuSequenceFileReaders.VALUE_FIELD))
+
+ override def getFileFormatShortName: String = "SequenceFileBinary"
+
+ /**
+ * Whether to use combine mode to merge multiple small files into larger batches.
+ * This improves GPU efficiency by reducing the number of small batches.
+ */
+ override def canUseCombine: Boolean = {
+ if (queryUsesInputFile) {
+ logDebug("Can't use combine mode because query uses 'input_file_xxx' function(s)")
+ false
+ } else {
+ val canUse = combineConf.combineThresholdSize > 0
+ if (!canUse) {
+ logDebug("Cannot use combine mode because the threshold size <= 0")
+ }
+ canUse
+ }
+ }
+
+ private def collectCombinedPartitionValues(
+ input: Array[HostMemoryBuffersWithMetaDataBase]): Array[(Long, InternalRow)] = {
+ val allPartValues = new ArrayBuffer[(Long, InternalRow)]()
+ input.foreach { buf =>
+ val partValues = buf.partitionedFile.partitionValues
+ buf match {
+ case empty: SequenceFileEmptyMetaData if empty.numRows > 0 =>
+ allPartValues.append((empty.numRows, partValues))
+ case meta: SequenceFileHostBuffersWithMetaData =>
+ allPartValues.append((meta.totalRows.toLong, partValues))
+ case _ =>
+ }
+ }
+ allPartValues.toArray
+ }
+
+ private def addPartitionValuesToBatch(
+ batch: ColumnarBatch,
+ singlePartValues: InternalRow,
+ combinedPartValues: Option[Array[(Long, InternalRow)]]): Iterator[ColumnarBatch] = {
+ combinedPartValues match {
+ case Some(partRowsAndValues) =>
+ val (rowsPerPart, partValues) = partRowsAndValues.unzip
+ BatchWithPartitionDataUtils.addPartitionValuesToBatch(
+ batch,
+ rowsPerPart,
+ partValues,
+ partitionSchema,
+ maxGpuColumnSizeBytes)
+ case None =>
+ BatchWithPartitionDataUtils.addSinglePartitionValueToBatch(
+ batch,
+ singlePartValues,
+ partitionSchema,
+ maxGpuColumnSizeBytes)
+ }
+ }
+
+ /**
+ * Combines multiple SequenceFile host memory buffers into a single buffer.
+ * This reduces the number of batches sent to the GPU, improving performance.
+ */
+ override def combineHMBs(
+ buffers: Array[HostMemoryBuffersWithMetaDataBase]): HostMemoryBuffersWithMetaDataBase = {
+ if (buffers.length == 1) {
+ logDebug("No need to combine because there is only one buffer.")
+ buffers.head
+ } else {
+ assert(buffers.length > 1)
+ logDebug(s"Got ${buffers.length} buffers, combining them")
+ doCombineHmbs(buffers)
+ }
+ }
+
+ /**
+ * Performs the actual combining of multiple SequenceFile buffers.
+ *
+ * OPTIMIZATION: Uses zero-copy approach similar to Parquet!
+ * Instead of copying data on CPU, we just collect buffer references
+ * and let GPU concatenate handle the merging (much faster due to high bandwidth).
+ */
+ private def doCombineHmbs(
+ input: Array[HostMemoryBuffersWithMetaDataBase]): HostMemoryBuffersWithMetaDataBase = {
+ val startCombineTime = System.currentTimeMillis()
+
+ // Separate empty and non-empty buffers
+ val (emptyBuffers, nonEmptyBuffers) = input.partition {
+ case _: SequenceFileEmptyMetaData => true
+ case meta: SequenceFileHostBuffersWithMetaData => meta.totalRows == 0
+ case _ => false
+ }
+
+ val allPartValues = collectCombinedPartitionValues(input)
+
+ // If all buffers are empty, return an empty combined result
+ if (nonEmptyBuffers.isEmpty) {
+ val totalBytesRead = input.map(_.bytesRead).sum
+ val firstPart = input.head.partitionedFile
+ emptyBuffers.foreach(_.close())
+ return SequenceFileEmptyMetaData(
+ firstPart,
+ totalBytesRead,
+ numRows = allPartValues.map(_._1).sum,
+ allPartValues = if (allPartValues.nonEmpty) Some(allPartValues) else None)
+ }
+
+ // Close empty buffers since we don't need them
+ emptyBuffers.foreach(_.close())
+
+ // Cast non-empty buffers to the correct type
+ val toCombine = nonEmptyBuffers.map(_.asInstanceOf[SequenceFileHostBuffersWithMetaData])
+
+ logDebug(s"Using zero-copy Combine mode, collecting ${toCombine.length} non-empty files, " +
+ s"files: ${toCombine.map(_.partitionedFile.filePath).mkString(",")}")
+
+ // ZERO-COPY: Just collect all chunks without copying data!
+ // The actual concatenation will happen on GPU (much faster)
+ val allKeyChunks = toCombine.flatMap(_.keyChunks)
+ val allValueChunks = toCombine.flatMap(_.valueChunks)
+ val totalRows = toCombine.map(_.totalRows).sum
+ val totalBytesRead = input.map(_.bytesRead).sum
+ val firstMeta = toCombine.head
+
+ val result = SequenceFileHostBuffersWithMetaData(
+ partitionedFile = firstMeta.partitionedFile,
+ memBuffersAndSizes = Array(SingleHMBAndMeta.empty(totalRows)),
+ bytesRead = totalBytesRead,
+ keyChunks = allKeyChunks,
+ valueChunks = allValueChunks,
+ totalRows = totalRows,
+ wantsKey = wantsKey,
+ wantsValue = wantsValue,
+ allPartValues = if (allPartValues.nonEmpty) Some(allPartValues) else None)
+
+ logDebug(s"Zero-copy combine took ${System.currentTimeMillis() - startCombineTime} ms, " +
+ s"collected ${toCombine.length} files with ${allKeyChunks.length} key chunks, " +
+ s"${allValueChunks.length} value chunks, total ${totalRows} rows, " +
+ s"task id: ${TaskContext.get().taskAttemptId()}")
+
+ result
+ }
+
+ override def getBatchRunner(
+ tc: TaskContext,
+ file: PartitionedFile,
+ config: Configuration,
+ filters: Array[Filter]): AsyncRunner[HostMemoryBuffersWithMetaDataBase] = {
+ new ReadBatchRunner(tc, file, config)
+ }
+
+ override def readBatches(
+ fileBufsAndMeta: HostMemoryBuffersWithMetaDataBase): Iterator[ColumnarBatch] = {
+ fileBufsAndMeta match {
+ case empty: SequenceFileEmptyMetaData =>
+ // No data, but we might need to emit partition values
+ GpuSemaphore.acquireIfNecessary(TaskContext.get())
+ val emptyBatch = new ColumnarBatch(Array.empty, empty.numRows.toInt)
+ addPartitionValuesToBatch(
+ emptyBatch,
+ empty.partitionedFile.partitionValues,
+ empty.allPartValues)
+
+ case meta: SequenceFileHostBuffersWithMetaData =>
+ GpuSemaphore.acquireIfNecessary(TaskContext.get())
+ val batch = buildColumnarBatchFromHostBuffers(meta)
+ closeOnExcept(batch) { _ =>
+ addPartitionValuesToBatch(
+ batch,
+ meta.partitionedFile.partitionValues,
+ meta.allPartValues)
+ }
+
+ case other =>
+ throw new RuntimeException(s"Unknown buffer type: ${other.getClass.getSimpleName}")
+ }
+ }
+
+ private def buildColumnarBatchFromHostBuffers(
+ meta: SequenceFileHostBuffersWithMetaData): ColumnarBatch = {
+ val numRows = meta.totalRows
+
+ if (numRows == 0 || requiredSchema.isEmpty) {
+ return new ColumnarBatch(Array.empty, numRows)
+ }
+
+ // Build device columns from host buffers
+ // If multiple chunks exist (combined mode), concatenate on GPU for better performance
+ val keyCol: Option[ColumnVector] = if (meta.wantsKey && meta.keyChunks.nonEmpty) {
+ Some(buildDeviceColumnFromChunks(meta.keyChunks))
+ } else None
+
+ val valueCol: Option[ColumnVector] = closeOnExcept(keyCol) { _ =>
+ if (meta.wantsValue && meta.valueChunks.nonEmpty) {
+ Some(buildDeviceColumnFromChunks(meta.valueChunks))
+ } else None
+ }
+
+ withResource(keyCol) { kc =>
+ withResource(valueCol) { vc =>
+ val cols: Array[SparkVector] = requiredSchema.fields.map { f =>
+ if (f.name.equalsIgnoreCase(GpuSequenceFileReaders.KEY_FIELD)) {
+ GpuColumnVector.from(kc.get.incRefCount(), BinaryType)
+ } else if (f.name.equalsIgnoreCase(GpuSequenceFileReaders.VALUE_FIELD)) {
+ GpuColumnVector.from(vc.get.incRefCount(), BinaryType)
+ } else {
+ GpuColumnVector.fromNull(numRows, f.dataType)
+ }
+ }
+ closeOnExcept(cols) { _ =>
+ new ColumnarBatch(cols, numRows)
+ }
+ }
+ }
+ }
+
+ /**
+ * Build a device column from multiple chunks using GPU concatenation.
+ * This is the key optimization: instead of copying on CPU, we transfer each chunk
+ * to GPU separately and use cudf::concatenate which is much faster.
+ */
+ private def buildDeviceColumnFromChunks(chunks: Array[SequenceFileChunk]): ColumnVector = {
+ if (chunks.length == 1) {
+ // Single chunk: use the original fast path
+ val chunk = chunks.head
+ buildDeviceColumnFromHostBuffers(chunk.dataBuffer, chunk.offsetsBuffer, chunk.numRows)
+ } else {
+ // Multiple chunks: transfer each to GPU and concatenate
+ // GPU concat is much faster than CPU copy + offset adjustment
+ val gpuCols = new ArrayBuffer[ColumnVector]()
+ try {
+ chunks.foreach { chunk =>
+ gpuCols += buildDeviceColumnFromHostBuffers(
+ chunk.dataBuffer, chunk.offsetsBuffer, chunk.numRows)
+ }
+ // Use cudf concatenate - this is highly optimized and uses GPU memory bandwidth
+ ColumnVector.concatenate(gpuCols.toArray: _*)
+ } finally {
+ gpuCols.foreach(_.close())
+ }
+ }
+ }
+
+ /**
+ * Build a device column (LIST) from host memory buffers.
+ * Uses proper nested HostColumnVector structure for efficient single copyToDevice().
+ *
+ * Note: The input buffers are expected to be exactly-sized (from getHostBuffersAndRelease).
+ * This method transfers ownership of the buffers to the HostColumnVector.
+ */
+ private def buildDeviceColumnFromHostBuffers(
+ dataBuffer: HostMemoryBuffer,
+ offsetsBuffer: HostMemoryBuffer,
+ numRows: Int): ColumnVector = {
+ // Chunks will also close these buffers, so keep one reference for chunk ownership.
+ dataBuffer.incRefCount()
+ offsetsBuffer.incRefCount()
+
+ // Get the actual data length from the final offset
+ val dataLen = offsetsBuffer.getInt(numRows.toLong * DType.INT32.getSizeInBytes)
+
+ // Create the child HostColumnVectorCore (UINT8 data)
+ val emptyChildren = new util.ArrayList[HostColumnVectorCore]()
+ val childCore = new HostColumnVectorCore(DType.UINT8, dataLen,
+ Optional.of[java.lang.Long](0L), dataBuffer, null, null, emptyChildren)
+
+ // Create the children list for the LIST column
+ val listChildren = new util.ArrayList[HostColumnVectorCore]()
+ listChildren.add(childCore)
+
+ // Create the LIST HostColumnVector with proper nested structure
+ // The HostColumnVector takes ownership of the buffers
+ val listHost = closeOnExcept(childCore) { _ =>
+ new HostColumnVector(DType.LIST, numRows,
+ Optional.of[java.lang.Long](0L), // nullCount = 0
+ null, // no data buffer for LIST type
+ null, // no validity buffer (no nulls)
+ offsetsBuffer, // offsets buffer
+ listChildren) // nested children containing the UINT8 child
+ }
+
+ // Single copyToDevice() handles the entire nested structure efficiently
+ withResource(listHost)(_.copyToDevice())
+ }
+
+ /**
+ * Async runner that reads a single SequenceFile to host memory buffers.
+ */
+ private class ReadBatchRunner(
+ taskContext: TaskContext,
+ partFile: PartitionedFile,
+ config: Configuration)
+ extends UnboundedAsyncRunner[HostMemoryBuffersWithMetaDataBase] with Logging {
+
+ override def callImpl(): HostMemoryBuffersWithMetaDataBase = {
+ TrampolineUtil.setTaskContext(taskContext)
+ RmmSpark.poolThreadWorkingOnTask(taskContext.taskAttemptId())
+ try {
+ doRead()
+ } catch {
+ case e: FileNotFoundException if ignoreMissingFiles =>
+ logWarning(s"Skipped missing file: ${partFile.filePath}", e)
+ SequenceFileEmptyMetaData(partFile, 0L)
+ case e: FileNotFoundException if !ignoreMissingFiles => throw e
+ case e: UnsupportedSequenceFileCompressionException => throw e
+ case e@(_: RuntimeException | _: IOException) if ignoreCorruptFiles =>
+ logWarning(s"Skipped corrupted file: ${partFile.filePath}", e)
+ SequenceFileEmptyMetaData(partFile, 0L)
+ } finally {
+ RmmSpark.poolThreadFinishedForTask(taskContext.taskAttemptId())
+ TrampolineUtil.unsetTaskContext()
+ }
+ }
+
+ private def doRead(): HostMemoryBuffersWithMetaDataBase = {
+ val startingBytesRead = fileSystemBytesRead()
+ val path = new org.apache.hadoop.fs.Path(new URI(partFile.filePath.toString))
+
+ withResource(new SequenceFile.Reader(config, SequenceFile.Reader.file(path))) { reader =>
+ // Check for compression before starting split processing.
+ // This remains an execution-time guard because planning-time sampling is conservative.
+ if (reader.isCompressed || reader.isBlockCompressed) {
+ val compressionType = reader.getCompressionType
+ val msg = s"SequenceFile reader does not support " +
+ s"compressed SequenceFiles (compressionType=$compressionType), file=$path"
+ throw new UnsupportedSequenceFileCompressionException(msg)
+ }
+
+ val start = partFile.start
+ if (start > 0) {
+ reader.sync(start)
+ }
+ val end = partFile.start + partFile.length
+
+ // Buffers for reading - reuse these across all records
+ val keyDataOut = new DataOutputBuffer()
+ val valueBytes = reader.createValueBytes()
+
+ // Pre-allocate buffers based on the split size for fewer growth copies.
+ // For uncompressed SequenceFiles, the value data is roughly proportional to the
+ // split size. Using a generous initial estimate avoids repeated doubling + copy
+ // operations (each doubling copies all existing data to a new buffer).
+ val splitSize = partFile.length
+ val initialSize = math.max(math.min(splitSize, 256L * 1024L * 1024L), 1024L * 1024L)
+ val estimatedRows = math.max((splitSize / 512).toInt, 1024) // ~512 bytes/record estimate
+ val initialRows = math.min(estimatedRows, 4 * 1024 * 1024) // cap at 4M rows
+
+ val keyBufferer = if (wantsKey) {
+ Some(new HostBinaryListBufferer(initialSize, initialRows))
+ } else None
+
+ val valueBufferer = closeOnExcept(keyBufferer) { _ =>
+ if (wantsValue) {
+ Some(new HostBinaryListBufferer(initialSize, initialRows))
+ } else None
+ }
+
+ // Reusable buffer for extracting value bytes from Hadoop ValueBytes.
+ // This avoids creating a new ByteArrayOutputStream per record (which was
+ // the #1 CPU-side performance bottleneck). DataOutputBuffer.getData() returns
+ // the internal array without copying, unlike ByteArrayOutputStream.toByteArray().
+ val valueDataOut = new DataOutputBuffer()
+
+ withResource(keyBufferer) { keyBuf =>
+ withResource(valueBufferer) { valBuf =>
+ var numRows = 0
+ var reachedEof = false
+
+ // Hadoop SequenceFile split boundary logic (matches SequenceFileRecordReader):
+ // 1. Get position BEFORE reading
+ // 2. Read the record
+ // 3. If posBeforeRead >= end AND syncSeen (from this read), DISCARD the record
+ // This ensures each record is processed by exactly one split.
+ while (!reachedEof) {
+ val posBeforeRead = reader.getPosition
+ keyDataOut.reset()
+ val recLen = reader.nextRaw(keyDataOut, valueBytes)
+ if (recLen < 0) {
+ // End of file reached
+ reachedEof = true
+ } else if (posBeforeRead >= end && reader.syncSeen()) {
+ // We were already past the split end, and this read crossed a sync marker.
+ // This record belongs to the next split - discard it.
+ reachedEof = true
+ } else {
+ if (wantsKey) {
+ val keyLen = keyDataOut.getLength
+ keyBuf.foreach { buf =>
+ GpuSequenceFileReaders.addBytesWritablePayload(
+ buf, keyDataOut.getData, 0, keyLen)
+ }
+ }
+ if (wantsValue) {
+ // Use reusable DataOutputBuffer instead of per-record ByteArrayOutputStream.
+ // getData() returns the internal array (zero-copy), then
+ // addBytesWritablePayload does a single copy to the host buffer.
+ valueDataOut.reset()
+ valueBytes.writeUncompressedBytes(valueDataOut)
+ valBuf.foreach { buf =>
+ GpuSequenceFileReaders.addBytesWritablePayload(
+ buf, valueDataOut.getData, 0, valueDataOut.getLength)
+ }
+ }
+ numRows += 1
+ }
+ }
+
+ val bytesRead = fileSystemBytesRead() - startingBytesRead
+
+ if (numRows == 0) {
+ SequenceFileEmptyMetaData(partFile, bytesRead)
+ } else {
+ // Extract host memory buffers from the streaming bufferers
+ // Create SequenceFileChunk for each column (key/value)
+ val keyChunks: Array[SequenceFileChunk] = keyBuf.map { kb =>
+ val (dataOpt, offsetsOpt) = kb.getHostBuffersAndRelease()
+ (dataOpt, offsetsOpt) match {
+ case (Some(data), Some(offsets)) =>
+ Array(SequenceFileChunk(data, offsets, numRows))
+ case _ => Array.empty[SequenceFileChunk]
+ }
+ }.getOrElse(Array.empty)
+
+ val valueChunks: Array[SequenceFileChunk] = closeOnExcept(keyChunks) { _ =>
+ valBuf.map { vb =>
+ val (dataOpt, offsetsOpt) = vb.getHostBuffersAndRelease()
+ (dataOpt, offsetsOpt) match {
+ case (Some(data), Some(offsets)) =>
+ Array(SequenceFileChunk(data, offsets, numRows))
+ case _ => Array.empty[SequenceFileChunk]
+ }
+ }.getOrElse(Array.empty)
+ }
+
+ SequenceFileHostBuffersWithMetaData(
+ partitionedFile = partFile,
+ memBuffersAndSizes = Array(SingleHMBAndMeta.empty(numRows)),
+ bytesRead = bytesRead,
+ keyChunks = keyChunks,
+ valueChunks = valueChunks,
+ totalRows = numRows,
+ wantsKey = wantsKey,
+ wantsValue = wantsValue)
+ }
+ }
+ }
+ }
+ }
+ }
+}
+
+case class GpuSequenceFileMultiFilePartitionReaderFactory(
+ @transient sqlConf: SQLConf,
+ broadcastedConf: Broadcast[SerializableConfiguration],
+ readDataSchema: StructType,
+ partitionSchema: StructType,
+ @transient rapidsConf: RapidsConf,
+ metrics: Map[String, GpuMetric],
+ queryUsesInputFile: Boolean)
+ extends MultiFilePartitionReaderFactoryBase(sqlConf, broadcastedConf, rapidsConf) {
+
+ // COALESCING mode is not beneficial for SequenceFile since decoding happens on CPU
+ // (using Hadoop's SequenceFile.Reader). There's no GPU-side decoding to amortize.
+ // However, COMBINE mode is supported to merge multiple small files into larger batches.
+ override val canUseCoalesceFilesReader: Boolean = false
+
+ override val canUseMultiThreadReader: Boolean =
+ rapidsConf.isSequenceFileMultiThreadReadEnabled
+
+ private val maxNumFileProcessed = rapidsConf.maxNumSequenceFilesParallel
+ private val ignoreMissingFiles = sqlConf.ignoreMissingFiles
+ private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles
+ private val poolConf = ThreadPoolConfBuilder(rapidsConf).build
+
+ // Combine configuration for merging small files into larger batches
+ private val combineThresholdSize = rapidsConf.getMultithreadedCombineThreshold
+ private val combineWaitTime = rapidsConf.getMultithreadedCombineWaitTime
+
+ override protected def getFileFormatShortName: String = "SequenceFileBinary"
+
+ override protected def buildBaseColumnarReaderForCloud(
+ files: Array[PartitionedFile],
+ conf: Configuration): PartitionReader[ColumnarBatch] = {
+ val combineConf = CombineConf(combineThresholdSize, combineWaitTime)
+ // Multi-threaded reader for cloud/parallel file reading with optional combining
+ new PartitionReaderWithBytesRead(
+ new MultiFileCloudSequenceFilePartitionReader(
+ conf,
+ files,
+ readDataSchema,
+ partitionSchema,
+ maxReadBatchSizeRows,
+ maxReadBatchSizeBytes,
+ maxGpuColumnSizeBytes,
+ poolConf,
+ maxNumFileProcessed,
+ metrics,
+ ignoreMissingFiles,
+ ignoreCorruptFiles,
+ queryUsesInputFile,
+ combineConf))
+ }
+
+ override protected def buildBaseColumnarReaderForCoalescing(
+ files: Array[PartitionedFile],
+ conf: Configuration): PartitionReader[ColumnarBatch] = {
+ // This should never be called since canUseCoalesceFilesReader = false
+ throw new IllegalStateException(
+ "COALESCING mode is not supported for SequenceFile. " +
+ "Use MULTITHREADED or AUTO instead.")
+ }
+}
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 3bdc45c1957..e3d0f234d64 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
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2020-2025, NVIDIA CORPORATION.
+ * Copyright (c) 2020-2026, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuSequenceFileSerializeFromObjectExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuSequenceFileSerializeFromObjectExec.scala
new file mode 100644
index 00000000000..1d8242c3770
--- /dev/null
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuSequenceFileSerializeFromObjectExec.scala
@@ -0,0 +1,225 @@
+/*
+ * Copyright (c) 2026, NVIDIA CORPORATION.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.rapids
+
+import scala.collection.mutable.ArrayBuffer
+
+import com.nvidia.spark.rapids._
+import com.nvidia.spark.rapids.GpuMetric._
+import com.nvidia.spark.rapids.sequencefile.GpuSequenceFileMultiFilePartitionReaderFactory
+import com.nvidia.spark.rapids.shims.{GpuDataSourceRDD, PartitionedFileUtilsShim}
+import org.apache.hadoop.fs.{FileStatus, Path}
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericInternalRow,
+ SortOrder, UnsafeProjection}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning,
+ UnknownPartitioning}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.datasources.FilePartition
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.util.SerializableConfiguration
+
+/**
+ * GPU replacement for SerializeFromObjectExec over SequenceFile object scans.
+ *
+ * The GPU columnar path bypasses the child RDD entirely and reads SequenceFiles
+ * directly using the multi-threaded reader with combine mode for optimal batch
+ * sizes and GPU utilization. This restores the same I/O path that the old
+ * logical-plan conversion used via GpuFileSourceScanExec.
+ *
+ * The CPU fallback path (doExecute) still uses the original child RDD.
+ */
+case class GpuSequenceFileSerializeFromObjectExec(
+ outputAttrs: Seq[Attribute],
+ child: SparkPlan,
+ goal: CoalesceSizeGoal,
+ inputPaths: Seq[String])(
+ @transient val rapidsConf: RapidsConf)
+ extends UnaryExecNode with GpuExec {
+
+ override def output: Seq[Attribute] = outputAttrs
+ override def outputPartitioning: Partitioning = UnknownPartitioning(0)
+ override def outputOrdering: Seq[SortOrder] = Nil
+ override def outputBatching: CoalesceGoal = goal
+ override def otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf)
+
+ override lazy val allMetrics: Map[String, GpuMetric] = Map(
+ OP_TIME_NEW ->
+ createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_OP_TIME_NEW),
+ NUM_OUTPUT_ROWS ->
+ createMetric(ESSENTIAL_LEVEL, DESCRIPTION_NUM_OUTPUT_ROWS),
+ NUM_OUTPUT_BATCHES ->
+ createMetric(MODERATE_LEVEL, DESCRIPTION_NUM_OUTPUT_BATCHES),
+ GPU_DECODE_TIME ->
+ createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_GPU_DECODE_TIME),
+ BUFFER_TIME ->
+ createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_BUFFER_TIME),
+ SCAN_TIME ->
+ createNanoTimingMetric(ESSENTIAL_LEVEL, DESCRIPTION_SCAN_TIME),
+ SCHEDULE_TIME ->
+ createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_SCHEDULE_TIME),
+ BUFFER_TIME_BUBBLE ->
+ createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_BUFFER_TIME_BUBBLE),
+ SCHEDULE_TIME_BUBBLE ->
+ createNanoTimingMetric(DEBUG_LEVEL,
+ DESCRIPTION_SCHEDULE_TIME_BUBBLE)
+ )
+
+ private lazy val readDataSchema: StructType = StructType(
+ outputAttrs.map(a => StructField(a.name, a.dataType, a.nullable)))
+
+ /**
+ * List all input files and bin-pack them into FilePartitions.
+ * Evaluated lazily on the driver.
+ */
+ @transient private lazy val filePartitions: Seq[FilePartition] = {
+ val session = SparkSession.active
+ val hadoopConf = session.sessionState.newHadoopConf()
+
+ val allFiles = new ArrayBuffer[FileStatus]()
+ inputPaths.foreach { pathStr =>
+ val path = new Path(pathStr)
+ val fs = path.getFileSystem(hadoopConf)
+ val statuses = fs.globStatus(path)
+ if (statuses != null) {
+ statuses.foreach { s =>
+ if (s.isFile) {
+ allFiles += s
+ } else {
+ val iter = fs.listFiles(s.getPath, true)
+ while (iter.hasNext) allFiles += iter.next()
+ }
+ }
+ }
+ }
+
+ // One PartitionedFile per file (no splitting; uncompressed
+ // SequenceFiles are handled as whole files by the reader).
+ // Use shim factory to handle String vs SparkPath across versions.
+ val splitFiles = allFiles.map { f =>
+ PartitionedFileUtilsShim.newPartitionedFile(
+ InternalRow.empty, f.getPath.toUri.toString, 0, f.getLen)
+ }.sortBy(_.length)(implicitly[Ordering[Long]].reverse)
+
+ val maxSplitBytes =
+ session.sessionState.conf.filesMaxPartitionBytes
+ FilePartition.getFilePartitions(
+ session, splitFiles.toSeq, maxSplitBytes)
+ }
+
+ /**
+ * Multi-threaded reader factory with combine mode support.
+ * Evaluated lazily on the driver.
+ */
+ @transient private lazy val readerFactory = {
+ val session = SparkSession.active
+ val hadoopConf = session.sessionState.newHadoopConf()
+ val broadcastedConf = session.sparkContext.broadcast(
+ new SerializableConfiguration(hadoopConf))
+ GpuSequenceFileMultiFilePartitionReaderFactory(
+ session.sessionState.conf,
+ broadcastedConf,
+ readDataSchema,
+ new StructType(), // no partition schema
+ rapidsConf,
+ allMetrics,
+ queryUsesInputFile = false)
+ }
+
+ // ---- CPU fallback (uses the original child RDD) -----------------------
+
+ override def doExecute(): RDD[InternalRow] = {
+ val localOutput = output
+ val childObjType = child.output.head.dataType
+ val outSchema = StructType(localOutput.map(a =>
+ StructField(a.name, a.dataType, a.nullable)))
+ child.execute().mapPartitionsWithIndexInternal { (index, it) =>
+ val unsafeProj = UnsafeProjection.create(outSchema)
+ unsafeProj.initialize(index)
+ it.map { row =>
+ val obj = row.get(0, childObjType)
+ val outRow = GpuSequenceFileSerializeFromObjectExec.projectObjectToOutputRow(
+ obj, localOutput)
+ unsafeProj(outRow).copy()
+ }
+ }
+ }
+
+ // ---- GPU columnar path (multi-threaded reader + combine) ---------------
+
+ override def internalDoExecuteColumnar(): RDD[ColumnarBatch] = {
+ val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS)
+ val scanTime = gpuLongMetric(SCAN_TIME)
+ GpuDataSourceRDD(
+ SparkSession.active.sparkContext, filePartitions, readerFactory
+ ).asInstanceOf[RDD[ColumnarBatch]].mapPartitionsInternal { batches =>
+ new Iterator[ColumnarBatch] {
+ override def hasNext: Boolean = scanTime.ns {
+ batches.hasNext
+ }
+ override def next(): ColumnarBatch = {
+ val batch = batches.next()
+ numOutputRows += batch.numRows()
+ batch
+ }
+ }
+ }
+ }
+
+ override protected def withNewChildInternal(
+ newChild: SparkPlan): SparkPlan = {
+ copy(child = newChild)(rapidsConf)
+ }
+}
+
+object GpuSequenceFileSerializeFromObjectExec {
+ private def sequenceFileFieldBytes(obj: Any, fieldName: String): Array[Byte] = {
+ obj match {
+ case bytes: Array[Byte] =>
+ bytes
+ case tuple: Product =>
+ if (fieldName.equalsIgnoreCase("key")) {
+ tuple.productElement(0).asInstanceOf[Array[Byte]]
+ } else {
+ tuple.productElement(1).asInstanceOf[Array[Byte]]
+ }
+ case other =>
+ throw new IllegalStateException(
+ s"Unexpected SequenceFile object type: ${other.getClass.getName}")
+ }
+ }
+
+ private[rapids] def projectObjectToOutputRow(
+ obj: Any,
+ outputAttrs: Seq[Attribute]): GenericInternalRow = {
+ val outRow = new GenericInternalRow(outputAttrs.length)
+ outputAttrs.zipWithIndex.foreach { case (attr, idx) =>
+ val bytes =
+ if (attr.name.equalsIgnoreCase("key") || attr.name.equalsIgnoreCase("value")) {
+ sequenceFileFieldBytes(obj, attr.name)
+ } else {
+ null
+ }
+ outRow.update(idx, bytes)
+ }
+ outRow
+ }
+}
diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/SequenceFileBinaryFileFormatSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/SequenceFileBinaryFileFormatSuite.scala
new file mode 100644
index 00000000000..f2392df8fa5
--- /dev/null
+++ b/tests/src/test/scala/com/nvidia/spark/rapids/SequenceFileBinaryFileFormatSuite.scala
@@ -0,0 +1,686 @@
+/*
+ * Copyright (c) 2026, 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, FileOutputStream}
+import java.nio.charset.StandardCharsets
+import java.nio.file.Files
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.{BytesWritable, SequenceFile}
+import org.apache.hadoop.io.SequenceFile.CompressionType
+import org.apache.hadoop.io.compress.DefaultCodec
+import org.apache.hadoop.mapred.{JobConf, SequenceFileAsBinaryInputFormat => OldSequenceFileAsBinaryInputFormat}
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat
+import org.scalatest.funsuite.AnyFunSuite
+
+import org.apache.spark.sql.{DataFrame, SparkSession}
+import org.apache.spark.sql.execution.RDDScanExec
+
+/**
+ * Unit tests for SequenceFile RDD read behavior with RAPIDS plugin enabled.
+ *
+ * RDD scans are preserved as-is (no logical-plan rewrite to FileFormat scan).
+ */
+class SequenceFileBinaryFileFormatSuite extends AnyFunSuite {
+
+ private def withRapidsSession(f: SparkSession => Unit): Unit = {
+ // Clear any existing sessions to ensure clean state
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+
+ val spark = SparkSession.builder()
+ .appName("SequenceFileBinaryFileFormatSuite")
+ .master("local[1]")
+ .config("spark.ui.enabled", "false")
+ .config("spark.sql.shuffle.partitions", "1")
+ // Register RAPIDS SQL extensions for logical plan rules
+ .config("spark.sql.extensions", "com.nvidia.spark.rapids.SQLExecPlugin")
+ .config("spark.plugins", "com.nvidia.spark.SQLPlugin")
+ .config("spark.rapids.sql.enabled", "true")
+ .getOrCreate()
+ try {
+ f(spark)
+ } finally {
+ spark.stop()
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ }
+ }
+
+ private def withPhysicalReplaceEnabledSession(f: SparkSession => Unit): Unit = {
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ val spark = SparkSession.builder()
+ .appName("SequenceFileBinaryFileFormatSuite-PhysicalReplace")
+ .master("local[1]")
+ .config("spark.ui.enabled", "false")
+ .config("spark.sql.shuffle.partitions", "1")
+ .config("spark.sql.extensions", "com.nvidia.spark.rapids.SQLExecPlugin")
+ .config("spark.plugins", "com.nvidia.spark.SQLPlugin")
+ .config("spark.rapids.sql.enabled", "true")
+ .config("spark.rapids.sql.format.sequencefile.rddScan.physicalReplace.enabled", "true")
+ .config("spark.rapids.sql.explain", "ALL")
+ .getOrCreate()
+ try {
+ f(spark)
+ } finally {
+ spark.stop()
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ }
+ }
+
+ private def hasGpuSequenceFileRDDScan(df: DataFrame): Boolean = {
+ df.queryExecution.executedPlan.collect {
+ case p if p.getClass.getSimpleName == "GpuSequenceFileSerializeFromObjectExec" => 1
+ }.nonEmpty
+ }
+
+ private def hasCpuRDDScan(df: DataFrame): Boolean = {
+ df.queryExecution.executedPlan.collect {
+ case _: RDDScanExec => 1
+ case p if p.getClass.getSimpleName == "ExternalRDDScanExec" => 1
+ }.nonEmpty
+ }
+
+ private def deleteRecursively(f: File): Unit = {
+ if (f.isDirectory) {
+ val children = f.listFiles()
+ if (children != null) {
+ children.foreach(deleteRecursively)
+ }
+ }
+ if (f.exists()) {
+ f.delete()
+ }
+ }
+
+ private def withTempDir(prefix: String)(f: File => Unit): Unit = {
+ val tmpDir = Files.createTempDirectory(prefix).toFile
+ try {
+ f(tmpDir)
+ } finally {
+ deleteRecursively(tmpDir)
+ }
+ }
+
+ /**
+ * Read a SequenceFile using the RDD path.
+ */
+ private def readSequenceFileViaRDD(spark: SparkSession, path: String): DataFrame = {
+ import spark.implicits._
+ val sc = spark.sparkContext
+ sc.newAPIHadoopFile(
+ path,
+ classOf[SequenceFileAsBinaryInputFormat],
+ classOf[BytesWritable],
+ classOf[BytesWritable]
+ ).map { case (k, v) =>
+ (SequenceFileBinaryFileFormatSuite.bytesWritablePayload(k.getBytes, k.getLength),
+ SequenceFileBinaryFileFormatSuite.bytesWritablePayload(v.getBytes, v.getLength))
+ }.toDF("key", "value")
+ }
+
+ /**
+ * Read only the value column from a SequenceFile (common pattern for protobuf payloads).
+ */
+ private def readSequenceFileValueOnly(spark: SparkSession, path: String): DataFrame = {
+ import spark.implicits._
+ val sc = spark.sparkContext
+ sc.newAPIHadoopFile(
+ path,
+ classOf[SequenceFileAsBinaryInputFormat],
+ classOf[BytesWritable],
+ classOf[BytesWritable]
+ ).map { case (_, v) =>
+ SequenceFileBinaryFileFormatSuite.bytesWritablePayload(v.getBytes, v.getLength)
+ }.toDF("value")
+ }
+
+ /**
+ * Read a SequenceFile via RDD and intentionally swap output names relative to tuple order.
+ */
+ private def readSequenceFileViaRDDWithSwappedNames(
+ spark: SparkSession,
+ path: String): DataFrame = {
+ import spark.implicits._
+ val sc = spark.sparkContext
+ sc.newAPIHadoopFile(
+ path,
+ classOf[SequenceFileAsBinaryInputFormat],
+ classOf[BytesWritable],
+ classOf[BytesWritable]
+ ).map { case (k, v) =>
+ (SequenceFileBinaryFileFormatSuite.bytesWritablePayload(k.getBytes, k.getLength),
+ SequenceFileBinaryFileFormatSuite.bytesWritablePayload(v.getBytes, v.getLength))
+ }.toDF("value", "key")
+ }
+
+ /**
+ * Write a SequenceFile with raw record format.
+ */
+ private def writeSequenceFile(
+ file: File,
+ conf: Configuration,
+ payloads: Array[Array[Byte]]): Unit = {
+ val path = new Path(file.toURI)
+ val writer = SequenceFile.createWriter(
+ conf,
+ SequenceFile.Writer.file(path),
+ SequenceFile.Writer.keyClass(classOf[BytesWritable]),
+ SequenceFile.Writer.valueClass(classOf[BytesWritable]),
+ SequenceFile.Writer.compression(CompressionType.NONE))
+ try {
+ payloads.zipWithIndex.foreach { case (p, idx) =>
+ val key = new BytesWritable(intToBytes(idx))
+ val value = new BytesWritable(p)
+ writer.append(key, value)
+ }
+ } finally {
+ writer.close()
+ }
+ }
+
+ private def writeCompressedSequenceFile(
+ file: File,
+ conf: Configuration,
+ payloads: Array[Array[Byte]]): Unit = {
+ val path = new Path(file.toURI)
+ val writer = SequenceFile.createWriter(
+ conf,
+ SequenceFile.Writer.file(path),
+ SequenceFile.Writer.keyClass(classOf[BytesWritable]),
+ SequenceFile.Writer.valueClass(classOf[BytesWritable]),
+ SequenceFile.Writer.compression(CompressionType.RECORD, new DefaultCodec()))
+ try {
+ payloads.zipWithIndex.foreach { case (p, idx) =>
+ val key = new BytesWritable(intToBytes(idx))
+ val value = new BytesWritable(p)
+ writer.append(key, value)
+ }
+ } finally {
+ writer.close()
+ }
+ }
+
+ private def intToBytes(i: Int): Array[Byte] = Array[Byte](
+ ((i >> 24) & 0xFF).toByte,
+ ((i >> 16) & 0xFF).toByte,
+ ((i >> 8) & 0xFF).toByte,
+ (i & 0xFF).toByte
+ )
+
+ private def bytesToInt(b: Array[Byte]): Int = {
+ require(b.length == 4, s"Expected 4 bytes, got ${b.length}")
+ ((b(0) & 0xFF) << 24) | ((b(1) & 0xFF) << 16) | ((b(2) & 0xFF) << 8) | (b(3) & 0xFF)
+ }
+
+ // ============================================================================
+ // Compression tests
+ // ============================================================================
+
+ test("Compressed SequenceFile is readable via preserved RDD scan") {
+ withTempDir("seqfile-compressed-test") { tmpDir =>
+ val file = new File(tmpDir, "compressed.seq")
+ val conf = new Configuration()
+ val payloads: Array[Array[Byte]] = Array(
+ Array[Byte](1, 2, 3),
+ "hello".getBytes(StandardCharsets.UTF_8)
+ )
+ writeCompressedSequenceFile(file, conf, payloads)
+
+ withRapidsSession { spark =>
+ val df = readSequenceFileValueOnly(spark, file.getAbsolutePath)
+ val got = df.collect().map(_.getAs[Array[Byte]](0)).sortBy(_.length)
+ val expected = payloads.sortBy(_.length)
+ assert(got.length == expected.length)
+ got.zip(expected).foreach { case (actual, exp) =>
+ assert(java.util.Arrays.equals(actual, exp),
+ s"Expected ${java.util.Arrays.toString(exp)}, got ${java.util.Arrays.toString(actual)}")
+ }
+ }
+ }
+ }
+
+ test("Corrupt SequenceFile header is handled gracefully") {
+ withTempDir("seqfile-corrupt-test") { tmpDir =>
+ val goodFile = new File(tmpDir, "good.seq")
+ val corruptFile = new File(tmpDir, "corrupt.seq")
+ val conf = new Configuration()
+ val payloads: Array[Array[Byte]] = Array(
+ Array[Byte](1, 2, 3),
+ "hello".getBytes(StandardCharsets.UTF_8)
+ )
+ writeSequenceFile(goodFile, conf, payloads)
+
+ val fos = new FileOutputStream(corruptFile)
+ try {
+ fos.write(Array[Byte](0x01, 0x02, 0x03, 0x04))
+ } finally {
+ fos.close()
+ }
+
+ withRapidsSession { spark =>
+ val path = s"${goodFile.getAbsolutePath},${corruptFile.getAbsolutePath}"
+ val thrown = intercept[Exception] {
+ readSequenceFileValueOnly(spark, path).collect()
+ }
+ assert(thrown.getMessage != null)
+ }
+ }
+ }
+
+ test("Swapped key/value output names still preserve query semantics") {
+ withTempDir("seqfile-physical-swapped-output-test") { tmpDir =>
+ val file = new File(tmpDir, "swapped.seq")
+ val conf = new Configuration()
+ val payloads = Array(
+ Array[Byte](1, 2, 3),
+ "swapped".getBytes(StandardCharsets.UTF_8))
+ writeSequenceFile(file, conf, payloads)
+
+ withPhysicalReplaceEnabledSession { spark =>
+ val df = readSequenceFileViaRDDWithSwappedNames(spark, file.getAbsolutePath)
+
+ val got = df.collect().map { row =>
+ (row.getAs[Array[Byte]]("value"), row.getAs[Array[Byte]]("key"))
+ }.sortBy { case (value, _) => bytesToInt(value) }
+
+ val expected = payloads.zipWithIndex.map { case (value, idx) =>
+ (intToBytes(idx), value)
+ }
+
+ assert(got.length == expected.length)
+ got.zip(expected).foreach { case ((actualValue, actualKey), (expectedKey, expectedValue)) =>
+ assert(java.util.Arrays.equals(actualValue, expectedKey))
+ assert(java.util.Arrays.equals(actualKey, expectedValue))
+ }
+ }
+ }
+ }
+
+ test("Physical replacement falls back to CPU for compressed SequenceFile") {
+ withTempDir("seqfile-physical-compressed-fallback-test") { tmpDir =>
+ val file = new File(tmpDir, "compressed.seq")
+ val conf = new Configuration()
+ val payloads = Array(
+ Array[Byte](8, 9, 10),
+ "compressed".getBytes(StandardCharsets.UTF_8))
+ writeCompressedSequenceFile(file, conf, payloads)
+
+ withPhysicalReplaceEnabledSession { spark =>
+ val df = readSequenceFileValueOnly(spark, file.getAbsolutePath)
+ assert(!hasGpuSequenceFileRDDScan(df),
+ s"Compressed input should not use SequenceFile GPU physical replacement exec:\n" +
+ s"${df.queryExecution.executedPlan}")
+ assert(hasCpuRDDScan(df), "Compressed input should remain on CPU scan path")
+ val got = df.collect().map(_.getAs[Array[Byte]](0)).sortBy(_.length)
+ val expected = payloads.sortBy(_.length)
+ assert(got.length == expected.length)
+ got.zip(expected).foreach { case (actual, exp) =>
+ assert(java.util.Arrays.equals(actual, exp))
+ }
+ }
+ }
+ }
+
+ test("Physical replacement falls back to CPU for complex lineage") {
+ withTempDir("seqfile-physical-complex-lineage-test") { tmpDir =>
+ val file = new File(tmpDir, "complex.seq")
+ val conf = new Configuration()
+ val payloads = Array(
+ "a".getBytes(StandardCharsets.UTF_8),
+ "bb".getBytes(StandardCharsets.UTF_8))
+ writeSequenceFile(file, conf, payloads)
+
+ withPhysicalReplaceEnabledSession { spark =>
+ import spark.implicits._
+ val sc = spark.sparkContext
+ val df = sc.newAPIHadoopFile(
+ file.getAbsolutePath,
+ classOf[SequenceFileAsBinaryInputFormat],
+ classOf[BytesWritable],
+ classOf[BytesWritable]
+ ).map { case (_, v) =>
+ SequenceFileBinaryFileFormatSuite.bytesWritablePayload(v.getBytes, v.getLength)
+ }.filter(_.length > 0)
+ .union(sc.parallelize(Seq(Array[Byte](7, 7, 7)), 1))
+ .filter(v => !(v.length == 3 && v(0) == 7.toByte && v(1) == 7.toByte && v(2) == 7.toByte))
+ .toDF("value")
+
+ assert(!hasGpuSequenceFileRDDScan(df),
+ s"Complex lineage should remain on CPU:\n${df.queryExecution.executedPlan}")
+ assert(hasCpuRDDScan(df))
+ assert(df.collect().length == payloads.length)
+ }
+ }
+ }
+
+ // ============================================================================
+ // Glob pattern tests
+ // ============================================================================
+
+ test("RDD path supports glob patterns in paths") {
+ withTempDir("seqfile-glob-test") { tmpDir =>
+ // Create subdirectories with data files
+ val subDir1 = new File(tmpDir, "2024/01")
+ val subDir2 = new File(tmpDir, "2024/02")
+ val subDir3 = new File(tmpDir, "2025/01")
+ subDir1.mkdirs()
+ subDir2.mkdirs()
+ subDir3.mkdirs()
+
+ val conf = new Configuration()
+
+ // Write different payloads to each subdirectory
+ val payloads1 = Array(Array[Byte](1, 1, 1))
+ val payloads2 = Array(Array[Byte](2, 2, 2))
+ val payloads3 = Array(Array[Byte](3, 3, 3))
+
+ writeSequenceFile(new File(subDir1, "part-00000.seq"), conf, payloads1)
+ writeSequenceFile(new File(subDir2, "part-00000.seq"), conf, payloads2)
+ writeSequenceFile(new File(subDir3, "part-00000.seq"), conf, payloads3)
+
+ withRapidsSession { spark =>
+ // Test glob pattern that matches subdirectories: 2024/*
+ val globPath = new File(tmpDir, "2024/*").getAbsolutePath
+ val df = readSequenceFileViaRDD(spark, globPath)
+
+ val results = df.select("value").collect().map(_.getAs[Array[Byte]](0))
+
+ // Should find files in 2024/01 and 2024/02 (2 files total)
+ assert(results.length == 2,
+ s"Expected 2 results from glob pattern '2024/*', got ${results.length}")
+
+ // Verify we got the exact expected payloads
+ val sortedResults = results.sortBy(_(0))
+ assert(java.util.Arrays.equals(sortedResults(0), payloads1(0)),
+ s"First result should be [1,1,1], got ${sortedResults(0).toSeq}")
+ assert(java.util.Arrays.equals(sortedResults(1), payloads2(0)),
+ s"Second result should be [2,2,2], got ${sortedResults(1).toSeq}")
+ }
+ }
+ }
+
+ test("RDD path supports recursive glob patterns") {
+ withTempDir("seqfile-recursive-glob-test") { tmpDir =>
+ // Create nested directory structure
+ val subDir1 = new File(tmpDir, "data/year=2024/month=01")
+ val subDir2 = new File(tmpDir, "data/year=2024/month=02")
+ subDir1.mkdirs()
+ subDir2.mkdirs()
+
+ val conf = new Configuration()
+
+ val payloads1 = Array(Array[Byte](10, 20, 30))
+ val payloads2 = Array(Array[Byte](40, 50, 60))
+
+ writeSequenceFile(new File(subDir1, "data.seq"), conf, payloads1)
+ writeSequenceFile(new File(subDir2, "data.seq"), conf, payloads2)
+
+ withRapidsSession { spark =>
+ // Test recursive glob pattern: data/year=2024/*/
+ val globPath = new File(tmpDir, "data/year=2024/*").getAbsolutePath
+ val df = readSequenceFileViaRDD(spark, globPath)
+
+ val results = df.select("value").collect().map(_.getAs[Array[Byte]](0))
+
+ assert(results.length == 2,
+ s"Expected 2 results from recursive glob, got ${results.length}")
+ }
+ }
+ }
+
+ test("RDD path handles glob pattern with no matches gracefully") {
+ withTempDir("seqfile-glob-nomatch-test") { tmpDir =>
+ // Create a single file
+ val file = new File(tmpDir, "test.seq")
+ val conf = new Configuration()
+ val payloads = Array(Array[Byte](1, 2, 3))
+ writeSequenceFile(file, conf, payloads)
+
+ withRapidsSession { spark =>
+ // Use a glob pattern that matches nothing
+ val globPath = new File(tmpDir, "nonexistent/*").getAbsolutePath
+
+ // Hadoop's newAPIHadoopFile throws InvalidInputException when glob matches 0 files.
+ // This happens at RDD creation time, before our conversion rule can do anything.
+ // We verify that this expected Hadoop behavior occurs.
+ val exception = intercept[org.apache.hadoop.mapreduce.lib.input.InvalidInputException] {
+ val df = readSequenceFileViaRDD(spark, globPath)
+ df.collect() // Force evaluation
+ }
+
+ assert(exception.getMessage.contains("matches 0 files"),
+ s"Expected 'matches 0 files' error, got: ${exception.getMessage}")
+ }
+ }
+ }
+
+ // ============================================================================
+ // Configuration tests
+ // ============================================================================
+
+ test("RDD SequenceFile path works without conversion config") {
+ withTempDir("seqfile-config-test") { tmpDir =>
+ val file = new File(tmpDir, "test.seq")
+ val conf = new Configuration()
+ val payloads = Array(Array[Byte](1, 2, 3))
+ writeSequenceFile(file, conf, payloads)
+
+ // Clear any existing sessions to ensure clean state
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+
+ // Create session without any SequenceFile conversion config.
+ val spark = SparkSession.builder()
+ .appName("SequenceFileBinaryFileFormatSuite-NoConversion")
+ .master("local[1]")
+ .config("spark.ui.enabled", "false")
+ // Register RAPIDS SQL extensions.
+ .config("spark.sql.extensions", "com.nvidia.spark.rapids.SQLExecPlugin")
+ .config("spark.plugins", "com.nvidia.spark.SQLPlugin")
+ .config("spark.rapids.sql.enabled", "true")
+ .getOrCreate()
+ try {
+ // This should work via the original RDD path.
+ val df = readSequenceFileValueOnly(spark, file.getAbsolutePath)
+ val results = df.collect()
+ assert(results.length == 1)
+
+ val expectedPayload = Array[Byte](1, 2, 3)
+ val actualBytes = results(0).getAs[Array[Byte]](0)
+ assert(java.util.Arrays.equals(actualBytes, expectedPayload),
+ s"Expected payload bytes ${java.util.Arrays.toString(expectedPayload)}, " +
+ s"but got ${java.util.Arrays.toString(actualBytes)}")
+ } finally {
+ spark.stop()
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ }
+ }
+ }
+
+ // ============================================================================
+ // Old API (hadoopRDD) tests
+ // ============================================================================
+
+ /**
+ * Read a SequenceFile using the old Hadoop API (hadoopRDD).
+ * This tests support for org.apache.hadoop.mapred.SequenceFileAsBinaryInputFormat.
+ */
+ private def readSequenceFileViaOldApi(spark: SparkSession, path: String): DataFrame = {
+ import spark.implicits._
+ val sc = spark.sparkContext
+ val jobConf = new JobConf(sc.hadoopConfiguration)
+ org.apache.hadoop.mapred.FileInputFormat.setInputPaths(jobConf, path)
+
+ sc.hadoopRDD(
+ jobConf,
+ classOf[OldSequenceFileAsBinaryInputFormat],
+ classOf[BytesWritable],
+ classOf[BytesWritable]
+ ).map { case (k, v) =>
+ (SequenceFileBinaryFileFormatSuite.bytesWritablePayload(k.getBytes, k.getLength),
+ SequenceFileBinaryFileFormatSuite.bytesWritablePayload(v.getBytes, v.getLength))
+ }.toDF("key", "value")
+ }
+
+ /**
+ * Read only the value column using old API hadoopRDD.
+ * This tests the pattern: rdd.map(...).toDF("value")
+ */
+ private def readSequenceFileValueOnlyViaOldApi(spark: SparkSession, path: String): DataFrame = {
+ import spark.implicits._
+ val sc = spark.sparkContext
+ val jobConf = new JobConf(sc.hadoopConfiguration)
+ org.apache.hadoop.mapred.FileInputFormat.setInputPaths(jobConf, path)
+
+ sc.hadoopRDD(
+ jobConf,
+ classOf[OldSequenceFileAsBinaryInputFormat],
+ classOf[BytesWritable],
+ classOf[BytesWritable]
+ ).map { case (_, v) =>
+ SequenceFileBinaryFileFormatSuite.bytesWritablePayload(v.getBytes, v.getLength)
+ }.toDF("value")
+ }
+
+ test("Old API hadoopRDD conversion reads key-value correctly") {
+ withTempDir("seqfile-oldapi-test") { tmpDir =>
+ val file = new File(tmpDir, "test.seq")
+ val conf = new Configuration()
+ val payloads: Array[Array[Byte]] = Array(
+ Array[Byte](1, 2, 3),
+ "hello".getBytes(StandardCharsets.UTF_8),
+ Array.fill[Byte](10)(42.toByte)
+ )
+ writeSequenceFile(file, conf, payloads)
+
+ withRapidsSession { spark =>
+ val df = readSequenceFileViaOldApi(spark, file.getAbsolutePath)
+
+ val got = df.select("key", "value")
+ .collect()
+ .map { row =>
+ val k = row.getAs[Array[Byte]](0)
+ val v = row.getAs[Array[Byte]](1)
+ (bytesToInt(k), v)
+ }
+ .sortBy(_._1)
+
+ assert(got.length == payloads.length)
+ got.foreach { case (idx, v) =>
+ assert(java.util.Arrays.equals(v, payloads(idx)),
+ s"Payload mismatch at index $idx: got ${java.util.Arrays.toString(v)}")
+ }
+ }
+ }
+ }
+
+ test("Old API hadoopRDD value-only conversion via toDF(\"value\")") {
+ withTempDir("seqfile-oldapi-valueonly-test") { tmpDir =>
+ val file = new File(tmpDir, "test.seq")
+ val conf = new Configuration()
+ val payloads: Array[Array[Byte]] = Array(
+ Array[Byte](10, 20, 30),
+ Array[Byte](40, 50, 60)
+ )
+ writeSequenceFile(file, conf, payloads)
+
+ withRapidsSession { spark =>
+ val df = readSequenceFileValueOnlyViaOldApi(spark, file.getAbsolutePath)
+
+ // Verify the schema only has "value" column
+ assert(df.schema.fieldNames.toSeq == Seq("value"),
+ s"Expected schema with only 'value' column, got: ${df.schema.fieldNames.mkString(", ")}")
+
+ val results = df.collect().map(_.getAs[Array[Byte]](0))
+ assert(results.length == payloads.length)
+
+ // Sort results to ensure consistent comparison
+ val sortedResults = results.sortBy(_(0))
+ val sortedPayloads = payloads.sortBy(_(0))
+
+ sortedResults.zip(sortedPayloads).zipWithIndex.foreach { case ((result, expected), idx) =>
+ assert(java.util.Arrays.equals(result, expected),
+ s"Mismatch at index $idx: got ${java.util.Arrays.toString(result)}, " +
+ s"expected ${java.util.Arrays.toString(expected)}")
+ }
+ }
+ }
+ }
+
+ test("Old API hadoopRDD with glob patterns") {
+ withTempDir("seqfile-oldapi-glob-test") { tmpDir =>
+ // Create subdirectories with data files
+ val subDir1 = new File(tmpDir, "part1")
+ val subDir2 = new File(tmpDir, "part2")
+ subDir1.mkdirs()
+ subDir2.mkdirs()
+
+ val conf = new Configuration()
+
+ val payloads1 = Array(Array[Byte](1, 1, 1))
+ val payloads2 = Array(Array[Byte](2, 2, 2))
+
+ writeSequenceFile(new File(subDir1, "data.seq"), conf, payloads1)
+ writeSequenceFile(new File(subDir2, "data.seq"), conf, payloads2)
+
+ withRapidsSession { spark =>
+ // Test glob pattern: part*
+ val globPath = new File(tmpDir, "part*").getAbsolutePath
+ val df = readSequenceFileViaOldApi(spark, globPath)
+
+ val results = df.select("value").collect().map(_.getAs[Array[Byte]](0))
+
+ assert(results.length == 2,
+ s"Expected 2 results from glob pattern 'part*', got ${results.length}")
+
+ val sortedResults = results.sortBy(_(0))
+ assert(java.util.Arrays.equals(sortedResults(0), payloads1(0)),
+ s"First result should be [1,1,1], got ${sortedResults(0).toSeq}")
+ assert(java.util.Arrays.equals(sortedResults(1), payloads2(0)),
+ s"Second result should be [2,2,2], got ${sortedResults(1).toSeq}")
+ }
+ }
+ }
+}
+
+object SequenceFileBinaryFileFormatSuite {
+ /**
+ * Extract payload from BytesWritable serialized form:
+ * 4-byte big-endian length prefix + payload bytes.
+ */
+ def bytesWritablePayload(bytes: Array[Byte], len: Int): Array[Byte] = {
+ if (len < 4) {
+ Array.emptyByteArray
+ } else {
+ val payloadLen = ((bytes(0) & 0xFF) << 24) |
+ ((bytes(1) & 0xFF) << 16) |
+ ((bytes(2) & 0xFF) << 8) |
+ (bytes(3) & 0xFF)
+ if (payloadLen > 0 && payloadLen <= len - 4) {
+ java.util.Arrays.copyOfRange(bytes, 4, 4 + payloadLen)
+ } else {
+ Array.emptyByteArray
+ }
+ }
+ }
+}
diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/SequenceFilePhysicalReplaceSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/SequenceFilePhysicalReplaceSuite.scala
new file mode 100644
index 00000000000..8ea1ff1c454
--- /dev/null
+++ b/tests/src/test/scala/com/nvidia/spark/rapids/SequenceFilePhysicalReplaceSuite.scala
@@ -0,0 +1,176 @@
+/*
+ * Copyright (c) 2026, 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.{DataOutputStream, File, FileOutputStream}
+import java.nio.charset.StandardCharsets
+import java.nio.file.Files
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.{BytesWritable, SequenceFile}
+import org.apache.hadoop.io.SequenceFile.CompressionType
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat
+import org.scalatest.funsuite.AnyFunSuite
+
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+class SequenceFilePhysicalReplaceSuite extends AnyFunSuite {
+
+ private def withPhysicalReplaceEnabledSession(f: SparkSession => Unit): Unit = {
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ val spark = SparkSession.builder()
+ .appName("SequenceFilePhysicalReplaceSuite")
+ .master("local[1]")
+ .config("spark.ui.enabled", "false")
+ .config("spark.sql.shuffle.partitions", "1")
+ .config("spark.sql.extensions", "com.nvidia.spark.rapids.SQLExecPlugin")
+ .config("spark.plugins", "com.nvidia.spark.SQLPlugin")
+ .config("spark.rapids.sql.enabled", "true")
+ .config("spark.rapids.sql.format.sequencefile.rddScan.physicalReplace.enabled", "true")
+ .config("spark.rapids.sql.explain", "ALL")
+ .getOrCreate()
+ try {
+ f(spark)
+ } finally {
+ spark.stop()
+ SparkSession.clearActiveSession()
+ SparkSession.clearDefaultSession()
+ }
+ }
+
+ private def withTempDir(prefix: String)(f: File => Unit): Unit = {
+ val tmpDir = Files.createTempDirectory(prefix).toFile
+ try {
+ f(tmpDir)
+ } finally {
+ def deleteRecursively(file: File): Unit = {
+ if (file.isDirectory) {
+ Option(file.listFiles()).getOrElse(Array.empty).foreach(deleteRecursively)
+ }
+ if (file.exists()) {
+ file.delete()
+ }
+ }
+ deleteRecursively(tmpDir)
+ }
+ }
+
+ private def writeSequenceFile(
+ file: File,
+ conf: Configuration,
+ payloads: Array[Array[Byte]]): Unit = {
+ val path = new Path(file.toURI)
+ val writer = SequenceFile.createWriter(
+ conf,
+ SequenceFile.Writer.file(path),
+ SequenceFile.Writer.keyClass(classOf[BytesWritable]),
+ SequenceFile.Writer.valueClass(classOf[BytesWritable]),
+ SequenceFile.Writer.compression(CompressionType.NONE))
+ try {
+ payloads.zipWithIndex.foreach { case (p, idx) =>
+ val key = new BytesWritable(Array[Byte](
+ ((idx >> 24) & 0xFF).toByte,
+ ((idx >> 16) & 0xFF).toByte,
+ ((idx >> 8) & 0xFF).toByte,
+ (idx & 0xFF).toByte))
+ val value = new BytesWritable(p)
+ writer.append(key, value)
+ }
+ } finally {
+ writer.close()
+ }
+ }
+
+ private def writeLegacySequenceFileHeader(file: File, isCompressed: Boolean): Unit = {
+ val out = new DataOutputStream(new FileOutputStream(file))
+ try {
+ out.writeByte('S')
+ out.writeByte('E')
+ out.writeByte('Q')
+ out.writeByte(4) // pre-block-compression header version
+ org.apache.hadoop.io.Text.writeString(out, classOf[BytesWritable].getName)
+ org.apache.hadoop.io.Text.writeString(out, classOf[BytesWritable].getName)
+ out.writeBoolean(isCompressed)
+ } finally {
+ out.close()
+ }
+ }
+
+ private def readSequenceFileValueOnly(spark: SparkSession, path: String): DataFrame = {
+ import spark.implicits._
+ val sc = spark.sparkContext
+ sc.newAPIHadoopFile(
+ path,
+ classOf[SequenceFileAsBinaryInputFormat],
+ classOf[BytesWritable],
+ classOf[BytesWritable]
+ ).map { case (_, v) =>
+ SequenceFileBinaryFileFormatSuite.bytesWritablePayload(v.getBytes, v.getLength)
+ }.toDF("value")
+ }
+
+ private def hasGpuSequenceFileRDDScan(df: DataFrame): Boolean = {
+ df.queryExecution.executedPlan.collect {
+ case p if p.getClass.getSimpleName == "GpuSequenceFileSerializeFromObjectExec" => 1
+ }.nonEmpty
+ }
+
+ test("Physical replacement hits GPU SequenceFile RDD scan for simple uncompressed path") {
+ withTempDir("seqfile-physical-hit-test") { tmpDir =>
+ val file = new File(tmpDir, "simple.seq")
+ val conf = new Configuration()
+ val payloads = Array(
+ Array[Byte](1, 2, 3),
+ "simple".getBytes(StandardCharsets.UTF_8))
+ writeSequenceFile(file, conf, payloads)
+
+ withPhysicalReplaceEnabledSession { spark =>
+ val df = readSequenceFileValueOnly(spark, file.getAbsolutePath)
+ assert(hasGpuSequenceFileRDDScan(df),
+ s"Expected GPU SequenceFile exec in plan:\n${df.queryExecution.executedPlan}")
+ val got = df.collect().map(_.getAs[Array[Byte]](0)).sortBy(_.length)
+ val expected = payloads.sortBy(_.length)
+ assert(got.length == expected.length)
+ got.zip(expected).foreach { case (actual, exp) =>
+ assert(java.util.Arrays.equals(actual, exp))
+ }
+ }
+ }
+ }
+
+ test("Legacy SequenceFile headers do not read block compression flag") {
+ withTempDir("seqfile-legacy-header-test") { tmpDir =>
+ val file = new File(tmpDir, "legacy.seq")
+ val conf = new Configuration()
+ writeLegacySequenceFileHeader(file, isCompressed = false)
+
+ val method = GpuSequenceFileSerializeFromObjectExecMeta.getClass.getDeclaredMethod(
+ "isCompressedSequenceFile",
+ classOf[Path],
+ classOf[Configuration])
+ method.setAccessible(true)
+ val isCompressed = method.invoke(
+ GpuSequenceFileSerializeFromObjectExecMeta,
+ new Path(file.toURI),
+ conf).asInstanceOf[Boolean]
+
+ assert(!isCompressed)
+ }
+ }
+}
diff --git a/tools/generated_files/330/operatorsScore.csv b/tools/generated_files/330/operatorsScore.csv
index b082fb264ee..f3c36ea8364 100644
--- a/tools/generated_files/330/operatorsScore.csv
+++ b/tools/generated_files/330/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/330/supportedExecs.csv b/tools/generated_files/330/supportedExecs.csv
index 8afd7c5d795..d6a64d21ae0 100644
--- a/tools/generated_files/330/supportedExecs.csv
+++ b/tools/generated_files/330/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/331/operatorsScore.csv b/tools/generated_files/331/operatorsScore.csv
index c553db0b399..ae4a42be82c 100644
--- a/tools/generated_files/331/operatorsScore.csv
+++ b/tools/generated_files/331/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/331/supportedExecs.csv b/tools/generated_files/331/supportedExecs.csv
index 8afd7c5d795..d6a64d21ae0 100644
--- a/tools/generated_files/331/supportedExecs.csv
+++ b/tools/generated_files/331/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/332/operatorsScore.csv b/tools/generated_files/332/operatorsScore.csv
index c553db0b399..ae4a42be82c 100644
--- a/tools/generated_files/332/operatorsScore.csv
+++ b/tools/generated_files/332/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/332/supportedExecs.csv b/tools/generated_files/332/supportedExecs.csv
index 8afd7c5d795..d6a64d21ae0 100644
--- a/tools/generated_files/332/supportedExecs.csv
+++ b/tools/generated_files/332/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/333/operatorsScore.csv b/tools/generated_files/333/operatorsScore.csv
index c553db0b399..ae4a42be82c 100644
--- a/tools/generated_files/333/operatorsScore.csv
+++ b/tools/generated_files/333/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/333/supportedExecs.csv b/tools/generated_files/333/supportedExecs.csv
index 8afd7c5d795..d6a64d21ae0 100644
--- a/tools/generated_files/333/supportedExecs.csv
+++ b/tools/generated_files/333/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/334/operatorsScore.csv b/tools/generated_files/334/operatorsScore.csv
index c553db0b399..ae4a42be82c 100644
--- a/tools/generated_files/334/operatorsScore.csv
+++ b/tools/generated_files/334/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/334/supportedExecs.csv b/tools/generated_files/334/supportedExecs.csv
index 8afd7c5d795..d6a64d21ae0 100644
--- a/tools/generated_files/334/supportedExecs.csv
+++ b/tools/generated_files/334/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/340/operatorsScore.csv b/tools/generated_files/340/operatorsScore.csv
index 6b215bdc0da..72e6115ebed 100644
--- a/tools/generated_files/340/operatorsScore.csv
+++ b/tools/generated_files/340/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/340/supportedExecs.csv b/tools/generated_files/340/supportedExecs.csv
index b3642faf097..aa7d0bca172 100644
--- a/tools/generated_files/340/supportedExecs.csv
+++ b/tools/generated_files/340/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/341/operatorsScore.csv b/tools/generated_files/341/operatorsScore.csv
index 6b215bdc0da..72e6115ebed 100644
--- a/tools/generated_files/341/operatorsScore.csv
+++ b/tools/generated_files/341/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/341/supportedExecs.csv b/tools/generated_files/341/supportedExecs.csv
index b3642faf097..aa7d0bca172 100644
--- a/tools/generated_files/341/supportedExecs.csv
+++ b/tools/generated_files/341/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/342/operatorsScore.csv b/tools/generated_files/342/operatorsScore.csv
index 6b215bdc0da..72e6115ebed 100644
--- a/tools/generated_files/342/operatorsScore.csv
+++ b/tools/generated_files/342/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/342/supportedExecs.csv b/tools/generated_files/342/supportedExecs.csv
index b3642faf097..aa7d0bca172 100644
--- a/tools/generated_files/342/supportedExecs.csv
+++ b/tools/generated_files/342/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/343/operatorsScore.csv b/tools/generated_files/343/operatorsScore.csv
index 6b215bdc0da..72e6115ebed 100644
--- a/tools/generated_files/343/operatorsScore.csv
+++ b/tools/generated_files/343/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/343/supportedExecs.csv b/tools/generated_files/343/supportedExecs.csv
index b3642faf097..aa7d0bca172 100644
--- a/tools/generated_files/343/supportedExecs.csv
+++ b/tools/generated_files/343/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/344/operatorsScore.csv b/tools/generated_files/344/operatorsScore.csv
index 6b215bdc0da..72e6115ebed 100644
--- a/tools/generated_files/344/operatorsScore.csv
+++ b/tools/generated_files/344/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/344/supportedExecs.csv b/tools/generated_files/344/supportedExecs.csv
index b3642faf097..aa7d0bca172 100644
--- a/tools/generated_files/344/supportedExecs.csv
+++ b/tools/generated_files/344/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/350/operatorsScore.csv b/tools/generated_files/350/operatorsScore.csv
index 6ad033830a6..4b8ac80ebef 100644
--- a/tools/generated_files/350/operatorsScore.csv
+++ b/tools/generated_files/350/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/350/supportedExecs.csv b/tools/generated_files/350/supportedExecs.csv
index b880795cb86..59b751e3f79 100644
--- a/tools/generated_files/350/supportedExecs.csv
+++ b/tools/generated_files/350/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/351/operatorsScore.csv b/tools/generated_files/351/operatorsScore.csv
index 6ad033830a6..4b8ac80ebef 100644
--- a/tools/generated_files/351/operatorsScore.csv
+++ b/tools/generated_files/351/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/351/supportedExecs.csv b/tools/generated_files/351/supportedExecs.csv
index b880795cb86..59b751e3f79 100644
--- a/tools/generated_files/351/supportedExecs.csv
+++ b/tools/generated_files/351/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/352/operatorsScore.csv b/tools/generated_files/352/operatorsScore.csv
index bf63bef242e..774e0013ebb 100644
--- a/tools/generated_files/352/operatorsScore.csv
+++ b/tools/generated_files/352/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/352/supportedExecs.csv b/tools/generated_files/352/supportedExecs.csv
index 7a151ecef50..518a1c3380e 100644
--- a/tools/generated_files/352/supportedExecs.csv
+++ b/tools/generated_files/352/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/353/operatorsScore.csv b/tools/generated_files/353/operatorsScore.csv
index bf63bef242e..774e0013ebb 100644
--- a/tools/generated_files/353/operatorsScore.csv
+++ b/tools/generated_files/353/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/353/supportedExecs.csv b/tools/generated_files/353/supportedExecs.csv
index 7a151ecef50..518a1c3380e 100644
--- a/tools/generated_files/353/supportedExecs.csv
+++ b/tools/generated_files/353/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/354/operatorsScore.csv b/tools/generated_files/354/operatorsScore.csv
index bf63bef242e..774e0013ebb 100644
--- a/tools/generated_files/354/operatorsScore.csv
+++ b/tools/generated_files/354/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/354/supportedExecs.csv b/tools/generated_files/354/supportedExecs.csv
index 7a151ecef50..518a1c3380e 100644
--- a/tools/generated_files/354/supportedExecs.csv
+++ b/tools/generated_files/354/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/355/operatorsScore.csv b/tools/generated_files/355/operatorsScore.csv
index bf63bef242e..774e0013ebb 100644
--- a/tools/generated_files/355/operatorsScore.csv
+++ b/tools/generated_files/355/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/355/supportedExecs.csv b/tools/generated_files/355/supportedExecs.csv
index 7a151ecef50..518a1c3380e 100644
--- a/tools/generated_files/355/supportedExecs.csv
+++ b/tools/generated_files/355/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/356/operatorsScore.csv b/tools/generated_files/356/operatorsScore.csv
index bf63bef242e..774e0013ebb 100644
--- a/tools/generated_files/356/operatorsScore.csv
+++ b/tools/generated_files/356/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/356/supportedExecs.csv b/tools/generated_files/356/supportedExecs.csv
index 7a151ecef50..518a1c3380e 100644
--- a/tools/generated_files/356/supportedExecs.csv
+++ b/tools/generated_files/356/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/357/operatorsScore.csv b/tools/generated_files/357/operatorsScore.csv
index bf63bef242e..774e0013ebb 100644
--- a/tools/generated_files/357/operatorsScore.csv
+++ b/tools/generated_files/357/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/357/supportedExecs.csv b/tools/generated_files/357/supportedExecs.csv
index 7a151ecef50..518a1c3380e 100644
--- a/tools/generated_files/357/supportedExecs.csv
+++ b/tools/generated_files/357/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/358/operatorsScore.csv b/tools/generated_files/358/operatorsScore.csv
index bf63bef242e..774e0013ebb 100644
--- a/tools/generated_files/358/operatorsScore.csv
+++ b/tools/generated_files/358/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/358/supportedExecs.csv b/tools/generated_files/358/supportedExecs.csv
index 7a151ecef50..518a1c3380e 100644
--- a/tools/generated_files/358/supportedExecs.csv
+++ b/tools/generated_files/358/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/operatorsScore.csv b/tools/generated_files/operatorsScore.csv
index b082fb264ee..f3c36ea8364 100644
--- a/tools/generated_files/operatorsScore.csv
+++ b/tools/generated_files/operatorsScore.csv
@@ -10,6 +10,7 @@ LocalLimitExec,3.0
ProjectExec,3.0
RangeExec,3.0
SampleExec,3.0
+SerializeFromObjectExec,3.0
SortExec,8.0
SubqueryBroadcastExec,3.0
TakeOrderedAndProjectExec,3.0
diff --git a/tools/generated_files/supportedExecs.csv b/tools/generated_files/supportedExecs.csv
index 8afd7c5d795..d6a64d21ae0 100644
--- a/tools/generated_files/supportedExecs.csv
+++ b/tools/generated_files/supportedExecs.csv
@@ -10,6 +10,7 @@ LocalLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS
ProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,S
RangeExec,S,None,Input/Output,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
SampleExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,S,S
+SerializeFromObjectExec,S,None,Input/Output,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA
SortExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
SubqueryBroadcastExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S
TakeOrderedAndProjectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS