From b7c7559b77b0b41172f39ecd9f0e3a7dfc6a4833 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Sat, 14 Feb 2026 14:51:24 +0530 Subject: [PATCH 01/13] feat: add native Rust Iceberg compaction with DataFusion and JNI bridge --- native/Cargo.lock | 1 + native/core/Cargo.toml | 1 + .../src/execution/iceberg_compaction_jni.rs | 421 +++++++++++++ native/core/src/execution/mod.rs | 1 + .../operators/iceberg_parquet_writer.rs | 591 ++++++++++++++++++ native/core/src/execution/operators/mod.rs | 4 + 6 files changed, 1019 insertions(+) create mode 100644 native/core/src/execution/iceberg_compaction_jni.rs create mode 100644 native/core/src/execution/operators/iceberg_parquet_writer.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index c1224c2a06..abcfd7892a 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1882,6 +1882,7 @@ dependencies = [ "rand 0.10.0", "regex", "reqwest", + "serde", "serde_json", "simd-adler32", "snap", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 07d4c6cc8f..1f65cfdc1a 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -79,6 +79,7 @@ object_store_opendal = {version = "0.55.0", optional = true} hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3"]} opendal = { version ="0.55.0", optional = true, features = ["services-hdfs"] } iceberg = { workspace = true } +serde = { version = "1.0", features = ["derive"] } serde_json = "1.0" uuid = "1.20.0" diff --git a/native/core/src/execution/iceberg_compaction_jni.rs b/native/core/src/execution/iceberg_compaction_jni.rs new file mode 100644 index 0000000000..775d0639c4 --- /dev/null +++ b/native/core/src/execution/iceberg_compaction_jni.rs @@ -0,0 +1,421 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! JNI bridge for Iceberg compaction operations. +//! +//! This module provides JNI functions for native Iceberg compaction (scan + write). +//! Commit is handled by Iceberg's Java API in Scala for reliability. + +use std::collections::HashMap; +use std::sync::Arc; + +use jni::objects::{JClass, JString}; +use jni::sys::jstring; +use jni::JNIEnv; +use serde::{Deserialize, Serialize}; +use tokio::runtime::Runtime; + +use crate::execution::operators::{IcebergCompactionResult, IcebergDataFileMetadata}; + +/// Configuration for Iceberg table metadata passed from JVM +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct IcebergTableConfig { + /// Table identifier (e.g., "db.table_name") + pub table_identifier: String, + /// Metadata file location + pub metadata_location: String, + /// Warehouse location + pub warehouse_location: String, + /// Current snapshot ID (for validation) + pub current_snapshot_id: Option, + /// File IO properties (for object store access) + pub file_io_properties: HashMap, +} + +impl IcebergTableConfig { + /// Deserialize from JSON + pub fn from_json(json: &str) -> Result { + serde_json::from_str(json).map_err(|e| format!("Failed to parse table config: {}", e)) + } + + /// Serialize to JSON + pub fn to_json(&self) -> Result { + serde_json::to_string(self).map_err(|e| format!("Failed to serialize table config: {}", e)) + } +} + +/// File scan task configuration passed from JVM for compaction +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct FileScanTaskConfig { + /// Data file path + pub file_path: String, + /// File size in bytes + pub file_size_bytes: u64, + /// Record count + pub record_count: u64, + /// Partition path (e.g., "year=2024/month=01" or "" for unpartitioned) + pub partition_path: String, + /// Partition spec ID + pub partition_spec_id: i32, + /// Start position in file (for split reads) + pub start: i64, + /// Length to read (for split reads) + pub length: i64, +} + +/// Compaction task configuration +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct CompactionTaskConfig { + /// Table configuration + pub table_config: IcebergTableConfig, + /// Files to compact (scan tasks) + pub file_scan_tasks: Vec, + /// Target file size for output + pub target_file_size_bytes: u64, + /// Compression codec (snappy, zstd, etc.) + pub compression: String, + /// Output data directory + pub data_dir: String, +} + +impl CompactionTaskConfig { + /// Deserialize from JSON + pub fn from_json(json: &str) -> Result { + serde_json::from_str(json).map_err(|e| format!("Failed to parse compaction config: {}", e)) + } +} + +/// Result of native compaction execution +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct NativeCompactionResult { + /// Whether compaction succeeded + pub success: bool, + /// Error message if failed + pub error_message: Option, + /// Compaction result with files to add/delete + pub result: Option, +} + +impl NativeCompactionResult { + /// Create a success result + pub fn success(result: IcebergCompactionResult) -> Self { + Self { + success: true, + error_message: None, + result: Some(result), + } + } + + /// Create a failure result + pub fn failure(error: String) -> Self { + Self { + success: false, + error_message: Some(error), + result: None, + } + } + + /// Serialize to JSON + pub fn to_json(&self) -> Result { + serde_json::to_string(self) + .map_err(|e| format!("Failed to serialize compaction result: {}", e)) + } +} + +/// Execute native Iceberg compaction. +/// +/// This function: +/// 1. Parses the compaction configuration from JSON +/// 2. Creates a native scan plan for the input files +/// 3. Writes compacted output using IcebergParquetWriterExec +/// 4. Returns metadata for new files (does NOT commit) +/// +/// # Arguments +/// * `compaction_config_json` - JSON string with CompactionTaskConfig +/// +/// # Returns +/// * JSON string with NativeCompactionResult +/// +/// # Safety +/// This function is called from JNI and expects valid JNI environment and string parameters. +#[no_mangle] +pub unsafe extern "system" fn Java_org_apache_comet_Native_executeIcebergCompaction( + mut env: JNIEnv, + _class: JClass, + compaction_config_json: JString, +) -> jstring { + let result = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| { + // Get config JSON from JNI + let config_json: String = env + .get_string(&compaction_config_json) + .map_err(|e| format!("Failed to get config string: {}", e))? + .into(); + + // Parse config + let config = CompactionTaskConfig::from_json(&config_json)?; + + // Execute compaction + execute_compaction_internal(&config) + })); + + let native_result = match result { + Ok(Ok(compaction_result)) => NativeCompactionResult::success(compaction_result), + Ok(Err(e)) => NativeCompactionResult::failure(e), + Err(_) => NativeCompactionResult::failure("Panic during compaction execution".to_string()), + }; + + let result_json = native_result + .to_json() + .unwrap_or_else(|e| format!(r#"{{"success":false,"error_message":"{}"}}"#, e)); + + env.new_string(&result_json) + .map(|s| s.into_raw()) + .unwrap_or(std::ptr::null_mut()) +} + +/// Internal compaction execution using DataFusion to read and write Parquet files +fn execute_compaction_internal( + config: &CompactionTaskConfig, +) -> Result { + use datafusion::prelude::*; + use parquet::arrow::ArrowWriter; + use parquet::basic::{Compression, ZstdLevel}; + use parquet::file::properties::WriterProperties; + use std::fs::File; + use uuid::Uuid; + + // Create tokio runtime for async operations + let rt = Runtime::new().map_err(|e| format!("Failed to create runtime: {}", e))?; + + rt.block_on(async { + // Build the list of files to delete (input files) + let files_to_delete: Vec = config + .file_scan_tasks + .iter() + .map(|t| t.file_path.clone()) + .collect(); + + if files_to_delete.is_empty() { + return Ok(IcebergCompactionResult::new()); + } + + // Create DataFusion context + let ctx = SessionContext::new(); + + // Convert file:// URIs to paths for reading + let file_paths: Vec = files_to_delete + .iter() + .map(|p| p.strip_prefix("file://").unwrap_or(p).to_string()) + .collect(); + + // Read all input Parquet files into a single DataFrame + let df = ctx + .read_parquet(file_paths, ParquetReadOptions::default()) + .await + .map_err(|e| format!("Failed to read input files: {}", e))?; + + // Collect all data into batches + let batches = df + .collect() + .await + .map_err(|e| format!("Failed to collect batches: {}", e))?; + + if batches.is_empty() { + return Ok(IcebergCompactionResult::new()); + } + + // Get schema from first batch + let schema = batches[0].schema(); + + // Calculate total rows + let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); + + // Generate output file path + let output_filename = format!("{}-compacted.parquet", Uuid::new_v4()); + let table_location = config + .table_config + .warehouse_location + .strip_prefix("file://") + .unwrap_or(&config.table_config.warehouse_location); + let output_path = format!( + "{}/{}/data/{}", + table_location, + config.table_config.table_identifier.replace('.', "/"), + output_filename + ); + + // Ensure data directory exists + if let Some(parent) = std::path::Path::new(&output_path).parent() { + std::fs::create_dir_all(parent) + .map_err(|e| format!("Failed to create output directory: {}", e))?; + } + + // Configure Parquet writer with compression + let compression = match config.compression.to_lowercase().as_str() { + "zstd" => Compression::ZSTD(ZstdLevel::try_new(3).unwrap()), + "snappy" => Compression::SNAPPY, + "lz4" => Compression::LZ4, + "gzip" => Compression::GZIP(Default::default()), + _ => Compression::ZSTD(ZstdLevel::try_new(3).unwrap()), + }; + + let props = WriterProperties::builder() + .set_compression(compression) + .build(); + + // Write compacted file + let file = File::create(&output_path) + .map_err(|e| format!("Failed to create output file: {}", e))?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), Some(props)) + .map_err(|e| format!("Failed to create Parquet writer: {}", e))?; + + for batch in &batches { + writer + .write(batch) + .map_err(|e| format!("Failed to write batch: {}", e))?; + } + + writer + .close() + .map_err(|e| format!("Failed to close writer: {}", e))?; + + // Get file size + let file_size = std::fs::metadata(&output_path) + .map(|m| m.len()) + .unwrap_or(0); + + // Build result + let mut result = IcebergCompactionResult::new(); + result.files_to_delete = files_to_delete; + result.total_rows = total_rows as u64; + result.total_bytes_written = file_size; + + // Add the new compacted file + let data_file = IcebergDataFileMetadata { + file_path: format!("file://{}", output_path), + file_format: "parquet".to_string(), + record_count: total_rows as u64, + file_size_in_bytes: file_size, + partition_json: config + .file_scan_tasks + .first() + .map(|t| t.partition_path.clone()) + .unwrap_or_default(), + column_sizes: HashMap::new(), + value_counts: HashMap::new(), + null_value_counts: HashMap::new(), + split_offsets: vec![4], // Parquet magic bytes offset + partition_spec_id: config + .file_scan_tasks + .first() + .map(|t| t.partition_spec_id) + .unwrap_or(0), + }; + result.files_to_add.push(data_file); + + Ok(result) + }) +} + +/// Get the version of the native Iceberg compaction library +/// +/// # Safety +/// This function is called from JNI and expects valid JNI environment. +#[no_mangle] +pub unsafe extern "system" fn Java_org_apache_comet_Native_getIcebergCompactionVersion( + env: JNIEnv, + _class: JClass, +) -> jstring { + let version = env!("CARGO_PKG_VERSION"); + env.new_string(version) + .map(|s| s.into_raw()) + .unwrap_or(std::ptr::null_mut()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_table_config_serialization() { + let config = IcebergTableConfig { + table_identifier: "db.test_table".to_string(), + metadata_location: "s3://bucket/warehouse/db/test_table/metadata/v1.json".to_string(), + warehouse_location: "s3://bucket/warehouse".to_string(), + current_snapshot_id: Some(12345), + file_io_properties: HashMap::from([( + "fs.s3a.access.key".to_string(), + "key".to_string(), + )]), + }; + + let json = config.to_json().unwrap(); + let parsed = IcebergTableConfig::from_json(&json).unwrap(); + + assert_eq!(parsed.table_identifier, config.table_identifier); + assert_eq!(parsed.current_snapshot_id, config.current_snapshot_id); + } + + #[test] + fn test_compaction_task_config() { + let config = CompactionTaskConfig { + table_config: IcebergTableConfig { + table_identifier: "db.table".to_string(), + metadata_location: "file:///tmp/metadata.json".to_string(), + warehouse_location: "file:///tmp/warehouse".to_string(), + current_snapshot_id: None, + file_io_properties: HashMap::new(), + }, + file_scan_tasks: vec![FileScanTaskConfig { + file_path: "file:///tmp/data/part-00000.parquet".to_string(), + file_size_bytes: 1024, + record_count: 100, + partition_path: "".to_string(), // unpartitioned + partition_spec_id: 0, + start: 0, + length: 1024, + }], + target_file_size_bytes: 128 * 1024 * 1024, + compression: "zstd".to_string(), + data_dir: "data".to_string(), + }; + + let json = serde_json::to_string(&config).unwrap(); + let parsed = CompactionTaskConfig::from_json(&json).unwrap(); + + assert_eq!(parsed.file_scan_tasks.len(), 1); + assert_eq!(parsed.target_file_size_bytes, 128 * 1024 * 1024); + } + + #[test] + fn test_native_compaction_result() { + let mut result = IcebergCompactionResult::new(); + result.files_to_delete = vec!["old1.parquet".to_string(), "old2.parquet".to_string()]; + result.files_to_add.push(IcebergDataFileMetadata::new( + "new.parquet".to_string(), + 2000, + 2048, + 0, + )); + + let native_result = NativeCompactionResult::success(result); + let json = native_result.to_json().unwrap(); + + assert!(json.contains("success")); + assert!(json.contains("old1.parquet")); + } +} diff --git a/native/core/src/execution/mod.rs b/native/core/src/execution/mod.rs index 85fc672461..53e5d35879 100644 --- a/native/core/src/execution/mod.rs +++ b/native/core/src/execution/mod.rs @@ -18,6 +18,7 @@ //! PoC of vectorization execution through JNI to Rust. pub mod columnar_to_row; pub mod expressions; +pub mod iceberg_compaction_jni; pub mod jni_api; pub(crate) mod metrics; pub mod operators; diff --git a/native/core/src/execution/operators/iceberg_parquet_writer.rs b/native/core/src/execution/operators/iceberg_parquet_writer.rs new file mode 100644 index 0000000000..a6a50744e7 --- /dev/null +++ b/native/core/src/execution/operators/iceberg_parquet_writer.rs @@ -0,0 +1,591 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Iceberg Parquet writer operator for writing RecordBatches to Parquet files +//! with Iceberg-compatible metadata (DataFile structures). + +use std::{ + any::Any, + collections::HashMap, + fmt::{self, Debug, Formatter}, + fs::File, + sync::Arc, +}; + +use arrow::datatypes::{Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use async_trait::async_trait; +use datafusion::{ + error::{DataFusionError, Result}, + execution::context::TaskContext, + physical_expr::EquivalenceProperties, + physical_plan::{ + execution_plan::{Boundedness, EmissionType}, + metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, + stream::RecordBatchStreamAdapter, + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, + SendableRecordBatchStream, Statistics, + }, +}; +use futures::TryStreamExt; +use parquet::{ + arrow::ArrowWriter, + basic::{Compression, ZstdLevel}, + file::properties::WriterProperties, +}; +use serde::{Deserialize, Serialize}; +use uuid::Uuid; + +use crate::execution::shuffle::CompressionCodec; + +/// Metadata for an Iceberg DataFile, returned after writing. +/// This structure mirrors iceberg-rust's DataFile but is serializable for JNI transport. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct IcebergDataFileMetadata { + /// Full URI for the file with FS scheme + pub file_path: String, + /// File format (always "parquet" for this writer) + pub file_format: String, + /// Number of records in this file + pub record_count: u64, + /// Total file size in bytes + pub file_size_in_bytes: u64, + /// Partition values as JSON (empty for unpartitioned tables) + pub partition_json: String, + /// Map from column id to the total size on disk + pub column_sizes: HashMap, + /// Map from column id to number of values + pub value_counts: HashMap, + /// Map from column id to number of null values + pub null_value_counts: HashMap, + /// Split offsets (row group offsets in Parquet) + pub split_offsets: Vec, + /// Partition spec ID + pub partition_spec_id: i32, +} + +impl IcebergDataFileMetadata { + /// Create a new IcebergDataFileMetadata + pub fn new( + file_path: String, + record_count: u64, + file_size_in_bytes: u64, + partition_spec_id: i32, + ) -> Self { + Self { + file_path, + file_format: "parquet".to_string(), + record_count, + file_size_in_bytes, + partition_json: "{}".to_string(), + column_sizes: HashMap::new(), + value_counts: HashMap::new(), + null_value_counts: HashMap::new(), + split_offsets: Vec::new(), + partition_spec_id, + } + } + + /// Set partition values from JSON + pub fn with_partition_json(mut self, partition_json: String) -> Self { + self.partition_json = partition_json; + self + } + + /// Set column sizes + pub fn with_column_sizes(mut self, column_sizes: HashMap) -> Self { + self.column_sizes = column_sizes; + self + } + + /// Set value counts + pub fn with_value_counts(mut self, value_counts: HashMap) -> Self { + self.value_counts = value_counts; + self + } + + /// Set null value counts + pub fn with_null_value_counts(mut self, null_value_counts: HashMap) -> Self { + self.null_value_counts = null_value_counts; + self + } + + /// Set split offsets + pub fn with_split_offsets(mut self, split_offsets: Vec) -> Self { + self.split_offsets = split_offsets; + self + } + + /// Serialize to JSON for JNI transport + pub fn to_json(&self) -> Result { + serde_json::to_string(self).map_err(|e| { + DataFusionError::Execution(format!("Failed to serialize DataFileMetadata: {}", e)) + }) + } + + /// Deserialize from JSON + pub fn from_json(json: &str) -> Result { + serde_json::from_str(json).map_err(|e| { + DataFusionError::Execution(format!("Failed to deserialize DataFileMetadata: {}", e)) + }) + } +} + +/// Iceberg Parquet writer operator that writes input batches to Parquet files +/// and produces DataFile metadata for Iceberg table commits. +#[derive(Debug)] +pub struct IcebergParquetWriterExec { + /// Input execution plan + input: Arc, + /// Table location (warehouse path) + table_location: String, + /// Data directory within table location + data_dir: String, + /// Compression codec + compression: CompressionCodec, + /// Target file size in bytes (for splitting large writes) + target_file_size_bytes: u64, + /// Partition spec ID + partition_spec_id: i32, + /// Column names for the output schema + column_names: Vec, + /// Column IDs for Iceberg schema (maps to column_names) + column_ids: Vec, + /// Object store configuration options + object_store_options: HashMap, + /// Metrics + metrics: ExecutionPlanMetricsSet, + /// Cache for plan properties + cache: PlanProperties, +} + +impl IcebergParquetWriterExec { + /// Create a new IcebergParquetWriterExec + #[allow(clippy::too_many_arguments)] + pub fn try_new( + input: Arc, + table_location: String, + data_dir: String, + compression: CompressionCodec, + target_file_size_bytes: u64, + partition_spec_id: i32, + column_names: Vec, + column_ids: Vec, + object_store_options: HashMap, + ) -> Result { + let input_partitioning = input.output_partitioning().clone(); + + let cache = PlanProperties::new( + EquivalenceProperties::new(Arc::clone(&input.schema())), + input_partitioning, + EmissionType::Final, + Boundedness::Bounded, + ); + + Ok(IcebergParquetWriterExec { + input, + table_location, + data_dir, + compression, + target_file_size_bytes, + partition_spec_id, + column_names, + column_ids, + object_store_options, + metrics: ExecutionPlanMetricsSet::new(), + cache, + }) + } + + fn compression_to_parquet(&self) -> Result { + match self.compression { + CompressionCodec::None => Ok(Compression::UNCOMPRESSED), + CompressionCodec::Zstd(level) => Ok(Compression::ZSTD(ZstdLevel::try_new(level)?)), + CompressionCodec::Lz4Frame => Ok(Compression::LZ4), + CompressionCodec::Snappy => Ok(Compression::SNAPPY), + } + } +} + +impl DisplayAs for IcebergParquetWriterExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!( + f, + "IcebergParquetWriterExec: table_location={}, compression={:?}, target_file_size={}", + self.table_location, self.compression, self.target_file_size_bytes + ) + } + DisplayFormatType::TreeRender => unimplemented!(), + } + } +} + +#[async_trait] +impl ExecutionPlan for IcebergParquetWriterExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "IcebergParquetWriterExec" + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn statistics(&self) -> Result { + self.input.partition_statistics(None) + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn schema(&self) -> SchemaRef { + Arc::new(Schema::empty()) + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + match children.len() { + 1 => Ok(Arc::new(IcebergParquetWriterExec::try_new( + Arc::clone(&children[0]), + self.table_location.clone(), + self.data_dir.clone(), + self.compression.clone(), + self.target_file_size_bytes, + self.partition_spec_id, + self.column_names.clone(), + self.column_ids.clone(), + self.object_store_options.clone(), + )?)), + _ => Err(DataFusionError::Internal( + "IcebergParquetWriterExec requires exactly one child".to_string(), + )), + } + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + let files_written = MetricBuilder::new(&self.metrics).counter("files_written", partition); + let bytes_written = MetricBuilder::new(&self.metrics).counter("bytes_written", partition); + let rows_written = MetricBuilder::new(&self.metrics).counter("rows_written", partition); + + let input = self.input.execute(partition, context)?; + let input_schema = self.input.schema(); + let compression = self.compression_to_parquet()?; + let column_names = self.column_names.clone(); + let column_ids = self.column_ids.clone(); + let table_location = self.table_location.clone(); + let data_dir = self.data_dir.clone(); + let target_file_size_bytes = self.target_file_size_bytes; + let partition_spec_id = self.partition_spec_id; + + assert_eq!(input_schema.fields().len(), column_names.len()); + + // Build output schema with correct column names + let fields: Vec<_> = input_schema + .fields() + .iter() + .enumerate() + .map(|(i, field)| Arc::new(field.as_ref().clone().with_name(&column_names[i]))) + .collect(); + let output_schema = Arc::new(Schema::new(fields)); + + let write_task = async move { + let mut stream = input; + let mut total_rows = 0u64; + let mut total_bytes = 0u64; + let mut file_index = 0usize; + let mut data_files: Vec = Vec::new(); + + // Current file state + let mut current_file_path: Option = None; + let mut current_writer: Option> = None; + let mut current_file_rows = 0u64; + let mut current_value_counts: HashMap = HashMap::new(); + let mut current_null_counts: HashMap = HashMap::new(); + + // Helper to finalize current file + let finalize_file = |writer: ArrowWriter, + file_path: &str, + file_rows: u64, + value_counts: HashMap, + null_counts: HashMap, + partition_spec_id: i32| + -> Result { + writer.close().map_err(|e| { + DataFusionError::Execution(format!("Failed to close writer: {}", e)) + })?; + + let local_path = file_path + .strip_prefix("file://") + .or_else(|| file_path.strip_prefix("file:")) + .unwrap_or(file_path); + let file_size = std::fs::metadata(local_path).map(|m| m.len()).unwrap_or(0); + + let metadata = IcebergDataFileMetadata::new( + file_path.to_string(), + file_rows, + file_size, + partition_spec_id, + ) + .with_value_counts(value_counts) + .with_null_value_counts(null_counts); + + Ok(metadata) + }; + + while let Some(batch_result) = stream.try_next().await.transpose() { + let batch = batch_result?; + let batch_rows = batch.num_rows() as u64; + + // Check if we need to start a new file + let need_new_file = current_writer.is_none() + || (target_file_size_bytes > 0 && current_file_rows >= target_file_size_bytes); + + if need_new_file { + // Finalize current file if exists + if let (Some(writer), Some(ref path)) = + (current_writer.take(), current_file_path.take()) + { + let metadata = finalize_file( + writer, + path, + current_file_rows, + std::mem::take(&mut current_value_counts), + std::mem::take(&mut current_null_counts), + partition_spec_id, + )?; + total_bytes += metadata.file_size_in_bytes; + data_files.push(metadata); + } + + // Start new file + file_index += 1; + let uuid = Uuid::now_v7(); + let new_file_path = format!( + "{}/{}/{:05}-{:05}-{}.parquet", + table_location, data_dir, partition, file_index, uuid + ); + + let local_path = new_file_path + .strip_prefix("file://") + .or_else(|| new_file_path.strip_prefix("file:")) + .unwrap_or(&new_file_path); + + if let Some(parent) = std::path::Path::new(local_path).parent() { + std::fs::create_dir_all(parent).map_err(|e| { + DataFusionError::Execution(format!("Failed to create directory: {}", e)) + })?; + } + + let file = File::create(local_path).map_err(|e| { + DataFusionError::Execution(format!("Failed to create file: {}", e)) + })?; + + let props = WriterProperties::builder() + .set_compression(compression) + .build(); + + let writer = + ArrowWriter::try_new(file, Arc::clone(&output_schema), Some(props)) + .map_err(|e| { + DataFusionError::Execution(format!( + "Failed to create writer: {}", + e + )) + })?; + + current_writer = Some(writer); + current_file_path = Some(new_file_path); + current_file_rows = 0; + } + + // Rename columns in batch + let renamed_batch = + RecordBatch::try_new(Arc::clone(&output_schema), batch.columns().to_vec()) + .map_err(|e| { + DataFusionError::Execution(format!( + "Failed to rename batch columns: {}", + e + )) + })?; + + // Update column statistics + for (i, col_id) in column_ids.iter().enumerate() { + let col = renamed_batch.column(i); + let null_count = col.null_count() as u64; + let value_count = col.len() as u64; + + *current_value_counts.entry(*col_id).or_insert(0) += value_count; + *current_null_counts.entry(*col_id).or_insert(0) += null_count; + } + + // Write batch + if let Some(ref mut writer) = current_writer { + writer.write(&renamed_batch).map_err(|e| { + DataFusionError::Execution(format!("Failed to write batch: {}", e)) + })?; + } + + current_file_rows += batch_rows; + total_rows += batch_rows; + } + + // Finalize last file + if let (Some(writer), Some(ref path)) = + (current_writer.take(), current_file_path.take()) + { + let metadata = finalize_file( + writer, + path, + current_file_rows, + current_value_counts, + current_null_counts, + partition_spec_id, + )?; + total_bytes += metadata.file_size_in_bytes; + data_files.push(metadata); + } + + // Update metrics + files_written.add(data_files.len()); + bytes_written.add(total_bytes as usize); + rows_written.add(total_rows as usize); + + eprintln!( + "IcebergParquetWriter: wrote {} files, {} bytes, {} rows", + data_files.len(), + total_bytes, + total_rows + ); + + Ok::<_, DataFusionError>(futures::stream::empty()) + }; + + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(write_task).try_flatten(), + ))) + } +} + +/// Result of Iceberg compaction operation +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct IcebergCompactionResult { + /// Files that were deleted (old fragmented files) + pub files_to_delete: Vec, + /// New compacted files with metadata + pub files_to_add: Vec, + /// Total rows processed + pub total_rows: u64, + /// Total bytes written + pub total_bytes_written: u64, +} + +impl IcebergCompactionResult { + /// Create a new compaction result + pub fn new() -> Self { + Self { + files_to_delete: Vec::new(), + files_to_add: Vec::new(), + total_rows: 0, + total_bytes_written: 0, + } + } + + /// Serialize to JSON for JNI transport + pub fn to_json(&self) -> Result { + serde_json::to_string(self).map_err(|e| { + DataFusionError::Execution(format!("Failed to serialize CompactionResult: {}", e)) + }) + } + + /// Deserialize from JSON + pub fn from_json(json: &str) -> Result { + serde_json::from_str(json).map_err(|e| { + DataFusionError::Execution(format!("Failed to deserialize CompactionResult: {}", e)) + }) + } +} + +impl Default for IcebergCompactionResult { + fn default() -> Self { + Self::new() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_data_file_metadata_serialization() { + let metadata = IcebergDataFileMetadata::new( + "s3://bucket/table/data/part-00000.parquet".to_string(), + 1000, + 1024 * 1024, + 0, + ) + .with_partition_json(r#"{"date": "2024-01-01"}"#.to_string()); + + let json = metadata.to_json().unwrap(); + let deserialized = IcebergDataFileMetadata::from_json(&json).unwrap(); + + assert_eq!(deserialized.file_path, metadata.file_path); + assert_eq!(deserialized.record_count, metadata.record_count); + assert_eq!(deserialized.file_size_in_bytes, metadata.file_size_in_bytes); + assert_eq!(deserialized.partition_json, metadata.partition_json); + } + + #[test] + fn test_compaction_result_serialization() { + let mut result = IcebergCompactionResult::new(); + result.files_to_delete = vec![ + "s3://bucket/table/data/old-1.parquet".to_string(), + "s3://bucket/table/data/old-2.parquet".to_string(), + ]; + result.files_to_add.push(IcebergDataFileMetadata::new( + "s3://bucket/table/data/compacted.parquet".to_string(), + 2000, + 2 * 1024 * 1024, + 0, + )); + result.total_rows = 2000; + result.total_bytes_written = 2 * 1024 * 1024; + + let json = result.to_json().unwrap(); + let deserialized = IcebergCompactionResult::from_json(&json).unwrap(); + + assert_eq!(deserialized.files_to_delete.len(), 2); + assert_eq!(deserialized.files_to_add.len(), 1); + assert_eq!(deserialized.total_rows, 2000); + } +} diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs index 07ee995367..6380324923 100644 --- a/native/core/src/execution/operators/mod.rs +++ b/native/core/src/execution/operators/mod.rs @@ -28,8 +28,12 @@ pub use scan::*; mod copy; mod expand; pub use expand::ExpandExec; +mod iceberg_parquet_writer; mod iceberg_scan; mod parquet_writer; +pub use iceberg_parquet_writer::{ + IcebergCompactionResult, IcebergDataFileMetadata, IcebergParquetWriterExec, +}; pub use parquet_writer::ParquetWriterExec; mod csv_scan; pub mod projection; From 9c01c5733d444d19c86b8631a989bdb765bdbf4a Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Sat, 14 Feb 2026 14:51:33 +0530 Subject: [PATCH 02/13] feat: add Scala JNI interface and CometNativeCompaction for Iceberg --- .../main/scala/org/apache/comet/Native.scala | 33 ++ .../comet/iceberg/IcebergReflection.scala | 66 +-- .../sql/comet/CometNativeCompaction.scala | 391 ++++++++++++++++++ 3 files changed, 467 insertions(+), 23 deletions(-) create mode 100644 spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index 55e0c70e72..a942f78c3f 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -248,4 +248,37 @@ class Native extends NativeBase { */ @native def columnarToRowClose(c2rHandle: Long): Unit + // Native Iceberg Compaction methods + + /** + * Execute native Iceberg compaction. + * + * This function: + * 1. Parses the compaction configuration from JSON 2. Creates a native scan plan for the + * input files 3. Writes compacted output using IcebergParquetWriterExec 4. Returns + * metadata for new files (does NOT commit) + * + * @param compactionConfigJson + * JSON string containing CompactionTaskConfig with: + * - table_config: IcebergTableConfig (table identifier, metadata location, etc.) + * - file_scan_tasks: Array of FileScanTaskConfig (files to compact) + * - target_file_size_bytes: Target size for output files + * - compression: Compression codec (snappy, zstd, etc.) + * - data_dir: Output data directory + * @return + * JSON string containing NativeCompactionResult with: + * - success: Boolean indicating if compaction succeeded + * - error_message: Error message if failed + * - result: IcebergCompactionResult with files_to_delete and files_to_add + */ + @native def executeIcebergCompaction(compactionConfigJson: String): String + + /** + * Get the version of the native Iceberg compaction library. + * + * @return + * Version string of the native library + */ + @native def getIcebergCompactionVersion(): String + } diff --git a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala index 7642749ad8..1922a3ec6a 100644 --- a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala +++ b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala @@ -154,39 +154,59 @@ object IcebergReflection extends Logging { /** * Gets the tasks from a SparkScan. * - * The tasks() method is protected in SparkScan, requiring reflection to access. + * SparkBatchQueryScan (via SparkPartitioningAwareScan) has tasks() method. SparkStagedScan (via + * SparkScan) has taskGroups() - we extract tasks from groups. */ def getTasks(scan: Any): Option[java.util.List[_]] = { - try { - val tasksMethod = scan.getClass.getSuperclass - .getDeclaredMethod("tasks") - tasksMethod.setAccessible(true) - Some(tasksMethod.invoke(scan).asInstanceOf[java.util.List[_]]) - } catch { - case e: Exception => - logError( - s"Iceberg reflection failure: Failed to get tasks from SparkScan: ${e.getMessage}") - None + // Try tasks() first (SparkPartitioningAwareScan hierarchy) + findMethodInHierarchy(scan.getClass, "tasks").flatMap { tasksMethod => + try { + return Some(tasksMethod.invoke(scan).asInstanceOf[java.util.List[_]]) + } catch { + case _: Exception => None + } + } + + // Fall back to taskGroups() (SparkScan hierarchy - used by SparkStagedScan) + findMethodInHierarchy(scan.getClass, "taskGroups").flatMap { taskGroupsMethod => + try { + val taskGroups = taskGroupsMethod.invoke(scan).asInstanceOf[java.util.List[_]] + // Extract individual tasks from each ScanTaskGroup + val allTasks = new java.util.ArrayList[Any]() + val iter = taskGroups.iterator() + while (iter.hasNext) { + val group = iter.next() + val tasksMethod = group.getClass.getMethod("tasks") + val groupTasks = tasksMethod.invoke(group).asInstanceOf[java.lang.Iterable[_]] + groupTasks.forEach(task => allTasks.add(task)) + } + Some(allTasks.asInstanceOf[java.util.List[_]]) + } catch { + case e: Exception => + logError( + s"Iceberg reflection failure: Failed to get tasks from SparkScan: ${e.getMessage}") + None + } } } /** * Gets the filter expressions from a SparkScan. * - * The filterExpressions() method is protected in SparkScan. + * The filterExpressions() method is protected in SparkScan. Uses findMethodInHierarchy to + * support both SparkBatchQueryScan and SparkStagedScan. */ def getFilterExpressions(scan: Any): Option[java.util.List[_]] = { - try { - val filterExpressionsMethod = scan.getClass.getSuperclass.getSuperclass - .getDeclaredMethod("filterExpressions") - filterExpressionsMethod.setAccessible(true) - Some(filterExpressionsMethod.invoke(scan).asInstanceOf[java.util.List[_]]) - } catch { - case e: Exception => - logError( - "Iceberg reflection failure: Failed to get filter expressions from SparkScan: " + - s"${e.getMessage}") - None + findMethodInHierarchy(scan.getClass, "filterExpressions").flatMap { filterExpressionsMethod => + try { + Some(filterExpressionsMethod.invoke(scan).asInstanceOf[java.util.List[_]]) + } catch { + case e: Exception => + logError( + "Iceberg reflection failure: Failed to get filter expressions from SparkScan: " + + s"${e.getMessage}") + None + } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala new file mode 100644 index 0000000000..b139ee993f --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import scala.collection.JavaConverters._ + +import org.apache.iceberg.{FileScanTask, Table} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule + +import org.apache.comet.Native + +/** Configuration for Iceberg table metadata passed to native code. */ +case class IcebergTableConfig( + table_identifier: String, + metadata_location: String, + warehouse_location: String, + current_snapshot_id: Option[Long], + file_io_properties: Map[String, String]) + +/** File scan task configuration for native compaction. */ +case class FileScanTaskConfig( + file_path: String, + file_size_bytes: Long, + record_count: Long, + partition_path: String, // e.g., "year=2024/month=01" or "" for unpartitioned + partition_spec_id: Int, + start: Long, + length: Long) + +/** Compaction task configuration for native execution. */ +case class CompactionTaskConfig( + table_config: IcebergTableConfig, + file_scan_tasks: Seq[FileScanTaskConfig], + target_file_size_bytes: Long, + compression: String, + data_dir: String) + +/** Iceberg DataFile metadata from native compaction. */ +case class IcebergDataFileMetadata( + file_path: String, + file_format: String, + record_count: Long, + file_size_in_bytes: Long, + partition_json: String, + column_sizes: Map[Int, Long], + value_counts: Map[Int, Long], + null_value_counts: Map[Int, Long], + split_offsets: Seq[Long], + partition_spec_id: Int) + +/** Result of native Iceberg compaction. */ +case class IcebergCompactionResult( + files_to_delete: Seq[String], + files_to_add: Seq[IcebergDataFileMetadata], + total_rows: Long, + total_bytes_written: Long) + +/** Native compaction execution result. */ +case class NativeCompactionResult( + success: Boolean, + error_message: Option[String], + result: Option[IcebergCompactionResult]) + +/** + * Native Iceberg compaction using Rust/DataFusion for scan+write, Java API for commit. + */ +class CometNativeCompaction(spark: SparkSession) extends Logging { + + private val objectMapper = new ObjectMapper().registerModule(DefaultScalaModule) + private val native = new Native() + + val DEFAULT_TARGET_FILE_SIZE: Long = 128 * 1024 * 1024 // 128 MB + val DEFAULT_COMPRESSION: String = "zstd" + + /** Rewrite data files using native scan+write with Iceberg Java API commit. */ + def rewriteDataFiles( + table: Table, + targetFileSizeBytes: Long = DEFAULT_TARGET_FILE_SIZE, + compression: String = DEFAULT_COMPRESSION): CompactionSummary = { + + logInfo(s"Starting native compaction for table ${table.name()}") + + val currentSnapshot = Option(table.currentSnapshot()) + val currentSnapshotId = currentSnapshot.map(_.snapshotId()) + val tableConfig = buildTableConfig(table, currentSnapshotId) + val fileGroups = planFileGroups(table, targetFileSizeBytes) + + if (fileGroups.isEmpty) { + logInfo("No files to compact") + return CompactionSummary( + filesDeleted = 0, + filesAdded = 0, + bytesDeleted = 0, + bytesAdded = 0, + rowsProcessed = 0) + } + + logInfo(s"Found ${fileGroups.size} file groups to compact") + + var totalFilesDeleted = 0 + var totalFilesAdded = 0 + var totalBytesDeleted = 0L + var totalBytesAdded = 0L + var totalRowsProcessed = 0L + + val allFilesToDelete = scala.collection.mutable.ArrayBuffer[String]() + val allFilesToAdd = scala.collection.mutable.ArrayBuffer[IcebergDataFileMetadata]() + + for ((group, groupIndex) <- fileGroups.zipWithIndex) { + logInfo( + s"Processing file group ${groupIndex + 1}/${fileGroups.size} " + + s"with ${group.size} files") + + val compactionConfig = buildCompactionConfig( + tableConfig, + group, + targetFileSizeBytes, + compression, + table.location()) + val result = executeNativeCompaction(compactionConfig) + + result match { + case NativeCompactionResult(true, _, Some(compactionResult)) => + allFilesToDelete ++= compactionResult.files_to_delete + allFilesToAdd ++= compactionResult.files_to_add + totalFilesDeleted += compactionResult.files_to_delete.size + totalFilesAdded += compactionResult.files_to_add.size + totalRowsProcessed += compactionResult.total_rows + totalBytesAdded += compactionResult.total_bytes_written + for (task <- group) { + totalBytesDeleted += task.file().fileSizeInBytes() + } + + logInfo( + s"Group ${groupIndex + 1} completed: " + + s"${compactionResult.files_to_delete.size} files deleted, " + + s"${compactionResult.files_to_add.size} files added") + + case NativeCompactionResult(false, Some(error), _) => + logError(s"Native compaction failed for group ${groupIndex + 1}: $error") + throw new RuntimeException(s"Native compaction failed: $error") + + case _ => + logError(s"Unexpected native compaction result for group ${groupIndex + 1}") + throw new RuntimeException("Unexpected native compaction result") + } + } + + if (allFilesToAdd.nonEmpty) { + logInfo( + s"Committing compaction: ${allFilesToDelete.size} files to delete, " + + s"${allFilesToAdd.size} files to add") + + val commitSuccess = commitCompaction(table, allFilesToDelete, allFilesToAdd) + + if (!commitSuccess) { + throw new RuntimeException("Failed to commit compaction results") + } + + logInfo("Compaction committed successfully") + } + + CompactionSummary( + filesDeleted = totalFilesDeleted, + filesAdded = totalFilesAdded, + bytesDeleted = totalBytesDeleted, + bytesAdded = totalBytesAdded, + rowsProcessed = totalRowsProcessed) + } + + private def buildTableConfig( + table: Table, + currentSnapshotId: Option[Long]): IcebergTableConfig = { + + val tableLocation = table.location() + val metadataLocation = + try { + table + .asInstanceOf[org.apache.iceberg.BaseTable] + .operations() + .current() + .metadataFileLocation() + } catch { + case _: Exception => s"$tableLocation/metadata/v1.metadata.json" + } + val warehouseLocation = tableLocation.substring(0, tableLocation.lastIndexOf('/')) + val fileIOProperties = table.properties().asScala.toMap + + IcebergTableConfig( + table_identifier = table.name(), + metadata_location = metadataLocation, + warehouse_location = warehouseLocation, + current_snapshot_id = currentSnapshotId, + file_io_properties = fileIOProperties) + } + + /** Plan file groups using bin-pack strategy. */ + private def planFileGroups(table: Table, targetFileSizeBytes: Long): Seq[Seq[FileScanTask]] = { + + val currentSnapshot = table.currentSnapshot() + if (currentSnapshot == null) { + return Seq.empty + } + + val scanTasks = table + .newScan() + .planFiles() + .iterator() + .asScala + .toSeq + + val smallFiles = scanTasks.filter(_.file().fileSizeInBytes() < targetFileSizeBytes) + + if (smallFiles.size < 2) { + return Seq.empty + } + + val groups = scala.collection.mutable.ArrayBuffer[Seq[FileScanTask]]() + var currentGroup = scala.collection.mutable.ArrayBuffer[FileScanTask]() + var currentGroupSize = 0L + + for (task <- smallFiles.sortBy(_.file().fileSizeInBytes())) { + if (currentGroupSize + task.file().fileSizeInBytes() > targetFileSizeBytes * 2) { + if (currentGroup.size >= 2) { + groups += currentGroup.toSeq + } + currentGroup = scala.collection.mutable.ArrayBuffer[FileScanTask]() + currentGroupSize = 0L + } + + currentGroup += task + currentGroupSize += task.file().fileSizeInBytes() + } + + if (currentGroup.size >= 2) { + groups += currentGroup.toSeq + } + + groups.toSeq + } + + private def buildCompactionConfig( + tableConfig: IcebergTableConfig, + tasks: Seq[FileScanTask], + targetFileSizeBytes: Long, + compression: String, + tableLocation: String): CompactionTaskConfig = { + + val fileScanTaskConfigs = tasks.map { task => + val partitionPath = task.spec().partitionToPath(task.file().partition()) + + FileScanTaskConfig( + file_path = task.file().path().toString, + file_size_bytes = task.file().fileSizeInBytes(), + record_count = task.file().recordCount(), + partition_path = partitionPath, + partition_spec_id = task.spec().specId(), + start = task.start(), + length = task.length()) + } + + CompactionTaskConfig( + table_config = tableConfig, + file_scan_tasks = fileScanTaskConfigs, + target_file_size_bytes = targetFileSizeBytes, + compression = compression, + data_dir = "data") + } + + /** Execute native compaction via JNI. */ + private def executeNativeCompaction(config: CompactionTaskConfig): NativeCompactionResult = { + val configJson = objectMapper.writeValueAsString(config) + + logDebug(s"Executing native compaction with config: $configJson") + + val resultJson = native.executeIcebergCompaction(configJson) + + logDebug(s"Native compaction result: $resultJson") + + objectMapper.readValue(resultJson, classOf[NativeCompactionResult]) + } + + /** Commit using Iceberg's Java RewriteFiles API. */ + private def commitCompaction( + table: Table, + filesToDelete: Seq[String], + filesToAdd: Seq[IcebergDataFileMetadata]): Boolean = { + + import org.apache.iceberg.{DataFile, DataFiles, FileFormat, PartitionSpec} + + try { + val specs = table.specs() + val deleteFiles: java.util.Set[DataFile] = new java.util.HashSet[DataFile]() + val deletePathSet = filesToDelete.toSet + val snapshot = table.currentSnapshot() + if (snapshot != null) { + import scala.jdk.CollectionConverters._ + val fileScanTasks = table.newScan().planFiles().iterator().asScala + fileScanTasks.foreach { task => + val dataFile = task.file() + if (deletePathSet.contains(dataFile.path().toString)) { + deleteFiles.add(dataFile) + } + } + } + + val addFiles: java.util.Set[DataFile] = new java.util.HashSet[DataFile]() + filesToAdd.foreach { m => + val spec: PartitionSpec = specs.get(m.partition_spec_id) + val builder = DataFiles + .builder(spec) + .withPath(m.file_path) + .withFormat(FileFormat.PARQUET) + .withFileSizeInBytes(m.file_size_in_bytes) + .withRecordCount(m.record_count) + if (m.partition_json != null && m.partition_json.nonEmpty && m.partition_json != "{}") { + builder.withPartitionPath(m.partition_json) + } + + addFiles.add(builder.build()) + } + + table.newRewrite().rewriteFiles(deleteFiles, addFiles).commit() + + logInfo( + s"Committed compaction: ${filesToDelete.size} files deleted, ${filesToAdd.size} files added") + true + } catch { + case e: Exception => + logError(s"Failed to commit compaction: ${e.getMessage}", e) + false + } + } +} + +/** Summary of compaction results. */ +case class CompactionSummary( + filesDeleted: Int, + filesAdded: Int, + bytesDeleted: Long, + bytesAdded: Long, + rowsProcessed: Long) { + + def compactionRatio: Double = { + if (bytesDeleted > 0) { + (bytesDeleted - bytesAdded).toDouble / bytesDeleted + } else { + 0.0 + } + } + + override def toString: String = { + f"CompactionSummary(files: $filesDeleted -> $filesAdded, " + + f"bytes: ${bytesDeleted / 1024 / 1024}%.1f MB -> ${bytesAdded / 1024 / 1024}%.1f MB, " + + f"rows: $rowsProcessed, ratio: ${compactionRatio * 100}%.1f%%)" + } +} + +object CometNativeCompaction { + def apply(spark: SparkSession): CometNativeCompaction = new CometNativeCompaction(spark) + + def isAvailable: Boolean = { + try { + val version = new Native().getIcebergCompactionVersion() + version != null && version.nonEmpty + } catch { + case _: UnsatisfiedLinkError | _: Exception => false + } + } +} From 80051d024d794c3400dfec795c549e9fdb01071e Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Sat, 14 Feb 2026 14:51:41 +0530 Subject: [PATCH 03/13] feat: add COMET_ICEBERG_COMPACTION_ENABLED config option --- common/src/main/scala/org/apache/comet/CometConf.scala | 10 ++++++++++ .../scala/org/apache/comet/rules/CometScanRule.scala | 8 ++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 49eb55479b..3dc61ebdfa 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -150,6 +150,16 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_ICEBERG_COMPACTION_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.iceberg.compaction.enabled") + .category(CATEGORY_TESTING) + .doc( + "Whether to enable Comet-accelerated Iceberg compaction. When enabled, " + + "CALL rewrite_data_files() uses Comet's native scan for the read path, " + + "reducing JVM overhead during compaction. Experimental.") + .booleanConf + .createWithDefault(false) + val COMET_CSV_V2_NATIVE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.csv.v2.enabled") .category(CATEGORY_TESTING) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index bb37515ab9..e015c0d39a 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -315,9 +315,13 @@ case class CometScanRule(session: SparkSession) } // Iceberg scan - detected by class name (works with unpatched Iceberg) + // SparkBatchQueryScan: normal query scans + // SparkStagedScan: compaction scans via ScanTaskSetManager case _ - if scanExec.scan.getClass.getName == - "org.apache.iceberg.spark.source.SparkBatchQueryScan" => + if Set( + "org.apache.iceberg.spark.source.SparkBatchQueryScan", + "org.apache.iceberg.spark.source.SparkStagedScan").contains( + scanExec.scan.getClass.getName) => val fallbackReasons = new ListBuffer[String]() // Native Iceberg scan requires both configs to be enabled From 9dca0f30b93675769c2529e0f5ecf34c49085ec6 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Sat, 14 Feb 2026 14:51:50 +0530 Subject: [PATCH 04/13] test: add Iceberg compaction unit tests and TPC-H benchmark --- .../CometIcebergCompactionBenchmarkTest.scala | 281 ++++++++++++ .../comet/CometIcebergCompactionSuite.scala | 421 ++++++++++++++++++ .../CometIcebergTPCCompactionBenchmark.scala | 378 ++++++++++++++++ 3 files changed, 1080 insertions(+) create mode 100644 spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala create mode 100644 spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala new file mode 100644 index 0000000000..e8bb623cdc --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala @@ -0,0 +1,281 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.io.File +import java.nio.file.Files + +import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.actions.SparkActions +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.comet.CometNativeCompaction + +/** + * Simple benchmark test for Iceberg compaction comparing Spark default vs Native compaction. Run + * with: mvn test -pl spark -Dsuites=org.apache.comet.CometIcebergCompactionBenchmarkTest + */ +class CometIcebergCompactionBenchmarkTest extends CometTestBase { + + private val dataLocation = "/tmp/tpch/sf1_parquet" + + private def icebergAvailable: Boolean = { + try { + Class.forName("org.apache.iceberg.catalog.Catalog") + true + } catch { + case _: ClassNotFoundException => false + } + } + + private def tpcDataAvailable: Boolean = { + new File(s"$dataLocation/lineitem").exists() + } + + private def withTempIcebergDir(f: File => Unit): Unit = { + val dir = Files.createTempDirectory("comet-benchmark").toFile + try { + f(dir) + } finally { + def deleteRecursively(file: File): Unit = { + if (file.isDirectory) file.listFiles().foreach(deleteRecursively) + file.delete() + } + deleteRecursively(dir) + } + } + + private def icebergCatalogConf(warehouseDir: File): Map[String, String] = Map( + "spark.sql.catalog.bench_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.bench_cat.type" -> "hadoop", + "spark.sql.catalog.bench_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_COMPACTION_ENABLED.key -> "true") + + private def runTableBenchmark( + warehouseDir: File, + sourceTable: String, + schema: String, + numFragments: Int, + rowsPerFragment: Int): (Long, Long, Double) = { + + val tableName = s"bench_cat.db.${sourceTable}_bench" + + // Create fragmented Iceberg table + spark.sql(s"CREATE TABLE $tableName ($schema) USING iceberg") + + // Insert fragments from TPC-H source + val cols = schema.split(",").map(_.trim.split(" ")(0)).mkString(", ") + for (i <- 0 until numFragments) { + spark.sql(s""" + INSERT INTO $tableName + SELECT $cols FROM parquet.`$dataLocation/$sourceTable` + LIMIT $rowsPerFragment OFFSET ${i * rowsPerFragment} + """) + } + + val filesBefore = spark.sql(s"SELECT * FROM $tableName.files").count() + + // Benchmark 1: Spark default compaction + val sparkStart = System.nanoTime() + val sparkTable = Spark3Util.loadIcebergTable(spark, tableName) + SparkActions.get(spark).rewriteDataFiles(sparkTable).binPack().execute() + val sparkDuration = (System.nanoTime() - sparkStart) / 1000000 + + // Re-create for native benchmark + spark.sql(s"DROP TABLE $tableName") + spark.sql(s"CREATE TABLE $tableName ($schema) USING iceberg") + for (i <- 0 until numFragments) { + spark.sql(s""" + INSERT INTO $tableName + SELECT $cols FROM parquet.`$dataLocation/$sourceTable` + LIMIT $rowsPerFragment OFFSET ${i * rowsPerFragment} + """) + } + + // Benchmark 2: Native compaction + val nativeStart = System.nanoTime() + val nativeTable = Spark3Util.loadIcebergTable(spark, tableName) + CometNativeCompaction(spark).rewriteDataFiles(nativeTable) + val nativeDuration = (System.nanoTime() - nativeStart) / 1000000 + + spark.sql(s"DROP TABLE $tableName") + + val speedup = if (nativeDuration > 0) sparkDuration.toDouble / nativeDuration else 0 + (sparkDuration, nativeDuration, speedup) + } + + test("TPC-H compaction benchmark: lineitem, orders, customer") { + assume(icebergAvailable, "Iceberg not available") + assume(tpcDataAvailable, s"TPC-H data not found at $dataLocation") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + val numFragments = 10 + val rowsPerFragment = 5000 + + // scalastyle:off println + println("\n" + "=" * 60) + println(" TPC-H ICEBERG COMPACTION BENCHMARK") + println(" Spark Default vs Native (Comet) Compaction") + println("=" * 60) + println(f"${"Table"}%-15s ${"Spark(ms)"}%12s ${"Native(ms)"}%12s ${"Speedup"}%10s") + println("-" * 60) + + // Lineitem benchmark + val lineitemSchema = + """l_orderkey BIGINT, l_partkey BIGINT, l_suppkey BIGINT, l_linenumber INT, + l_quantity DOUBLE, l_extendedprice DOUBLE, l_discount DOUBLE, l_tax DOUBLE, + l_returnflag STRING, l_linestatus STRING""" + val (lSpark, lNative, lSpeedup) = + runTableBenchmark( + warehouseDir, + "lineitem", + lineitemSchema, + numFragments, + rowsPerFragment) + println(f"${"lineitem"}%-15s $lSpark%12d $lNative%12d ${lSpeedup}%9.2fx") + + // Orders benchmark + val ordersSchema = + """o_orderkey BIGINT, o_custkey BIGINT, o_orderstatus STRING, o_totalprice DOUBLE, + o_orderdate DATE, o_orderpriority STRING, o_clerk STRING, o_shippriority INT, + o_comment STRING""" + val (oSpark, oNative, oSpeedup) = + runTableBenchmark(warehouseDir, "orders", ordersSchema, numFragments, rowsPerFragment) + println(f"${"orders"}%-15s $oSpark%12d $oNative%12d ${oSpeedup}%9.2fx") + + // Customer benchmark + val customerSchema = + """c_custkey BIGINT, c_name STRING, c_address STRING, c_nationkey BIGINT, + c_phone STRING, c_acctbal DOUBLE, c_mktsegment STRING, c_comment STRING""" + val (cSpark, cNative, cSpeedup) = + runTableBenchmark( + warehouseDir, + "customer", + customerSchema, + numFragments, + rowsPerFragment) + println(f"${"customer"}%-15s $cSpark%12d $cNative%12d ${cSpeedup}%9.2fx") + + println("-" * 60) + val avgSpeedup = (lSpeedup + oSpeedup + cSpeedup) / 3 + println( + f"${"AVERAGE"}%-15s ${lSpark + oSpark + cSpark}%12d ${lNative + oNative + cNative}%12d ${avgSpeedup}%9.2fx") + println("=" * 60 + "\n") + // scalastyle:on println + } + } + } + + test("benchmark: Spark vs Native compaction on lineitem (SF0.01 subset)") { + assume(icebergAvailable, "Iceberg not available") + assume(tpcDataAvailable, s"TPC-H data not found at $dataLocation") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + val tableName = "bench_cat.db.lineitem_bench" + val numFragments = 10 + val rowsPerFragment = 1000 + + // Create fragmented Iceberg table + spark.sql(s""" + CREATE TABLE $tableName ( + l_orderkey BIGINT, l_partkey BIGINT, l_suppkey BIGINT, l_linenumber INT, + l_quantity DOUBLE, l_extendedprice DOUBLE, l_discount DOUBLE, + l_tax DOUBLE, l_returnflag STRING, l_linestatus STRING + ) USING iceberg + """) + + // Insert fragments from TPC-H lineitem + for (i <- 0 until numFragments) { + spark.sql(s""" + INSERT INTO $tableName + SELECT l_orderkey, l_partkey, l_suppkey, l_linenumber, + l_quantity, l_extendedprice, l_discount, l_tax, + l_returnflag, l_linestatus + FROM parquet.`$dataLocation/lineitem` + LIMIT $rowsPerFragment OFFSET ${i * rowsPerFragment} + """) + } + + val filesBefore = spark.sql(s"SELECT * FROM $tableName.files").count() + val rowCount = spark.sql(s"SELECT COUNT(*) FROM $tableName").first().getLong(0) + + // scalastyle:off println + println(s"\n========== COMPACTION BENCHMARK ==========") + println(s"Table: $tableName") + println(s"Files before: $filesBefore, Rows: $rowCount") + println("=" * 45) + + // Benchmark 1: Spark default compaction + val sparkStart = System.nanoTime() + val sparkTable = Spark3Util.loadIcebergTable(spark, tableName) + SparkActions.get(spark).rewriteDataFiles(sparkTable).binPack().execute() + val sparkDuration = (System.nanoTime() - sparkStart) / 1000000 + + spark.sql(s"REFRESH TABLE $tableName") + val filesAfterSpark = spark.sql(s"SELECT * FROM $tableName.files").count() + println(s"Spark compaction: ${sparkDuration}ms ($filesBefore -> $filesAfterSpark files)") + + // Re-create fragmented table for native benchmark + spark.sql(s"DROP TABLE $tableName") + spark.sql(s""" + CREATE TABLE $tableName ( + l_orderkey BIGINT, l_partkey BIGINT, l_suppkey BIGINT, l_linenumber INT, + l_quantity DOUBLE, l_extendedprice DOUBLE, l_discount DOUBLE, + l_tax DOUBLE, l_returnflag STRING, l_linestatus STRING + ) USING iceberg + """) + + for (i <- 0 until numFragments) { + spark.sql(s""" + INSERT INTO $tableName + SELECT l_orderkey, l_partkey, l_suppkey, l_linenumber, + l_quantity, l_extendedprice, l_discount, l_tax, + l_returnflag, l_linestatus + FROM parquet.`$dataLocation/lineitem` + LIMIT $rowsPerFragment OFFSET ${i * rowsPerFragment} + """) + } + + // Benchmark 2: Native compaction + val nativeStart = System.nanoTime() + val nativeTable = Spark3Util.loadIcebergTable(spark, tableName) + CometNativeCompaction(spark).rewriteDataFiles(nativeTable) + val nativeDuration = (System.nanoTime() - nativeStart) / 1000000 + + spark.sql(s"REFRESH TABLE $tableName") + val filesAfterNative = spark.sql(s"SELECT * FROM $tableName.files").count() + println( + s"Native compaction: ${nativeDuration}ms ($filesBefore -> $filesAfterNative files)") + + val speedup = if (nativeDuration > 0) sparkDuration.toDouble / nativeDuration else 0 + println(s"Speedup: ${f"$speedup%.2f"}x") + println("=" * 45 + "\n") + // scalastyle:on println + + spark.sql(s"DROP TABLE $tableName") + } + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala new file mode 100644 index 0000000000..6d3773f6be --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala @@ -0,0 +1,421 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.io.File +import java.nio.file.Files + +import org.apache.iceberg.spark.Spark3Util +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.comet.CometNativeCompaction + +/** Integration tests for native Iceberg compaction using CometNativeCompaction. */ +class CometIcebergCompactionSuite extends CometTestBase { + + private def icebergAvailable: Boolean = { + try { + Class.forName("org.apache.iceberg.catalog.Catalog") + true + } catch { + case _: ClassNotFoundException => false + } + } + + private def withTempIcebergDir(f: File => Unit): Unit = { + val dir = Files.createTempDirectory("comet-compaction-test").toFile + try { + f(dir) + } finally { + def deleteRecursively(file: File): Unit = { + if (file.isDirectory) file.listFiles().foreach(deleteRecursively) + file.delete() + } + deleteRecursively(dir) + } + } + + private def icebergCatalogConf(warehouseDir: File): Map[String, String] = Map( + "spark.sql.catalog.compact_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.compact_cat.type" -> "hadoop", + "spark.sql.catalog.compact_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_COMPACTION_ENABLED.key -> "true") + + private def loadIcebergTable(tableName: String): org.apache.iceberg.Table = { + Spark3Util.loadIcebergTable(spark, tableName) + } + + // ============== Basic Native Compaction Tests ============== + + test("native compaction compacts fragmented files") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.frag_table ( + id BIGINT, + name STRING, + value DOUBLE + ) USING iceberg + """) + + for (i <- 0 until 10) { + spark.sql(s"INSERT INTO compact_cat.db.frag_table VALUES ($i, 'name_$i', ${i * 1.5})") + } + + val filesBefore = + spark.sql("SELECT file_path FROM compact_cat.db.frag_table.files").count() + assert(filesBefore >= 5, s"Expected multiple files, got $filesBefore") + + val rowsBefore = + spark.sql("SELECT count(*) FROM compact_cat.db.frag_table").collect()(0).getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.frag_table") + val nativeCompaction = CometNativeCompaction(spark) + val summary = nativeCompaction.rewriteDataFiles(icebergTable) + + assert(summary.filesDeleted > 0, "Should delete files") + assert(summary.filesAdded > 0, "Should add files") + + spark.sql("REFRESH TABLE compact_cat.db.frag_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM compact_cat.db.frag_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + val filesAfter = + spark.sql("SELECT file_path FROM compact_cat.db.frag_table.files").count() + assert(filesAfter < filesBefore, s"Expected fewer files: $filesBefore -> $filesAfter") + + spark.sql("DROP TABLE compact_cat.db.frag_table") + } + } + } + + test("native compaction preserves data correctness") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.correct_table ( + id BIGINT, + name STRING + ) USING iceberg + """) + + for (i <- 1 to 20) { + spark.sql(s"INSERT INTO compact_cat.db.correct_table VALUES ($i, 'row_$i')") + } + + val dataBefore = spark + .sql("SELECT id, name FROM compact_cat.db.correct_table ORDER BY id") + .collect() + .map(r => (r.getLong(0), r.getString(1))) + + val icebergTable = loadIcebergTable("compact_cat.db.correct_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.correct_table") + val dataAfter = spark + .sql("SELECT id, name FROM compact_cat.db.correct_table ORDER BY id") + .collect() + .map(r => (r.getLong(0), r.getString(1))) + + assert(dataBefore.toSeq == dataAfter.toSeq, "Data must be identical after compaction") + + spark.sql("DROP TABLE compact_cat.db.correct_table") + } + } + } + + // ============== Partitioned Table Tests ============== + + // TODO: Native compaction doesn't yet support partitioned tables correctly. + // Partition column values stored in partition paths (not data files) are not preserved. + ignore("native compaction on partitioned table preserves partition boundaries") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.partitioned_table ( + id BIGINT, + category STRING, + value DOUBLE + ) USING iceberg + PARTITIONED BY (category) + """) + + for (i <- 1 to 15) { + val cat = if (i % 3 == 0) "A" else if (i % 3 == 1) "B" else "C" + spark.sql( + s"INSERT INTO compact_cat.db.partitioned_table VALUES ($i, '$cat', ${i * 1.5})") + } + + val dataBefore = spark + .sql("SELECT id, category, value FROM compact_cat.db.partitioned_table ORDER BY id") + .collect() + + val icebergTable = loadIcebergTable("compact_cat.db.partitioned_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.partitioned_table") + val dataAfter = spark + .sql("SELECT id, category, value FROM compact_cat.db.partitioned_table ORDER BY id") + .collect() + + assert(dataBefore.toSeq == dataAfter.toSeq, "Data must be identical after compaction") + + val partitions = spark + .sql("SELECT DISTINCT category FROM compact_cat.db.partitioned_table") + .collect() + .map(_.getString(0)) + assert(partitions.toSet == Set("A", "B", "C"), "Partitions should be preserved") + + spark.sql("DROP TABLE compact_cat.db.partitioned_table") + } + } + } + + test("native compaction on date-partitioned table") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.date_part_table ( + id BIGINT, + event_date DATE, + data STRING + ) USING iceberg + PARTITIONED BY (days(event_date)) + """) + + for (day <- 1 to 5; i <- 1 to 3) { + spark.sql(s""" + INSERT INTO compact_cat.db.date_part_table + VALUES (${(day - 1) * 3 + i}, DATE '2024-01-0$day', 'data_$i') + """) + } + + val rowsBefore = + spark.sql("SELECT count(*) FROM compact_cat.db.date_part_table").collect()(0).getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.date_part_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.date_part_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM compact_cat.db.date_part_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.date_part_table") + } + } + } + + // ============== Copy-on-Write vs Merge-on-Read Tests ============== + + test("native compaction on Copy-on-Write table") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.cow_table ( + id BIGINT, + value STRING + ) USING iceberg + TBLPROPERTIES ( + 'write.delete.mode' = 'copy-on-write', + 'write.update.mode' = 'copy-on-write' + ) + """) + + for (i <- 1 to 10) { + spark.sql(s"INSERT INTO compact_cat.db.cow_table VALUES ($i, 'v$i')") + } + + spark.sql("UPDATE compact_cat.db.cow_table SET value = 'updated' WHERE id <= 3") + + val rowsBefore = + spark.sql("SELECT count(*) FROM compact_cat.db.cow_table").collect()(0).getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.cow_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.cow_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM compact_cat.db.cow_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.cow_table") + } + } + } + + test("native compaction on Merge-on-Read table with delete files") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.mor_table ( + id BIGINT, + value STRING + ) USING iceberg + TBLPROPERTIES ( + 'write.delete.mode' = 'merge-on-read', + 'write.update.mode' = 'merge-on-read' + ) + """) + + for (i <- 1 to 10) { + spark.sql(s"INSERT INTO compact_cat.db.mor_table VALUES ($i, 'v$i')") + } + + spark.sql("DELETE FROM compact_cat.db.mor_table WHERE id IN (2, 4, 6)") + + val dataBefore = spark + .sql("SELECT id, value FROM compact_cat.db.mor_table ORDER BY id") + .collect() + .map(r => (r.getLong(0), r.getString(1))) + + val icebergTable = loadIcebergTable("compact_cat.db.mor_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.mor_table") + val dataAfter = spark + .sql("SELECT id, value FROM compact_cat.db.mor_table ORDER BY id") + .collect() + .map(r => (r.getLong(0), r.getString(1))) + + assert(dataBefore.toSeq == dataAfter.toSeq, "Data must be identical after compaction") + assert(!dataAfter.map(_._1).contains(2L), "Deleted rows should remain deleted") + + spark.sql("DROP TABLE compact_cat.db.mor_table") + } + } + } + + // ============== Schema Variation Tests ============== + + test("native compaction with complex schema (all common types)") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.complex_schema ( + id BIGINT, + int_col INT, + float_col FLOAT, + double_col DOUBLE, + decimal_col DECIMAL(10, 2), + string_col STRING, + bool_col BOOLEAN, + date_col DATE, + ts_col TIMESTAMP + ) USING iceberg + """) + + for (i <- 1 to 10) { + spark.sql(s""" + INSERT INTO compact_cat.db.complex_schema VALUES + ($i, $i, ${i * 0.5}f, ${i * 1.5}, ${i * 10.25}, 'str_$i', + ${i % 2 == 0}, DATE '2024-01-0${(i % 9) + 1}', + TIMESTAMP '2024-01-01 0${i % 10}:00:00') + """) + } + + val dataBefore = spark + .sql("SELECT * FROM compact_cat.db.complex_schema ORDER BY id") + .collect() + + val icebergTable = loadIcebergTable("compact_cat.db.complex_schema") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.complex_schema") + val dataAfter = spark + .sql("SELECT * FROM compact_cat.db.complex_schema ORDER BY id") + .collect() + + assert( + dataBefore.length == dataAfter.length, + s"Row count changed: ${dataBefore.length} -> ${dataAfter.length}") + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Data must be identical after compaction") + + spark.sql("DROP TABLE compact_cat.db.complex_schema") + } + } + } + + test("native compaction with nullable columns") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.nullable_table ( + id BIGINT, + nullable_str STRING, + nullable_int INT + ) USING iceberg + """) + + for (i <- 1 to 10) { + val strVal = if (i % 3 == 0) "NULL" else s"'value_$i'" + val intVal = if (i % 2 == 0) "NULL" else s"$i" + spark.sql(s"INSERT INTO compact_cat.db.nullable_table VALUES ($i, $strVal, $intVal)") + } + + val dataBefore = spark + .sql("SELECT * FROM compact_cat.db.nullable_table ORDER BY id") + .collect() + + val icebergTable = loadIcebergTable("compact_cat.db.nullable_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.nullable_table") + val dataAfter = spark + .sql("SELECT * FROM compact_cat.db.nullable_table ORDER BY id") + .collect() + + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Data with nulls must be identical after compaction") + + spark.sql("DROP TABLE compact_cat.db.nullable_table") + } + } + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala new file mode 100644 index 0000000000..9d35052de8 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala @@ -0,0 +1,378 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.benchmark + +import java.io.{FileOutputStream, PrintStream} +import java.io.File +import java.util.Locale + +import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.actions.SparkActions +import org.apache.spark.sql.comet.CometNativeCompaction + +import org.apache.comet.CometConf + +/** + * Benchmark to measure Iceberg compaction performance using TPC-H dataset. Compares Spark default + * compaction (SparkBinPackDataRewriter) vs Comet-accelerated compaction + * (CometBinPackRewriteRunner with iceberg-rust ReplaceDataFilesAction). + * + * To run this benchmark: + * {{{ + * // Set scale factor in GB + * scale_factor=1 + * + * // GenTPCHData to create the data set at /tmp/tpch/sf1_parquet + * cd $COMET_HOME + * make benchmark-org.apache.spark.sql.GenTPCHData -- --location /tmp --scaleFactor ${scale_factor} + * + * // Run the Iceberg compaction benchmark + * SPARK_GENERATE_BENCHMARK_FILES=1 make benchmark-org.apache.spark.sql.benchmark.CometIcebergTPCCompactionBenchmark -- --data-location /tmp/tpch/sf${scale_factor}_parquet + * }}} + * + * Results will be written to "spark/benchmarks/CometIcebergTPCCompactionBenchmark-results.txt". + */ +object CometIcebergTPCCompactionBenchmark extends CometBenchmarkBase { + + // TPC-H tables to use for compaction benchmarks + // lineitem is the largest and most representative for compaction workloads + val compactionTables: Seq[String] = Seq("lineitem", "orders", "customer") + + // Partitioned table benchmarks use lineitem partitioned by l_shipdate + val partitionedBenchmarkEnabled: Boolean = true + + override def runCometBenchmark(mainArgs: Array[String]): Unit = { + val benchmarkArgs = new IcebergTPCBenchmarkArguments(mainArgs) + + runBenchmark("Iceberg TPC-H Compaction Benchmark") { + runIcebergCompactionBenchmarks(benchmarkArgs.dataLocation, benchmarkArgs.numFragments) + } + } + + private def runIcebergCompactionBenchmarks(dataLocation: String, numFragments: Int): Unit = { + // Print header + printHeader() + + // Non-partitioned table benchmarks + compactionTables.foreach { tableName => + runTableCompactionBenchmark(dataLocation, tableName, numFragments) + } + + // Partitioned table benchmark (lineitem partitioned by shipdate month) + if (partitionedBenchmarkEnabled) { + runPartitionedTableBenchmark(dataLocation, numFragments) + } + + // Print footer + printFooter() + } + + private def printToOutput(text: String): Unit = { + // scalastyle:off println + println(text) + output.foreach { os => + os.write(text.getBytes) + os.write('\n') + } + // scalastyle:on println + } + + private def printHeader(): Unit = { + val sep = "-" * 90 + val colHeader = + f"${"Table"}%-15s ${"Rows"}%10s ${"Files"}%8s ${"Spark(ms)"}%12s ${"Native(ms)"}%12s ${"Speedup"}%10s" + printToOutput(s"\n$sep") + printToOutput(" Iceberg Compaction Benchmark: Spark Default vs Comet Native") + printToOutput(sep) + printToOutput(colHeader) + printToOutput(sep) + } + + private def printFooter(): Unit = { + printToOutput("-" * 90) + } + + private def writeResult( + tableName: String, + rowCount: Long, + filesBefore: Long, + filesAfter: Long, + sparkMs: Long, + nativeMs: Long, + speedup: Double): Unit = { + val speedupStr = if (speedup > 0) f"$speedup%.2fx" else "N/A" + val sparkStr = if (sparkMs > 0) sparkMs.toString else "N/A" + val line = + f"$tableName%-15s ${rowCount / 1000}%9dK $filesBefore%3d->$filesAfter%-3d $sparkStr%12s $nativeMs%12d $speedupStr%10s" + printToOutput(line) + } + + /** + * Run compaction benchmark for partitioned Iceberg table (lineitem by shipdate month). + */ + private def runPartitionedTableBenchmark(dataLocation: String, numFragments: Int): Unit = { + val tableFilePath = resolveTablePath(dataLocation, "lineitem") + + withIcebergWarehouse { (warehouseDir, catalog) => + val icebergTableName = s"$catalog.db.lineitem_partitioned" + + // Create fragmented partitioned table + createFragmentedPartitionedTable(icebergTableName, tableFilePath, numFragments) + val rowCount = spark.sql(s"SELECT COUNT(*) FROM $icebergTableName").first().getLong(0) + val fileCount = spark.sql(s"SELECT * FROM $icebergTableName.files").count() + + // Measure native compaction on partitioned table (single run) + val nativeStart = System.nanoTime() + val nativeTable = Spark3Util.loadIcebergTable(spark, icebergTableName) + new CometNativeCompaction(spark).rewriteDataFiles(nativeTable) + val nativeTimeMs = (System.nanoTime() - nativeStart) / 1000000 + val nativeFilesAfter = spark.sql(s"SELECT * FROM $icebergTableName.files").count() + + // Write result + writeResult(s"lineitem_part", rowCount, fileCount, nativeFilesAfter, 0, nativeTimeMs, 0) + + spark.sql(s"DROP TABLE IF EXISTS $icebergTableName") + } + } + + /** + * Create fragmented partitioned Iceberg table from TPC-H lineitem. + */ + private def createFragmentedPartitionedTable( + icebergTable: String, + sourceParquetPath: String, + numFragments: Int): Unit = { + + val sourceDF = spark.read.parquet(sourceParquetPath) + val totalRows = sourceDF.count() + val rowsPerFragment = totalRows / numFragments + + // Create partitioned Iceberg table by l_shipmode + spark.sql(s""" + CREATE TABLE $icebergTable ( + l_orderkey BIGINT, l_partkey BIGINT, l_suppkey BIGINT, l_linenumber INT, + l_quantity DECIMAL(15,2), l_extendedprice DECIMAL(15,2), l_discount DECIMAL(15,2), + l_tax DECIMAL(15,2), l_returnflag STRING, l_linestatus STRING, l_shipdate DATE, + l_commitdate DATE, l_receiptdate DATE, l_shipinstruct STRING, l_shipmode STRING, + l_comment STRING + ) USING iceberg PARTITIONED BY (l_shipmode) + """) + + val schema = sourceDF.schema.fieldNames.mkString(", ") + for (i <- 0 until numFragments) { + val offset = i * rowsPerFragment + spark.sql(s""" + INSERT INTO $icebergTable + SELECT $schema FROM parquet.`$sourceParquetPath` + LIMIT $rowsPerFragment OFFSET $offset + """) + } + } + + /** + * Run compaction benchmark for a specific TPC-H table. + */ + private def runTableCompactionBenchmark( + dataLocation: String, + tableName: String, + numFragments: Int): Unit = { + + val tableFilePath = resolveTablePath(dataLocation, tableName) + + withIcebergWarehouse { (warehouseDir, catalog) => + val icebergTableName = s"$catalog.db.${tableName}_iceberg" + + // Create fragmented table once to measure metadata + createFragmentedIcebergTable(icebergTableName, tableFilePath, numFragments) + val rowCount = spark.sql(s"SELECT COUNT(*) FROM $icebergTableName").first().getLong(0) + val fileCount = spark.sql(s"SELECT * FROM $icebergTableName.files").count() + + // Measure Spark compaction (single run - compaction is destructive) + val sparkStart = System.nanoTime() + val sparkTable = Spark3Util.loadIcebergTable(spark, icebergTableName) + SparkActions.get(spark).rewriteDataFiles(sparkTable).binPack().execute() + val sparkTimeMs = (System.nanoTime() - sparkStart) / 1000000 + val sparkFilesAfter = spark.sql(s"SELECT * FROM $icebergTableName.files").count() + + // Re-create fragmented table for native benchmark + spark.sql(s"DROP TABLE IF EXISTS $icebergTableName") + createFragmentedIcebergTable(icebergTableName, tableFilePath, numFragments) + + // Measure native compaction (single run) + val nativeStart = System.nanoTime() + val nativeTable = Spark3Util.loadIcebergTable(spark, icebergTableName) + new CometNativeCompaction(spark).rewriteDataFiles(nativeTable) + val nativeTimeMs = (System.nanoTime() - nativeStart) / 1000000 + val nativeFilesAfter = spark.sql(s"SELECT * FROM $icebergTableName.files").count() + + // Calculate speedup + val speedup = if (nativeTimeMs > 0) sparkTimeMs.toDouble / nativeTimeMs.toDouble else 0.0 + + // Write result + writeResult( + tableName, + rowCount, + fileCount, + sparkFilesAfter, + sparkTimeMs, + nativeTimeMs, + speedup) + + spark.sql(s"DROP TABLE IF EXISTS $icebergTableName") + } + } + + /** + * Create a fragmented Iceberg table by importing TPC-H Parquet data in multiple batches. + */ + private def createFragmentedIcebergTable( + icebergTable: String, + sourceParquetPath: String, + numFragments: Int): Unit = { + + // Read the source Parquet data + val sourceDF = spark.read.parquet(sourceParquetPath) + val totalRows = sourceDF.count() + val rowsPerFragment = totalRows / numFragments + + // Create the Iceberg table + sourceDF.limit(0).writeTo(icebergTable).using("iceberg").create() + + // Insert data in fragments to create multiple small files + val schema = sourceDF.schema.fieldNames.mkString(", ") + + for (i <- 0 until numFragments) { + val offset = i * rowsPerFragment + spark.sql(s""" + INSERT INTO $icebergTable + SELECT $schema FROM parquet.`$sourceParquetPath` + LIMIT $rowsPerFragment OFFSET $offset + """) + } + } + + /** + * Resolve the path to a TPC-H table, supporting both with and without .parquet extension. + */ + private def resolveTablePath(dataLocation: String, tableName: String): String = { + val pathDefault = s"$dataLocation/$tableName" + val pathAlt = s"$dataLocation/$tableName.parquet" + + if (new File(pathDefault).exists()) { + pathDefault + } else if (new File(pathAlt).exists()) { + pathAlt + } else { + throw new java.io.FileNotFoundException( + s"TPC-H table $tableName not found at $pathDefault or $pathAlt") + } + } + + /** + * Helper to use Iceberg warehouse with catalog configuration. + */ + private def withIcebergWarehouse(f: (File, String) => Unit): Unit = { + withTempPath { dir => + val warehouseDir = new File(dir, "iceberg-warehouse") + val catalogName = "tpc_iceberg_cat" + + spark.conf.set(s"spark.sql.catalog.$catalogName", "org.apache.iceberg.spark.SparkCatalog") + spark.conf.set(s"spark.sql.catalog.$catalogName.type", "hadoop") + spark.conf.set(s"spark.sql.catalog.$catalogName.warehouse", warehouseDir.getAbsolutePath) + + spark.sql(s"CREATE NAMESPACE IF NOT EXISTS $catalogName.db") + + try { + f(warehouseDir, catalogName) + } finally { + spark.conf.unset(s"spark.sql.catalog.$catalogName") + spark.conf.unset(s"spark.sql.catalog.$catalogName.type") + spark.conf.unset(s"spark.sql.catalog.$catalogName.warehouse") + } + } + } +} + +/** + * Command line arguments for Iceberg TPC compaction benchmark. + */ +class IcebergTPCBenchmarkArguments(val args: Array[String]) { + var dataLocation: String = null + var numFragments: Int = 20 + + parseArgs(args.toList) + validateArguments() + + private def optionMatch(optionName: String, s: String): Boolean = { + optionName == s.toLowerCase(Locale.ROOT) + } + + private def parseArgs(inputArgs: List[String]): Unit = { + var args = inputArgs + + while (args.nonEmpty) { + args match { + case optName :: value :: tail if optionMatch("--data-location", optName) => + dataLocation = value + args = tail + + case optName :: value :: tail if optionMatch("--num-fragments", optName) => + numFragments = value.toInt + args = tail + + case _ => + // scalastyle:off println + System.err.println("Unknown/unsupported param " + args) + // scalastyle:on println + printUsageAndExit(1) + } + } + } + + private def printUsageAndExit(exitCode: Int): Unit = { + // scalastyle:off + System.err.println(""" + |Usage: spark-submit --class [Options] + |Options: + | --data-location Path to TPC-H Parquet data (required) + | --num-fragments Number of fragments to create for compaction (default: 20) + | + |------------------------------------------------------------------------------------------------------------------ + |This benchmark measures Iceberg compaction performance using TPC-H data. + | + |To generate TPC-H data: + | make benchmark-org.apache.spark.sql.GenTPCHData -- --location /tmp --scaleFactor 1 + | + |Then run the benchmark: + | SPARK_GENERATE_BENCHMARK_FILES=1 make benchmark-org.apache.spark.sql.benchmark.CometIcebergTPCCompactionBenchmark -- --data-location /tmp/tpch/sf1_parquet + """.stripMargin) + // scalastyle:on + System.exit(exitCode) + } + + private def validateArguments(): Unit = { + if (dataLocation == null) { + // scalastyle:off println + System.err.println("Must specify --data-location") + // scalastyle:on println + printUsageAndExit(-1) + } + } +} From 1df0011092a950f6d6e0391b95673f3205a49d7f Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Sat, 14 Feb 2026 15:35:15 +0530 Subject: [PATCH 05/13] test: add comprehensive Iceberg compaction tests for partitions, schema evolution, nested types --- .../comet/CometIcebergCompactionSuite.scala | 499 ++++++++++++++++++ 1 file changed, 499 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala index 6d3773f6be..4987e0069b 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala @@ -418,4 +418,503 @@ class CometIcebergCompactionSuite extends CometTestBase { } } } + + // ============== Partition Transform Tests ============== + + test("native compaction on bucket-partitioned table") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.bucket_table ( + id BIGINT, + category STRING, + value DOUBLE + ) USING iceberg + PARTITIONED BY (bucket(4, id)) + """) + + for (i <- 1 to 20) { + spark.sql(s"INSERT INTO compact_cat.db.bucket_table VALUES ($i, 'cat_$i', ${i * 1.5})") + } + + val rowsBefore = + spark.sql("SELECT count(*) FROM compact_cat.db.bucket_table").collect()(0).getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.bucket_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.bucket_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM compact_cat.db.bucket_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.bucket_table") + } + } + } + + test("native compaction on truncate-partitioned table") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.truncate_table ( + id BIGINT, + name STRING, + value DOUBLE + ) USING iceberg + PARTITIONED BY (truncate(3, name)) + """) + + for (i <- 1 to 15) { + spark.sql( + s"INSERT INTO compact_cat.db.truncate_table VALUES ($i, 'name_$i', ${i * 1.5})") + } + + val rowsBefore = + spark.sql("SELECT count(*) FROM compact_cat.db.truncate_table").collect()(0).getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.truncate_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.truncate_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM compact_cat.db.truncate_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.truncate_table") + } + } + } + + test("native compaction on month-partitioned table") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.month_part_table ( + id BIGINT, + event_ts TIMESTAMP, + data STRING + ) USING iceberg + PARTITIONED BY (month(event_ts)) + """) + + for (month <- 1 to 3; i <- 1 to 3) { + val monthStr = f"$month%02d" + spark.sql(s""" + INSERT INTO compact_cat.db.month_part_table + VALUES (${(month - 1) * 3 + i}, TIMESTAMP '2024-$monthStr-15 10:00:00', 'data_$i') + """) + } + + val rowsBefore = + spark + .sql("SELECT count(*) FROM compact_cat.db.month_part_table") + .collect()(0) + .getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.month_part_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.month_part_table") + val rowsAfter = + spark + .sql("SELECT count(*) FROM compact_cat.db.month_part_table") + .collect()(0) + .getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.month_part_table") + } + } + } + + test("native compaction on hour-partitioned table") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.hour_part_table ( + id BIGINT, + event_ts TIMESTAMP, + data STRING + ) USING iceberg + PARTITIONED BY (hour(event_ts)) + """) + + for (hour <- 1 to 4; i <- 1 to 2) { + val hourStr = f"$hour%02d" + spark.sql(s""" + INSERT INTO compact_cat.db.hour_part_table + VALUES (${(hour - 1) * 2 + i}, TIMESTAMP '2024-01-15 $hourStr:30:00', 'data_$i') + """) + } + + val rowsBefore = + spark.sql("SELECT count(*) FROM compact_cat.db.hour_part_table").collect()(0).getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.hour_part_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.hour_part_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM compact_cat.db.hour_part_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.hour_part_table") + } + } + } + + // ============== Multiple Partition Columns ============== + + test("native compaction on multi-column partitioned table") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.multi_part_table ( + id BIGINT, + region STRING, + event_date DATE, + value DOUBLE + ) USING iceberg + PARTITIONED BY (days(event_date), bucket(2, region)) + """) + + val regions = Seq("US", "EU", "APAC") + for (day <- 1 to 3; region <- regions) { + spark.sql(s""" + INSERT INTO compact_cat.db.multi_part_table + VALUES (${day * 10 + regions.indexOf(region)}, '$region', + DATE '2024-01-0$day', ${day * 1.5}) + """) + } + + val rowsBefore = + spark + .sql("SELECT count(*) FROM compact_cat.db.multi_part_table") + .collect()(0) + .getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.multi_part_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.multi_part_table") + val rowsAfter = + spark + .sql("SELECT count(*) FROM compact_cat.db.multi_part_table") + .collect()(0) + .getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.multi_part_table") + } + } + } + + // ============== Schema Evolution Tests ============== + + test("native compaction after schema evolution (add column)") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.schema_evo_table ( + id BIGINT, + name STRING + ) USING iceberg + """) + + for (i <- 1 to 5) { + spark.sql(s"INSERT INTO compact_cat.db.schema_evo_table VALUES ($i, 'name_$i')") + } + + spark.sql("ALTER TABLE compact_cat.db.schema_evo_table ADD COLUMN value DOUBLE") + + for (i <- 6 to 10) { + spark.sql( + s"INSERT INTO compact_cat.db.schema_evo_table VALUES ($i, 'name_$i', ${i * 1.5})") + } + + val rowsBefore = + spark + .sql("SELECT count(*) FROM compact_cat.db.schema_evo_table") + .collect()(0) + .getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.schema_evo_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.schema_evo_table") + val rowsAfter = + spark + .sql("SELECT count(*) FROM compact_cat.db.schema_evo_table") + .collect()(0) + .getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + val nullCount = spark + .sql("SELECT count(*) FROM compact_cat.db.schema_evo_table WHERE value IS NULL") + .collect()(0) + .getLong(0) + assert(nullCount == 5, s"Expected 5 nulls for old rows, got $nullCount") + + spark.sql("DROP TABLE compact_cat.db.schema_evo_table") + } + } + } + + // ============== Nested Type Tests ============== + + test("native compaction with struct column") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.struct_table ( + id BIGINT, + info STRUCT + ) USING iceberg + """) + + for (i <- 1 to 10) { + spark.sql( + s"INSERT INTO compact_cat.db.struct_table VALUES ($i, named_struct('name', 'n$i', 'age', $i))") + } + + val dataBefore = spark + .sql("SELECT id, info.name, info.age FROM compact_cat.db.struct_table ORDER BY id") + .collect() + + val icebergTable = loadIcebergTable("compact_cat.db.struct_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.struct_table") + val dataAfter = spark + .sql("SELECT id, info.name, info.age FROM compact_cat.db.struct_table ORDER BY id") + .collect() + + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Struct data must be identical after compaction") + + spark.sql("DROP TABLE compact_cat.db.struct_table") + } + } + } + + test("native compaction with array column") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.array_table ( + id BIGINT, + tags ARRAY + ) USING iceberg + """) + + for (i <- 1 to 10) { + spark.sql( + s"INSERT INTO compact_cat.db.array_table VALUES ($i, array('tag_${i}_a', 'tag_${i}_b'))") + } + + val dataBefore = spark + .sql("SELECT id, tags FROM compact_cat.db.array_table ORDER BY id") + .collect() + + val icebergTable = loadIcebergTable("compact_cat.db.array_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.array_table") + val dataAfter = spark + .sql("SELECT id, tags FROM compact_cat.db.array_table ORDER BY id") + .collect() + + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Array data must be identical after compaction") + + spark.sql("DROP TABLE compact_cat.db.array_table") + } + } + } + + test("native compaction with map column") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.map_table ( + id BIGINT, + properties MAP + ) USING iceberg + """) + + for (i <- 1 to 10) { + spark.sql( + s"INSERT INTO compact_cat.db.map_table VALUES ($i, map('key_$i', $i, 'val_$i', ${i * 10}))") + } + + val dataBefore = spark + .sql("SELECT id, properties FROM compact_cat.db.map_table ORDER BY id") + .collect() + + val icebergTable = loadIcebergTable("compact_cat.db.map_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.map_table") + val dataAfter = spark + .sql("SELECT id, properties FROM compact_cat.db.map_table ORDER BY id") + .collect() + + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Map data must be identical after compaction") + + spark.sql("DROP TABLE compact_cat.db.map_table") + } + } + } + + // ============== Table Properties Tests ============== + + test("native compaction with custom table properties") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.props_table ( + id BIGINT, + value STRING + ) USING iceberg + TBLPROPERTIES ( + 'write.parquet.compression-codec' = 'zstd', + 'write.parquet.compression-level' = '3', + 'commit.retry.num-retries' = '5' + ) + """) + + for (i <- 1 to 10) { + spark.sql(s"INSERT INTO compact_cat.db.props_table VALUES ($i, 'v$i')") + } + + val rowsBefore = + spark.sql("SELECT count(*) FROM compact_cat.db.props_table").collect()(0).getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.props_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.props_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM compact_cat.db.props_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.props_table") + } + } + } + + // ============== Large Decimal Tests ============== + + test("native compaction with large decimal precision") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.decimal_table ( + id BIGINT, + small_dec DECIMAL(10, 2), + large_dec DECIMAL(28, 10) + ) USING iceberg + """) + + for (i <- 1 to 10) { + spark.sql(s""" + INSERT INTO compact_cat.db.decimal_table VALUES + ($i, ${i * 100.25}, ${i * 1000000.1234567890}) + """) + } + + val dataBefore = spark + .sql("SELECT * FROM compact_cat.db.decimal_table ORDER BY id") + .collect() + + val icebergTable = loadIcebergTable("compact_cat.db.decimal_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.decimal_table") + val dataAfter = spark + .sql("SELECT * FROM compact_cat.db.decimal_table ORDER BY id") + .collect() + + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Decimal data must be identical after compaction") + + spark.sql("DROP TABLE compact_cat.db.decimal_table") + } + } + } + + // ============== Binary/UUID Tests ============== + + test("native compaction with binary column") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + spark.sql(""" + CREATE TABLE compact_cat.db.binary_table ( + id BIGINT, + data BINARY + ) USING iceberg + """) + + for (i <- 1 to 10) { + spark.sql( + s"INSERT INTO compact_cat.db.binary_table VALUES ($i, cast('binary_data_$i' as binary))") + } + + val rowsBefore = + spark.sql("SELECT count(*) FROM compact_cat.db.binary_table").collect()(0).getLong(0) + + val icebergTable = loadIcebergTable("compact_cat.db.binary_table") + CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + spark.sql("REFRESH TABLE compact_cat.db.binary_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM compact_cat.db.binary_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE compact_cat.db.binary_table") + } + } + } } From ad88f6eac6e22201ac58fd1d19784bb662ca3174 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Sat, 14 Feb 2026 15:41:15 +0530 Subject: [PATCH 06/13] test: add file count validation and Spark vs Native comparison test --- .../comet/CometIcebergCompactionSuite.scala | 244 +++++++++++++++--- 1 file changed, 202 insertions(+), 42 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala index 4987e0069b..b4c5004192 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionSuite.scala @@ -440,16 +440,33 @@ class CometIcebergCompactionSuite extends CometTestBase { spark.sql(s"INSERT INTO compact_cat.db.bucket_table VALUES ($i, 'cat_$i', ${i * 1.5})") } - val rowsBefore = - spark.sql("SELECT count(*) FROM compact_cat.db.bucket_table").collect()(0).getLong(0) + val filesBefore = + spark.sql("SELECT file_path FROM compact_cat.db.bucket_table.files").count() + assert(filesBefore >= 10, s"Expected multiple files, got $filesBefore") + + val dataBefore = spark + .sql("SELECT id, category, value FROM compact_cat.db.bucket_table ORDER BY id") + .collect() val icebergTable = loadIcebergTable("compact_cat.db.bucket_table") - CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + val summary = CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + assert(summary.filesDeleted > 0, "Should delete files") + assert(summary.filesAdded > 0, "Should add files") + assert(summary.filesAdded < summary.filesDeleted, "Should reduce file count") spark.sql("REFRESH TABLE compact_cat.db.bucket_table") - val rowsAfter = - spark.sql("SELECT count(*) FROM compact_cat.db.bucket_table").collect()(0).getLong(0) - assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + val filesAfter = + spark.sql("SELECT file_path FROM compact_cat.db.bucket_table.files").count() + assert(filesAfter < filesBefore, s"Expected fewer files: $filesBefore -> $filesAfter") + + val dataAfter = spark + .sql("SELECT id, category, value FROM compact_cat.db.bucket_table ORDER BY id") + .collect() + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Data must be identical after compaction") spark.sql("DROP TABLE compact_cat.db.bucket_table") } @@ -476,16 +493,32 @@ class CometIcebergCompactionSuite extends CometTestBase { s"INSERT INTO compact_cat.db.truncate_table VALUES ($i, 'name_$i', ${i * 1.5})") } - val rowsBefore = - spark.sql("SELECT count(*) FROM compact_cat.db.truncate_table").collect()(0).getLong(0) + val filesBefore = + spark.sql("SELECT file_path FROM compact_cat.db.truncate_table.files").count() + assert(filesBefore >= 10, s"Expected multiple files, got $filesBefore") + + val dataBefore = spark + .sql("SELECT id, name, value FROM compact_cat.db.truncate_table ORDER BY id") + .collect() val icebergTable = loadIcebergTable("compact_cat.db.truncate_table") - CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + val summary = CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + assert(summary.filesDeleted > 0, "Should delete files") + assert(summary.filesAdded > 0, "Should add files") spark.sql("REFRESH TABLE compact_cat.db.truncate_table") - val rowsAfter = - spark.sql("SELECT count(*) FROM compact_cat.db.truncate_table").collect()(0).getLong(0) - assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + val filesAfter = + spark.sql("SELECT file_path FROM compact_cat.db.truncate_table.files").count() + assert(filesAfter < filesBefore, s"Expected fewer files: $filesBefore -> $filesAfter") + + val dataAfter = spark + .sql("SELECT id, name, value FROM compact_cat.db.truncate_table ORDER BY id") + .collect() + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Data must be identical after compaction") spark.sql("DROP TABLE compact_cat.db.truncate_table") } @@ -515,22 +548,32 @@ class CometIcebergCompactionSuite extends CometTestBase { """) } - val rowsBefore = - spark - .sql("SELECT count(*) FROM compact_cat.db.month_part_table") - .collect()(0) - .getLong(0) + val filesBefore = + spark.sql("SELECT file_path FROM compact_cat.db.month_part_table.files").count() + assert(filesBefore >= 5, s"Expected multiple files, got $filesBefore") + + val dataBefore = spark + .sql("SELECT id, event_ts, data FROM compact_cat.db.month_part_table ORDER BY id") + .collect() val icebergTable = loadIcebergTable("compact_cat.db.month_part_table") - CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + val summary = CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + assert(summary.filesDeleted > 0, "Should delete files") + assert(summary.filesAdded > 0, "Should add files") spark.sql("REFRESH TABLE compact_cat.db.month_part_table") - val rowsAfter = - spark - .sql("SELECT count(*) FROM compact_cat.db.month_part_table") - .collect()(0) - .getLong(0) - assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + val filesAfter = + spark.sql("SELECT file_path FROM compact_cat.db.month_part_table.files").count() + assert(filesAfter < filesBefore, s"Expected fewer files: $filesBefore -> $filesAfter") + + val dataAfter = spark + .sql("SELECT id, event_ts, data FROM compact_cat.db.month_part_table ORDER BY id") + .collect() + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Data must be identical after compaction") spark.sql("DROP TABLE compact_cat.db.month_part_table") } @@ -560,16 +603,32 @@ class CometIcebergCompactionSuite extends CometTestBase { """) } - val rowsBefore = - spark.sql("SELECT count(*) FROM compact_cat.db.hour_part_table").collect()(0).getLong(0) + val filesBefore = + spark.sql("SELECT file_path FROM compact_cat.db.hour_part_table.files").count() + assert(filesBefore >= 5, s"Expected multiple files, got $filesBefore") + + val dataBefore = spark + .sql("SELECT id, event_ts, data FROM compact_cat.db.hour_part_table ORDER BY id") + .collect() val icebergTable = loadIcebergTable("compact_cat.db.hour_part_table") - CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + val summary = CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + assert(summary.filesDeleted > 0, "Should delete files") + assert(summary.filesAdded > 0, "Should add files") spark.sql("REFRESH TABLE compact_cat.db.hour_part_table") - val rowsAfter = - spark.sql("SELECT count(*) FROM compact_cat.db.hour_part_table").collect()(0).getLong(0) - assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + val filesAfter = + spark.sql("SELECT file_path FROM compact_cat.db.hour_part_table.files").count() + assert(filesAfter < filesBefore, s"Expected fewer files: $filesBefore -> $filesAfter") + + val dataAfter = spark + .sql("SELECT id, event_ts, data FROM compact_cat.db.hour_part_table ORDER BY id") + .collect() + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Data must be identical after compaction") spark.sql("DROP TABLE compact_cat.db.hour_part_table") } @@ -603,22 +662,32 @@ class CometIcebergCompactionSuite extends CometTestBase { """) } - val rowsBefore = - spark - .sql("SELECT count(*) FROM compact_cat.db.multi_part_table") - .collect()(0) - .getLong(0) + val filesBefore = + spark.sql("SELECT file_path FROM compact_cat.db.multi_part_table.files").count() + assert(filesBefore >= 5, s"Expected multiple files, got $filesBefore") + + val dataBefore = spark + .sql("SELECT id, region, event_date, value FROM compact_cat.db.multi_part_table ORDER BY id") + .collect() val icebergTable = loadIcebergTable("compact_cat.db.multi_part_table") - CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + val summary = CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + assert(summary.filesDeleted > 0, "Should delete files") + assert(summary.filesAdded > 0, "Should add files") spark.sql("REFRESH TABLE compact_cat.db.multi_part_table") - val rowsAfter = - spark - .sql("SELECT count(*) FROM compact_cat.db.multi_part_table") - .collect()(0) - .getLong(0) - assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + val filesAfter = + spark.sql("SELECT file_path FROM compact_cat.db.multi_part_table.files").count() + assert(filesAfter < filesBefore, s"Expected fewer files: $filesBefore -> $filesAfter") + + val dataAfter = spark + .sql("SELECT id, region, event_date, value FROM compact_cat.db.multi_part_table ORDER BY id") + .collect() + assert( + dataBefore.map(_.toString()).toSeq == dataAfter.map(_.toString()).toSeq, + "Data must be identical after compaction") spark.sql("DROP TABLE compact_cat.db.multi_part_table") } @@ -917,4 +986,95 @@ class CometIcebergCompactionSuite extends CometTestBase { } } } + + // ============== Spark Default vs Native Compaction Comparison ============== + + test("native compaction produces same result as Spark default compaction") { + assume(icebergAvailable, "Iceberg not available in classpath") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(icebergCatalogConf(warehouseDir).toSeq: _*) { + // Create two identical tables + spark.sql(""" + CREATE TABLE compact_cat.db.spark_table ( + id BIGINT, + name STRING, + value DOUBLE + ) USING iceberg + """) + + spark.sql(""" + CREATE TABLE compact_cat.db.native_table ( + id BIGINT, + name STRING, + value DOUBLE + ) USING iceberg + """) + + // Insert same data into both tables + for (i <- 1 to 15) { + spark.sql(s"INSERT INTO compact_cat.db.spark_table VALUES ($i, 'name_$i', ${i * 1.5})") + spark.sql(s"INSERT INTO compact_cat.db.native_table VALUES ($i, 'name_$i', ${i * 1.5})") + } + + // Verify both tables have same fragmented state + val sparkFilesBefore = + spark.sql("SELECT file_path FROM compact_cat.db.spark_table.files").count() + val nativeFilesBefore = + spark.sql("SELECT file_path FROM compact_cat.db.native_table.files").count() + assert(sparkFilesBefore == nativeFilesBefore, "Both tables should start with same files") + + // Run Spark default compaction + val sparkTable = loadIcebergTable("compact_cat.db.spark_table") + import org.apache.iceberg.spark.actions.SparkActions + SparkActions.get(spark).rewriteDataFiles(sparkTable).execute() + + // Run Native compaction + val nativeTable = loadIcebergTable("compact_cat.db.native_table") + CometNativeCompaction(spark).rewriteDataFiles(nativeTable) + + // Refresh tables + spark.sql("REFRESH TABLE compact_cat.db.spark_table") + spark.sql("REFRESH TABLE compact_cat.db.native_table") + + // Compare file counts after compaction + val sparkFilesAfter = + spark.sql("SELECT file_path FROM compact_cat.db.spark_table.files").count() + val nativeFilesAfter = + spark.sql("SELECT file_path FROM compact_cat.db.native_table.files").count() + + assert(sparkFilesAfter < sparkFilesBefore, "Spark compaction should reduce files") + assert(nativeFilesAfter < nativeFilesBefore, "Native compaction should reduce files") + + // Compare data - both should produce identical results + val sparkData = spark + .sql("SELECT id, name, value FROM compact_cat.db.spark_table ORDER BY id") + .collect() + .map(_.toString()) + + val nativeData = spark + .sql("SELECT id, name, value FROM compact_cat.db.native_table ORDER BY id") + .collect() + .map(_.toString()) + + assert( + sparkData.toSeq == nativeData.toSeq, + "Spark and Native compaction should produce identical data") + + // Verify row counts match + val sparkRows = + spark.sql("SELECT count(*) FROM compact_cat.db.spark_table").collect()(0).getLong(0) + val nativeRows = + spark.sql("SELECT count(*) FROM compact_cat.db.native_table").collect()(0).getLong(0) + assert( + sparkRows == nativeRows, + s"Row counts differ: Spark=$sparkRows, Native=$nativeRows") + assert(sparkRows == 15, s"Expected 15 rows, got $sparkRows") + + spark.sql("DROP TABLE compact_cat.db.spark_table") + spark.sql("DROP TABLE compact_cat.db.native_table") + } + } + } } From 95738231735b2d7e476730ac4eb2b081d9803934 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Mon, 16 Feb 2026 09:59:12 +0530 Subject: [PATCH 07/13] fix: scalastyle errors - remove unused imports and unnecessary string interpolation --- .../apache/comet/CometIcebergCompactionBenchmarkTest.scala | 2 +- .../sql/benchmark/CometIcebergTPCCompactionBenchmark.scala | 5 +---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala index e8bb623cdc..e07070150e 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala @@ -222,7 +222,7 @@ class CometIcebergCompactionBenchmarkTest extends CometTestBase { val rowCount = spark.sql(s"SELECT COUNT(*) FROM $tableName").first().getLong(0) // scalastyle:off println - println(s"\n========== COMPACTION BENCHMARK ==========") + println("\n========== COMPACTION BENCHMARK ==========") println(s"Table: $tableName") println(s"Files before: $filesBefore, Rows: $rowCount") println("=" * 45) diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala index 9d35052de8..4141f476db 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.benchmark -import java.io.{FileOutputStream, PrintStream} import java.io.File import java.util.Locale @@ -27,8 +26,6 @@ import org.apache.iceberg.spark.Spark3Util import org.apache.iceberg.spark.actions.SparkActions import org.apache.spark.sql.comet.CometNativeCompaction -import org.apache.comet.CometConf - /** * Benchmark to measure Iceberg compaction performance using TPC-H dataset. Compares Spark default * compaction (SparkBinPackDataRewriter) vs Comet-accelerated compaction @@ -146,7 +143,7 @@ object CometIcebergTPCCompactionBenchmark extends CometBenchmarkBase { val nativeFilesAfter = spark.sql(s"SELECT * FROM $icebergTableName.files").count() // Write result - writeResult(s"lineitem_part", rowCount, fileCount, nativeFilesAfter, 0, nativeTimeMs, 0) + writeResult("lineitem_part", rowCount, fileCount, nativeFilesAfter, 0, nativeTimeMs, 0) spark.sql(s"DROP TABLE IF EXISTS $icebergTableName") } From b9b015c8721d7b00c226fe3006eca48ef27c2f24 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Mon, 16 Feb 2026 10:04:37 +0530 Subject: [PATCH 08/13] fix: Scala 2.13 compilation errors and unused parameter warnings --- .../CometIcebergCompactionBenchmarkTest.scala | 8 ++------ .../CometIcebergTPCCompactionBenchmark.scala | 5 ++--- .../spark/sql/comet/CometNativeCompaction.scala | 17 ++++++++--------- 3 files changed, 12 insertions(+), 18 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala index e07070150e..f1a46edbf0 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala @@ -69,8 +69,8 @@ class CometIcebergCompactionBenchmarkTest extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_ICEBERG_COMPACTION_ENABLED.key -> "true") + // scalastyle:off parameter.number private def runTableBenchmark( - warehouseDir: File, sourceTable: String, schema: String, numFragments: Int, @@ -91,8 +91,6 @@ class CometIcebergCompactionBenchmarkTest extends CometTestBase { """) } - val filesBefore = spark.sql(s"SELECT * FROM $tableName.files").count() - // Benchmark 1: Spark default compaction val sparkStart = System.nanoTime() val sparkTable = Spark3Util.loadIcebergTable(spark, tableName) @@ -147,7 +145,6 @@ class CometIcebergCompactionBenchmarkTest extends CometTestBase { l_returnflag STRING, l_linestatus STRING""" val (lSpark, lNative, lSpeedup) = runTableBenchmark( - warehouseDir, "lineitem", lineitemSchema, numFragments, @@ -160,7 +157,7 @@ class CometIcebergCompactionBenchmarkTest extends CometTestBase { o_orderdate DATE, o_orderpriority STRING, o_clerk STRING, o_shippriority INT, o_comment STRING""" val (oSpark, oNative, oSpeedup) = - runTableBenchmark(warehouseDir, "orders", ordersSchema, numFragments, rowsPerFragment) + runTableBenchmark("orders", ordersSchema, numFragments, rowsPerFragment) println(f"${"orders"}%-15s $oSpark%12d $oNative%12d ${oSpeedup}%9.2fx") // Customer benchmark @@ -169,7 +166,6 @@ class CometIcebergCompactionBenchmarkTest extends CometTestBase { c_phone STRING, c_acctbal DOUBLE, c_mktsegment STRING, c_comment STRING""" val (cSpark, cNative, cSpeedup) = runTableBenchmark( - warehouseDir, "customer", customerSchema, numFragments, diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala index 4141f476db..b6335de481 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometIcebergTPCCompactionBenchmark.scala @@ -127,7 +127,7 @@ object CometIcebergTPCCompactionBenchmark extends CometBenchmarkBase { private def runPartitionedTableBenchmark(dataLocation: String, numFragments: Int): Unit = { val tableFilePath = resolveTablePath(dataLocation, "lineitem") - withIcebergWarehouse { (warehouseDir, catalog) => + withIcebergWarehouse { (_, catalog) => val icebergTableName = s"$catalog.db.lineitem_partitioned" // Create fragmented partitioned table @@ -193,7 +193,7 @@ object CometIcebergTPCCompactionBenchmark extends CometBenchmarkBase { val tableFilePath = resolveTablePath(dataLocation, tableName) - withIcebergWarehouse { (warehouseDir, catalog) => + withIcebergWarehouse { (_, catalog) => val icebergTableName = s"$catalog.db.${tableName}_iceberg" // Create fragmented table once to measure metadata @@ -217,7 +217,6 @@ object CometIcebergTPCCompactionBenchmark extends CometBenchmarkBase { val nativeTable = Spark3Util.loadIcebergTable(spark, icebergTableName) new CometNativeCompaction(spark).rewriteDataFiles(nativeTable) val nativeTimeMs = (System.nanoTime() - nativeStart) / 1000000 - val nativeFilesAfter = spark.sql(s"SELECT * FROM $icebergTableName.files").count() // Calculate speedup val speedup = if (nativeTimeMs > 0) sparkTimeMs.toDouble / nativeTimeMs.toDouble else 0.0 diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala index b139ee993f..46160dec2c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala @@ -136,8 +136,7 @@ class CometNativeCompaction(spark: SparkSession) extends Logging { tableConfig, group, targetFileSizeBytes, - compression, - table.location()) + compression) val result = executeNativeCompaction(compactionConfig) result match { @@ -172,7 +171,8 @@ class CometNativeCompaction(spark: SparkSession) extends Logging { s"Committing compaction: ${allFilesToDelete.size} files to delete, " + s"${allFilesToAdd.size} files to add") - val commitSuccess = commitCompaction(table, allFilesToDelete, allFilesToAdd) + val commitSuccess = + commitCompaction(table, allFilesToDelete.toSeq, allFilesToAdd.toSeq) if (!commitSuccess) { throw new RuntimeException("Failed to commit compaction results") @@ -264,8 +264,7 @@ class CometNativeCompaction(spark: SparkSession) extends Logging { tableConfig: IcebergTableConfig, tasks: Seq[FileScanTask], targetFileSizeBytes: Long, - compression: String, - tableLocation: String): CompactionTaskConfig = { + compression: String): CompactionTaskConfig = { val fileScanTaskConfigs = tasks.map { task => val partitionPath = task.spec().partitionToPath(task.file().partition()) @@ -312,12 +311,12 @@ class CometNativeCompaction(spark: SparkSession) extends Logging { try { val specs = table.specs() val deleteFiles: java.util.Set[DataFile] = new java.util.HashSet[DataFile]() - val deletePathSet = filesToDelete.toSet val snapshot = table.currentSnapshot() if (snapshot != null) { - import scala.jdk.CollectionConverters._ - val fileScanTasks = table.newScan().planFiles().iterator().asScala - fileScanTasks.foreach { task => + val deletePathSet = filesToDelete.toSet + val fileScanTasks = table.newScan().planFiles().iterator() + while (fileScanTasks.hasNext) { + val task = fileScanTasks.next() val dataFile = task.file() if (deletePathSet.contains(dataFile.path().toString)) { deleteFiles.add(dataFile) From 326e6cc257f36fbd1acad86f622b302283e68733 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Mon, 16 Feb 2026 10:04:37 +0530 Subject: [PATCH 09/13] fix: Scala 2.13 compilation errors and unused parameter warnings --- .../comet/CometIcebergCompactionBenchmarkTest.scala | 12 ++---------- .../spark/sql/comet/CometNativeCompaction.scala | 7 ++----- 2 files changed, 4 insertions(+), 15 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala index f1a46edbf0..93eb9a4624 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionBenchmarkTest.scala @@ -144,11 +144,7 @@ class CometIcebergCompactionBenchmarkTest extends CometTestBase { l_quantity DOUBLE, l_extendedprice DOUBLE, l_discount DOUBLE, l_tax DOUBLE, l_returnflag STRING, l_linestatus STRING""" val (lSpark, lNative, lSpeedup) = - runTableBenchmark( - "lineitem", - lineitemSchema, - numFragments, - rowsPerFragment) + runTableBenchmark("lineitem", lineitemSchema, numFragments, rowsPerFragment) println(f"${"lineitem"}%-15s $lSpark%12d $lNative%12d ${lSpeedup}%9.2fx") // Orders benchmark @@ -165,11 +161,7 @@ class CometIcebergCompactionBenchmarkTest extends CometTestBase { """c_custkey BIGINT, c_name STRING, c_address STRING, c_nationkey BIGINT, c_phone STRING, c_acctbal DOUBLE, c_mktsegment STRING, c_comment STRING""" val (cSpark, cNative, cSpeedup) = - runTableBenchmark( - "customer", - customerSchema, - numFragments, - rowsPerFragment) + runTableBenchmark("customer", customerSchema, numFragments, rowsPerFragment) println(f"${"customer"}%-15s $cSpark%12d $cNative%12d ${cSpeedup}%9.2fx") println("-" * 60) diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala index 46160dec2c..61ce4095ac 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala @@ -132,11 +132,8 @@ class CometNativeCompaction(spark: SparkSession) extends Logging { s"Processing file group ${groupIndex + 1}/${fileGroups.size} " + s"with ${group.size} files") - val compactionConfig = buildCompactionConfig( - tableConfig, - group, - targetFileSizeBytes, - compression) + val compactionConfig = + buildCompactionConfig(tableConfig, group, targetFileSizeBytes, compression) val result = executeNativeCompaction(compactionConfig) result match { From aa78c268bf2a939a2fb8328f3d5aae9a7ace0c2f Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 17 Feb 2026 00:36:28 +0530 Subject: [PATCH 10/13] feat: move CometNativeCompaction to main scope with provided Iceberg dep --- spark/pom.xml | 6 +++--- .../org/apache/spark/sql/comet/CometNativeCompaction.scala | 3 ++- 2 files changed, 5 insertions(+), 4 deletions(-) rename spark/src/{test => main}/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala (99%) diff --git a/spark/pom.xml b/spark/pom.xml index a9cd72f51b..8e3dd2095b 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -186,7 +186,7 @@ under the License. org.apache.iceberg iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} 1.5.2 - test + provided @@ -214,7 +214,7 @@ under the License. org.apache.iceberg iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} 1.8.1 - test + provided @@ -239,7 +239,7 @@ under the License. org.apache.iceberg iceberg-spark-runtime-${spark.version.short}_${scala.binary.version} 1.10.0 - test + provided diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala similarity index 99% rename from spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala index 61ce4095ac..5e8cb6dc67 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeCompaction.scala @@ -340,7 +340,8 @@ class CometNativeCompaction(spark: SparkSession) extends Logging { table.newRewrite().rewriteFiles(deleteFiles, addFiles).commit() logInfo( - s"Committed compaction: ${filesToDelete.size} files deleted, ${filesToAdd.size} files added") + s"Committed compaction: ${filesToDelete.size} files deleted, " + + s"${filesToAdd.size} files added") true } catch { case e: Exception => From 23a8dd2efd6319a61aae36d35b40d1d7ae848095 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 17 Feb 2026 00:36:49 +0530 Subject: [PATCH 11/13] fix: add enforcer ignore for Iceberg uber jar duplicate classes --- pom.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pom.xml b/pom.xml index 1b33fc4757..09081050ee 100644 --- a/pom.xml +++ b/pom.xml @@ -1135,6 +1135,15 @@ under the License. com.google.thirdparty.publicsuffix.PublicSuffixType + + + org.apache.iceberg + * + + org.jetbrains.annotations.* + org.intellij.lang.annotations.* + + true true From 38095c3860727d902e83eee334792d24593ee0f3 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 17 Feb 2026 00:36:59 +0530 Subject: [PATCH 12/13] feat: add CometCompactionRule to intercept CALL rewrite_data_files --- .../scala/org/apache/comet/CometConf.scala | 8 +- .../comet/CometSparkSessionExtensions.scala | 7 +- .../comet/rules/CometCompactionRule.scala | 104 ++++++++++++++++++ .../sql/comet/CometNativeCompactionExec.scala | 67 +++++++++++ 4 files changed, 182 insertions(+), 4 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/rules/CometCompactionRule.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometNativeCompactionExec.scala diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 3dc61ebdfa..f1ee0e1868 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -152,11 +152,13 @@ object CometConf extends ShimCometConf { val COMET_ICEBERG_COMPACTION_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.iceberg.compaction.enabled") - .category(CATEGORY_TESTING) + .category(CATEGORY_EXEC) .doc( "Whether to enable Comet-accelerated Iceberg compaction. When enabled, " + - "CALL rewrite_data_files() uses Comet's native scan for the read path, " + - "reducing JVM overhead during compaction. Experimental.") + "CALL rewrite_data_files() is intercepted and executed via Comet's native " + + "Rust/DataFusion engine for direct Parquet read/write, bypassing Spark's " + + "DAG execution. Only bin-pack strategy is supported; sort and z-order " + + "fall back to Spark's default. Requires Iceberg on the classpath. Experimental.") .booleanConf .createWithDefault(false) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 5839570684..3245938264 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.internal.SQLConf import org.apache.comet.CometConf._ -import org.apache.comet.rules.{CometExecRule, CometScanRule, EliminateRedundantTransitions} +import org.apache.comet.rules.{CometCompactionRule, CometExecRule, CometScanRule, EliminateRedundantTransitions} import org.apache.comet.shims.ShimCometSparkSessionExtensions /** @@ -47,6 +47,7 @@ class CometSparkSessionExtensions override def apply(extensions: SparkSessionExtensions): Unit = { extensions.injectColumnar { session => CometScanColumnar(session) } extensions.injectColumnar { session => CometExecColumnar(session) } + extensions.injectColumnar { session => CometCompactionColumnar(session) } extensions.injectQueryStagePrepRule { session => CometScanRule(session) } extensions.injectQueryStagePrepRule { session => CometExecRule(session) } } @@ -61,6 +62,10 @@ class CometSparkSessionExtensions override def postColumnarTransitions: Rule[SparkPlan] = EliminateRedundantTransitions(session) } + + case class CometCompactionColumnar(session: SparkSession) extends ColumnarRule { + override def preColumnarTransitions: Rule[SparkPlan] = CometCompactionRule(session) + } } object CometSparkSessionExtensions extends Logging { diff --git a/spark/src/main/scala/org/apache/comet/rules/CometCompactionRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometCompactionRule.scala new file mode 100644 index 0000000000..d370b4c722 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/rules/CometCompactionRule.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet.rules + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.comet.{CometNativeCompaction, CometNativeCompactionExec} +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.SparkPlan + +import org.apache.comet.CometConf + +/** + * Replaces Iceberg's CallExec targeting RewriteDataFilesProcedure with native Comet compaction. + * + * Uses reflection to detect CallExec (from Iceberg extensions) to avoid hard compile-time + * dependency. Only active when spark.comet.iceberg.compaction.enabled is true and native + * compaction is available. Currently supports Spark 3.x only (Spark 4.0 uses InvokeProcedures at + * the analysis phase, handled separately via shim). + */ +case class CometCompactionRule(session: SparkSession) extends Rule[SparkPlan] with Logging { + + private val CALL_EXEC_CLASS = "org.apache.spark.sql.execution.datasources.v2.CallExec" + private val REWRITE_PROCEDURE_NAME = "RewriteDataFilesProcedure" + + override def apply(plan: SparkPlan): SparkPlan = { + if (!isEnabled) return plan + + plan.transformUp { + case exec if isRewriteCallExec(exec) => + replaceWithNative(exec).getOrElse(exec) + } + } + + private def isEnabled: Boolean = + CometConf.COMET_ICEBERG_COMPACTION_ENABLED.get(session.sessionState.conf) && + CometNativeCompaction.isAvailable + + private def isRewriteCallExec(plan: SparkPlan): Boolean = { + plan.getClass.getName == CALL_EXEC_CLASS && { + try { + val proc = plan.getClass.getMethod("procedure").invoke(plan) + proc.getClass.getSimpleName == REWRITE_PROCEDURE_NAME + } catch { case _: Exception => false } + } + } + + private def replaceWithNative(exec: SparkPlan): Option[SparkPlan] = { + try { + val proc = exec.getClass.getMethod("procedure").invoke(exec) + val input = exec.getClass.getMethod("input").invoke(exec).asInstanceOf[InternalRow] + + // Only intercept bin-pack strategy (default when strategy is null) + if (!input.isNullAt(1)) { + val strategy = input.getUTF8String(1).toString + if (!strategy.equalsIgnoreCase("binpack")) { + logInfo(s"Native compaction skipped: unsupported strategy '$strategy'") + return None + } + } + + val tableCatalog = extractTableCatalog(proc) + val tableIdent = parseIdentifier(input.getUTF8String(0).toString) + + logInfo(s"Replacing CallExec with CometNativeCompactionExec for $tableIdent") + Some(CometNativeCompactionExec(exec.output, tableCatalog, tableIdent)) + } catch { + case e: Exception => + logWarning(s"Cannot replace with native compaction: ${e.getMessage}") + None + } + } + + /** Extract TableCatalog from BaseProcedure via reflection (field is private). */ + private def extractTableCatalog(procedure: Any): TableCatalog = { + val field = procedure.getClass.getSuperclass.getDeclaredField("tableCatalog") + field.setAccessible(true) + field.get(procedure).asInstanceOf[TableCatalog] + } + + private def parseIdentifier(identStr: String): Identifier = { + val parts = identStr.split("\\.") + Identifier.of(parts.dropRight(1), parts.last) + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeCompactionExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeCompactionExec.scala new file mode 100644 index 0000000000..2a1d69649b --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeCompactionExec.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericInternalRow} +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec + +/** + * Executes Iceberg compaction via Comet's native Rust/DataFusion engine. Replaces CallExec for + * RewriteDataFilesProcedure when native compaction is enabled. + * + * Output row is built dynamically to match the procedure's output schema, which varies across + * Iceberg versions (e.g. removed_delete_files_count added in later versions). + */ +case class CometNativeCompactionExec( + output: Seq[Attribute], + @transient tableCatalog: TableCatalog, + tableIdent: Identifier) + extends LeafV2CommandExec + with Logging { + + override protected def run(): Seq[InternalRow] = { + val spark = SparkSession.active + val icebergTable = tableCatalog + .loadTable(tableIdent) + .asInstanceOf[SparkTable] + .table() + + logInfo(s"Executing native compaction for $tableIdent") + val summary = CometNativeCompaction(spark).rewriteDataFiles(icebergTable) + + val fieldValues: Map[String, Any] = Map( + "rewritten_data_files_count" -> summary.filesDeleted, + "added_data_files_count" -> summary.filesAdded, + "rewritten_bytes_count" -> summary.bytesDeleted, + "failed_data_files_count" -> 0, + "removed_delete_files_count" -> 0) + + val values = output.map(attr => fieldValues.getOrElse(attr.name, 0)) + Seq(new GenericInternalRow(values.toArray)) + } + + override def simpleString(maxFields: Int): String = + s"CometNativeCompactionExec[$tableIdent]" +} From e2e9c3317b24207aa1c5448a0271f75d12b6a063 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Tue, 17 Feb 2026 00:37:09 +0530 Subject: [PATCH 13/13] test: add integration tests for CALL rewrite_data_files procedure --- ...CometIcebergCompactionProcedureSuite.scala | 246 ++++++++++++++++++ 1 file changed, 246 insertions(+) create mode 100644 spark/src/test/scala/org/apache/comet/CometIcebergCompactionProcedureSuite.scala diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergCompactionProcedureSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionProcedureSuite.scala new file mode 100644 index 0000000000..575b0cc096 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometIcebergCompactionProcedureSuite.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.comet + +import java.io.File +import java.nio.file.Files + +import org.apache.spark.SparkConf +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.comet.CometNativeCompaction + +/** + * Integration tests for CALL rewrite_data_files() procedure intercepted by CometCompactionRule. + * Verifies that the SQL procedure path routes through native compaction when enabled. + */ +class CometIcebergCompactionProcedureSuite extends CometTestBase { + + private val icebergExtensions = + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions" + + override protected def sparkConf: SparkConf = { + val conf = super.sparkConf + val existing = conf.get("spark.sql.extensions", "") + val extensions = + if (existing.isEmpty) icebergExtensions + else s"$existing,$icebergExtensions" + conf.set("spark.sql.extensions", extensions) + } + + private def icebergAvailable: Boolean = { + try { + Class.forName("org.apache.iceberg.catalog.Catalog") + true + } catch { + case _: ClassNotFoundException => false + } + } + + private def withTempIcebergDir(f: File => Unit): Unit = { + val dir = Files.createTempDirectory("comet-procedure-test").toFile + try { + f(dir) + } finally { + def deleteRecursively(file: File): Unit = { + if (file.isDirectory) file.listFiles().foreach(deleteRecursively) + file.delete() + } + deleteRecursively(dir) + } + } + + private def catalogConf(warehouseDir: File, compactionEnabled: Boolean): Map[String, String] = + Map( + "spark.sql.catalog.proc_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.proc_cat.type" -> "hadoop", + "spark.sql.catalog.proc_cat.warehouse" -> warehouseDir.getAbsolutePath, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_ICEBERG_COMPACTION_ENABLED.key -> compactionEnabled.toString) + + private def createFragmentedTable(tableName: String, rowCount: Int): Unit = { + spark.sql(s""" + CREATE TABLE $tableName ( + id BIGINT, + name STRING, + value DOUBLE + ) USING iceberg + """) + for (i <- 1 to rowCount) { + spark.sql(s"INSERT INTO $tableName VALUES ($i, 'name_$i', ${i * 1.5})") + } + } + + // ============== SQL Procedure Tests ============== + + test("CALL rewrite_data_files uses native compaction when enabled") { + assume(icebergAvailable, "Iceberg not available") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(catalogConf(warehouseDir, compactionEnabled = true).toSeq: _*) { + createFragmentedTable("proc_cat.db.proc_table", 10) + + val filesBefore = + spark.sql("SELECT file_path FROM proc_cat.db.proc_table.files").count() + assert(filesBefore >= 5, s"Expected fragmented files, got $filesBefore") + + val rowsBefore = + spark.sql("SELECT count(*) FROM proc_cat.db.proc_table").collect()(0).getLong(0) + + val result = + spark.sql("CALL proc_cat.system.rewrite_data_files(table => 'db.proc_table')") + val resultRow = result.collect() + + assert(resultRow.length == 1, "Procedure should return one result row") + val fields = result.schema.fieldNames.toSeq + Seq("rewritten_data_files_count", "added_data_files_count", "rewritten_bytes_count") + .foreach(f => assert(fields.contains(f), s"Missing field $f in $fields")) + + val rewrittenCount = resultRow(0).getInt(0) + val addedCount = resultRow(0).getInt(1) + assert(rewrittenCount > 0, "Should rewrite files") + assert(addedCount > 0, "Should add compacted files") + + spark.sql("REFRESH TABLE proc_cat.db.proc_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM proc_cat.db.proc_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + val filesAfter = + spark.sql("SELECT file_path FROM proc_cat.db.proc_table.files").count() + assert( + filesAfter < filesBefore, + s"File count should decrease: $filesBefore -> $filesAfter") + + spark.sql("DROP TABLE proc_cat.db.proc_table") + } + } + } + + test("CALL rewrite_data_files falls back to Spark when config disabled") { + assume(icebergAvailable, "Iceberg not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(catalogConf(warehouseDir, compactionEnabled = false).toSeq: _*) { + createFragmentedTable("proc_cat.db.fallback_table", 10) + + val rowsBefore = + spark.sql("SELECT count(*) FROM proc_cat.db.fallback_table").collect()(0).getLong(0) + + val result = + spark.sql("CALL proc_cat.system.rewrite_data_files(table => 'db.fallback_table')") + val resultRow = result.collect() + + assert(resultRow.length == 1, "Spark procedure should return one result row") + + spark.sql("REFRESH TABLE proc_cat.db.fallback_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM proc_cat.db.fallback_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE proc_cat.db.fallback_table") + } + } + } + + test("CALL rewrite_data_files with binpack strategy uses native compaction") { + assume(icebergAvailable, "Iceberg not available") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(catalogConf(warehouseDir, compactionEnabled = true).toSeq: _*) { + createFragmentedTable("proc_cat.db.binpack_table", 10) + + val rowsBefore = + spark.sql("SELECT count(*) FROM proc_cat.db.binpack_table").collect()(0).getLong(0) + + val result = spark.sql( + "CALL proc_cat.system.rewrite_data_files(table => 'db.binpack_table', strategy => 'binpack')") + val resultRow = result.collect() + + assert(resultRow.length == 1) + assert(resultRow(0).getInt(0) > 0, "Should rewrite files with binpack") + + spark.sql("REFRESH TABLE proc_cat.db.binpack_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM proc_cat.db.binpack_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE proc_cat.db.binpack_table") + } + } + } + + test("CALL rewrite_data_files with sort strategy falls back to Spark") { + assume(icebergAvailable, "Iceberg not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(catalogConf(warehouseDir, compactionEnabled = true).toSeq: _*) { + createFragmentedTable("proc_cat.db.sort_table", 10) + + val rowsBefore = + spark.sql("SELECT count(*) FROM proc_cat.db.sort_table").collect()(0).getLong(0) + + // Sort strategy not supported by native compaction, should fall back to Spark + val result = spark.sql( + "CALL proc_cat.system.rewrite_data_files(table => 'db.sort_table', strategy => 'sort', sort_order => 'id')") + val resultRow = result.collect() + + assert(resultRow.length == 1, "Spark fallback should still return results") + + spark.sql("REFRESH TABLE proc_cat.db.sort_table") + val rowsAfter = + spark.sql("SELECT count(*) FROM proc_cat.db.sort_table").collect()(0).getLong(0) + assert(rowsAfter == rowsBefore, s"Row count changed: $rowsBefore -> $rowsAfter") + + spark.sql("DROP TABLE proc_cat.db.sort_table") + } + } + } + + test("CALL rewrite_data_files preserves data correctness") { + assume(icebergAvailable, "Iceberg not available") + assume(CometNativeCompaction.isAvailable, "Native compaction not available") + + withTempIcebergDir { warehouseDir => + withSQLConf(catalogConf(warehouseDir, compactionEnabled = true).toSeq: _*) { + createFragmentedTable("proc_cat.db.correct_table", 15) + + val dataBefore = spark + .sql("SELECT id, name, value FROM proc_cat.db.correct_table ORDER BY id") + .collect() + .map(_.toString()) + + spark.sql("CALL proc_cat.system.rewrite_data_files(table => 'db.correct_table')") + + spark.sql("REFRESH TABLE proc_cat.db.correct_table") + val dataAfter = spark + .sql("SELECT id, name, value FROM proc_cat.db.correct_table ORDER BY id") + .collect() + .map(_.toString()) + + assert(dataBefore.toSeq == dataAfter.toSeq, "Data must be identical after procedure call") + + spark.sql("DROP TABLE proc_cat.db.correct_table") + } + } + } +}