From b7c7559b77b0b41172f39ecd9f0e3a7dfc6a4833 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Sat, 14 Feb 2026 14:51:24 +0530 Subject: [PATCH 01/15] 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/15] 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/15] 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/15] 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/15] 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/15] 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/15] 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/15] 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/15] 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/15] 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/15] 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/15] 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/15] 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") + } + } + } +} From 3af0b12f3303d0d1c224fe131cc0b13be72f7dd2 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 25 Feb 2026 11:30:02 +0530 Subject: [PATCH 14/15] Merge upstream/main into feature/iceberg-compaction-benchmark --- .github/dependabot.yml | 16 + .github/workflows/pr_build_linux.yml | 24 +- .github/workflows/stale.yml | 2 +- .gitignore | 2 + {dev/benchmarks => benchmarks/tpc}/.gitignore | 0 benchmarks/tpc/README.md | 365 +++ benchmarks/tpc/create-iceberg-tables.py | 171 ++ .../tpc}/drop-caches.sh | 0 benchmarks/tpc/engines/comet-hashjoin.toml | 35 + .../tpc/engines/comet-iceberg-hashjoin.toml | 48 + benchmarks/tpc/engines/comet-iceberg.toml | 47 + benchmarks/tpc/engines/comet.toml | 34 + benchmarks/tpc/engines/gluten.toml | 34 + benchmarks/tpc/engines/spark.toml | 19 + .../tpc}/generate-comparison.py | 130 +- benchmarks/tpc/infra/docker/Dockerfile | 58 + .../tpc/infra/docker/Dockerfile.build-comet | 76 + .../infra/docker/docker-compose-laptop.yml | 98 + .../tpc/infra/docker/docker-compose.yml | 132 ++ benchmarks/tpc/queries/tpcds/q1.sql | 26 + benchmarks/tpc/queries/tpcds/q10.sql | 60 + benchmarks/tpc/queries/tpcds/q11.sql | 82 + benchmarks/tpc/queries/tpcds/q12.sql | 35 + benchmarks/tpc/queries/tpcds/q13.sql | 53 + benchmarks/tpc/queries/tpcds/q14.sql | 211 ++ benchmarks/tpc/queries/tpcds/q15.sql | 21 + benchmarks/tpc/queries/tpcds/q16.sql | 32 + benchmarks/tpc/queries/tpcds/q17.sql | 46 + benchmarks/tpc/queries/tpcds/q18.sql | 35 + benchmarks/tpc/queries/tpcds/q19.sql | 26 + benchmarks/tpc/queries/tpcds/q2.sql | 61 + benchmarks/tpc/queries/tpcds/q20.sql | 31 + benchmarks/tpc/queries/tpcds/q21.sql | 31 + benchmarks/tpc/queries/tpcds/q22.sql | 21 + benchmarks/tpc/queries/tpcds/q23.sql | 108 + benchmarks/tpc/queries/tpcds/q24.sql | 108 + benchmarks/tpc/queries/tpcds/q25.sql | 49 + benchmarks/tpc/queries/tpcds/q26.sql | 22 + benchmarks/tpc/queries/tpcds/q27.sql | 24 + benchmarks/tpc/queries/tpcds/q28.sql | 54 + benchmarks/tpc/queries/tpcds/q29.sql | 48 + benchmarks/tpc/queries/tpcds/q3.sql | 22 + benchmarks/tpc/queries/tpcds/q30.sql | 32 + benchmarks/tpc/queries/tpcds/q31.sql | 53 + benchmarks/tpc/queries/tpcds/q32.sql | 29 + benchmarks/tpc/queries/tpcds/q33.sql | 76 + benchmarks/tpc/queries/tpcds/q34.sql | 32 + benchmarks/tpc/queries/tpcds/q35.sql | 59 + benchmarks/tpc/queries/tpcds/q36.sql | 31 + benchmarks/tpc/queries/tpcds/q37.sql | 18 + benchmarks/tpc/queries/tpcds/q38.sql | 24 + benchmarks/tpc/queries/tpcds/q39.sql | 55 + benchmarks/tpc/queries/tpcds/q4.sql | 117 + benchmarks/tpc/queries/tpcds/q40.sql | 29 + benchmarks/tpc/queries/tpcds/q41.sql | 53 + benchmarks/tpc/queries/tpcds/q42.sql | 23 + benchmarks/tpc/queries/tpcds/q43.sql | 20 + benchmarks/tpc/queries/tpcds/q44.sql | 36 + benchmarks/tpc/queries/tpcds/q45.sql | 21 + benchmarks/tpc/queries/tpcds/q46.sql | 36 + benchmarks/tpc/queries/tpcds/q47.sql | 52 + benchmarks/tpc/queries/tpcds/q48.sql | 68 + benchmarks/tpc/queries/tpcds/q49.sql | 130 ++ benchmarks/tpc/queries/tpcds/q5.sql | 129 ++ benchmarks/tpc/queries/tpcds/q50.sql | 60 + benchmarks/tpc/queries/tpcds/q51.sql | 46 + benchmarks/tpc/queries/tpcds/q52.sql | 23 + benchmarks/tpc/queries/tpcds/q53.sql | 29 + benchmarks/tpc/queries/tpcds/q54.sql | 57 + benchmarks/tpc/queries/tpcds/q55.sql | 15 + benchmarks/tpc/queries/tpcds/q56.sql | 70 + benchmarks/tpc/queries/tpcds/q57.sql | 49 + benchmarks/tpc/queries/tpcds/q58.sql | 66 + benchmarks/tpc/queries/tpcds/q59.sql | 45 + benchmarks/tpc/queries/tpcds/q6.sql | 27 + benchmarks/tpc/queries/tpcds/q60.sql | 79 + benchmarks/tpc/queries/tpcds/q61.sql | 45 + benchmarks/tpc/queries/tpcds/q62.sql | 36 + benchmarks/tpc/queries/tpcds/q63.sql | 30 + benchmarks/tpc/queries/tpcds/q64.sql | 122 + benchmarks/tpc/queries/tpcds/q65.sql | 30 + benchmarks/tpc/queries/tpcds/q66.sql | 221 ++ benchmarks/tpc/queries/tpcds/q67.sql | 45 + benchmarks/tpc/queries/tpcds/q68.sql | 43 + benchmarks/tpc/queries/tpcds/q69.sql | 48 + benchmarks/tpc/queries/tpcds/q7.sql | 22 + benchmarks/tpc/queries/tpcds/q70.sql | 39 + benchmarks/tpc/queries/tpcds/q71.sql | 41 + benchmarks/tpc/queries/tpcds/q72.sql | 30 + benchmarks/tpc/queries/tpcds/q73.sql | 29 + benchmarks/tpc/queries/tpcds/q74.sql | 62 + benchmarks/tpc/queries/tpcds/q75.sql | 71 + benchmarks/tpc/queries/tpcds/q76.sql | 25 + benchmarks/tpc/queries/tpcds/q77.sql | 109 + benchmarks/tpc/queries/tpcds/q78.sql | 59 + benchmarks/tpc/queries/tpcds/q79.sql | 24 + benchmarks/tpc/queries/tpcds/q8.sql | 109 + benchmarks/tpc/queries/tpcds/q80.sql | 97 + benchmarks/tpc/queries/tpcds/q81.sql | 32 + benchmarks/tpc/queries/tpcds/q82.sql | 18 + benchmarks/tpc/queries/tpcds/q83.sql | 68 + benchmarks/tpc/queries/tpcds/q84.sql | 22 + benchmarks/tpc/queries/tpcds/q85.sql | 85 + benchmarks/tpc/queries/tpcds/q86.sql | 27 + benchmarks/tpc/queries/tpcds/q87.sql | 24 + benchmarks/tpc/queries/tpcds/q88.sql | 95 + benchmarks/tpc/queries/tpcds/q89.sql | 29 + benchmarks/tpc/queries/tpcds/q9.sql | 52 + benchmarks/tpc/queries/tpcds/q90.sql | 23 + benchmarks/tpc/queries/tpcds/q91.sql | 32 + benchmarks/tpc/queries/tpcds/q92.sql | 31 + benchmarks/tpc/queries/tpcds/q93.sql | 19 + benchmarks/tpc/queries/tpcds/q94.sql | 30 + benchmarks/tpc/queries/tpcds/q95.sql | 33 + benchmarks/tpc/queries/tpcds/q96.sql | 17 + benchmarks/tpc/queries/tpcds/q97.sql | 26 + benchmarks/tpc/queries/tpcds/q98.sql | 34 + benchmarks/tpc/queries/tpcds/q99.sql | 36 + benchmarks/tpc/queries/tpch/q1.sql | 23 + benchmarks/tpc/queries/tpch/q10.sql | 33 + benchmarks/tpc/queries/tpch/q11.sql | 29 + benchmarks/tpc/queries/tpch/q12.sql | 30 + benchmarks/tpc/queries/tpch/q13.sql | 22 + benchmarks/tpc/queries/tpch/q14.sql | 15 + benchmarks/tpc/queries/tpch/q15.sql | 33 + benchmarks/tpc/queries/tpch/q16.sql | 32 + benchmarks/tpc/queries/tpch/q17.sql | 19 + benchmarks/tpc/queries/tpch/q18.sql | 34 + benchmarks/tpc/queries/tpch/q19.sql | 37 + benchmarks/tpc/queries/tpch/q2.sql | 45 + benchmarks/tpc/queries/tpch/q20.sql | 39 + benchmarks/tpc/queries/tpch/q21.sql | 41 + benchmarks/tpc/queries/tpch/q22.sql | 39 + benchmarks/tpc/queries/tpch/q3.sql | 24 + benchmarks/tpc/queries/tpch/q4.sql | 23 + benchmarks/tpc/queries/tpch/q5.sql | 26 + benchmarks/tpc/queries/tpch/q6.sql | 11 + benchmarks/tpc/queries/tpch/q7.sql | 41 + benchmarks/tpc/queries/tpch/q8.sql | 39 + benchmarks/tpc/queries/tpch/q9.sql | 34 + benchmarks/tpc/run.py | 397 ++++ .../benchmarks => benchmarks/tpc}/tpcbench.py | 40 +- .../scala/org/apache/comet/CometConf.scala | 81 +- dev/benchmarks/README.md | 151 -- dev/benchmarks/blaze-tpcds.sh | 53 - dev/benchmarks/blaze-tpch.sh | 53 - dev/benchmarks/comet-tpcds.sh | 53 - dev/benchmarks/comet-tpch-iceberg.sh | 114 - dev/benchmarks/comet-tpch.sh | 55 - dev/benchmarks/create-iceberg-tpch.py | 88 - dev/benchmarks/gluten-tpcds.sh | 53 - dev/benchmarks/gluten-tpch.sh | 53 - dev/benchmarks/spark-tpcds.sh | 45 - dev/benchmarks/spark-tpch.sh | 46 - dev/diffs/3.4.3.diff | 2 +- dev/diffs/3.5.8.diff | 2 +- dev/diffs/4.0.1.diff | 2 +- dev/release/rat_exclude_files.txt | 1 + docs/source/about/gluten_comparison.md | 2 +- .../benchmark-results/tpc-ds.md | 2 +- .../benchmark-results/tpc-h.md | 2 +- docs/source/contributor-guide/benchmarking.md | 2 +- .../contributor-guide/benchmarking_aws_ec2.md | 15 +- docs/source/contributor-guide/development.md | 73 + docs/source/contributor-guide/ffi.md | 7 +- .../source/contributor-guide/parquet_scans.md | 120 +- docs/source/contributor-guide/roadmap.md | 27 +- .../source/user-guide/latest/compatibility.md | 2 +- docs/spark_expressions_support.md | 2 +- native/Cargo.lock | 166 +- native/Cargo.toml | 8 +- native/core/Cargo.toml | 4 +- native/core/src/execution/columnar_to_row.rs | 14 +- native/core/src/execution/jni_api.rs | 147 +- .../src/execution/operators/iceberg_scan.rs | 6 +- native/core/src/execution/planner.rs | 130 +- native/core/src/execution/spark_config.rs | 8 + native/fs-hdfs/Cargo.toml | 4 +- native/proto/src/proto/operator.proto | 3 + native/spark-expr/Cargo.toml | 8 + .../spark-expr/benches/cast_from_boolean.rs | 89 + .../benches/cast_int_to_timestamp.rs | 131 ++ .../src/array_funcs/array_repeat.rs | 216 -- native/spark-expr/src/array_funcs/mod.rs | 2 - .../src/bitwise_funcs/bitwise_count.rs | 57 +- native/spark-expr/src/comet_scalar_funcs.rs | 10 +- .../src/conversion_funcs/boolean.rs | 196 ++ .../spark-expr/src/conversion_funcs/cast.rs | 2007 ++--------------- native/spark-expr/src/conversion_funcs/mod.rs | 3 + .../spark-expr/src/conversion_funcs/string.rs | 1632 ++++++++++++++ .../spark-expr/src/conversion_funcs/utils.rs | 119 + pom.xml | 32 + spark/pom.xml | 17 + .../org/apache/comet/CometExecIterator.scala | 5 + .../apache/comet/expressions/CometCast.scala | 55 +- .../comet/iceberg/IcebergReflection.scala | 26 + .../apache/comet/rules/CometScanRule.scala | 17 +- .../apache/comet/serde/QueryPlanSerde.scala | 3 +- .../scala/org/apache/comet/serde/arrays.scala | 37 +- .../operator/CometDataWritingCommand.scala | 2 +- .../operator/CometIcebergNativeScan.scala | 19 +- .../org/apache/comet/serde/strings.scala | 31 +- .../comet/CometNativeColumnarToRowExec.scala | 5 - .../iceberg/rest/RESTCatalogAdapter.java | 28 + .../expressions/array/array_repeat.sql | 8 +- .../sql-tests/expressions/bitwise/bitwise.sql | 2 +- .../sql-tests/expressions/hash/crc32.sql | 32 + .../expressions/string/concat_ws.sql | 4 +- .../string/regexp_replace_enabled.sql | 2 +- .../expressions/string/rlike_enabled.sql | 2 +- .../org/apache/comet/CometCastSuite.scala | 122 +- .../apache/comet/CometExpressionSuite.scala | 10 +- .../apache/comet/CometFuzzIcebergSuite.scala | 2 +- .../org/apache/comet/CometFuzzTestBase.scala | 16 +- .../org/apache/comet/CometFuzzTestSuite.scala | 2 +- .../apache/comet/IcebergReadFromS3Suite.scala | 74 +- .../parquet/CometParquetWriterSuite.scala | 55 +- .../benchmark/CometTPCDSMicroBenchmark.scala | 2 +- .../comet/iceberg/RESTCatalogHelper.scala | 25 +- .../comet/iceberg/RESTCatalogHelper.scala | 25 +- 220 files changed, 10708 insertions(+), 3380 deletions(-) rename {dev/benchmarks => benchmarks/tpc}/.gitignore (100%) create mode 100644 benchmarks/tpc/README.md create mode 100644 benchmarks/tpc/create-iceberg-tables.py rename {dev/benchmarks => benchmarks/tpc}/drop-caches.sh (100%) create mode 100644 benchmarks/tpc/engines/comet-hashjoin.toml create mode 100644 benchmarks/tpc/engines/comet-iceberg-hashjoin.toml create mode 100644 benchmarks/tpc/engines/comet-iceberg.toml create mode 100644 benchmarks/tpc/engines/comet.toml create mode 100644 benchmarks/tpc/engines/gluten.toml create mode 100644 benchmarks/tpc/engines/spark.toml rename {dev/benchmarks => benchmarks/tpc}/generate-comparison.py (61%) create mode 100644 benchmarks/tpc/infra/docker/Dockerfile create mode 100644 benchmarks/tpc/infra/docker/Dockerfile.build-comet create mode 100644 benchmarks/tpc/infra/docker/docker-compose-laptop.yml create mode 100644 benchmarks/tpc/infra/docker/docker-compose.yml create mode 100644 benchmarks/tpc/queries/tpcds/q1.sql create mode 100644 benchmarks/tpc/queries/tpcds/q10.sql create mode 100644 benchmarks/tpc/queries/tpcds/q11.sql create mode 100644 benchmarks/tpc/queries/tpcds/q12.sql create mode 100644 benchmarks/tpc/queries/tpcds/q13.sql create mode 100644 benchmarks/tpc/queries/tpcds/q14.sql create mode 100644 benchmarks/tpc/queries/tpcds/q15.sql create mode 100644 benchmarks/tpc/queries/tpcds/q16.sql create mode 100644 benchmarks/tpc/queries/tpcds/q17.sql create mode 100644 benchmarks/tpc/queries/tpcds/q18.sql create mode 100644 benchmarks/tpc/queries/tpcds/q19.sql create mode 100644 benchmarks/tpc/queries/tpcds/q2.sql create mode 100644 benchmarks/tpc/queries/tpcds/q20.sql create mode 100644 benchmarks/tpc/queries/tpcds/q21.sql create mode 100644 benchmarks/tpc/queries/tpcds/q22.sql create mode 100644 benchmarks/tpc/queries/tpcds/q23.sql create mode 100644 benchmarks/tpc/queries/tpcds/q24.sql create mode 100644 benchmarks/tpc/queries/tpcds/q25.sql create mode 100644 benchmarks/tpc/queries/tpcds/q26.sql create mode 100644 benchmarks/tpc/queries/tpcds/q27.sql create mode 100644 benchmarks/tpc/queries/tpcds/q28.sql create mode 100644 benchmarks/tpc/queries/tpcds/q29.sql create mode 100644 benchmarks/tpc/queries/tpcds/q3.sql create mode 100644 benchmarks/tpc/queries/tpcds/q30.sql create mode 100644 benchmarks/tpc/queries/tpcds/q31.sql create mode 100644 benchmarks/tpc/queries/tpcds/q32.sql create mode 100644 benchmarks/tpc/queries/tpcds/q33.sql create mode 100644 benchmarks/tpc/queries/tpcds/q34.sql create mode 100644 benchmarks/tpc/queries/tpcds/q35.sql create mode 100644 benchmarks/tpc/queries/tpcds/q36.sql create mode 100644 benchmarks/tpc/queries/tpcds/q37.sql create mode 100644 benchmarks/tpc/queries/tpcds/q38.sql create mode 100644 benchmarks/tpc/queries/tpcds/q39.sql create mode 100644 benchmarks/tpc/queries/tpcds/q4.sql create mode 100644 benchmarks/tpc/queries/tpcds/q40.sql create mode 100644 benchmarks/tpc/queries/tpcds/q41.sql create mode 100644 benchmarks/tpc/queries/tpcds/q42.sql create mode 100644 benchmarks/tpc/queries/tpcds/q43.sql create mode 100644 benchmarks/tpc/queries/tpcds/q44.sql create mode 100644 benchmarks/tpc/queries/tpcds/q45.sql create mode 100644 benchmarks/tpc/queries/tpcds/q46.sql create mode 100644 benchmarks/tpc/queries/tpcds/q47.sql create mode 100644 benchmarks/tpc/queries/tpcds/q48.sql create mode 100644 benchmarks/tpc/queries/tpcds/q49.sql create mode 100644 benchmarks/tpc/queries/tpcds/q5.sql create mode 100644 benchmarks/tpc/queries/tpcds/q50.sql create mode 100644 benchmarks/tpc/queries/tpcds/q51.sql create mode 100644 benchmarks/tpc/queries/tpcds/q52.sql create mode 100644 benchmarks/tpc/queries/tpcds/q53.sql create mode 100644 benchmarks/tpc/queries/tpcds/q54.sql create mode 100644 benchmarks/tpc/queries/tpcds/q55.sql create mode 100644 benchmarks/tpc/queries/tpcds/q56.sql create mode 100644 benchmarks/tpc/queries/tpcds/q57.sql create mode 100644 benchmarks/tpc/queries/tpcds/q58.sql create mode 100644 benchmarks/tpc/queries/tpcds/q59.sql create mode 100644 benchmarks/tpc/queries/tpcds/q6.sql create mode 100644 benchmarks/tpc/queries/tpcds/q60.sql create mode 100644 benchmarks/tpc/queries/tpcds/q61.sql create mode 100644 benchmarks/tpc/queries/tpcds/q62.sql create mode 100644 benchmarks/tpc/queries/tpcds/q63.sql create mode 100644 benchmarks/tpc/queries/tpcds/q64.sql create mode 100644 benchmarks/tpc/queries/tpcds/q65.sql create mode 100644 benchmarks/tpc/queries/tpcds/q66.sql create mode 100644 benchmarks/tpc/queries/tpcds/q67.sql create mode 100644 benchmarks/tpc/queries/tpcds/q68.sql create mode 100644 benchmarks/tpc/queries/tpcds/q69.sql create mode 100644 benchmarks/tpc/queries/tpcds/q7.sql create mode 100644 benchmarks/tpc/queries/tpcds/q70.sql create mode 100644 benchmarks/tpc/queries/tpcds/q71.sql create mode 100644 benchmarks/tpc/queries/tpcds/q72.sql create mode 100644 benchmarks/tpc/queries/tpcds/q73.sql create mode 100644 benchmarks/tpc/queries/tpcds/q74.sql create mode 100644 benchmarks/tpc/queries/tpcds/q75.sql create mode 100644 benchmarks/tpc/queries/tpcds/q76.sql create mode 100644 benchmarks/tpc/queries/tpcds/q77.sql create mode 100644 benchmarks/tpc/queries/tpcds/q78.sql create mode 100644 benchmarks/tpc/queries/tpcds/q79.sql create mode 100644 benchmarks/tpc/queries/tpcds/q8.sql create mode 100644 benchmarks/tpc/queries/tpcds/q80.sql create mode 100644 benchmarks/tpc/queries/tpcds/q81.sql create mode 100644 benchmarks/tpc/queries/tpcds/q82.sql create mode 100644 benchmarks/tpc/queries/tpcds/q83.sql create mode 100644 benchmarks/tpc/queries/tpcds/q84.sql create mode 100644 benchmarks/tpc/queries/tpcds/q85.sql create mode 100644 benchmarks/tpc/queries/tpcds/q86.sql create mode 100644 benchmarks/tpc/queries/tpcds/q87.sql create mode 100644 benchmarks/tpc/queries/tpcds/q88.sql create mode 100644 benchmarks/tpc/queries/tpcds/q89.sql create mode 100644 benchmarks/tpc/queries/tpcds/q9.sql create mode 100644 benchmarks/tpc/queries/tpcds/q90.sql create mode 100644 benchmarks/tpc/queries/tpcds/q91.sql create mode 100644 benchmarks/tpc/queries/tpcds/q92.sql create mode 100644 benchmarks/tpc/queries/tpcds/q93.sql create mode 100644 benchmarks/tpc/queries/tpcds/q94.sql create mode 100644 benchmarks/tpc/queries/tpcds/q95.sql create mode 100644 benchmarks/tpc/queries/tpcds/q96.sql create mode 100644 benchmarks/tpc/queries/tpcds/q97.sql create mode 100644 benchmarks/tpc/queries/tpcds/q98.sql create mode 100644 benchmarks/tpc/queries/tpcds/q99.sql create mode 100644 benchmarks/tpc/queries/tpch/q1.sql create mode 100644 benchmarks/tpc/queries/tpch/q10.sql create mode 100644 benchmarks/tpc/queries/tpch/q11.sql create mode 100644 benchmarks/tpc/queries/tpch/q12.sql create mode 100644 benchmarks/tpc/queries/tpch/q13.sql create mode 100644 benchmarks/tpc/queries/tpch/q14.sql create mode 100644 benchmarks/tpc/queries/tpch/q15.sql create mode 100644 benchmarks/tpc/queries/tpch/q16.sql create mode 100644 benchmarks/tpc/queries/tpch/q17.sql create mode 100644 benchmarks/tpc/queries/tpch/q18.sql create mode 100644 benchmarks/tpc/queries/tpch/q19.sql create mode 100644 benchmarks/tpc/queries/tpch/q2.sql create mode 100644 benchmarks/tpc/queries/tpch/q20.sql create mode 100644 benchmarks/tpc/queries/tpch/q21.sql create mode 100644 benchmarks/tpc/queries/tpch/q22.sql create mode 100644 benchmarks/tpc/queries/tpch/q3.sql create mode 100644 benchmarks/tpc/queries/tpch/q4.sql create mode 100644 benchmarks/tpc/queries/tpch/q5.sql create mode 100644 benchmarks/tpc/queries/tpch/q6.sql create mode 100644 benchmarks/tpc/queries/tpch/q7.sql create mode 100644 benchmarks/tpc/queries/tpch/q8.sql create mode 100644 benchmarks/tpc/queries/tpch/q9.sql create mode 100755 benchmarks/tpc/run.py rename {dev/benchmarks => benchmarks/tpc}/tpcbench.py (87%) delete mode 100644 dev/benchmarks/README.md delete mode 100755 dev/benchmarks/blaze-tpcds.sh delete mode 100755 dev/benchmarks/blaze-tpch.sh delete mode 100755 dev/benchmarks/comet-tpcds.sh delete mode 100755 dev/benchmarks/comet-tpch-iceberg.sh delete mode 100755 dev/benchmarks/comet-tpch.sh delete mode 100644 dev/benchmarks/create-iceberg-tpch.py delete mode 100755 dev/benchmarks/gluten-tpcds.sh delete mode 100755 dev/benchmarks/gluten-tpch.sh delete mode 100755 dev/benchmarks/spark-tpcds.sh delete mode 100755 dev/benchmarks/spark-tpch.sh create mode 100644 native/spark-expr/benches/cast_from_boolean.rs create mode 100644 native/spark-expr/benches/cast_int_to_timestamp.rs delete mode 100644 native/spark-expr/src/array_funcs/array_repeat.rs create mode 100644 native/spark-expr/src/conversion_funcs/boolean.rs create mode 100644 native/spark-expr/src/conversion_funcs/string.rs create mode 100644 native/spark-expr/src/conversion_funcs/utils.rs create mode 100644 spark/src/test/resources/sql-tests/expressions/hash/crc32.sql diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 62534a0503..0b6965d565 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -37,6 +37,22 @@ updates: patterns: - "prost*" - "pbjson*" + # Catch-all: group only minor/patch into a single PR, + # excluding deps we want always separate (and excluding arrow/parquet which have their own group) + all-other-cargo-deps: + applies-to: version-updates + patterns: + - "*" + exclude-patterns: + - "arrow*" + - "parquet" + - "object_store" + - "sqlparser" + - "prost*" + - "pbjson*" + update-types: + - "minor" + - "patch" - package-ecosystem: "github-actions" directory: "/" schedule: diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index a7b4189496..ee62eb9730 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -67,12 +67,12 @@ jobs: build-native: needs: lint name: Build Native Library - runs-on: ubuntu-latest + runs-on: ${{ github.repository_owner == 'apache' && format('runs-on={0},family=m8a,cpu=8,image=ubuntu24-full-x64,extras=s3-cache,disk=large,tag=datafusion', github.run_id) || 'ubuntu-latest' }} container: image: amd64/rust steps: + - uses: runs-on/action@cd2b598b0515d39d78c38a02d529db87d2196d1e # v2.0.3 - uses: actions/checkout@v6 - - name: Setup Rust toolchain uses: ./.github/actions/setup-builder with: @@ -120,10 +120,12 @@ jobs: linux-test-rust: needs: lint name: ubuntu-latest/rust-test - runs-on: ubuntu-latest + runs-on: ${{ github.repository_owner == 'apache' && format('runs-on={0},family=m8a,cpu=16,image=ubuntu24-full-x64,extras=s3-cache,disk=large,tag=datafusion', github.run_id) || 'ubuntu-latest' }} container: image: amd64/rust steps: + - uses: runs-on/action@cd2b598b0515d39d78c38a02d529db87d2196d1e # v2.0.3 + - uses: actions/checkout@v6 - name: Setup Rust & Java toolchain @@ -161,7 +163,6 @@ jobs: needs: build-native strategy: matrix: - os: [ubuntu-latest] # the goal with these profiles is to get coverage of all Java, Scala, and Spark # versions without testing all possible combinations, which would be overkill profile: @@ -273,14 +274,15 @@ jobs: value: | org.apache.spark.sql.CometToPrettyStringSuite fail-fast: false - name: ${{ matrix.os }}/${{ matrix.profile.name }}/${{ matrix.profile.scan_impl }} [${{ matrix.suite.name }}] - runs-on: ${{ matrix.os }} + name: ${{ matrix.profile.name }}/${{ matrix.profile.scan_impl }} [${{ matrix.suite.name }}] + runs-on: ${{ github.repository_owner == 'apache' && format('runs-on={0},family=m8a,cpu=16,image=ubuntu24-full-x64,extras=s3-cache,disk=large,tag=datafusion-comet', github.run_id) || 'ubuntu-latest' }} container: image: amd64/rust env: JAVA_TOOL_OPTIONS: ${{ matrix.profile.java_version == '17' && '--add-exports=java.base/sun.nio.ch=ALL-UNNAMED --add-exports=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED' || '' }} steps: + - uses: runs-on/action@cd2b598b0515d39d78c38a02d529db87d2196d1e # v2.0.3 - uses: actions/checkout@v6 - name: Setup Rust & Java toolchain @@ -310,7 +312,7 @@ jobs: - name: Java test steps uses: ./.github/actions/java-test with: - artifact_name: ${{ matrix.os }}-${{ matrix.profile.name }}-${{ matrix.suite.name }}-${{ github.run_id }}-${{ github.run_number }}-${{ github.run_attempt }} + artifact_name: ${{ matrix.profile.name }}-${{ matrix.suite.name }}-${{ github.run_id }}-${{ github.run_number }}-${{ github.run_attempt }} suites: ${{ matrix.suite.name == 'sql' && matrix.profile.name == 'Spark 3.4, JDK 11, Scala 2.12' && '' || matrix.suite.value }} maven_opts: ${{ matrix.profile.maven_opts }} scan_impl: ${{ matrix.profile.scan_impl }} @@ -321,10 +323,12 @@ jobs: verify-benchmark-results-tpch: needs: build-native name: Verify TPC-H Results - runs-on: ubuntu-latest + runs-on: ${{ github.repository_owner == 'apache' && format('runs-on={0},family=m8a,cpu=16,image=ubuntu24-full-x64,extras=s3-cache,disk=large,tag=datafusion', github.run_id) || 'ubuntu-latest' }} container: image: amd64/rust steps: + - uses: runs-on/action@cd2b598b0515d39d78c38a02d529db87d2196d1e # v2.0.3 + - uses: actions/checkout@v6 - name: Setup Rust & Java toolchain @@ -373,7 +377,7 @@ jobs: verify-benchmark-results-tpcds: needs: build-native name: Verify TPC-DS Results (${{ matrix.join }}) - runs-on: ubuntu-latest + runs-on: ${{ github.repository_owner == 'apache' && format('runs-on={0},family=m8a,cpu=16,image=ubuntu24-full-x64,extras=s3-cache,disk=large,tag=datafusion', github.run_id) || 'ubuntu-latest' }} container: image: amd64/rust strategy: @@ -381,6 +385,8 @@ jobs: join: [sort_merge, broadcast, hash] fail-fast: false steps: + - uses: runs-on/action@cd2b598b0515d39d78c38a02d529db87d2196d1e # v2.0.3 + - uses: actions/checkout@v6 - name: Setup Rust & Java toolchain diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 08b36fa663..cef1d4ac18 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -27,7 +27,7 @@ jobs: issues: write pull-requests: write steps: - - uses: actions/stale@997185467fa4f803885201cee163a9f38240193d # v10.1.1 + - uses: actions/stale@b5d41d4e1d5dceea10e7104786b73624c18a190f # v10.2.0 with: stale-pr-message: "Thank you for your contribution. Unfortunately, this pull request is stale because it has been open 60 days with no activity. Please remove the stale label or comment or this will be closed in 7 days." days-before-pr-stale: 60 diff --git a/.gitignore b/.gitignore index 05b37627bd..15cac247ed 100644 --- a/.gitignore +++ b/.gitignore @@ -17,8 +17,10 @@ filtered_rat.txt dev/dist apache-rat-*.jar venv +.venv dev/release/comet-rm/workdir spark/benchmarks .DS_Store comet-event-trace.json __pycache__ +output diff --git a/dev/benchmarks/.gitignore b/benchmarks/tpc/.gitignore similarity index 100% rename from dev/benchmarks/.gitignore rename to benchmarks/tpc/.gitignore diff --git a/benchmarks/tpc/README.md b/benchmarks/tpc/README.md new file mode 100644 index 0000000000..eb1fb04791 --- /dev/null +++ b/benchmarks/tpc/README.md @@ -0,0 +1,365 @@ + + +# Comet Benchmarking Scripts + +This directory contains scripts used for generating benchmark results that are published in this repository and in +the Comet documentation. + +For full instructions on running these benchmarks on an EC2 instance, see the [Comet Benchmarking on EC2 Guide]. + +[Comet Benchmarking on EC2 Guide]: https://datafusion.apache.org/comet/contributor-guide/benchmarking_aws_ec2.html + +## Setup + +TPC queries are bundled in `benchmarks/tpc/queries/` (derived from TPC-H/DS under the TPC Fair Use Policy). + +## Usage + +All benchmarks are run via `run.py`: + +``` +python3 run.py --engine --benchmark [options] +``` + +| Option | Description | +| -------------- | ------------------------------------------------ | +| `--engine` | Engine name (matches a TOML file in `engines/`) | +| `--benchmark` | `tpch` or `tpcds` | +| `--iterations` | Number of iterations (default: 1) | +| `--output` | Output directory (default: `.`) | +| `--query` | Run a single query number | +| `--no-restart` | Skip Spark master/worker restart | +| `--dry-run` | Print the spark-submit command without executing | + +Available engines: `spark`, `comet`, `comet-iceberg`, `gluten` + +## Example usage + +Set Spark environment variables: + +```shell +export SPARK_HOME=/opt/spark-3.5.3-bin-hadoop3/ +export SPARK_MASTER=spark://yourhostname:7077 +``` + +Set path to data (TPC queries are bundled in `benchmarks/tpc/queries/`): + +```shell +export TPCH_DATA=/mnt/bigdata/tpch/sf100/ +``` + +Run Spark benchmark: + +```shell +export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 +sudo ./drop-caches.sh +python3 run.py --engine spark --benchmark tpch +``` + +Run Comet benchmark: + +```shell +export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 +export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar +sudo ./drop-caches.sh +python3 run.py --engine comet --benchmark tpch +``` + +Run Gluten benchmark: + +```shell +export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 +export GLUTEN_JAR=/opt/gluten/gluten-velox-bundle-spark3.5_2.12-linux_amd64-1.4.0.jar +sudo ./drop-caches.sh +python3 run.py --engine gluten --benchmark tpch +``` + +Preview a command without running it: + +```shell +python3 run.py --engine comet --benchmark tpch --dry-run +``` + +Generating charts: + +```shell +python3 generate-comparison.py --benchmark tpch --labels "Spark 3.5.3" "Comet 0.9.0" "Gluten 1.4.0" --title "TPC-H @ 100 GB (single executor, 8 cores, local Parquet files)" spark-tpch-1752338506381.json comet-tpch-1752337818039.json gluten-tpch-1752337474344.json +``` + +## Engine Configuration + +Each engine is defined by a TOML file in `engines/`. The config specifies JARs, Spark conf overrides, +required environment variables, and optional defaults/exports. See existing files for examples. + +## Iceberg Benchmarking + +Comet includes native Iceberg support via iceberg-rust integration. This enables benchmarking TPC-H queries +against Iceberg tables with native scan acceleration. + +### Prerequisites + +Download the Iceberg Spark runtime JAR (required for running the benchmark): + +```shell +wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar +export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar +``` + +Note: Table creation uses `--packages` which auto-downloads the dependency. + +### Create Iceberg tables + +Convert existing Parquet data to Iceberg format using `create-iceberg-tables.py`. +The script configures the Iceberg catalog automatically -- no `--conf` flags needed. + +```shell +export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse +mkdir -p $ICEBERG_WAREHOUSE + +# TPC-H +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + --conf spark.driver.memory=8G \ + --conf spark.executor.instances=2 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=16 \ + --conf spark.executor.memory=16g \ + create-iceberg-tables.py \ + --benchmark tpch \ + --parquet-path $TPCH_DATA \ + --warehouse $ICEBERG_WAREHOUSE + +# TPC-DS +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + --conf spark.driver.memory=8G \ + --conf spark.executor.instances=2 \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=16 \ + --conf spark.executor.memory=16g \ + create-iceberg-tables.py \ + --benchmark tpcds \ + --parquet-path $TPCDS_DATA \ + --warehouse $ICEBERG_WAREHOUSE +``` + +### Run Iceberg benchmark + +```shell +export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 +export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar +export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar +export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse +sudo ./drop-caches.sh +python3 run.py --engine comet-iceberg --benchmark tpch +``` + +The benchmark uses `spark.comet.scan.icebergNative.enabled=true` to enable Comet's native iceberg-rust +integration. Verify native scanning is active by checking for `CometIcebergNativeScanExec` in the +physical plan output. + +### create-iceberg-tables.py options + +| Option | Required | Default | Description | +| ---------------- | -------- | -------------- | ----------------------------------- | +| `--benchmark` | Yes | | `tpch` or `tpcds` | +| `--parquet-path` | Yes | | Path to source Parquet data | +| `--warehouse` | Yes | | Path to Iceberg warehouse directory | +| `--catalog` | No | `local` | Iceberg catalog name | +| `--database` | No | benchmark name | Database name for the tables | + +## Running with Docker + +A Docker Compose setup is provided in `infra/docker/` for running benchmarks in an isolated +Spark standalone cluster. The Docker image supports both **Linux (amd64)** and **macOS (arm64)** +via architecture-agnostic Java symlinks created at build time. + +### Build the image + +The image must be built for the correct platform to match the native libraries in the +engine JARs (e.g. Comet bundles `libcomet.so` for a specific OS/arch). + +```shell +docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . +``` + +### Building a compatible Comet JAR + +The Comet JAR contains platform-specific native libraries (`libcomet.so` / `libcomet.dylib`). +A JAR built on the host may not work inside the Docker container due to OS, architecture, +or glibc version mismatches. Use `Dockerfile.build-comet` to build a JAR with compatible +native libraries: + +- **macOS (Apple Silicon):** The host JAR contains `darwin/aarch64` libraries which + won't work in Linux containers. You **must** use the build Dockerfile. +- **Linux:** If your host glibc version differs from the container's, the native library + will fail to load with a `GLIBC_x.xx not found` error. The build Dockerfile uses + Ubuntu 20.04 (glibc 2.31) for broad compatibility. Use it if you see + `UnsatisfiedLinkError` mentioning glibc when running benchmarks. + +```shell +mkdir -p output +docker build -t comet-builder \ + -f benchmarks/tpc/infra/docker/Dockerfile.build-comet . +docker run --rm -v $(pwd)/output:/output comet-builder +export COMET_JAR=$(pwd)/output/comet-spark-spark3.5_2.12-*.jar +``` + +### Platform notes + +**macOS (Apple Silicon):** Docker Desktop is required. + +- **Memory:** Docker Desktop defaults to a small memory allocation (often 8 GB) which + is not enough for Spark benchmarks. Go to **Docker Desktop > Settings > Resources > + Memory** and increase it to at least 48 GB (each worker requests 16 GB for its executor + plus overhead, and the driver needs 8 GB). Without enough memory, executors will be + OOM-killed (exit code 137). +- **File Sharing:** You may need to add your data directory (e.g. `/opt`) to + **Docker Desktop > Settings > Resources > File Sharing** before mounting host volumes. + +**Linux (amd64):** Docker uses cgroup memory limits directly without a VM layer. No +special Docker configuration is needed, but you may still need to build the Comet JAR +using `Dockerfile.build-comet` (see above) if your host glibc version doesn't match +the container's. + +The Docker image auto-detects the container architecture (amd64/arm64) and sets up +arch-agnostic Java symlinks. The compose file uses `BENCH_JAVA_HOME` (not `JAVA_HOME`) +to avoid inheriting the host's Java path into the container. + +### Start the cluster + +Set environment variables pointing to your host paths, then start the Spark master and +two workers: + +```shell +export DATA_DIR=/mnt/bigdata/tpch/sf100 +export RESULTS_DIR=/tmp/bench-results +export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar + +mkdir -p $RESULTS_DIR/spark-events +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d +``` + +Set `COMET_JAR`, `GLUTEN_JAR`, or `ICEBERG_JAR` to the host path of the engine JAR you +want to use. Each JAR is mounted individually into the container, so you can easily switch +between versions by changing the path and restarting. + +### Run benchmarks + +Use `docker compose run --rm` to execute benchmarks. The `--rm` flag removes the +container when it exits, preventing port conflicts on subsequent runs. Pass +`--no-restart` since the cluster is already managed by Compose, and `--output /results` +so that output files land in the mounted results directory: + +```shell +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ + run --rm -p 4040:4040 bench \ + python3 /opt/benchmarks/run.py \ + --engine comet --benchmark tpch --output /results --no-restart +``` + +The `-p 4040:4040` flag exposes the Spark Application UI on the host. The following +UIs are available during a benchmark run: + +| UI | URL | +| ----------------- | ---------------------- | +| Spark Master | http://localhost:8080 | +| Worker 1 | http://localhost:8081 | +| Worker 2 | http://localhost:8082 | +| Spark Application | http://localhost:4040 | +| History Server | http://localhost:18080 | + +> **Note:** The Master UI links to the Application UI using the container's internal +> hostname, which is not reachable from the host. Use `http://localhost:4040` directly +> to access the Application UI. + +The Spark Application UI is only available while a benchmark is running. To inspect +completed runs, uncomment the `history-server` service in `docker-compose.yml` and +restart the cluster. The History Server reads event logs from `$RESULTS_DIR/spark-events`. + +For Gluten (requires Java 8), you must restart the **entire cluster** with `JAVA_HOME` +set so that all services (master, workers, and bench) use Java 8: + +```shell +export BENCH_JAVA_HOME=/usr/lib/jvm/java-8-openjdk +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml down +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d + +docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml \ + run --rm bench \ + python3 /opt/benchmarks/run.py \ + --engine gluten --benchmark tpch --output /results --no-restart +``` + +> **Important:** Only passing `-e JAVA_HOME=...` to the `bench` container is not +> sufficient -- the workers also need Java 8 or Gluten will fail at runtime with +> `sun.misc.Unsafe` errors. Unset `BENCH_JAVA_HOME` (or switch it back to Java 17) +> and restart the cluster before running Comet or Spark benchmarks. + +### Memory limits + +Two compose files are provided for different hardware profiles: + +| File | Workers | Total memory | Use case | +| --------------------------- | ------- | ------------ | ------------------------------ | +| `docker-compose.yml` | 2 | ~74 GB | SF100+ on a workstation/server | +| `docker-compose-laptop.yml` | 1 | ~12 GB | SF1–SF10 on a laptop | + +**`docker-compose.yml`** (workstation default): + +| Container | Container limit (`mem_limit`) | Spark JVM allocation | +| -------------- | ----------------------------- | ------------------------- | +| spark-worker-1 | 32 GB | 16 GB executor + overhead | +| spark-worker-2 | 32 GB | 16 GB executor + overhead | +| bench (driver) | 10 GB | 8 GB driver | +| **Total** | **74 GB** | | + +Configure via environment variables: `WORKER_MEM_LIMIT` (default: 32g per worker), +`BENCH_MEM_LIMIT` (default: 10g), `WORKER_MEMORY` (default: 16g, Spark executor memory), +`WORKER_CORES` (default: 8). + +### Running on a laptop with small scale factors + +For local development or testing with small scale factors (e.g. SF1 or SF10), use the +laptop compose file which runs a single worker with reduced memory: + +```shell +docker compose -f benchmarks/tpc/infra/docker/docker-compose-laptop.yml up -d +``` + +This starts one worker (4 GB executor inside an 8 GB container) and a 4 GB bench +container, totaling approximately **12 GB** of memory. + +The benchmark scripts request 2 executor instances and 16 max cores by default +(`run.py`). Spark will simply use whatever resources are available on the single worker, +so no script changes are needed. + +### Comparing Parquet vs Iceberg performance + +Run both benchmarks and compare: + +```shell +python3 generate-comparison.py --benchmark tpch \ + --labels "Comet (Parquet)" "Comet (Iceberg)" \ + --title "TPC-H @ 100 GB: Parquet vs Iceberg" \ + comet-tpch-*.json comet-iceberg-tpch-*.json +``` diff --git a/benchmarks/tpc/create-iceberg-tables.py b/benchmarks/tpc/create-iceberg-tables.py new file mode 100644 index 0000000000..219969bda7 --- /dev/null +++ b/benchmarks/tpc/create-iceberg-tables.py @@ -0,0 +1,171 @@ +# 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. + +""" +Convert TPC-H or TPC-DS Parquet data to Iceberg tables. + +Usage: + spark-submit \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + create-iceberg-tables.py \ + --benchmark tpch \ + --parquet-path /path/to/tpch/parquet \ + --warehouse /path/to/iceberg-warehouse + + spark-submit \ + --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ + create-iceberg-tables.py \ + --benchmark tpcds \ + --parquet-path /path/to/tpcds/parquet \ + --warehouse /path/to/iceberg-warehouse +""" + +import argparse +import os +import sys +from pyspark.sql import SparkSession +import time + +TPCH_TABLES = [ + "customer", + "lineitem", + "nation", + "orders", + "part", + "partsupp", + "region", + "supplier", +] + +TPCDS_TABLES = [ + "call_center", + "catalog_page", + "catalog_returns", + "catalog_sales", + "customer", + "customer_address", + "customer_demographics", + "date_dim", + "time_dim", + "household_demographics", + "income_band", + "inventory", + "item", + "promotion", + "reason", + "ship_mode", + "store", + "store_returns", + "store_sales", + "warehouse", + "web_page", + "web_returns", + "web_sales", + "web_site", +] + +BENCHMARK_TABLES = { + "tpch": TPCH_TABLES, + "tpcds": TPCDS_TABLES, +} + + +def main(benchmark: str, parquet_path: str, warehouse: str, catalog: str, database: str): + table_names = BENCHMARK_TABLES[benchmark] + + # Validate paths before starting Spark + errors = [] + if not os.path.isdir(parquet_path): + errors.append(f"Error: --parquet-path '{parquet_path}' does not exist or is not a directory") + if not os.path.isdir(warehouse): + errors.append(f"Error: --warehouse '{warehouse}' does not exist or is not a directory. " + "Create it with: mkdir -p " + warehouse) + if errors: + for e in errors: + print(e, file=sys.stderr) + sys.exit(1) + + spark = SparkSession.builder \ + .appName(f"Create Iceberg {benchmark.upper()} Tables") \ + .config(f"spark.sql.catalog.{catalog}", "org.apache.iceberg.spark.SparkCatalog") \ + .config(f"spark.sql.catalog.{catalog}.type", "hadoop") \ + .config(f"spark.sql.catalog.{catalog}.warehouse", warehouse) \ + .getOrCreate() + + # Set the Iceberg catalog as the current catalog so that + # namespace operations are routed correctly + spark.sql(f"USE {catalog}") + + # Create namespace if it doesn't exist + try: + spark.sql(f"CREATE NAMESPACE IF NOT EXISTS {database}") + except Exception: + # Namespace may already exist + pass + + for table in table_names: + parquet_table_path = f"{parquet_path}/{table}.parquet" + iceberg_table = f"{catalog}.{database}.{table}" + + print(f"Converting {parquet_table_path} -> {iceberg_table}") + start_time = time.time() + + # Drop table if exists to allow re-running + spark.sql(f"DROP TABLE IF EXISTS {iceberg_table}") + + # Read parquet and write as Iceberg + df = spark.read.parquet(parquet_table_path) + df.writeTo(iceberg_table).using("iceberg").create() + + row_count = spark.table(iceberg_table).count() + elapsed = time.time() - start_time + print(f" Created {iceberg_table} with {row_count} rows in {elapsed:.2f}s") + + print(f"\nAll {benchmark.upper()} tables created successfully!") + print(f"Tables available at: {catalog}.{database}.*") + + spark.stop() + + +if __name__ == "__main__": + parser = argparse.ArgumentParser( + description="Convert TPC-H or TPC-DS Parquet data to Iceberg tables" + ) + parser.add_argument( + "--benchmark", required=True, choices=["tpch", "tpcds"], + help="Benchmark whose tables to convert (tpch or tpcds)" + ) + parser.add_argument( + "--parquet-path", required=True, + help="Path to Parquet data directory" + ) + parser.add_argument( + "--warehouse", required=True, + help="Path to Iceberg warehouse directory" + ) + parser.add_argument( + "--catalog", default="local", + help="Iceberg catalog name (default: 'local')" + ) + parser.add_argument( + "--database", default=None, + help="Database name to create tables in (defaults to benchmark name)" + ) + args = parser.parse_args() + + database = args.database if args.database else args.benchmark + main(args.benchmark, args.parquet_path, args.warehouse, args.catalog, database) diff --git a/dev/benchmarks/drop-caches.sh b/benchmarks/tpc/drop-caches.sh similarity index 100% rename from dev/benchmarks/drop-caches.sh rename to benchmarks/tpc/drop-caches.sh diff --git a/benchmarks/tpc/engines/comet-hashjoin.toml b/benchmarks/tpc/engines/comet-hashjoin.toml new file mode 100644 index 0000000000..1aa4957241 --- /dev/null +++ b/benchmarks/tpc/engines/comet-hashjoin.toml @@ -0,0 +1,35 @@ +# 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. + +[engine] +name = "comet-hashjoin" + +[env] +required = ["COMET_JAR"] + +[spark_submit] +jars = ["$COMET_JAR"] +driver_class_path = ["$COMET_JAR"] + +[spark_conf] +"spark.driver.extraClassPath" = "$COMET_JAR" +"spark.executor.extraClassPath" = "$COMET_JAR" +"spark.plugins" = "org.apache.spark.CometPlugin" +"spark.shuffle.manager" = "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" +"spark.comet.scan.impl" = "native_datafusion" +"spark.comet.exec.replaceSortMergeJoin" = "true" +"spark.comet.expression.Cast.allowIncompatible" = "true" diff --git a/benchmarks/tpc/engines/comet-iceberg-hashjoin.toml b/benchmarks/tpc/engines/comet-iceberg-hashjoin.toml new file mode 100644 index 0000000000..84a5333728 --- /dev/null +++ b/benchmarks/tpc/engines/comet-iceberg-hashjoin.toml @@ -0,0 +1,48 @@ +# 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. + +[engine] +name = "comet-iceberg-hashjoin" + +[env] +required = ["COMET_JAR", "ICEBERG_JAR", "ICEBERG_WAREHOUSE"] + +[env.defaults] +ICEBERG_CATALOG = "local" + +[spark_submit] +jars = ["$COMET_JAR", "$ICEBERG_JAR"] +driver_class_path = ["$COMET_JAR", "$ICEBERG_JAR"] + +[spark_conf] +"spark.driver.extraClassPath" = "$COMET_JAR:$ICEBERG_JAR" +"spark.executor.extraClassPath" = "$COMET_JAR:$ICEBERG_JAR" +"spark.plugins" = "org.apache.spark.CometPlugin" +"spark.shuffle.manager" = "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" +"spark.comet.exec.replaceSortMergeJoin" = "true" +"spark.comet.expression.Cast.allowIncompatible" = "true" +"spark.comet.enabled" = "true" +"spark.comet.exec.enabled" = "true" +"spark.comet.scan.icebergNative.enabled" = "true" +"spark.comet.explainFallback.enabled" = "true" +"spark.sql.catalog.${ICEBERG_CATALOG}" = "org.apache.iceberg.spark.SparkCatalog" +"spark.sql.catalog.${ICEBERG_CATALOG}.type" = "hadoop" +"spark.sql.catalog.${ICEBERG_CATALOG}.warehouse" = "$ICEBERG_WAREHOUSE" +"spark.sql.defaultCatalog" = "${ICEBERG_CATALOG}" + +[tpcbench_args] +use_iceberg = true diff --git a/benchmarks/tpc/engines/comet-iceberg.toml b/benchmarks/tpc/engines/comet-iceberg.toml new file mode 100644 index 0000000000..3654f359e3 --- /dev/null +++ b/benchmarks/tpc/engines/comet-iceberg.toml @@ -0,0 +1,47 @@ +# 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. + +[engine] +name = "comet-iceberg" + +[env] +required = ["COMET_JAR", "ICEBERG_JAR", "ICEBERG_WAREHOUSE"] + +[env.defaults] +ICEBERG_CATALOG = "local" + +[spark_submit] +jars = ["$COMET_JAR", "$ICEBERG_JAR"] +driver_class_path = ["$COMET_JAR", "$ICEBERG_JAR"] + +[spark_conf] +"spark.driver.extraClassPath" = "$COMET_JAR:$ICEBERG_JAR" +"spark.executor.extraClassPath" = "$COMET_JAR:$ICEBERG_JAR" +"spark.plugins" = "org.apache.spark.CometPlugin" +"spark.shuffle.manager" = "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" +"spark.comet.expression.Cast.allowIncompatible" = "true" +"spark.comet.enabled" = "true" +"spark.comet.exec.enabled" = "true" +"spark.comet.scan.icebergNative.enabled" = "true" +"spark.comet.explainFallback.enabled" = "true" +"spark.sql.catalog.${ICEBERG_CATALOG}" = "org.apache.iceberg.spark.SparkCatalog" +"spark.sql.catalog.${ICEBERG_CATALOG}.type" = "hadoop" +"spark.sql.catalog.${ICEBERG_CATALOG}.warehouse" = "$ICEBERG_WAREHOUSE" +"spark.sql.defaultCatalog" = "${ICEBERG_CATALOG}" + +[tpcbench_args] +use_iceberg = true diff --git a/benchmarks/tpc/engines/comet.toml b/benchmarks/tpc/engines/comet.toml new file mode 100644 index 0000000000..05b2cb22ba --- /dev/null +++ b/benchmarks/tpc/engines/comet.toml @@ -0,0 +1,34 @@ +# 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. + +[engine] +name = "comet" + +[env] +required = ["COMET_JAR"] + +[spark_submit] +jars = ["$COMET_JAR"] +driver_class_path = ["$COMET_JAR"] + +[spark_conf] +"spark.driver.extraClassPath" = "$COMET_JAR" +"spark.executor.extraClassPath" = "$COMET_JAR" +"spark.plugins" = "org.apache.spark.CometPlugin" +"spark.shuffle.manager" = "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" +"spark.comet.scan.impl" = "native_datafusion" +"spark.comet.expression.Cast.allowIncompatible" = "true" diff --git a/benchmarks/tpc/engines/gluten.toml b/benchmarks/tpc/engines/gluten.toml new file mode 100644 index 0000000000..20165788c2 --- /dev/null +++ b/benchmarks/tpc/engines/gluten.toml @@ -0,0 +1,34 @@ +# 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. + +[engine] +name = "gluten" + +[env] +required = ["GLUTEN_JAR"] +exports = { TZ = "UTC" } + +[spark_submit] +jars = ["$GLUTEN_JAR"] + +[spark_conf] +"spark.plugins" = "org.apache.gluten.GlutenPlugin" +"spark.driver.extraClassPath" = "${GLUTEN_JAR}" +"spark.executor.extraClassPath" = "${GLUTEN_JAR}" +"spark.gluten.sql.columnar.forceShuffledHashJoin" = "true" +"spark.shuffle.manager" = "org.apache.spark.shuffle.sort.ColumnarShuffleManager" +"spark.sql.session.timeZone" = "UTC" diff --git a/benchmarks/tpc/engines/spark.toml b/benchmarks/tpc/engines/spark.toml new file mode 100644 index 0000000000..c02e7a6ad2 --- /dev/null +++ b/benchmarks/tpc/engines/spark.toml @@ -0,0 +1,19 @@ +# 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. + +[engine] +name = "spark" diff --git a/dev/benchmarks/generate-comparison.py b/benchmarks/tpc/generate-comparison.py similarity index 61% rename from dev/benchmarks/generate-comparison.py rename to benchmarks/tpc/generate-comparison.py index eb57cc1e45..e5058a3bfa 100644 --- a/dev/benchmarks/generate-comparison.py +++ b/benchmarks/tpc/generate-comparison.py @@ -17,19 +17,88 @@ import argparse import json +import logging import matplotlib.pyplot as plt import numpy as np +logging.basicConfig(level=logging.INFO) +logger = logging.getLogger(__name__) + def geomean(data): return np.prod(data) ** (1 / len(data)) -def generate_query_rel_speedup_chart(baseline, comparison, label1: str, label2: str, benchmark: str, title: str): +def get_durations(result, query_key): + """Extract durations from a query result, supporting both old (list) and new (dict) formats.""" + value = result[query_key] + if isinstance(value, dict): + return value["durations"] + return value + +def get_all_queries(results): + """Return the sorted union of all query keys across all result sets.""" + all_keys = set() + for result in results: + all_keys.update(result.keys()) + # Filter to numeric query keys and sort numerically + numeric_keys = [] + for k in all_keys: + try: + numeric_keys.append(int(k)) + except ValueError: + pass + return sorted(numeric_keys) + +def get_common_queries(results, labels): + """Return queries present in ALL result sets, warning about queries missing from some files.""" + all_queries = get_all_queries(results) + common = [] + for query in all_queries: + key = str(query) + present = [labels[i] for i, r in enumerate(results) if key in r] + missing = [labels[i] for i, r in enumerate(results) if key not in r] + if missing: + logger.warning(f"Query {query}: present in [{', '.join(present)}] but missing from [{', '.join(missing)}]") + if not missing: + common.append(query) + return common + +def check_result_consistency(results, labels, benchmark): + """Log warnings if row counts or result hashes differ across result sets.""" + all_queries = get_all_queries(results) + for query in all_queries: + key = str(query) + row_counts = [] + hashes = [] + for i, result in enumerate(results): + if key not in result: + continue + value = result[key] + if not isinstance(value, dict): + continue + if "row_count" in value: + row_counts.append((labels[i], value["row_count"])) + if "result_hash" in value: + hashes.append((labels[i], value["result_hash"])) + + if len(row_counts) > 1: + counts = set(rc for _, rc in row_counts) + if len(counts) > 1: + details = ", ".join(f"{label}={rc}" for label, rc in row_counts) + logger.warning(f"Query {query}: row count mismatch: {details}") + + if len(hashes) > 1: + hash_values = set(h for _, h in hashes) + if len(hash_values) > 1: + details = ", ".join(f"{label}={h}" for label, h in hashes) + logger.warning(f"Query {query}: result hash mismatch: {details}") + +def generate_query_rel_speedup_chart(baseline, comparison, label1: str, label2: str, benchmark: str, title: str, common_queries=None): + if common_queries is None: + common_queries = range(1, query_count(benchmark)+1) results = [] - for query in range(1, query_count(benchmark)+1): - if query == 999: - continue - a = np.median(np.array(baseline[str(query)])) - b = np.median(np.array(comparison[str(query)])) + for query in common_queries: + a = np.median(np.array(get_durations(baseline, str(query)))) + b = np.median(np.array(get_durations(comparison, str(query)))) if a > b: speedup = a/b-1 else: @@ -80,13 +149,13 @@ def generate_query_rel_speedup_chart(baseline, comparison, label1: str, label2: # Save the plot as an image file plt.savefig(f'{benchmark}_queries_speedup_rel.png', format='png') -def generate_query_abs_speedup_chart(baseline, comparison, label1: str, label2: str, benchmark: str, title: str): +def generate_query_abs_speedup_chart(baseline, comparison, label1: str, label2: str, benchmark: str, title: str, common_queries=None): + if common_queries is None: + common_queries = range(1, query_count(benchmark)+1) results = [] - for query in range(1, query_count(benchmark)+1): - if query == 999: - continue - a = np.median(np.array(baseline[str(query)])) - b = np.median(np.array(comparison[str(query)])) + for query in common_queries: + a = np.median(np.array(get_durations(baseline, str(query)))) + b = np.median(np.array(get_durations(comparison, str(query)))) speedup = a-b results.append(("q" + str(query), round(speedup, 1))) @@ -130,17 +199,17 @@ def generate_query_abs_speedup_chart(baseline, comparison, label1: str, label2: # Save the plot as an image file plt.savefig(f'{benchmark}_queries_speedup_abs.png', format='png') -def generate_query_comparison_chart(results, labels, benchmark: str, title: str): +def generate_query_comparison_chart(results, labels, benchmark: str, title: str, common_queries=None): + if common_queries is None: + common_queries = range(1, query_count(benchmark)+1) queries = [] benches = [] for _ in results: benches.append([]) - for query in range(1, query_count(benchmark)+1): - if query == 999: - continue + for query in common_queries: queries.append("q" + str(query)) for i in range(0, len(results)): - benches[i].append(np.median(np.array(results[i][str(query)]))) + benches[i].append(np.median(np.array(get_durations(results[i], str(query))))) # Define the width of the bars bar_width = 0.3 @@ -168,17 +237,17 @@ def generate_query_comparison_chart(results, labels, benchmark: str, title: str) # Save the plot as an image file plt.savefig(f'{benchmark}_queries_compare.png', format='png') -def generate_summary(results, labels, benchmark: str, title: str): +def generate_summary(results, labels, benchmark: str, title: str, common_queries=None): + if common_queries is None: + common_queries = range(1, query_count(benchmark)+1) timings = [] for _ in results: timings.append(0) - num_queries = query_count(benchmark) - for query in range(1, num_queries + 1): - if query == 999: - continue + num_queries = len([q for q in common_queries]) + for query in common_queries: for i in range(0, len(results)): - timings[i] += np.median(np.array(results[i][str(query)])) + timings[i] += np.median(np.array(get_durations(results[i], str(query)))) # Create figure and axis fig, ax = plt.subplots() @@ -186,7 +255,7 @@ def generate_summary(results, labels, benchmark: str, title: str): # Add title and labels ax.set_title(title) - ax.set_ylabel(f'Time in seconds to run all {num_queries} {benchmark} queries (lower is better)') + ax.set_ylabel(f'Time in seconds to run {num_queries} {benchmark} queries (lower is better)') times = [round(x,0) for x in timings] @@ -213,11 +282,16 @@ def main(files, labels, benchmark: str, title: str): for filename in files: with open(filename) as f: results.append(json.load(f)) - generate_summary(results, labels, benchmark, title) - generate_query_comparison_chart(results, labels, benchmark, title) + check_result_consistency(results, labels, benchmark) + common_queries = get_common_queries(results, labels) + if not common_queries: + logger.error("No queries found in common across all result files") + return + generate_summary(results, labels, benchmark, title, common_queries) + generate_query_comparison_chart(results, labels, benchmark, title, common_queries) if len(files) == 2: - generate_query_abs_speedup_chart(results[0], results[1], labels[0], labels[1], benchmark, title) - generate_query_rel_speedup_chart(results[0], results[1], labels[0], labels[1], benchmark, title) + generate_query_abs_speedup_chart(results[0], results[1], labels[0], labels[1], benchmark, title, common_queries) + generate_query_rel_speedup_chart(results[0], results[1], labels[0], labels[1], benchmark, title, common_queries) if __name__ == '__main__': argparse = argparse.ArgumentParser(description='Generate comparison') diff --git a/benchmarks/tpc/infra/docker/Dockerfile b/benchmarks/tpc/infra/docker/Dockerfile new file mode 100644 index 0000000000..60567536ad --- /dev/null +++ b/benchmarks/tpc/infra/docker/Dockerfile @@ -0,0 +1,58 @@ +# 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. + +# Benchmark image for running TPC-H and TPC-DS benchmarks across engines +# (Spark, Comet, Gluten). +# +# Build (from repository root): +# docker build -t comet-bench -f benchmarks/tpc/infra/docker/Dockerfile . + +ARG SPARK_IMAGE=apache/spark:3.5.2-python3 +FROM ${SPARK_IMAGE} + +USER root + +# Install Java 8 (Gluten) and Java 17 (Comet) plus Python 3. +RUN apt-get update \ + && apt-get install -y --no-install-recommends \ + openjdk-8-jdk-headless \ + openjdk-17-jdk-headless \ + python3 python3-pip procps \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + +# Default to Java 17 (override with JAVA_HOME at runtime for Gluten). +# Detect architecture (amd64 or arm64) so the image works on both Linux and macOS. +ARG TARGETARCH +RUN ln -s /usr/lib/jvm/java-17-openjdk-${TARGETARCH} /usr/lib/jvm/java-17-openjdk && \ + ln -s /usr/lib/jvm/java-8-openjdk-${TARGETARCH} /usr/lib/jvm/java-8-openjdk +ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk + +# Copy the benchmark scripts into the image. +COPY benchmarks/tpc/run.py /opt/benchmarks/run.py +COPY benchmarks/tpc/tpcbench.py /opt/benchmarks/tpcbench.py +COPY benchmarks/tpc/engines /opt/benchmarks/engines +COPY benchmarks/tpc/queries /opt/benchmarks/queries +COPY benchmarks/tpc/create-iceberg-tables.py /opt/benchmarks/create-iceberg-tables.py +COPY benchmarks/tpc/generate-comparison.py /opt/benchmarks/generate-comparison.py + +# Engine JARs are bind-mounted or copied in at runtime via --jars. +# Data and query paths are also bind-mounted. + +WORKDIR /opt/benchmarks + +# Defined in the base apache/spark image. +ARG spark_uid +USER ${spark_uid} diff --git a/benchmarks/tpc/infra/docker/Dockerfile.build-comet b/benchmarks/tpc/infra/docker/Dockerfile.build-comet new file mode 100644 index 0000000000..af5a0257a0 --- /dev/null +++ b/benchmarks/tpc/infra/docker/Dockerfile.build-comet @@ -0,0 +1,76 @@ +# 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. + +# Build a Comet JAR with native libraries for the current platform. +# +# This is useful on macOS (Apple Silicon) where the host-built JAR contains +# darwin/aarch64 native libraries but Docker containers need linux/aarch64. +# +# Usage (from repository root): +# docker build -t comet-builder -f benchmarks/tpc/infra/docker/Dockerfile.build-comet . +# docker run --rm -v $(pwd)/output:/output comet-builder +# +# The JAR is copied to ./output/ on the host. + +# Use Ubuntu 20.04 to match the GLIBC version (2.31) in apache/spark images. +FROM ubuntu:20.04 AS builder + +ARG TARGETARCH +ENV DEBIAN_FRONTEND=noninteractive + +# Install build dependencies: Java 17, Maven wrapper prerequisites, GCC 11. +# Ubuntu 20.04's default GCC 9 has a memcmp bug (GCC #95189) that breaks aws-lc-sys. +RUN apt-get update && apt-get install -y --no-install-recommends \ + openjdk-17-jdk-headless \ + curl ca-certificates git pkg-config \ + libssl-dev unzip software-properties-common \ + && add-apt-repository -y ppa:ubuntu-toolchain-r/test \ + && apt-get update \ + && apt-get install -y --no-install-recommends gcc-11 g++-11 make \ + && update-alternatives --install /usr/bin/gcc gcc /usr/bin/gcc-11 110 \ + && update-alternatives --install /usr/bin/g++ g++ /usr/bin/g++-11 110 \ + && update-alternatives --install /usr/bin/cc cc /usr/bin/gcc-11 110 \ + && apt-get clean && rm -rf /var/lib/apt/lists/* + +# Install protoc 25.x (Ubuntu 22.04's protoc is too old for proto3 optional fields). +ARG PROTOC_VERSION=25.6 +RUN ARCH=$(uname -m) && \ + if [ "$ARCH" = "aarch64" ]; then PROTOC_ARCH="linux-aarch_64"; \ + else PROTOC_ARCH="linux-x86_64"; fi && \ + curl -sLO "https://github.com/protocolbuffers/protobuf/releases/download/v${PROTOC_VERSION}/protoc-${PROTOC_VERSION}-${PROTOC_ARCH}.zip" && \ + unzip -o "protoc-${PROTOC_VERSION}-${PROTOC_ARCH}.zip" -d /usr/local bin/protoc && \ + rm "protoc-${PROTOC_VERSION}-${PROTOC_ARCH}.zip" && \ + protoc --version + +# Set JAVA_HOME and LD_LIBRARY_PATH so the Rust build can find libjvm. +RUN ln -s /usr/lib/jvm/java-17-openjdk-${TARGETARCH} /usr/lib/jvm/java-17-openjdk +ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk +ENV LD_LIBRARY_PATH=${JAVA_HOME}/lib/server + +# Install Rust. +RUN curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y +ENV PATH="/root/.cargo/bin:${PATH}" + +WORKDIR /build + +# Copy the full source tree. +COPY . . + +# Build native code + package the JAR (skip tests). +RUN make release-nogit + +# The entrypoint copies the built JAR to /output (bind-mounted from host). +RUN mkdir -p /output +CMD ["sh", "-c", "cp spark/target/comet-spark-spark3.5_2.12-*-SNAPSHOT.jar /output/ && echo 'Comet JAR copied to /output/' && ls -lh /output/*.jar"] diff --git a/benchmarks/tpc/infra/docker/docker-compose-laptop.yml b/benchmarks/tpc/infra/docker/docker-compose-laptop.yml new file mode 100644 index 0000000000..bc882ae7b9 --- /dev/null +++ b/benchmarks/tpc/infra/docker/docker-compose-laptop.yml @@ -0,0 +1,98 @@ +# 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. + +# Lightweight Spark standalone cluster for TPC benchmarks on a laptop. +# +# Single worker, ~12 GB total memory. Suitable for SF1-SF10 testing. +# +# Usage: +# export COMET_JAR=/path/to/comet-spark-0.10.0.jar +# docker compose -f benchmarks/tpc/infra/docker/docker-compose-laptop.yml up -d +# +# Environment variables (set in .env or export before running): +# BENCH_IMAGE - Docker image to use (default: comet-bench) +# DATA_DIR - Host path to TPC data (default: /tmp/tpc-data) +# RESULTS_DIR - Host path for results output (default: /tmp/bench-results) +# COMET_JAR - Host path to Comet JAR +# GLUTEN_JAR - Host path to Gluten JAR +# ICEBERG_JAR - Host path to Iceberg Spark runtime JAR +# BENCH_JAVA_HOME - Java home inside container (default: /usr/lib/jvm/java-17-openjdk) +# Set to /usr/lib/jvm/java-8-openjdk for Gluten + +x-volumes: &volumes + - ${DATA_DIR:-/tmp/tpc-data}:/data:ro + - ${RESULTS_DIR:-/tmp/bench-results}:/results + - ${COMET_JAR:-/dev/null}:/jars/comet.jar:ro + - ${GLUTEN_JAR:-/dev/null}:/jars/gluten.jar:ro + - ${ICEBERG_JAR:-/dev/null}:/jars/iceberg.jar:ro + - ${RESULTS_DIR:-/tmp/bench-results}/logs:/opt/spark/logs + - ${RESULTS_DIR:-/tmp/bench-results}/work:/opt/spark/work + +services: + spark-master: + image: ${BENCH_IMAGE:-comet-bench} + container_name: spark-master + hostname: spark-master + command: /opt/spark/sbin/start-master.sh --host spark-master + ports: + - "7077:7077" + - "8080:8080" + volumes: *volumes + environment: + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} + - SPARK_MASTER_HOST=spark-master + - SPARK_NO_DAEMONIZE=true + + spark-worker-1: + image: ${BENCH_IMAGE:-comet-bench} + container_name: spark-worker-1 + hostname: spark-worker-1 + depends_on: + - spark-master + command: /opt/spark/sbin/start-worker.sh spark://spark-master:7077 + ports: + - "8081:8081" + volumes: *volumes + environment: + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} + - SPARK_WORKER_CORES=4 + - SPARK_WORKER_MEMORY=4g + - SPARK_NO_DAEMONIZE=true + mem_limit: 8g + memswap_limit: 8g + + bench: + image: ${BENCH_IMAGE:-comet-bench} + container_name: bench-runner + depends_on: + - spark-master + - spark-worker-1 + # Override 'command' to run a specific benchmark, e.g.: + # docker compose run bench python3 /opt/benchmarks/run.py \ + # --engine comet --benchmark tpch --no-restart + command: ["echo", "Use 'docker compose run bench python3 /opt/benchmarks/run.py ...' to run benchmarks"] + volumes: *volumes + environment: + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} + - SPARK_HOME=/opt/spark + - SPARK_MASTER=spark://spark-master:7077 + - COMET_JAR=/jars/comet.jar + - GLUTEN_JAR=/jars/gluten.jar + - ICEBERG_JAR=/jars/iceberg.jar + - TPCH_DATA=/data + - TPCDS_DATA=/data + - SPARK_EVENT_LOG_DIR=/results/spark-events + mem_limit: 4g + memswap_limit: 4g diff --git a/benchmarks/tpc/infra/docker/docker-compose.yml b/benchmarks/tpc/infra/docker/docker-compose.yml new file mode 100644 index 0000000000..5a76a5d6ec --- /dev/null +++ b/benchmarks/tpc/infra/docker/docker-compose.yml @@ -0,0 +1,132 @@ +# 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. + +# Spark standalone cluster for TPC benchmarks. +# +# Two workers are used so that shuffles go through the network stack, +# which better reflects real cluster behavior. +# +# Usage: +# export COMET_JAR=/path/to/comet-spark-0.10.0.jar +# docker compose -f benchmarks/tpc/infra/docker/docker-compose.yml up -d +# +# Environment variables (set in .env or export before running): +# BENCH_IMAGE - Docker image to use (default: comet-bench) +# DATA_DIR - Host path to TPC data (default: /tmp/tpc-data) +# RESULTS_DIR - Host path for results output (default: /tmp/bench-results) +# COMET_JAR - Host path to Comet JAR +# GLUTEN_JAR - Host path to Gluten JAR +# ICEBERG_JAR - Host path to Iceberg Spark runtime JAR +# WORKER_MEM_LIMIT - Hard memory limit per worker container (default: 32g) +# BENCH_MEM_LIMIT - Hard memory limit for the bench runner (default: 10g) +# BENCH_JAVA_HOME - Java home inside container (default: /usr/lib/jvm/java-17-openjdk) +# Set to /usr/lib/jvm/java-8-openjdk for Gluten + +x-volumes: &volumes + - ${DATA_DIR:-/tmp/tpc-data}:/data:ro + - ${RESULTS_DIR:-/tmp/bench-results}:/results + - ${COMET_JAR:-/dev/null}:/jars/comet.jar:ro + - ${GLUTEN_JAR:-/dev/null}:/jars/gluten.jar:ro + - ${ICEBERG_JAR:-/dev/null}:/jars/iceberg.jar:ro + - ${RESULTS_DIR:-/tmp/bench-results}/logs:/opt/spark/logs + - ${RESULTS_DIR:-/tmp/bench-results}/work:/opt/spark/work + +x-worker: &worker + image: ${BENCH_IMAGE:-comet-bench} + depends_on: + - spark-master + command: /opt/spark/sbin/start-worker.sh spark://spark-master:7077 + volumes: *volumes + environment: + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} + - SPARK_WORKER_CORES=${WORKER_CORES:-8} + - SPARK_WORKER_MEMORY=${WORKER_MEMORY:-16g} + - SPARK_NO_DAEMONIZE=true + mem_limit: ${WORKER_MEM_LIMIT:-32g} + memswap_limit: ${WORKER_MEM_LIMIT:-32g} + +services: + spark-master: + image: ${BENCH_IMAGE:-comet-bench} + container_name: spark-master + hostname: spark-master + command: /opt/spark/sbin/start-master.sh --host spark-master + ports: + - "7077:7077" + - "8080:8080" + volumes: *volumes + environment: + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} + - SPARK_MASTER_HOST=spark-master + - SPARK_NO_DAEMONIZE=true + + spark-worker-1: + <<: *worker + container_name: spark-worker-1 + hostname: spark-worker-1 + ports: + - "8081:8081" + + spark-worker-2: + <<: *worker + container_name: spark-worker-2 + hostname: spark-worker-2 + ports: + - "8082:8081" + + bench: + image: ${BENCH_IMAGE:-comet-bench} + container_name: bench-runner + depends_on: + - spark-master + - spark-worker-1 + - spark-worker-2 + # Override 'command' to run a specific benchmark, e.g.: + # docker compose run bench python3 /opt/benchmarks/run.py \ + # --engine comet --benchmark tpch --no-restart + command: ["echo", "Use 'docker compose run bench python3 /opt/benchmarks/run.py ...' to run benchmarks"] + volumes: *volumes + environment: + - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} + - SPARK_HOME=/opt/spark + - SPARK_MASTER=spark://spark-master:7077 + - COMET_JAR=/jars/comet.jar + - GLUTEN_JAR=/jars/gluten.jar + - ICEBERG_JAR=/jars/iceberg.jar + - TPCH_DATA=/data + - TPCDS_DATA=/data + - SPARK_EVENT_LOG_DIR=/results/spark-events + mem_limit: ${BENCH_MEM_LIMIT:-10g} + memswap_limit: ${BENCH_MEM_LIMIT:-10g} + + # Uncomment to enable the Spark History Server for inspecting completed + # benchmark runs at http://localhost:18080. Requires event logs in + # $RESULTS_DIR/spark-events (created by `mkdir -p $RESULTS_DIR/spark-events` + # before starting the cluster). + # + # history-server: + # image: ${BENCH_IMAGE:-comet-bench} + # container_name: spark-history + # hostname: spark-history + # command: /opt/spark/sbin/start-history-server.sh + # ports: + # - "18080:18080" + # volumes: + # - ${RESULTS_DIR:-/tmp/bench-results}:/results:ro + # environment: + # - JAVA_HOME=${BENCH_JAVA_HOME:-/usr/lib/jvm/java-17-openjdk} + # - SPARK_HISTORY_OPTS=-Dspark.history.fs.logDirectory=/results/spark-events + # - SPARK_NO_DAEMONIZE=true + diff --git a/benchmarks/tpc/queries/tpcds/q1.sql b/benchmarks/tpc/queries/tpcds/q1.sql new file mode 100644 index 0000000000..31bd9eb045 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q1.sql @@ -0,0 +1,26 @@ +-- CometBench-DS query 1 derived from TPC-DS query 1 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with customer_total_return as +(select sr_customer_sk as ctr_customer_sk +,sr_store_sk as ctr_store_sk +,sum(SR_RETURN_AMT_INC_TAX) as ctr_total_return +from store_returns +,date_dim +where sr_returned_date_sk = d_date_sk +and d_year =1999 +group by sr_customer_sk +,sr_store_sk) + select c_customer_id +from customer_total_return ctr1 +,store +,customer +where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 +from customer_total_return ctr2 +where ctr1.ctr_store_sk = ctr2.ctr_store_sk) +and s_store_sk = ctr1.ctr_store_sk +and s_state = 'TN' +and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q10.sql b/benchmarks/tpc/queries/tpcds/q10.sql new file mode 100644 index 0000000000..9ddfefb25d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q10.sql @@ -0,0 +1,60 @@ +-- CometBench-DS query 10 derived from TPC-DS query 10 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_county in ('Clinton County','Platte County','Franklin County','Louisa County','Harmon County') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 3 and 3+3) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 3 ANd 3+3) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 3 and 3+3)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q11.sql b/benchmarks/tpc/queries/tpcds/q11.sql new file mode 100644 index 0000000000..abc1e14ae4 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q11.sql @@ -0,0 +1,82 @@ +-- CometBench-DS query 11 derived from TPC-DS query 11 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ss_ext_list_price-ss_ext_discount_amt) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ws_ext_list_price-ws_ext_discount_amt) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 1999 + and t_s_secyear.dyear = 1999+1 + and t_w_firstyear.dyear = 1999 + and t_w_secyear.dyear = 1999+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else 0.0 end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else 0.0 end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q12.sql b/benchmarks/tpc/queries/tpcds/q12.sql new file mode 100644 index 0000000000..e2e46d92d8 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q12.sql @@ -0,0 +1,35 @@ +-- CometBench-DS query 12 derived from TPC-DS query 12 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ws_ext_sales_price) as itemrevenue + ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over + (partition by i_class) as revenueratio +from + web_sales + ,item + ,date_dim +where + ws_item_sk = i_item_sk + and i_category in ('Jewelry', 'Books', 'Women') + and ws_sold_date_sk = d_date_sk + and d_date between cast('2002-03-22' as date) + and (cast('2002-03-22' as date) + INTERVAL '30 DAYS') +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q13.sql b/benchmarks/tpc/queries/tpcds/q13.sql new file mode 100644 index 0000000000..9cf55cc15d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q13.sql @@ -0,0 +1,53 @@ +-- CometBench-DS query 13 derived from TPC-DS query 13 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select avg(ss_quantity) + ,avg(ss_ext_sales_price) + ,avg(ss_ext_wholesale_cost) + ,sum(ss_ext_wholesale_cost) + from store_sales + ,store + ,customer_demographics + ,household_demographics + ,customer_address + ,date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and((ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'U' + and cd_education_status = '4 yr Degree' + and ss_sales_price between 100.00 and 150.00 + and hd_dep_count = 3 + )or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'S' + and cd_education_status = 'Unknown' + and ss_sales_price between 50.00 and 100.00 + and hd_dep_count = 1 + ) or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'D' + and cd_education_status = '2 yr Degree' + and ss_sales_price between 150.00 and 200.00 + and hd_dep_count = 1 + )) + and((ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('CO', 'MI', 'MN') + and ss_net_profit between 100 and 200 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('NC', 'NY', 'TX') + and ss_net_profit between 150 and 300 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('CA', 'NE', 'TN') + and ss_net_profit between 50 and 250 + )) +; + diff --git a/benchmarks/tpc/queries/tpcds/q14.sql b/benchmarks/tpc/queries/tpcds/q14.sql new file mode 100644 index 0000000000..f242c1eaa6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q14.sql @@ -0,0 +1,211 @@ +-- CometBench-DS query 14 derived from TPC-DS query 14 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1999 AND 1999 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1999 AND 1999 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1999 AND 1999 + 2) + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as + (select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2) x) + select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales) + from( + select 'store' channel, i_brand_id,i_class_id + ,i_category_id,sum(ss_quantity*ss_list_price) sales + , count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales) + union all + select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales + from catalog_sales + ,item + ,date_dim + where cs_item_sk in (select ss_item_sk from cross_items) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales) + union all + select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales + from web_sales + ,item + ,date_dim + where ws_item_sk in (select ss_item_sk from cross_items) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1999+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales) + ) y + group by rollup (channel, i_brand_id,i_class_id,i_category_id) + order by channel,i_brand_id,i_class_id,i_category_id + LIMIT 100; +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1999 AND 1999 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1999 AND 1999 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1999 AND 1999 + 2) x + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as +(select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1999 and 1999 + 2) x) + select this_year.channel ty_channel + ,this_year.i_brand_id ty_brand + ,this_year.i_class_id ty_class + ,this_year.i_category_id ty_category + ,this_year.sales ty_sales + ,this_year.number_sales ty_number_sales + ,last_year.channel ly_channel + ,last_year.i_brand_id ly_brand + ,last_year.i_class_id ly_class + ,last_year.i_category_id ly_category + ,last_year.sales ly_sales + ,last_year.number_sales ly_number_sales + from + (select 'store' channel, i_brand_id,i_class_id,i_category_id + ,sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1999 + 1 + and d_moy = 12 + and d_dom = 14) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) this_year, + (select 'store' channel, i_brand_id,i_class_id + ,i_category_id, sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1999 + and d_moy = 12 + and d_dom = 14) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) last_year + where this_year.i_brand_id= last_year.i_brand_id + and this_year.i_class_id = last_year.i_class_id + and this_year.i_category_id = last_year.i_category_id + order by this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q15.sql b/benchmarks/tpc/queries/tpcds/q15.sql new file mode 100644 index 0000000000..37f0d2791f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q15.sql @@ -0,0 +1,21 @@ +-- CometBench-DS query 15 derived from TPC-DS query 15 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select ca_zip + ,sum(cs_sales_price) + from catalog_sales + ,customer + ,customer_address + ,date_dim + where cs_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', + '85392', '85460', '80348', '81792') + or ca_state in ('CA','WA','GA') + or cs_sales_price > 500) + and cs_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2002 + group by ca_zip + order by ca_zip + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q16.sql b/benchmarks/tpc/queries/tpcds/q16.sql new file mode 100644 index 0000000000..5a4cb1cecd --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q16.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 16 derived from TPC-DS query 16 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + count(distinct cs_order_number) as `order count` + ,sum(cs_ext_ship_cost) as `total shipping cost` + ,sum(cs_net_profit) as `total net profit` +from + catalog_sales cs1 + ,date_dim + ,customer_address + ,call_center +where + d_date between '1999-5-01' and + (cast('1999-5-01' as date) + INTERVAL '60 DAYS') +and cs1.cs_ship_date_sk = d_date_sk +and cs1.cs_ship_addr_sk = ca_address_sk +and ca_state = 'ID' +and cs1.cs_call_center_sk = cc_call_center_sk +and cc_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County' +) +and exists (select * + from catalog_sales cs2 + where cs1.cs_order_number = cs2.cs_order_number + and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) +and not exists(select * + from catalog_returns cr1 + where cs1.cs_order_number = cr1.cr_order_number) +order by count(distinct cs_order_number) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q17.sql b/benchmarks/tpc/queries/tpcds/q17.sql new file mode 100644 index 0000000000..37602e2a8a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q17.sql @@ -0,0 +1,46 @@ +-- CometBench-DS query 17 derived from TPC-DS query 17 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id + ,i_item_desc + ,s_state + ,count(ss_quantity) as store_sales_quantitycount + ,avg(ss_quantity) as store_sales_quantityave + ,stddev_samp(ss_quantity) as store_sales_quantitystdev + ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov + ,count(sr_return_quantity) as store_returns_quantitycount + ,avg(sr_return_quantity) as store_returns_quantityave + ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev + ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov + ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave + ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev + ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov + from store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where d1.d_quarter_name = '1999Q1' + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_quarter_name in ('1999Q1','1999Q2','1999Q3') + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_quarter_name in ('1999Q1','1999Q2','1999Q3') + group by i_item_id + ,i_item_desc + ,s_state + order by i_item_id + ,i_item_desc + ,s_state + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q18.sql b/benchmarks/tpc/queries/tpcds/q18.sql new file mode 100644 index 0000000000..fa89d2db86 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q18.sql @@ -0,0 +1,35 @@ +-- CometBench-DS query 18 derived from TPC-DS query 18 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id, + ca_country, + ca_state, + ca_county, + avg( cast(cs_quantity as decimal(12,2))) agg1, + avg( cast(cs_list_price as decimal(12,2))) agg2, + avg( cast(cs_coupon_amt as decimal(12,2))) agg3, + avg( cast(cs_sales_price as decimal(12,2))) agg4, + avg( cast(cs_net_profit as decimal(12,2))) agg5, + avg( cast(c_birth_year as decimal(12,2))) agg6, + avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 + from catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd1.cd_demo_sk and + cs_bill_customer_sk = c_customer_sk and + cd1.cd_gender = 'M' and + cd1.cd_education_status = 'Primary' and + c_current_cdemo_sk = cd2.cd_demo_sk and + c_current_addr_sk = ca_address_sk and + c_birth_month in (1,2,9,5,11,3) and + d_year = 1998 and + ca_state in ('MS','NE','IA' + ,'MI','GA','NY','CO') + group by rollup (i_item_id, ca_country, ca_state, ca_county) + order by ca_country, + ca_state, + ca_county, + i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q19.sql b/benchmarks/tpc/queries/tpcds/q19.sql new file mode 100644 index 0000000000..54cc69519f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q19.sql @@ -0,0 +1,26 @@ +-- CometBench-DS query 19 derived from TPC-DS query 19 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item,customer,customer_address,store + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=8 + and d_moy=11 + and d_year=1999 + and ss_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and substr(ca_zip,1,5) <> substr(s_zip,1,5) + and ss_store_sk = s_store_sk + group by i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact + order by ext_price desc + ,i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact + LIMIT 100 ; + diff --git a/benchmarks/tpc/queries/tpcds/q2.sql b/benchmarks/tpc/queries/tpcds/q2.sql new file mode 100644 index 0000000000..77b0dccf8c --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q2.sql @@ -0,0 +1,61 @@ +-- CometBench-DS query 2 derived from TPC-DS query 2 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with wscs as + (select sold_date_sk + ,sales_price + from (select ws_sold_date_sk sold_date_sk + ,ws_ext_sales_price sales_price + from web_sales + union all + select cs_sold_date_sk sold_date_sk + ,cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + (select d_week_seq, + sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales + from wscs + ,date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) + select d_week_seq1 + ,round(sun_sales1/sun_sales2,2) + ,round(mon_sales1/mon_sales2,2) + ,round(tue_sales1/tue_sales2,2) + ,round(wed_sales1/wed_sales2,2) + ,round(thu_sales1/thu_sales2,2) + ,round(fri_sales1/fri_sales2,2) + ,round(sat_sales1/sat_sales2,2) + from + (select wswscs.d_week_seq d_week_seq1 + ,sun_sales sun_sales1 + ,mon_sales mon_sales1 + ,tue_sales tue_sales1 + ,wed_sales wed_sales1 + ,thu_sales thu_sales1 + ,fri_sales fri_sales1 + ,sat_sales sat_sales1 + from wswscs,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 2000) y, + (select wswscs.d_week_seq d_week_seq2 + ,sun_sales sun_sales2 + ,mon_sales mon_sales2 + ,tue_sales tue_sales2 + ,wed_sales wed_sales2 + ,thu_sales thu_sales2 + ,fri_sales fri_sales2 + ,sat_sales sat_sales2 + from wswscs + ,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 2000+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1; + diff --git a/benchmarks/tpc/queries/tpcds/q20.sql b/benchmarks/tpc/queries/tpcds/q20.sql new file mode 100644 index 0000000000..47531bd9bd --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q20.sql @@ -0,0 +1,31 @@ +-- CometBench-DS query 20 derived from TPC-DS query 20 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(cs_ext_sales_price) as itemrevenue + ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over + (partition by i_class) as revenueratio + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and i_category in ('Children', 'Sports', 'Music') + and cs_sold_date_sk = d_date_sk + and d_date between cast('2002-04-01' as date) + and (cast('2002-04-01' as date) + INTERVAL '30 DAYS') + group by i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + order by i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q21.sql b/benchmarks/tpc/queries/tpcds/q21.sql new file mode 100644 index 0000000000..ecc7b2ea44 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q21.sql @@ -0,0 +1,31 @@ +-- CometBench-DS query 21 derived from TPC-DS query 21 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select * + from(select w_warehouse_name + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2000-05-19' as date)) + then inv_quantity_on_hand + else 0 end) as inv_before + ,sum(case when (cast(d_date as date) >= cast ('2000-05-19' as date)) + then inv_quantity_on_hand + else 0 end) as inv_after + from inventory + ,warehouse + ,item + ,date_dim + where i_current_price between 0.99 and 1.49 + and i_item_sk = inv_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_date between (cast ('2000-05-19' as date) - INTERVAL '30 DAYS') + and (cast ('2000-05-19' as date) + INTERVAL '30 DAYS') + group by w_warehouse_name, i_item_id) x + where (case when inv_before > 0 + then inv_after / inv_before + else null + end) between 2.0/3.0 and 3.0/2.0 + order by w_warehouse_name + ,i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q22.sql b/benchmarks/tpc/queries/tpcds/q22.sql new file mode 100644 index 0000000000..416c4879c5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q22.sql @@ -0,0 +1,21 @@ +-- CometBench-DS query 22 derived from TPC-DS query 22 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_product_name + ,i_brand + ,i_class + ,i_category + ,avg(inv_quantity_on_hand) qoh + from inventory + ,date_dim + ,item + where inv_date_sk=d_date_sk + and inv_item_sk=i_item_sk + and d_month_seq between 1201 and 1201 + 11 + group by rollup(i_product_name + ,i_brand + ,i_class + ,i_category) +order by qoh, i_product_name, i_brand, i_class, i_category + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q23.sql b/benchmarks/tpc/queries/tpcds/q23.sql new file mode 100644 index 0000000000..220bf5bf72 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q23.sql @@ -0,0 +1,108 @@ +-- CometBench-DS query 23 derived from TPC-DS query 23 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk)), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * +from + max_store_sales)) + select sum(sales) + from (select cs_quantity*cs_list_price sales + from catalog_sales + ,date_dim + where d_year = 2000 + and d_moy = 3 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + union all + select ws_quantity*ws_list_price sales + from web_sales + ,date_dim + where d_year = 2000 + and d_moy = 3 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) + LIMIT 100; +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (2000,2000 + 1,2000 + 2,2000 + 3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2000,2000+1,2000+2,2000+3) + group by c_customer_sk)), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * + from max_store_sales)) + select c_last_name,c_first_name,sales + from (select c_last_name,c_first_name,sum(cs_quantity*cs_list_price) sales + from catalog_sales + ,customer + ,date_dim + where d_year = 2000 + and d_moy = 3 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and cs_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name + union all + select c_last_name,c_first_name,sum(ws_quantity*ws_list_price) sales + from web_sales + ,customer + ,date_dim + where d_year = 2000 + and d_moy = 3 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and ws_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name) + order by c_last_name,c_first_name,sales + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q24.sql b/benchmarks/tpc/queries/tpcds/q24.sql new file mode 100644 index 0000000000..119f86267b --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q24.sql @@ -0,0 +1,108 @@ +-- CometBench-DS query 24 derived from TPC-DS query 24 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_net_profit) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip +and s_market_id=10 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'orchid' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_net_profit) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_current_addr_sk = ca_address_sk + and c_birth_country <> upper(ca_country) + and s_zip = ca_zip + and s_market_id = 10 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'green' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +order by c_last_name + ,c_first_name + ,s_store_name +; + diff --git a/benchmarks/tpc/queries/tpcds/q25.sql b/benchmarks/tpc/queries/tpcds/q25.sql new file mode 100644 index 0000000000..eda5d6f724 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q25.sql @@ -0,0 +1,49 @@ +-- CometBench-DS query 25 derived from TPC-DS query 25 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,min(ss_net_profit) as store_sales_profit + ,min(sr_net_loss) as store_returns_loss + ,min(cs_net_profit) as catalog_sales_profit + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 2002 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 10 + and d2.d_year = 2002 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_moy between 4 and 10 + and d3.d_year = 2002 + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q26.sql b/benchmarks/tpc/queries/tpcds/q26.sql new file mode 100644 index 0000000000..44cc067b3a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q26.sql @@ -0,0 +1,22 @@ +-- CometBench-DS query 26 derived from TPC-DS query 26 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 + from catalog_sales, customer_demographics, date_dim, item, promotion + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd_demo_sk and + cs_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'M' and + cd_education_status = '4 yr Degree' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2000 + group by i_item_id + order by i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q27.sql b/benchmarks/tpc/queries/tpcds/q27.sql new file mode 100644 index 0000000000..d141d56a8a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q27.sql @@ -0,0 +1,24 @@ +-- CometBench-DS query 27 derived from TPC-DS query 27 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id, + s_state, grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, store, item + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_store_sk = s_store_sk and + ss_cdemo_sk = cd_demo_sk and + cd_gender = 'M' and + cd_marital_status = 'U' and + cd_education_status = 'Secondary' and + d_year = 2000 and + s_state in ('TN','TN', 'TN', 'TN', 'TN', 'TN') + group by rollup (i_item_id, s_state) + order by i_item_id + ,s_state + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q28.sql b/benchmarks/tpc/queries/tpcds/q28.sql new file mode 100644 index 0000000000..a34b079dea --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q28.sql @@ -0,0 +1,54 @@ +-- CometBench-DS query 28 derived from TPC-DS query 28 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select * +from (select avg(ss_list_price) B1_LP + ,count(ss_list_price) B1_CNT + ,count(distinct ss_list_price) B1_CNTD + from store_sales + where ss_quantity between 0 and 5 + and (ss_list_price between 28 and 28+10 + or ss_coupon_amt between 12573 and 12573+1000 + or ss_wholesale_cost between 33 and 33+20)) B1, + (select avg(ss_list_price) B2_LP + ,count(ss_list_price) B2_CNT + ,count(distinct ss_list_price) B2_CNTD + from store_sales + where ss_quantity between 6 and 10 + and (ss_list_price between 143 and 143+10 + or ss_coupon_amt between 5562 and 5562+1000 + or ss_wholesale_cost between 45 and 45+20)) B2, + (select avg(ss_list_price) B3_LP + ,count(ss_list_price) B3_CNT + ,count(distinct ss_list_price) B3_CNTD + from store_sales + where ss_quantity between 11 and 15 + and (ss_list_price between 159 and 159+10 + or ss_coupon_amt between 2807 and 2807+1000 + or ss_wholesale_cost between 24 and 24+20)) B3, + (select avg(ss_list_price) B4_LP + ,count(ss_list_price) B4_CNT + ,count(distinct ss_list_price) B4_CNTD + from store_sales + where ss_quantity between 16 and 20 + and (ss_list_price between 24 and 24+10 + or ss_coupon_amt between 3706 and 3706+1000 + or ss_wholesale_cost between 46 and 46+20)) B4, + (select avg(ss_list_price) B5_LP + ,count(ss_list_price) B5_CNT + ,count(distinct ss_list_price) B5_CNTD + from store_sales + where ss_quantity between 21 and 25 + and (ss_list_price between 76 and 76+10 + or ss_coupon_amt between 2096 and 2096+1000 + or ss_wholesale_cost between 50 and 50+20)) B5, + (select avg(ss_list_price) B6_LP + ,count(ss_list_price) B6_CNT + ,count(distinct ss_list_price) B6_CNTD + from store_sales + where ss_quantity between 26 and 30 + and (ss_list_price between 169 and 169+10 + or ss_coupon_amt between 10672 and 10672+1000 + or ss_wholesale_cost between 58 and 58+20)) B6 + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q29.sql b/benchmarks/tpc/queries/tpcds/q29.sql new file mode 100644 index 0000000000..3d6a7f8d0b --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q29.sql @@ -0,0 +1,48 @@ +-- CometBench-DS query 29 derived from TPC-DS query 29 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,stddev_samp(ss_quantity) as store_sales_quantity + ,stddev_samp(sr_return_quantity) as store_returns_quantity + ,stddev_samp(cs_quantity) as catalog_sales_quantity + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1999 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 4 + 3 + and d2.d_year = 1999 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_year in (1999,1999+1,1999+2) + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q3.sql b/benchmarks/tpc/queries/tpcds/q3.sql new file mode 100644 index 0000000000..2890c02e85 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q3.sql @@ -0,0 +1,22 @@ +-- CometBench-DS query 3 derived from TPC-DS query 3 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_net_profit) sum_agg + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 445 + and dt.d_moy=12 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,sum_agg desc + ,brand_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q30.sql b/benchmarks/tpc/queries/tpcds/q30.sql new file mode 100644 index 0000000000..658653381f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q30.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 30 derived from TPC-DS query 30 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with customer_total_return as + (select wr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(wr_return_amt) as ctr_total_return + from web_returns + ,date_dim + ,customer_address + where wr_returned_date_sk = d_date_sk + and d_year =2000 + and wr_returning_addr_sk = ca_address_sk + group by wr_returning_customer_sk + ,ca_state) + select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'KS' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date_sk,ctr_total_return + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q31.sql b/benchmarks/tpc/queries/tpcds/q31.sql new file mode 100644 index 0000000000..509a86fa43 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q31.sql @@ -0,0 +1,53 @@ +-- CometBench-DS query 31 derived from TPC-DS query 31 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ss as + (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales + from store_sales,date_dim,customer_address + where ss_sold_date_sk = d_date_sk + and ss_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year), + ws as + (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales + from web_sales,date_dim,customer_address + where ws_sold_date_sk = d_date_sk + and ws_bill_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year) + select + ss1.ca_county + ,ss1.d_year + ,ws2.web_sales/ws1.web_sales web_q1_q2_increase + ,ss2.store_sales/ss1.store_sales store_q1_q2_increase + ,ws3.web_sales/ws2.web_sales web_q2_q3_increase + ,ss3.store_sales/ss2.store_sales store_q2_q3_increase + from + ss ss1 + ,ss ss2 + ,ss ss3 + ,ws ws1 + ,ws ws2 + ,ws ws3 + where + ss1.d_qoy = 1 + and ss1.d_year = 1999 + and ss1.ca_county = ss2.ca_county + and ss2.d_qoy = 2 + and ss2.d_year = 1999 + and ss2.ca_county = ss3.ca_county + and ss3.d_qoy = 3 + and ss3.d_year = 1999 + and ss1.ca_county = ws1.ca_county + and ws1.d_qoy = 1 + and ws1.d_year = 1999 + and ws1.ca_county = ws2.ca_county + and ws2.d_qoy = 2 + and ws2.d_year = 1999 + and ws1.ca_county = ws3.ca_county + and ws3.d_qoy = 3 + and ws3.d_year =1999 + and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end + > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end + and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end + > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end + order by ss1.ca_county; + diff --git a/benchmarks/tpc/queries/tpcds/q32.sql b/benchmarks/tpc/queries/tpcds/q32.sql new file mode 100644 index 0000000000..3b968974f7 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q32.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 32 derived from TPC-DS query 32 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select sum(cs_ext_discount_amt) as `excess discount amount` +from + catalog_sales + ,item + ,date_dim +where +i_manufact_id = 283 +and i_item_sk = cs_item_sk +and d_date between '1999-02-22' and + (cast('1999-02-22' as date) + INTERVAL '90 DAYS') +and d_date_sk = cs_sold_date_sk +and cs_ext_discount_amt + > ( + select + 1.3 * avg(cs_ext_discount_amt) + from + catalog_sales + ,date_dim + where + cs_item_sk = i_item_sk + and d_date between '1999-02-22' and + (cast('1999-02-22' as date) + INTERVAL '90 DAYS') + and d_date_sk = cs_sold_date_sk + ) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q33.sql b/benchmarks/tpc/queries/tpcds/q33.sql new file mode 100644 index 0000000000..97f1f0fbf8 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q33.sql @@ -0,0 +1,76 @@ +-- CometBench-DS query 33 derived from TPC-DS query 33 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ss as ( + select + i_manufact_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 4 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + cs as ( + select + i_manufact_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 4 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + ws as ( + select + i_manufact_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 4 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id) + select i_manufact_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_manufact_id + order by total_sales + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q34.sql b/benchmarks/tpc/queries/tpcds/q34.sql new file mode 100644 index 0000000000..2c772481ef --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q34.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 34 derived from TPC-DS query 34 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) + and (household_demographics.hd_buy_potential = '501-1000' or + household_demographics.hd_buy_potential = 'Unknown') + and household_demographics.hd_vehicle_count > 0 + and (case when household_demographics.hd_vehicle_count > 0 + then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count + else null + end) > 1.2 + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dn,customer + where ss_customer_sk = c_customer_sk + and cnt between 15 and 20 + order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number; + diff --git a/benchmarks/tpc/queries/tpcds/q35.sql b/benchmarks/tpc/queries/tpcds/q35.sql new file mode 100644 index 0000000000..f116d84b22 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q35.sql @@ -0,0 +1,59 @@ +-- CometBench-DS query 35 derived from TPC-DS query 35 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + count(*) cnt1, + max(cd_dep_count), + stddev_samp(cd_dep_count), + stddev_samp(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + max(cd_dep_employed_count), + stddev_samp(cd_dep_employed_count), + stddev_samp(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + max(cd_dep_college_count), + stddev_samp(cd_dep_college_count), + stddev_samp(cd_dep_college_count) + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2000 and + d_qoy < 4) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2000 and + d_qoy < 4) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2000 and + d_qoy < 4)) + group by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q36.sql b/benchmarks/tpc/queries/tpcds/q36.sql new file mode 100644 index 0000000000..0b5d84adcf --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q36.sql @@ -0,0 +1,31 @@ +-- CometBench-DS query 36 derived from TPC-DS query 36 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,item + ,store + where + d1.d_year = 2001 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and s_state in ('TN','TN','TN','TN', + 'TN','TN','TN','TN') + group by rollup(i_category,i_class) + order by + lochierarchy desc + ,case when lochierarchy = 0 then i_category end + ,rank_within_parent + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q37.sql b/benchmarks/tpc/queries/tpcds/q37.sql new file mode 100644 index 0000000000..25ca6689aa --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q37.sql @@ -0,0 +1,18 @@ +-- CometBench-DS query 37 derived from TPC-DS query 37 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, catalog_sales + where i_current_price between 26 and 26 + 30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2001-06-09' as date) and (cast('2001-06-09' as date) + INTERVAL '60 DAYS') + and i_manufact_id in (744,884,722,693) + and inv_quantity_on_hand between 100 and 500 + and cs_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q38.sql b/benchmarks/tpc/queries/tpcds/q38.sql new file mode 100644 index 0000000000..563be4e807 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q38.sql @@ -0,0 +1,24 @@ +-- CometBench-DS query 38 derived from TPC-DS query 38 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select count(*) from ( + select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1190 and 1190 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1190 and 1190 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1190 and 1190 + 11 +) hot_cust + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q39.sql b/benchmarks/tpc/queries/tpcds/q39.sql new file mode 100644 index 0000000000..41d981501c --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q39.sql @@ -0,0 +1,55 @@ +-- CometBench-DS query 39 derived from TPC-DS query 39 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =2001 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=1 + and inv2.d_moy=1+1 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +; +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =2001 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=1 + and inv2.d_moy=1+1 + and inv1.cov > 1.5 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +; + diff --git a/benchmarks/tpc/queries/tpcds/q4.sql b/benchmarks/tpc/queries/tpcds/q4.sql new file mode 100644 index 0000000000..8edacf2f9d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q4.sql @@ -0,0 +1,117 @@ +-- CometBench-DS query 4 derived from TPC-DS query 4 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total + ,'c' sale_type + from customer + ,catalog_sales + ,date_dim + where c_customer_sk = cs_bill_customer_sk + and cs_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year +union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_c_firstyear + ,year_total t_c_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_c_secyear.customer_id + and t_s_firstyear.customer_id = t_c_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_c_firstyear.sale_type = 'c' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_c_secyear.sale_type = 'c' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 2001 + and t_s_secyear.dyear = 2001+1 + and t_c_firstyear.dyear = 2001 + and t_c_secyear.dyear = 2001+1 + and t_w_firstyear.dyear = 2001 + and t_w_secyear.dyear = 2001+1 + and t_s_firstyear.year_total > 0 + and t_c_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q40.sql b/benchmarks/tpc/queries/tpcds/q40.sql new file mode 100644 index 0000000000..b52966894e --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q40.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 40 derived from TPC-DS query 40 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + w_state + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('2002-05-18' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before + ,sum(case when (cast(d_date as date) >= cast ('2002-05-18' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after + from + catalog_sales left outer join catalog_returns on + (cs_order_number = cr_order_number + and cs_item_sk = cr_item_sk) + ,warehouse + ,item + ,date_dim + where + i_current_price between 0.99 and 1.49 + and i_item_sk = cs_item_sk + and cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and d_date between (cast ('2002-05-18' as date) - INTERVAL '30 DAYS') + and (cast ('2002-05-18' as date) + INTERVAL '30 DAYS') + group by + w_state,i_item_id + order by w_state,i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q41.sql b/benchmarks/tpc/queries/tpcds/q41.sql new file mode 100644 index 0000000000..df2db88833 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q41.sql @@ -0,0 +1,53 @@ +-- CometBench-DS query 41 derived from TPC-DS query 41 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select distinct(i_product_name) + from item i1 + where i_manufact_id between 668 and 668+40 + and (select count(*) as item_cnt + from item + where (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'cream' or i_color = 'ghost') and + (i_units = 'Ton' or i_units = 'Gross') and + (i_size = 'economy' or i_size = 'small') + ) or + (i_category = 'Women' and + (i_color = 'midnight' or i_color = 'burlywood') and + (i_units = 'Tsp' or i_units = 'Bundle') and + (i_size = 'medium' or i_size = 'extra large') + ) or + (i_category = 'Men' and + (i_color = 'lavender' or i_color = 'azure') and + (i_units = 'Each' or i_units = 'Lb') and + (i_size = 'large' or i_size = 'N/A') + ) or + (i_category = 'Men' and + (i_color = 'chocolate' or i_color = 'steel') and + (i_units = 'N/A' or i_units = 'Dozen') and + (i_size = 'economy' or i_size = 'small') + ))) or + (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'floral' or i_color = 'royal') and + (i_units = 'Unknown' or i_units = 'Tbl') and + (i_size = 'economy' or i_size = 'small') + ) or + (i_category = 'Women' and + (i_color = 'navy' or i_color = 'forest') and + (i_units = 'Bunch' or i_units = 'Dram') and + (i_size = 'medium' or i_size = 'extra large') + ) or + (i_category = 'Men' and + (i_color = 'cyan' or i_color = 'indian') and + (i_units = 'Carton' or i_units = 'Cup') and + (i_size = 'large' or i_size = 'N/A') + ) or + (i_category = 'Men' and + (i_color = 'coral' or i_color = 'pale') and + (i_units = 'Pallet' or i_units = 'Gram') and + (i_size = 'economy' or i_size = 'small') + )))) > 0 + order by i_product_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q42.sql b/benchmarks/tpc/queries/tpcds/q42.sql new file mode 100644 index 0000000000..3c565534a9 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q42.sql @@ -0,0 +1,23 @@ +-- CometBench-DS query 42 derived from TPC-DS query 42 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select dt.d_year + ,item.i_category_id + ,item.i_category + ,sum(ss_ext_sales_price) + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=11 + and dt.d_year=1998 + group by dt.d_year + ,item.i_category_id + ,item.i_category + order by sum(ss_ext_sales_price) desc,dt.d_year + ,item.i_category_id + ,item.i_category + LIMIT 100 ; + diff --git a/benchmarks/tpc/queries/tpcds/q43.sql b/benchmarks/tpc/queries/tpcds/q43.sql new file mode 100644 index 0000000000..1aaf123c9a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q43.sql @@ -0,0 +1,20 @@ +-- CometBench-DS query 43 derived from TPC-DS query 43 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select s_store_name, s_store_id, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from date_dim, store_sales, store + where d_date_sk = ss_sold_date_sk and + s_store_sk = ss_store_sk and + s_gmt_offset = -5 and + d_year = 2000 + group by s_store_name, s_store_id + order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q44.sql b/benchmarks/tpc/queries/tpcds/q44.sql new file mode 100644 index 0000000000..f7e474249f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q44.sql @@ -0,0 +1,36 @@ +-- CometBench-DS query 44 derived from TPC-DS query 44 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing +from(select * + from (select item_sk,rank() over (order by rank_col asc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 6 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 6 + and ss_hdemo_sk is null + group by ss_store_sk))V1)V11 + where rnk < 11) asceding, + (select * + from (select item_sk,rank() over (order by rank_col desc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 6 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 6 + and ss_hdemo_sk is null + group by ss_store_sk))V2)V21 + where rnk < 11) descending, +item i1, +item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk=asceding.item_sk + and i2.i_item_sk=descending.item_sk +order by asceding.rnk + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q45.sql b/benchmarks/tpc/queries/tpcds/q45.sql new file mode 100644 index 0000000000..dd77ce6cb3 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q45.sql @@ -0,0 +1,21 @@ +-- CometBench-DS query 45 derived from TPC-DS query 45 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select ca_zip, ca_city, sum(ws_sales_price) + from web_sales, customer, customer_address, date_dim, item + where ws_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ws_item_sk = i_item_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') + or + i_item_id in (select i_item_id + from item + where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) + ) + and ws_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2000 + group by ca_zip, ca_city + order by ca_zip, ca_city + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q46.sql b/benchmarks/tpc/queries/tpcds/q46.sql new file mode 100644 index 0000000000..1321f134ef --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q46.sql @@ -0,0 +1,36 @@ +-- CometBench-DS query 46 derived from TPC-DS query 46 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and (household_demographics.hd_dep_count = 3 or + household_demographics.hd_vehicle_count= 1) + and date_dim.d_dow in (6,0) + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_city in ('Midway','Fairview','Fairview','Midway','Fairview') + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q47.sql b/benchmarks/tpc/queries/tpcds/q47.sql new file mode 100644 index 0000000000..5b29007c03 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q47.sql @@ -0,0 +1,52 @@ +-- CometBench-DS query 47 derived from TPC-DS query 47 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with v1 as( + select i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + s_store_name, s_company_name + order by d_year, d_moy) rn + from item, store_sales, date_dim, store + where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + ( + d_year = 2001 or + ( d_year = 2001-1 and d_moy =12) or + ( d_year = 2001+1 and d_moy =1) + ) + group by i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 as( + select v1.i_category, v1.i_brand, v1.s_store_name, v1.s_company_name + ,v1.d_year + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1.s_store_name = v1_lag.s_store_name and + v1.s_store_name = v1_lead.s_store_name and + v1.s_company_name = v1_lag.s_company_name and + v1.s_company_name = v1_lead.s_company_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2001 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, nsum + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q48.sql b/benchmarks/tpc/queries/tpcds/q48.sql new file mode 100644 index 0000000000..cf08721423 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q48.sql @@ -0,0 +1,68 @@ +-- CometBench-DS query 48 derived from TPC-DS query 48 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select sum (ss_quantity) + from store_sales, store, customer_demographics, customer_address, date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and + ( + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'W' + and + cd_education_status = '2 yr Degree' + and + ss_sales_price between 100.00 and 150.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'S' + and + cd_education_status = 'Advanced Degree' + and + ss_sales_price between 50.00 and 100.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'D' + and + cd_education_status = 'Primary' + and + ss_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('IL', 'KY', 'OR') + and ss_net_profit between 0 and 2000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('VA', 'FL', 'AL') + and ss_net_profit between 150 and 3000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('OK', 'IA', 'TX') + and ss_net_profit between 50 and 25000 + ) + ) +; + diff --git a/benchmarks/tpc/queries/tpcds/q49.sql b/benchmarks/tpc/queries/tpcds/q49.sql new file mode 100644 index 0000000000..3e37517bc4 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q49.sql @@ -0,0 +1,130 @@ +-- CometBench-DS query 49 derived from TPC-DS query 49 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select channel, item, return_ratio, return_rank, currency_rank from + (select + 'web' as channel + ,web.item + ,web.return_ratio + ,web.return_rank + ,web.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select ws.ws_item_sk as item + ,(cast(sum(coalesce(wr.wr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(wr.wr_return_amt,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + web_sales ws left outer join web_returns wr + on (ws.ws_order_number = wr.wr_order_number and + ws.ws_item_sk = wr.wr_item_sk) + ,date_dim + where + wr.wr_return_amt > 10000 + and ws.ws_net_profit > 1 + and ws.ws_net_paid > 0 + and ws.ws_quantity > 0 + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by ws.ws_item_sk + ) in_web + ) web + where + ( + web.return_rank <= 10 + or + web.currency_rank <= 10 + ) + union + select + 'catalog' as channel + ,catalog.item + ,catalog.return_ratio + ,catalog.return_rank + ,catalog.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select + cs.cs_item_sk as item + ,(cast(sum(coalesce(cr.cr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(cr.cr_return_amount,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + catalog_sales cs left outer join catalog_returns cr + on (cs.cs_order_number = cr.cr_order_number and + cs.cs_item_sk = cr.cr_item_sk) + ,date_dim + where + cr.cr_return_amount > 10000 + and cs.cs_net_profit > 1 + and cs.cs_net_paid > 0 + and cs.cs_quantity > 0 + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by cs.cs_item_sk + ) in_cat + ) catalog + where + ( + catalog.return_rank <= 10 + or + catalog.currency_rank <=10 + ) + union + select + 'store' as channel + ,store.item + ,store.return_ratio + ,store.return_rank + ,store.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select sts.ss_item_sk as item + ,(cast(sum(coalesce(sr.sr_return_quantity,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(sr.sr_return_amt,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + store_sales sts left outer join store_returns sr + on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk) + ,date_dim + where + sr.sr_return_amt > 10000 + and sts.ss_net_profit > 1 + and sts.ss_net_paid > 0 + and sts.ss_quantity > 0 + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by sts.ss_item_sk + ) in_store + ) store + where ( + store.return_rank <= 10 + or + store.currency_rank <= 10 + ) + ) + order by 1,4,5,2 + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q5.sql b/benchmarks/tpc/queries/tpcds/q5.sql new file mode 100644 index 0000000000..68fda358e9 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q5.sql @@ -0,0 +1,129 @@ +-- CometBench-DS query 5 derived from TPC-DS query 5 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ssr as + (select s_store_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select ss_store_sk as store_sk, + ss_sold_date_sk as date_sk, + ss_ext_sales_price as sales_price, + ss_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from store_sales + union all + select sr_store_sk as store_sk, + sr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + sr_return_amt as return_amt, + sr_net_loss as net_loss + from store_returns + ) salesreturns, + date_dim, + store + where date_sk = d_date_sk + and d_date between cast('2001-08-04' as date) + and (cast('2001-08-04' as date) + INTERVAL '14 DAYS') + and store_sk = s_store_sk + group by s_store_id) + , + csr as + (select cp_catalog_page_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select cs_catalog_page_sk as page_sk, + cs_sold_date_sk as date_sk, + cs_ext_sales_price as sales_price, + cs_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from catalog_sales + union all + select cr_catalog_page_sk as page_sk, + cr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + cr_return_amount as return_amt, + cr_net_loss as net_loss + from catalog_returns + ) salesreturns, + date_dim, + catalog_page + where date_sk = d_date_sk + and d_date between cast('2001-08-04' as date) + and (cast('2001-08-04' as date) + INTERVAL '14 DAYS') + and page_sk = cp_catalog_page_sk + group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( select ws_web_site_sk as wsr_web_site_sk, + ws_sold_date_sk as date_sk, + ws_ext_sales_price as sales_price, + ws_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from web_sales + union all + select ws_web_site_sk as wsr_web_site_sk, + wr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + wr_return_amt as return_amt, + wr_net_loss as net_loss + from web_returns left outer join web_sales on + ( wr_item_sk = ws_item_sk + and wr_order_number = ws_order_number) + ) salesreturns, + date_dim, + web_site + where date_sk = d_date_sk + and d_date between cast('2001-08-04' as date) + and (cast('2001-08-04' as date) + INTERVAL '14 DAYS') + and wsr_web_site_sk = web_site_sk + group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , 'store' || s_store_id as id + , sales + , returns + , (profit - profit_loss) as profit + from ssr + union all + select 'catalog channel' as channel + , 'catalog_page' || cp_catalog_page_id as id + , sales + , returns + , (profit - profit_loss) as profit + from csr + union all + select 'web channel' as channel + , 'web_site' || web_site_id as id + , sales + , returns + , (profit - profit_loss) as profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q50.sql b/benchmarks/tpc/queries/tpcds/q50.sql new file mode 100644 index 0000000000..82aa472004 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q50.sql @@ -0,0 +1,60 @@ +-- CometBench-DS query 50 derived from TPC-DS query 50 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as `30 days` + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and + (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as `31-60 days` + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and + (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as `61-90 days` + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and + (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as `91-120 days` + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as `>120 days` +from + store_sales + ,store_returns + ,store + ,date_dim d1 + ,date_dim d2 +where + d2.d_year = 2002 +and d2.d_moy = 8 +and ss_ticket_number = sr_ticket_number +and ss_item_sk = sr_item_sk +and ss_sold_date_sk = d1.d_date_sk +and sr_returned_date_sk = d2.d_date_sk +and ss_customer_sk = sr_customer_sk +and ss_store_sk = s_store_sk +group by + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +order by s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q51.sql b/benchmarks/tpc/queries/tpcds/q51.sql new file mode 100644 index 0000000000..55ed6c462f --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q51.sql @@ -0,0 +1,46 @@ +-- CometBench-DS query 51 derived from TPC-DS query 51 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +WITH web_v1 as ( +select + ws_item_sk item_sk, d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from web_sales + ,date_dim +where ws_sold_date_sk=d_date_sk + and d_month_seq between 1215 and 1215+11 + and ws_item_sk is not NULL +group by ws_item_sk, d_date), +store_v1 as ( +select + ss_item_sk item_sk, d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from store_sales + ,date_dim +where ss_sold_date_sk=d_date_sk + and d_month_seq between 1215 and 1215+11 + and ss_item_sk is not NULL +group by ss_item_sk, d_date) + select * +from (select item_sk + ,d_date + ,web_sales + ,store_sales + ,max(web_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative + ,max(store_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative + from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk + ,case when web.d_date is not null then web.d_date else store.d_date end d_date + ,web.cume_sales web_sales + ,store.cume_sales store_sales + from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + )x )y +where web_cumulative > store_cumulative +order by item_sk + ,d_date + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q52.sql b/benchmarks/tpc/queries/tpcds/q52.sql new file mode 100644 index 0000000000..fc133fa7a5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q52.sql @@ -0,0 +1,23 @@ +-- CometBench-DS query 52 derived from TPC-DS query 52 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) ext_price + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=11 + and dt.d_year=2000 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,ext_price desc + ,brand_id + LIMIT 100 ; + diff --git a/benchmarks/tpc/queries/tpcds/q53.sql b/benchmarks/tpc/queries/tpcds/q53.sql new file mode 100644 index 0000000000..cff50f65e8 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q53.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 53 derived from TPC-DS query 53 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select * from +(select i_manufact_id, +sum(ss_sales_price) sum_sales, +avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and +ss_sold_date_sk = d_date_sk and +ss_store_sk = s_store_sk and +d_month_seq in (1197,1197+1,1197+2,1197+3,1197+4,1197+5,1197+6,1197+7,1197+8,1197+9,1197+10,1197+11) and +((i_category in ('Books','Children','Electronics') and +i_class in ('personal','portable','reference','self-help') and +i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) +or(i_category in ('Women','Music','Men') and +i_class in ('accessories','classical','fragrances','pants') and +i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manufact_id, d_qoy ) tmp1 +where case when avg_quarterly_sales > 0 + then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales + else null end > 0.1 +order by avg_quarterly_sales, + sum_sales, + i_manufact_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q54.sql b/benchmarks/tpc/queries/tpcds/q54.sql new file mode 100644 index 0000000000..1316ba2399 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q54.sql @@ -0,0 +1,57 @@ +-- CometBench-DS query 54 derived from TPC-DS query 54 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with my_customers as ( + select distinct c_customer_sk + , c_current_addr_sk + from + ( select cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + from catalog_sales + union all + select ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + from web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + where sold_date_sk = d_date_sk + and item_sk = i_item_sk + and i_category = 'Men' + and i_class = 'shirts' + and c_customer_sk = cs_or_ws_sales.customer_sk + and d_moy = 4 + and d_year = 1998 + ) + , my_revenue as ( + select c_customer_sk, + sum(ss_ext_sales_price) as revenue + from my_customers, + store_sales, + customer_address, + store, + date_dim + where c_current_addr_sk = ca_address_sk + and ca_county = s_county + and ca_state = s_state + and ss_sold_date_sk = d_date_sk + and c_customer_sk = ss_customer_sk + and d_month_seq between (select distinct d_month_seq+1 + from date_dim where d_year = 1998 and d_moy = 4) + and (select distinct d_month_seq+3 + from date_dim where d_year = 1998 and d_moy = 4) + group by c_customer_sk + ) + , segments as + (select cast((revenue/50) as int) as segment + from my_revenue + ) + select segment, count(*) as num_customers, segment*50 as segment_base + from segments + group by segment + order by segment, num_customers + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q55.sql b/benchmarks/tpc/queries/tpcds/q55.sql new file mode 100644 index 0000000000..b448279ded --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q55.sql @@ -0,0 +1,15 @@ +-- CometBench-DS query 55 derived from TPC-DS query 55 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_brand_id brand_id, i_brand brand, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=20 + and d_moy=12 + and d_year=1998 + group by i_brand, i_brand_id + order by ext_price desc, i_brand_id + LIMIT 100 ; + diff --git a/benchmarks/tpc/queries/tpcds/q56.sql b/benchmarks/tpc/queries/tpcds/q56.sql new file mode 100644 index 0000000000..4fb34dd3e4 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q56.sql @@ -0,0 +1,70 @@ +-- CometBench-DS query 56 derived from TPC-DS query 56 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ss as ( + select i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where i_item_id in (select + i_item_id +from item +where i_color in ('powder','goldenrod','bisque')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id), + cs as ( + select i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('powder','goldenrod','bisque')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id), + ws as ( + select i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('powder','goldenrod','bisque')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1998 + and d_moy = 5 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_item_id) + select i_item_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by total_sales, + i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q57.sql b/benchmarks/tpc/queries/tpcds/q57.sql new file mode 100644 index 0000000000..3dc568848a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q57.sql @@ -0,0 +1,49 @@ +-- CometBench-DS query 57 derived from TPC-DS query 57 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with v1 as( + select i_category, i_brand, + cc_name, + d_year, d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) over + (partition by i_category, i_brand, + cc_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + cc_name + order by d_year, d_moy) rn + from item, catalog_sales, date_dim, call_center + where cs_item_sk = i_item_sk and + cs_sold_date_sk = d_date_sk and + cc_call_center_sk= cs_call_center_sk and + ( + d_year = 2000 or + ( d_year = 2000-1 and d_moy =12) or + ( d_year = 2000+1 and d_moy =1) + ) + group by i_category, i_brand, + cc_name , d_year, d_moy), + v2 as( + select v1.cc_name + ,v1.d_year, v1.d_moy + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1. cc_name = v1_lag. cc_name and + v1. cc_name = v1_lead. cc_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2000 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, psum + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q58.sql b/benchmarks/tpc/queries/tpcds/q58.sql new file mode 100644 index 0000000000..3243c0ac8a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q58.sql @@ -0,0 +1,66 @@ +-- CometBench-DS query 58 derived from TPC-DS query 58 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ss_items as + (select i_item_id item_id + ,sum(ss_ext_sales_price) ss_item_rev + from store_sales + ,item + ,date_dim + where ss_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '2000-02-12')) + and ss_sold_date_sk = d_date_sk + group by i_item_id), + cs_items as + (select i_item_id item_id + ,sum(cs_ext_sales_price) cs_item_rev + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '2000-02-12')) + and cs_sold_date_sk = d_date_sk + group by i_item_id), + ws_items as + (select i_item_id item_id + ,sum(ws_ext_sales_price) ws_item_rev + from web_sales + ,item + ,date_dim + where ws_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq =(select d_week_seq + from date_dim + where d_date = '2000-02-12')) + and ws_sold_date_sk = d_date_sk + group by i_item_id) + select ss_items.item_id + ,ss_item_rev + ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev + ,cs_item_rev + ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev + ,ws_item_rev + ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev + ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average + from ss_items,cs_items,ws_items + where ss_items.item_id=cs_items.item_id + and ss_items.item_id=ws_items.item_id + and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + order by item_id + ,ss_item_rev + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q59.sql b/benchmarks/tpc/queries/tpcds/q59.sql new file mode 100644 index 0000000000..cd92847be6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q59.sql @@ -0,0 +1,45 @@ +-- CometBench-DS query 59 derived from TPC-DS query 59 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with wss as + (select d_week_seq, + ss_store_sk, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + group by d_week_seq,ss_store_sk + ) + select s_store_name1,s_store_id1,d_week_seq1 + ,sun_sales1/sun_sales2,mon_sales1/mon_sales2 + ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2 + ,fri_sales1/fri_sales2,sat_sales1/sat_sales2 + from + (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1 + ,s_store_id s_store_id1,sun_sales sun_sales1 + ,mon_sales mon_sales1,tue_sales tue_sales1 + ,wed_sales wed_sales1,thu_sales thu_sales1 + ,fri_sales fri_sales1,sat_sales sat_sales1 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1206 and 1206 + 11) y, + (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2 + ,s_store_id s_store_id2,sun_sales sun_sales2 + ,mon_sales mon_sales2,tue_sales tue_sales2 + ,wed_sales wed_sales2,thu_sales thu_sales2 + ,fri_sales fri_sales2,sat_sales sat_sales2 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1206+ 12 and 1206 + 23) x + where s_store_id1=s_store_id2 + and d_week_seq1=d_week_seq2-52 + order by s_store_name1,s_store_id1,d_week_seq1 + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q6.sql b/benchmarks/tpc/queries/tpcds/q6.sql new file mode 100644 index 0000000000..9e5a5ba8e3 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q6.sql @@ -0,0 +1,27 @@ +-- CometBench-DS query 6 derived from TPC-DS query 6 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select a.ca_state state, count(*) cnt + from customer_address a + ,customer c + ,store_sales s + ,date_dim d + ,item i + where a.ca_address_sk = c.c_current_addr_sk + and c.c_customer_sk = s.ss_customer_sk + and s.ss_sold_date_sk = d.d_date_sk + and s.ss_item_sk = i.i_item_sk + and d.d_month_seq = + (select distinct (d_month_seq) + from date_dim + where d_year = 1998 + and d_moy = 3 ) + and i.i_current_price > 1.2 * + (select avg(j.i_current_price) + from item j + where j.i_category = i.i_category) + group by a.ca_state + having count(*) >= 10 + order by cnt, a.ca_state + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q60.sql b/benchmarks/tpc/queries/tpcds/q60.sql new file mode 100644 index 0000000000..cf1f885524 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q60.sql @@ -0,0 +1,79 @@ +-- CometBench-DS query 60 derived from TPC-DS query 60 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ss as ( + select + i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Shoes')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 10 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + cs as ( + select + i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Shoes')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 10 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + ws as ( + select + i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Shoes')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2001 + and d_moy = 10 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id) + select + i_item_id +,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by i_item_id + ,total_sales + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q61.sql b/benchmarks/tpc/queries/tpcds/q61.sql new file mode 100644 index 0000000000..ef2e89b971 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q61.sql @@ -0,0 +1,45 @@ +-- CometBench-DS query 61 derived from TPC-DS query 61 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 +from + (select sum(ss_ext_sales_price) promotions + from store_sales + ,store + ,promotion + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_promo_sk = p_promo_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -6 + and i_category = 'Sports' + and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') + and s_gmt_offset = -6 + and d_year = 2002 + and d_moy = 11) promotional_sales, + (select sum(ss_ext_sales_price) total + from store_sales + ,store + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -6 + and i_category = 'Sports' + and s_gmt_offset = -6 + and d_year = 2002 + and d_moy = 11) all_sales +order by promotions, total + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q62.sql b/benchmarks/tpc/queries/tpcds/q62.sql new file mode 100644 index 0000000000..5d14451765 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q62.sql @@ -0,0 +1,36 @@ +-- CometBench-DS query 62 derived from TPC-DS query 62 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as `30 days` + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and + (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as `31-60 days` + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and + (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as `61-90 days` + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and + (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as `91-120 days` + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as `>120 days` +from + web_sales + ,warehouse + ,ship_mode + ,web_site + ,date_dim +where + d_month_seq between 1217 and 1217 + 11 +and ws_ship_date_sk = d_date_sk +and ws_warehouse_sk = w_warehouse_sk +and ws_ship_mode_sk = sm_ship_mode_sk +and ws_web_site_sk = web_site_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,web_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q63.sql b/benchmarks/tpc/queries/tpcds/q63.sql new file mode 100644 index 0000000000..e903fca6a0 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q63.sql @@ -0,0 +1,30 @@ +-- CometBench-DS query 63 derived from TPC-DS query 63 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select * +from (select i_manager_id + ,sum(ss_sales_price) sum_sales + ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales + from item + ,store_sales + ,date_dim + ,store + where ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and d_month_seq in (1181,1181+1,1181+2,1181+3,1181+4,1181+5,1181+6,1181+7,1181+8,1181+9,1181+10,1181+11) + and (( i_category in ('Books','Children','Electronics') + and i_class in ('personal','portable','reference','self-help') + and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) + or( i_category in ('Women','Music','Men') + and i_class in ('accessories','classical','fragrances','pants') + and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manager_id, d_moy) tmp1 +where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 +order by i_manager_id + ,avg_monthly_sales + ,sum_sales + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q64.sql b/benchmarks/tpc/queries/tpcds/q64.sql new file mode 100644 index 0000000000..bc8f1676fc --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q64.sql @@ -0,0 +1,122 @@ +-- CometBench-DS query 64 derived from TPC-DS query 64 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with cs_ui as + (select cs_item_sk + ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund + from catalog_sales + ,catalog_returns + where cs_item_sk = cr_item_sk + and cs_order_number = cr_order_number + group by cs_item_sk + having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)), +cross_sales as + (select i_product_name product_name + ,i_item_sk item_sk + ,s_store_name store_name + ,s_zip store_zip + ,ad1.ca_street_number b_street_number + ,ad1.ca_street_name b_street_name + ,ad1.ca_city b_city + ,ad1.ca_zip b_zip + ,ad2.ca_street_number c_street_number + ,ad2.ca_street_name c_street_name + ,ad2.ca_city c_city + ,ad2.ca_zip c_zip + ,d1.d_year as syear + ,d2.d_year as fsyear + ,d3.d_year s2year + ,count(*) cnt + ,sum(ss_wholesale_cost) s1 + ,sum(ss_list_price) s2 + ,sum(ss_coupon_amt) s3 + FROM store_sales + ,store_returns + ,cs_ui + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,customer + ,customer_demographics cd1 + ,customer_demographics cd2 + ,promotion + ,household_demographics hd1 + ,household_demographics hd2 + ,customer_address ad1 + ,customer_address ad2 + ,income_band ib1 + ,income_band ib2 + ,item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk= cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk and + ss_item_sk = i_item_sk and + ss_item_sk = sr_item_sk and + ss_ticket_number = sr_ticket_number and + ss_item_sk = cs_ui.cs_item_sk and + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk and + c_first_sales_date_sk = d2.d_date_sk and + c_first_shipto_date_sk = d3.d_date_sk and + ss_promo_sk = p_promo_sk and + hd1.hd_income_band_sk = ib1.ib_income_band_sk and + hd2.hd_income_band_sk = ib2.ib_income_band_sk and + cd1.cd_marital_status <> cd2.cd_marital_status and + i_color in ('light','cyan','burnished','green','almond','smoke') and + i_current_price between 22 and 22 + 10 and + i_current_price between 22 + 1 and 22 + 15 +group by i_product_name + ,i_item_sk + ,s_store_name + ,s_zip + ,ad1.ca_street_number + ,ad1.ca_street_name + ,ad1.ca_city + ,ad1.ca_zip + ,ad2.ca_street_number + ,ad2.ca_street_name + ,ad2.ca_city + ,ad2.ca_zip + ,d1.d_year + ,d2.d_year + ,d3.d_year +) +select cs1.product_name + ,cs1.store_name + ,cs1.store_zip + ,cs1.b_street_number + ,cs1.b_street_name + ,cs1.b_city + ,cs1.b_zip + ,cs1.c_street_number + ,cs1.c_street_name + ,cs1.c_city + ,cs1.c_zip + ,cs1.syear + ,cs1.cnt + ,cs1.s1 as s11 + ,cs1.s2 as s21 + ,cs1.s3 as s31 + ,cs2.s1 as s12 + ,cs2.s2 as s22 + ,cs2.s3 as s32 + ,cs2.syear + ,cs2.cnt +from cross_sales cs1,cross_sales cs2 +where cs1.item_sk=cs2.item_sk and + cs1.syear = 2001 and + cs2.syear = 2001 + 1 and + cs2.cnt <= cs1.cnt and + cs1.store_name = cs2.store_name and + cs1.store_zip = cs2.store_zip +order by cs1.product_name + ,cs1.store_name + ,cs2.cnt + ,cs1.s1 + ,cs2.s1; + diff --git a/benchmarks/tpc/queries/tpcds/q65.sql b/benchmarks/tpc/queries/tpcds/q65.sql new file mode 100644 index 0000000000..d00f247199 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q65.sql @@ -0,0 +1,30 @@ +-- CometBench-DS query 65 derived from TPC-DS query 65 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand + from store, item, + (select ss_store_sk, avg(revenue) as ave + from + (select ss_store_sk, ss_item_sk, + sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1186 and 1186+11 + group by ss_store_sk, ss_item_sk) sa + group by ss_store_sk) sb, + (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1186 and 1186+11 + group by ss_store_sk, ss_item_sk) sc + where sb.ss_store_sk = sc.ss_store_sk and + sc.revenue <= 0.1 * sb.ave and + s_store_sk = sc.ss_store_sk and + i_item_sk = sc.ss_item_sk + order by s_store_name, i_item_desc + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q66.sql b/benchmarks/tpc/queries/tpcds/q66.sql new file mode 100644 index 0000000000..9fb527c2d5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q66.sql @@ -0,0 +1,221 @@ +-- CometBench-DS query 66 derived from TPC-DS query 66 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + ,sum(jan_sales) as jan_sales + ,sum(feb_sales) as feb_sales + ,sum(mar_sales) as mar_sales + ,sum(apr_sales) as apr_sales + ,sum(may_sales) as may_sales + ,sum(jun_sales) as jun_sales + ,sum(jul_sales) as jul_sales + ,sum(aug_sales) as aug_sales + ,sum(sep_sales) as sep_sales + ,sum(oct_sales) as oct_sales + ,sum(nov_sales) as nov_sales + ,sum(dec_sales) as dec_sales + ,sum(jan_sales/w_warehouse_sq_ft) as jan_sales_per_sq_foot + ,sum(feb_sales/w_warehouse_sq_ft) as feb_sales_per_sq_foot + ,sum(mar_sales/w_warehouse_sq_ft) as mar_sales_per_sq_foot + ,sum(apr_sales/w_warehouse_sq_ft) as apr_sales_per_sq_foot + ,sum(may_sales/w_warehouse_sq_ft) as may_sales_per_sq_foot + ,sum(jun_sales/w_warehouse_sq_ft) as jun_sales_per_sq_foot + ,sum(jul_sales/w_warehouse_sq_ft) as jul_sales_per_sq_foot + ,sum(aug_sales/w_warehouse_sq_ft) as aug_sales_per_sq_foot + ,sum(sep_sales/w_warehouse_sq_ft) as sep_sales_per_sq_foot + ,sum(oct_sales/w_warehouse_sq_ft) as oct_sales_per_sq_foot + ,sum(nov_sales/w_warehouse_sq_ft) as nov_sales_per_sq_foot + ,sum(dec_sales/w_warehouse_sq_ft) as dec_sales_per_sq_foot + ,sum(jan_net) as jan_net + ,sum(feb_net) as feb_net + ,sum(mar_net) as mar_net + ,sum(apr_net) as apr_net + ,sum(may_net) as may_net + ,sum(jun_net) as jun_net + ,sum(jul_net) as jul_net + ,sum(aug_net) as aug_net + ,sum(sep_net) as sep_net + ,sum(oct_net) as oct_net + ,sum(nov_net) as nov_net + ,sum(dec_net) as dec_net + from ( + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,'FEDEX' || ',' || 'GERMA' as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then ws_ext_list_price* ws_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then ws_ext_list_price* ws_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then ws_ext_list_price* ws_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then ws_ext_list_price* ws_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then ws_ext_list_price* ws_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then ws_ext_list_price* ws_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then ws_ext_list_price* ws_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then ws_ext_list_price* ws_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then ws_ext_list_price* ws_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then ws_ext_list_price* ws_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then ws_ext_list_price* ws_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then ws_ext_list_price* ws_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then ws_net_profit * ws_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then ws_net_profit * ws_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then ws_net_profit * ws_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then ws_net_profit * ws_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then ws_net_profit * ws_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then ws_net_profit * ws_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then ws_net_profit * ws_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then ws_net_profit * ws_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then ws_net_profit * ws_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then ws_net_profit * ws_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then ws_net_profit * ws_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then ws_net_profit * ws_quantity else 0 end) as dec_net + from + web_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + ws_warehouse_sk = w_warehouse_sk + and ws_sold_date_sk = d_date_sk + and ws_sold_time_sk = t_time_sk + and ws_ship_mode_sk = sm_ship_mode_sk + and d_year = 2001 + and t_time between 19072 and 19072+28800 + and sm_carrier in ('FEDEX','GERMA') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + union all + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,'FEDEX' || ',' || 'GERMA' as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then cs_sales_price* cs_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then cs_sales_price* cs_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then cs_sales_price* cs_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then cs_sales_price* cs_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then cs_sales_price* cs_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then cs_sales_price* cs_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then cs_sales_price* cs_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then cs_sales_price* cs_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then cs_sales_price* cs_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then cs_sales_price* cs_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then cs_sales_price* cs_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then cs_sales_price* cs_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then cs_net_paid * cs_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then cs_net_paid * cs_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then cs_net_paid * cs_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then cs_net_paid * cs_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then cs_net_paid * cs_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then cs_net_paid * cs_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then cs_net_paid * cs_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then cs_net_paid * cs_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then cs_net_paid * cs_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then cs_net_paid * cs_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then cs_net_paid * cs_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then cs_net_paid * cs_quantity else 0 end) as dec_net + from + catalog_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and cs_sold_time_sk = t_time_sk + and cs_ship_mode_sk = sm_ship_mode_sk + and d_year = 2001 + and t_time between 19072 AND 19072+28800 + and sm_carrier in ('FEDEX','GERMA') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + ) x + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + order by w_warehouse_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q67.sql b/benchmarks/tpc/queries/tpcds/q67.sql new file mode 100644 index 0000000000..50b7e125bd --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q67.sql @@ -0,0 +1,45 @@ +-- CometBench-DS query 67 derived from TPC-DS query 67 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select * +from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rank() over (partition by i_category order by sumsales desc) rk + from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales + from store_sales + ,date_dim + ,store + ,item + where ss_sold_date_sk=d_date_sk + and ss_item_sk=i_item_sk + and ss_store_sk = s_store_sk + and d_month_seq between 1194 and 1194+11 + group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2 +where rk <= 100 +order by i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rk + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q68.sql b/benchmarks/tpc/queries/tpcds/q68.sql new file mode 100644 index 0000000000..fb69c8d045 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q68.sql @@ -0,0 +1,43 @@ +-- CometBench-DS query 68 derived from TPC-DS query 68 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,extended_price + ,extended_tax + ,list_price + from (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_ext_sales_price) extended_price + ,sum(ss_ext_list_price) list_price + ,sum(ss_ext_tax) extended_tax + from store_sales + ,date_dim + ,store + ,household_demographics + ,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_dep_count = 8 or + household_demographics.hd_vehicle_count= 3) + and date_dim.d_year in (2000,2000+1,2000+2) + and store.s_city in ('Midway','Fairview') + group by ss_ticket_number + ,ss_customer_sk + ,ss_addr_sk,ca_city) dn + ,customer + ,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,ss_ticket_number + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q69.sql b/benchmarks/tpc/queries/tpcds/q69.sql new file mode 100644 index 0000000000..d9bb0ff013 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q69.sql @@ -0,0 +1,48 @@ +-- CometBench-DS query 69 derived from TPC-DS query 69 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_state in ('IN','VA','MS') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 2 and 2+2) and + (not exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 2 and 2+2) and + not exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 2 and 2+2)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q7.sql b/benchmarks/tpc/queries/tpcds/q7.sql new file mode 100644 index 0000000000..02e55dddf7 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q7.sql @@ -0,0 +1,22 @@ +-- CometBench-DS query 7 derived from TPC-DS query 7 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, item, promotion + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_cdemo_sk = cd_demo_sk and + ss_promo_sk = p_promo_sk and + cd_gender = 'M' and + cd_marital_status = 'M' and + cd_education_status = '4 yr Degree' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 2001 + group by i_item_id + order by i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q70.sql b/benchmarks/tpc/queries/tpcds/q70.sql new file mode 100644 index 0000000000..a5b67fae2a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q70.sql @@ -0,0 +1,39 @@ +-- CometBench-DS query 70 derived from TPC-DS query 70 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + sum(ss_net_profit) as total_sum + ,s_state + ,s_county + ,grouping(s_state)+grouping(s_county) as lochierarchy + ,rank() over ( + partition by grouping(s_state)+grouping(s_county), + case when grouping(s_county) = 0 then s_state end + order by sum(ss_net_profit) desc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,store + where + d1.d_month_seq between 1180 and 1180+11 + and d1.d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + and s_state in + ( select s_state + from (select s_state as s_state, + rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking + from store_sales, store, date_dim + where d_month_seq between 1180 and 1180+11 + and d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + group by s_state + ) tmp1 + where ranking <= 5 + ) + group by rollup(s_state,s_county) + order by + lochierarchy desc + ,case when lochierarchy = 0 then s_state end + ,rank_within_parent + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q71.sql b/benchmarks/tpc/queries/tpcds/q71.sql new file mode 100644 index 0000000000..04ce711a31 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q71.sql @@ -0,0 +1,41 @@ +-- CometBench-DS query 71 derived from TPC-DS query 71 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_brand_id brand_id, i_brand brand,t_hour,t_minute, + sum(ext_price) ext_price + from item, (select ws_ext_sales_price as ext_price, + ws_sold_date_sk as sold_date_sk, + ws_item_sk as sold_item_sk, + ws_sold_time_sk as time_sk + from web_sales,date_dim + where d_date_sk = ws_sold_date_sk + and d_moy=11 + and d_year=2001 + union all + select cs_ext_sales_price as ext_price, + cs_sold_date_sk as sold_date_sk, + cs_item_sk as sold_item_sk, + cs_sold_time_sk as time_sk + from catalog_sales,date_dim + where d_date_sk = cs_sold_date_sk + and d_moy=11 + and d_year=2001 + union all + select ss_ext_sales_price as ext_price, + ss_sold_date_sk as sold_date_sk, + ss_item_sk as sold_item_sk, + ss_sold_time_sk as time_sk + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + and d_moy=11 + and d_year=2001 + ) tmp,time_dim + where + sold_item_sk = i_item_sk + and i_manager_id=1 + and time_sk = t_time_sk + and (t_meal_time = 'breakfast' or t_meal_time = 'dinner') + group by i_brand, i_brand_id,t_hour,t_minute + order by ext_price desc, i_brand_id + ; + diff --git a/benchmarks/tpc/queries/tpcds/q72.sql b/benchmarks/tpc/queries/tpcds/q72.sql new file mode 100644 index 0000000000..d89d62ce9e --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q72.sql @@ -0,0 +1,30 @@ +-- CometBench-DS query 72 derived from TPC-DS query 72 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_desc + ,w_warehouse_name + ,d1.d_week_seq + ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo + ,sum(case when p_promo_sk is not null then 1 else 0 end) promo + ,count(*) total_cnt +from catalog_sales +join inventory on (cs_item_sk = inv_item_sk) +join warehouse on (w_warehouse_sk=inv_warehouse_sk) +join item on (i_item_sk = cs_item_sk) +join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) +join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) +join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) +join date_dim d2 on (inv_date_sk = d2.d_date_sk) +join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) +left outer join promotion on (cs_promo_sk=p_promo_sk) +left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) +where d1.d_week_seq = d2.d_week_seq + and inv_quantity_on_hand < cs_quantity + and d3.d_date > d1.d_date + 5 + and hd_buy_potential = '501-1000' + and d1.d_year = 1999 + and cd_marital_status = 'S' +group by i_item_desc,w_warehouse_name,d1.d_week_seq +order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q73.sql b/benchmarks/tpc/queries/tpcds/q73.sql new file mode 100644 index 0000000000..b21caa6e8a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q73.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 73 derived from TPC-DS query 73 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_buy_potential = '1001-5000' or + household_demographics.hd_buy_potential = '5001-10000') + and household_demographics.hd_vehicle_count > 0 + and case when household_demographics.hd_vehicle_count > 0 then + household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dj,customer + where ss_customer_sk = c_customer_sk + and cnt between 1 and 5 + order by cnt desc, c_last_name asc; + diff --git a/benchmarks/tpc/queries/tpcds/q74.sql b/benchmarks/tpc/queries/tpcds/q74.sql new file mode 100644 index 0000000000..04af4b3fd0 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q74.sql @@ -0,0 +1,62 @@ +-- CometBench-DS query 74 derived from TPC-DS query 74 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,stddev_samp(ss_net_paid) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2001,2001+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,stddev_samp(ws_net_paid) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + and d_year in (2001,2001+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + ) + select + t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.year = 2001 + and t_s_secyear.year = 2001+1 + and t_w_firstyear.year = 2001 + and t_w_secyear.year = 2001+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + order by 3,2,1 + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q75.sql b/benchmarks/tpc/queries/tpcds/q75.sql new file mode 100644 index 0000000000..8e739147d0 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q75.sql @@ -0,0 +1,71 @@ +-- CometBench-DS query 75 derived from TPC-DS query 75 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +WITH all_sales AS ( + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,SUM(sales_cnt) AS sales_cnt + ,SUM(sales_amt) AS sales_amt + FROM (SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt + ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt + FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk + JOIN date_dim ON d_date_sk=cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number + AND cs_item_sk=cr_item_sk) + WHERE i_category='Shoes' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt + ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt + FROM store_sales JOIN item ON i_item_sk=ss_item_sk + JOIN date_dim ON d_date_sk=ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number + AND ss_item_sk=sr_item_sk) + WHERE i_category='Shoes' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt + ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt + FROM web_sales JOIN item ON i_item_sk=ws_item_sk + JOIN date_dim ON d_date_sk=ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number=wr_order_number + AND ws_item_sk=wr_item_sk) + WHERE i_category='Shoes') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) + SELECT prev_yr.d_year AS prev_year + ,curr_yr.d_year AS year + ,curr_yr.i_brand_id + ,curr_yr.i_class_id + ,curr_yr.i_category_id + ,curr_yr.i_manufact_id + ,prev_yr.sales_cnt AS prev_yr_cnt + ,curr_yr.sales_cnt AS curr_yr_cnt + ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff + ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff + FROM all_sales curr_yr, all_sales prev_yr + WHERE curr_yr.i_brand_id=prev_yr.i_brand_id + AND curr_yr.i_class_id=prev_yr.i_class_id + AND curr_yr.i_category_id=prev_yr.i_category_id + AND curr_yr.i_manufact_id=prev_yr.i_manufact_id + AND curr_yr.d_year=2000 + AND prev_yr.d_year=2000-1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 + ORDER BY sales_cnt_diff,sales_amt_diff + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q76.sql b/benchmarks/tpc/queries/tpcds/q76.sql new file mode 100644 index 0000000000..98fc8ddb2a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q76.sql @@ -0,0 +1,25 @@ +-- CometBench-DS query 76 derived from TPC-DS query 76 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( + SELECT 'store' as channel, 'ss_customer_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_customer_sk IS NULL + AND ss_sold_date_sk=d_date_sk + AND ss_item_sk=i_item_sk + UNION ALL + SELECT 'web' as channel, 'ws_ship_hdemo_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_ship_hdemo_sk IS NULL + AND ws_sold_date_sk=d_date_sk + AND ws_item_sk=i_item_sk + UNION ALL + SELECT 'catalog' as channel, 'cs_bill_customer_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_bill_customer_sk IS NULL + AND cs_sold_date_sk=d_date_sk + AND cs_item_sk=i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q77.sql b/benchmarks/tpc/queries/tpcds/q77.sql new file mode 100644 index 0000000000..81e8ccf80d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q77.sql @@ -0,0 +1,109 @@ +-- CometBench-DS query 77 derived from TPC-DS query 77 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ss as + (select s_store_sk, + sum(ss_ext_sales_price) as sales, + sum(ss_net_profit) as profit + from store_sales, + date_dim, + store + where ss_sold_date_sk = d_date_sk + and d_date between cast('2001-08-11' as date) + and (cast('2001-08-11' as date) + INTERVAL '30 DAYS') + and ss_store_sk = s_store_sk + group by s_store_sk) + , + sr as + (select s_store_sk, + sum(sr_return_amt) as returns, + sum(sr_net_loss) as profit_loss + from store_returns, + date_dim, + store + where sr_returned_date_sk = d_date_sk + and d_date between cast('2001-08-11' as date) + and (cast('2001-08-11' as date) + INTERVAL '30 DAYS') + and sr_store_sk = s_store_sk + group by s_store_sk), + cs as + (select cs_call_center_sk, + sum(cs_ext_sales_price) as sales, + sum(cs_net_profit) as profit + from catalog_sales, + date_dim + where cs_sold_date_sk = d_date_sk + and d_date between cast('2001-08-11' as date) + and (cast('2001-08-11' as date) + INTERVAL '30 DAYS') + group by cs_call_center_sk + ), + cr as + (select cr_call_center_sk, + sum(cr_return_amount) as returns, + sum(cr_net_loss) as profit_loss + from catalog_returns, + date_dim + where cr_returned_date_sk = d_date_sk + and d_date between cast('2001-08-11' as date) + and (cast('2001-08-11' as date) + INTERVAL '30 DAYS') + group by cr_call_center_sk + ), + ws as + ( select wp_web_page_sk, + sum(ws_ext_sales_price) as sales, + sum(ws_net_profit) as profit + from web_sales, + date_dim, + web_page + where ws_sold_date_sk = d_date_sk + and d_date between cast('2001-08-11' as date) + and (cast('2001-08-11' as date) + INTERVAL '30 DAYS') + and ws_web_page_sk = wp_web_page_sk + group by wp_web_page_sk), + wr as + (select wp_web_page_sk, + sum(wr_return_amt) as returns, + sum(wr_net_loss) as profit_loss + from web_returns, + date_dim, + web_page + where wr_returned_date_sk = d_date_sk + and d_date between cast('2001-08-11' as date) + and (cast('2001-08-11' as date) + INTERVAL '30 DAYS') + and wr_web_page_sk = wp_web_page_sk + group by wp_web_page_sk) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , ss.s_store_sk as id + , sales + , coalesce(returns, 0) as returns + , (profit - coalesce(profit_loss,0)) as profit + from ss left join sr + on ss.s_store_sk = sr.s_store_sk + union all + select 'catalog channel' as channel + , cs_call_center_sk as id + , sales + , returns + , (profit - profit_loss) as profit + from cs + , cr + union all + select 'web channel' as channel + , ws.wp_web_page_sk as id + , sales + , coalesce(returns, 0) returns + , (profit - coalesce(profit_loss,0)) as profit + from ws left join wr + on ws.wp_web_page_sk = wr.wp_web_page_sk + ) x + group by rollup (channel, id) + order by channel + ,id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q78.sql b/benchmarks/tpc/queries/tpcds/q78.sql new file mode 100644 index 0000000000..2cb7a97da2 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q78.sql @@ -0,0 +1,59 @@ +-- CometBench-DS query 78 derived from TPC-DS query 78 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ws as + (select d_year AS ws_sold_year, ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + from web_sales + left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk + join date_dim on ws_sold_date_sk = d_date_sk + where wr_order_number is null + group by d_year, ws_item_sk, ws_bill_customer_sk + ), +cs as + (select d_year AS cs_sold_year, cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + from catalog_sales + left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk + join date_dim on cs_sold_date_sk = d_date_sk + where cr_order_number is null + group by d_year, cs_item_sk, cs_bill_customer_sk + ), +ss as + (select d_year AS ss_sold_year, ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + from store_sales + left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk + join date_dim on ss_sold_date_sk = d_date_sk + where sr_ticket_number is null + group by d_year, ss_item_sk, ss_customer_sk + ) + select +ss_customer_sk, +round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio, +ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price, +coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty, +coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost, +coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price +from ss +left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk) +left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk) +where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=2001 +order by + ss_customer_sk, + ss_qty desc, ss_wc desc, ss_sp desc, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + ratio + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q79.sql b/benchmarks/tpc/queries/tpcds/q79.sql new file mode 100644 index 0000000000..3680d08400 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q79.sql @@ -0,0 +1,24 @@ +-- CometBench-DS query 79 derived from TPC-DS query 79 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,store.s_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (household_demographics.hd_dep_count = 0 or household_demographics.hd_vehicle_count > 4) + and date_dim.d_dow = 1 + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_number_employees between 200 and 295 + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer + where ss_customer_sk = c_customer_sk + order by c_last_name,c_first_name,substr(s_city,1,30), profit + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q8.sql b/benchmarks/tpc/queries/tpcds/q8.sql new file mode 100644 index 0000000000..63235faf45 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q8.sql @@ -0,0 +1,109 @@ +-- CometBench-DS query 8 derived from TPC-DS query 8 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select s_store_name + ,sum(ss_net_profit) + from store_sales + ,date_dim + ,store, + (select ca_zip + from ( + SELECT substr(ca_zip,1,5) ca_zip + FROM customer_address + WHERE substr(ca_zip,1,5) IN ( + '19100','41548','51640','49699','88329','55986', + '85119','19510','61020','95452','26235', + '51102','16733','42819','27823','90192', + '31905','28865','62197','23750','81398', + '95288','45114','82060','12313','25218', + '64386','46400','77230','69271','43672', + '36521','34217','13017','27936','42766', + '59233','26060','27477','39981','93402', + '74270','13932','51731','71642','17710', + '85156','21679','70840','67191','39214', + '35273','27293','17128','15458','31615', + '60706','67657','54092','32775','14683', + '32206','62543','43053','11297','58216', + '49410','14710','24501','79057','77038', + '91286','32334','46298','18326','67213', + '65382','40315','56115','80162','55956', + '81583','73588','32513','62880','12201', + '11592','17014','83832','61796','57872', + '78829','69912','48524','22016','26905', + '48511','92168','63051','25748','89786', + '98827','86404','53029','37524','14039', + '50078','34487','70142','18697','40129', + '60642','42810','62667','57183','46414', + '58463','71211','46364','34851','54884', + '25382','25239','74126','21568','84204', + '13607','82518','32982','36953','86001', + '79278','21745','64444','35199','83181', + '73255','86177','98043','90392','13882', + '47084','17859','89526','42072','20233', + '52745','75000','22044','77013','24182', + '52554','56138','43440','86100','48791', + '21883','17096','15965','31196','74903', + '19810','35763','92020','55176','54433', + '68063','71919','44384','16612','32109', + '28207','14762','89933','10930','27616', + '56809','14244','22733','33177','29784', + '74968','37887','11299','34692','85843', + '83663','95421','19323','17406','69264', + '28341','50150','79121','73974','92917', + '21229','32254','97408','46011','37169', + '18146','27296','62927','68812','47734', + '86572','12620','80252','50173','27261', + '29534','23488','42184','23695','45868', + '12910','23429','29052','63228','30731', + '15747','25827','22332','62349','56661', + '44652','51862','57007','22773','40361', + '65238','19327','17282','44708','35484', + '34064','11148','92729','22995','18833', + '77528','48917','17256','93166','68576', + '71096','56499','35096','80551','82424', + '17700','32748','78969','46820','57725', + '46179','54677','98097','62869','83959', + '66728','19716','48326','27420','53458', + '69056','84216','36688','63957','41469', + '66843','18024','81950','21911','58387', + '58103','19813','34581','55347','17171', + '35914','75043','75088','80541','26802', + '28849','22356','57721','77084','46385', + '59255','29308','65885','70673','13306', + '68788','87335','40987','31654','67560', + '92309','78116','65961','45018','16548', + '67092','21818','33716','49449','86150', + '12156','27574','43201','50977','52839', + '33234','86611','71494','17823','57172', + '59869','34086','51052','11320','39717', + '79604','24672','70555','38378','91135', + '15567','21606','74994','77168','38607', + '27384','68328','88944','40203','37893', + '42726','83549','48739','55652','27543', + '23109','98908','28831','45011','47525', + '43870','79404','35780','42136','49317', + '14574','99586','21107','14302','83882', + '81272','92552','14916','87533','86518', + '17862','30741','96288','57886','30304', + '24201','79457','36728','49833','35182', + '20108','39858','10804','47042','20439', + '54708','59027','82499','75311','26548', + '53406','92060','41152','60446','33129', + '43979','16903','60319','35550','33887', + '25463','40343','20726','44429') + intersect + select ca_zip + from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk and + c_preferred_cust_flag='Y' + group by ca_zip + having count(*) > 10)A1)A2) V1 + where ss_store_sk = s_store_sk + and ss_sold_date_sk = d_date_sk + and d_qoy = 1 and d_year = 2000 + and (substr(s_zip,1,2) = substr(V1.ca_zip,1,2)) + group by s_store_name + order by s_store_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q80.sql b/benchmarks/tpc/queries/tpcds/q80.sql new file mode 100644 index 0000000000..8f0a9c63d5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q80.sql @@ -0,0 +1,97 @@ +-- CometBench-DS query 80 derived from TPC-DS query 80 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ssr as + (select s_store_id as store_id, + sum(ss_ext_sales_price) as sales, + sum(coalesce(sr_return_amt, 0)) as returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit + from store_sales left outer join store_returns on + (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number), + date_dim, + store, + item, + promotion + where ss_sold_date_sk = d_date_sk + and d_date between cast('2002-08-04' as date) + and (cast('2002-08-04' as date) + INTERVAL '30 DAYS') + and ss_store_sk = s_store_sk + and ss_item_sk = i_item_sk + and i_current_price > 50 + and ss_promo_sk = p_promo_sk + and p_channel_tv = 'N' + group by s_store_id) + , + csr as + (select cp_catalog_page_id as catalog_page_id, + sum(cs_ext_sales_price) as sales, + sum(coalesce(cr_return_amount, 0)) as returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit + from catalog_sales left outer join catalog_returns on + (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number), + date_dim, + catalog_page, + item, + promotion + where cs_sold_date_sk = d_date_sk + and d_date between cast('2002-08-04' as date) + and (cast('2002-08-04' as date) + INTERVAL '30 DAYS') + and cs_catalog_page_sk = cp_catalog_page_sk + and cs_item_sk = i_item_sk + and i_current_price > 50 + and cs_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(ws_ext_sales_price) as sales, + sum(coalesce(wr_return_amt, 0)) as returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit + from web_sales left outer join web_returns on + (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number), + date_dim, + web_site, + item, + promotion + where ws_sold_date_sk = d_date_sk + and d_date between cast('2002-08-04' as date) + and (cast('2002-08-04' as date) + INTERVAL '30 DAYS') + and ws_web_site_sk = web_site_sk + and ws_item_sk = i_item_sk + and i_current_price > 50 + and ws_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , 'store' || store_id as id + , sales + , returns + , profit + from ssr + union all + select 'catalog channel' as channel + , 'catalog_page' || catalog_page_id as id + , sales + , returns + , profit + from csr + union all + select 'web channel' as channel + , 'web_site' || web_site_id as id + , sales + , returns + , profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q81.sql b/benchmarks/tpc/queries/tpcds/q81.sql new file mode 100644 index 0000000000..a768764099 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q81.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 81 derived from TPC-DS query 81 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with customer_total_return as + (select cr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(cr_return_amt_inc_tax) as ctr_total_return + from catalog_returns + ,date_dim + ,customer_address + where cr_returned_date_sk = d_date_sk + and d_year =1998 + and cr_returning_addr_sk = ca_address_sk + group by cr_returning_customer_sk + ,ca_state ) + select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'TX' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q82.sql b/benchmarks/tpc/queries/tpcds/q82.sql new file mode 100644 index 0000000000..98c002cec6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q82.sql @@ -0,0 +1,18 @@ +-- CometBench-DS query 82 derived from TPC-DS query 82 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, store_sales + where i_current_price between 69 and 69+30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('1998-06-06' as date) and (cast('1998-06-06' as date) + INTERVAL '60 DAYS') + and i_manufact_id in (105,513,180,137) + and inv_quantity_on_hand between 100 and 500 + and ss_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q83.sql b/benchmarks/tpc/queries/tpcds/q83.sql new file mode 100644 index 0000000000..85f25c7eea --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q83.sql @@ -0,0 +1,68 @@ +-- CometBench-DS query 83 derived from TPC-DS query 83 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with sr_items as + (select i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + from store_returns, + item, + date_dim + where sr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2000-04-29','2000-09-09','2000-11-02'))) + and sr_returned_date_sk = d_date_sk + group by i_item_id), + cr_items as + (select i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + from catalog_returns, + item, + date_dim + where cr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2000-04-29','2000-09-09','2000-11-02'))) + and cr_returned_date_sk = d_date_sk + group by i_item_id), + wr_items as + (select i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + from web_returns, + item, + date_dim + where wr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('2000-04-29','2000-09-09','2000-11-02'))) + and wr_returned_date_sk = d_date_sk + group by i_item_id) + select sr_items.item_id + ,sr_item_qty + ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev + ,cr_item_qty + ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev + ,wr_item_qty + ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev + ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average + from sr_items + ,cr_items + ,wr_items + where sr_items.item_id=cr_items.item_id + and sr_items.item_id=wr_items.item_id + order by sr_items.item_id + ,sr_item_qty + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q84.sql b/benchmarks/tpc/queries/tpcds/q84.sql new file mode 100644 index 0000000000..89fa1e44df --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q84.sql @@ -0,0 +1,22 @@ +-- CometBench-DS query 84 derived from TPC-DS query 84 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select c_customer_id as customer_id + , coalesce(c_last_name,'') || ', ' || coalesce(c_first_name,'') as customername + from customer + ,customer_address + ,customer_demographics + ,household_demographics + ,income_band + ,store_returns + where ca_city = 'White Oak' + and c_current_addr_sk = ca_address_sk + and ib_lower_bound >= 45626 + and ib_upper_bound <= 45626 + 50000 + and ib_income_band_sk = hd_income_band_sk + and cd_demo_sk = c_current_cdemo_sk + and hd_demo_sk = c_current_hdemo_sk + and sr_cdemo_sk = cd_demo_sk + order by c_customer_id + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q85.sql b/benchmarks/tpc/queries/tpcds/q85.sql new file mode 100644 index 0000000000..54e53149c5 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q85.sql @@ -0,0 +1,85 @@ +-- CometBench-DS query 85 derived from TPC-DS query 85 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) + from web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason + where ws_web_page_sk = wp_web_page_sk + and ws_item_sk = wr_item_sk + and ws_order_number = wr_order_number + and ws_sold_date_sk = d_date_sk and d_year = 2001 + and cd1.cd_demo_sk = wr_refunded_cdemo_sk + and cd2.cd_demo_sk = wr_returning_cdemo_sk + and ca_address_sk = wr_refunded_addr_sk + and r_reason_sk = wr_reason_sk + and + ( + ( + cd1.cd_marital_status = 'D' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Primary' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 100.00 and 150.00 + ) + or + ( + cd1.cd_marital_status = 'U' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Unknown' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 50.00 and 100.00 + ) + or + ( + cd1.cd_marital_status = 'M' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Advanced Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ca_country = 'United States' + and + ca_state in ('SC', 'IN', 'VA') + and ws_net_profit between 100 and 200 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('WA', 'KS', 'KY') + and ws_net_profit between 150 and 300 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('SD', 'WI', 'NE') + and ws_net_profit between 50 and 250 + ) + ) +group by r_reason_desc +order by substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q86.sql b/benchmarks/tpc/queries/tpcds/q86.sql new file mode 100644 index 0000000000..c8d2d08b84 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q86.sql @@ -0,0 +1,27 @@ +-- CometBench-DS query 86 derived from TPC-DS query 86 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + sum(ws_net_paid) as total_sum + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ws_net_paid) desc) as rank_within_parent + from + web_sales + ,date_dim d1 + ,item + where + d1.d_month_seq between 1205 and 1205+11 + and d1.d_date_sk = ws_sold_date_sk + and i_item_sk = ws_item_sk + group by rollup(i_category,i_class) + order by + lochierarchy desc, + case when lochierarchy = 0 then i_category end, + rank_within_parent + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q87.sql b/benchmarks/tpc/queries/tpcds/q87.sql new file mode 100644 index 0000000000..86e50e9ba6 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q87.sql @@ -0,0 +1,24 @@ +-- CometBench-DS query 87 derived from TPC-DS query 87 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select count(*) +from ((select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189+11) + except + (select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189+11) + except + (select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1189 and 1189+11) +) cool_cust +; + diff --git a/benchmarks/tpc/queries/tpcds/q88.sql b/benchmarks/tpc/queries/tpcds/q88.sql new file mode 100644 index 0000000000..d663b2738a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q88.sql @@ -0,0 +1,95 @@ +-- CometBench-DS query 88 derived from TPC-DS query 88 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select * +from + (select count(*) h8_30_to_9 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s1, + (select count(*) h9_to_9_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s2, + (select count(*) h9_30_to_10 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s3, + (select count(*) h10_to_10_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s4, + (select count(*) h10_30_to_11 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s5, + (select count(*) h11_to_11_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s6, + (select count(*) h11_30_to_12 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s7, + (select count(*) h12_to_12_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 12 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2) or + (household_demographics.hd_dep_count = 4 and household_demographics.hd_vehicle_count<=4+2)) + and store.s_store_name = 'ese') s8 +; + diff --git a/benchmarks/tpc/queries/tpcds/q89.sql b/benchmarks/tpc/queries/tpcds/q89.sql new file mode 100644 index 0000000000..a9359808f0 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q89.sql @@ -0,0 +1,29 @@ +-- CometBench-DS query 89 derived from TPC-DS query 89 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select * +from( +select i_category, i_class, i_brand, + s_store_name, s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + d_year in (2001) and + ((i_category in ('Children','Jewelry','Home') and + i_class in ('infants','birdal','flatware') + ) + or (i_category in ('Electronics','Music','Books') and + i_class in ('audio','classical','science') + )) +group by i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 +order by sum_sales - avg_monthly_sales, s_store_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q9.sql b/benchmarks/tpc/queries/tpcds/q9.sql new file mode 100644 index 0000000000..7dab30e028 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q9.sql @@ -0,0 +1,52 @@ +-- CometBench-DS query 9 derived from TPC-DS query 9 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select case when (select count(*) + from store_sales + where ss_quantity between 1 and 20) > 31002 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 1 and 20) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 1 and 20) end bucket1 , + case when (select count(*) + from store_sales + where ss_quantity between 21 and 40) > 588 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 21 and 40) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 21 and 40) end bucket2, + case when (select count(*) + from store_sales + where ss_quantity between 41 and 60) > 2456 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 41 and 60) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 41 and 60) end bucket3, + case when (select count(*) + from store_sales + where ss_quantity between 61 and 80) > 21645 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 61 and 80) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 61 and 80) end bucket4, + case when (select count(*) + from store_sales + where ss_quantity between 81 and 100) > 20553 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 81 and 100) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 81 and 100) end bucket5 +from reason +where r_reason_sk = 1 +; + diff --git a/benchmarks/tpc/queries/tpcds/q90.sql b/benchmarks/tpc/queries/tpcds/q90.sql new file mode 100644 index 0000000000..6c296c313d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q90.sql @@ -0,0 +1,23 @@ +-- CometBench-DS query 90 derived from TPC-DS query 90 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio + from ( select count(*) amc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 9 and 9+1 + and household_demographics.hd_dep_count = 2 + and web_page.wp_char_count between 5000 and 5200) at, + ( select count(*) pmc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 15 and 15+1 + and household_demographics.hd_dep_count = 2 + and web_page.wp_char_count between 5000 and 5200) pt + order by am_pm_ratio + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q91.sql b/benchmarks/tpc/queries/tpcds/q91.sql new file mode 100644 index 0000000000..a9ebbc090a --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q91.sql @@ -0,0 +1,32 @@ +-- CometBench-DS query 91 derived from TPC-DS query 91 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +from + call_center, + catalog_returns, + date_dim, + customer, + customer_address, + customer_demographics, + household_demographics +where + cr_call_center_sk = cc_call_center_sk +and cr_returned_date_sk = d_date_sk +and cr_returning_customer_sk= c_customer_sk +and cd_demo_sk = c_current_cdemo_sk +and hd_demo_sk = c_current_hdemo_sk +and ca_address_sk = c_current_addr_sk +and d_year = 2002 +and d_moy = 11 +and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') + or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) +and hd_buy_potential like 'Unknown%' +and ca_gmt_offset = -6 +group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status +order by sum(cr_net_loss) desc; + diff --git a/benchmarks/tpc/queries/tpcds/q92.sql b/benchmarks/tpc/queries/tpcds/q92.sql new file mode 100644 index 0000000000..63dab6f474 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q92.sql @@ -0,0 +1,31 @@ +-- CometBench-DS query 92 derived from TPC-DS query 92 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + sum(ws_ext_discount_amt) as `Excess Discount Amount` +from + web_sales + ,item + ,date_dim +where +i_manufact_id = 914 +and i_item_sk = ws_item_sk +and d_date between '2001-01-25' and + (cast('2001-01-25' as date) + INTERVAL '90 DAYS') +and d_date_sk = ws_sold_date_sk +and ws_ext_discount_amt + > ( + SELECT + 1.3 * avg(ws_ext_discount_amt) + FROM + web_sales + ,date_dim + WHERE + ws_item_sk = i_item_sk + and d_date between '2001-01-25' and + (cast('2001-01-25' as date) + INTERVAL '90 DAYS') + and d_date_sk = ws_sold_date_sk + ) +order by sum(ws_ext_discount_amt) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q93.sql b/benchmarks/tpc/queries/tpcds/q93.sql new file mode 100644 index 0000000000..e09acd8505 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q93.sql @@ -0,0 +1,19 @@ +-- CometBench-DS query 93 derived from TPC-DS query 93 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select ss_customer_sk + ,sum(act_sales) sumsales + from (select ss_item_sk + ,ss_ticket_number + ,ss_customer_sk + ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price + else (ss_quantity*ss_sales_price) end act_sales + from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk + and sr_ticket_number = ss_ticket_number) + ,reason + where sr_reason_sk = r_reason_sk + and r_reason_desc = 'Did not get it on time') t + group by ss_customer_sk + order by sumsales, ss_customer_sk + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q94.sql b/benchmarks/tpc/queries/tpcds/q94.sql new file mode 100644 index 0000000000..79e528352d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q94.sql @@ -0,0 +1,30 @@ +-- CometBench-DS query 94 derived from TPC-DS query 94 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + count(distinct ws_order_number) as `order count` + ,sum(ws_ext_ship_cost) as `total shipping cost` + ,sum(ws_net_profit) as `total net profit` +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '1999-4-01' and + (cast('1999-4-01' as date) + INTERVAL '60 DAYS') +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'WI' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and exists (select * + from web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +and not exists(select * + from web_returns wr1 + where ws1.ws_order_number = wr1.wr_order_number) +order by count(distinct ws_order_number) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q95.sql b/benchmarks/tpc/queries/tpcds/q95.sql new file mode 100644 index 0000000000..f29717711b --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q95.sql @@ -0,0 +1,33 @@ +-- CometBench-DS query 95 derived from TPC-DS query 95 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ws_wh as +(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 + from web_sales ws1,web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + select + count(distinct ws_order_number) as `order count` + ,sum(ws_ext_ship_cost) as `total shipping cost` + ,sum(ws_net_profit) as `total net profit` +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '2002-5-01' and + (cast('2002-5-01' as date) + INTERVAL '60 DAYS') +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'MA' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and ws1.ws_order_number in (select ws_order_number + from ws_wh) +and ws1.ws_order_number in (select wr_order_number + from web_returns,ws_wh + where wr_order_number = ws_wh.ws_order_number) +order by count(distinct ws_order_number) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q96.sql b/benchmarks/tpc/queries/tpcds/q96.sql new file mode 100644 index 0000000000..9642a924e1 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q96.sql @@ -0,0 +1,17 @@ +-- CometBench-DS query 96 derived from TPC-DS query 96 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select count(*) +from store_sales + ,household_demographics + ,time_dim, store +where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and household_demographics.hd_dep_count = 5 + and store.s_store_name = 'ese' +order by count(*) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q97.sql b/benchmarks/tpc/queries/tpcds/q97.sql new file mode 100644 index 0000000000..10b29bd126 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q97.sql @@ -0,0 +1,26 @@ +-- CometBench-DS query 97 derived from TPC-DS query 97 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +with ssci as ( +select ss_customer_sk customer_sk + ,ss_item_sk item_sk +from store_sales,date_dim +where ss_sold_date_sk = d_date_sk + and d_month_seq between 1211 and 1211 + 11 +group by ss_customer_sk + ,ss_item_sk), +csci as( + select cs_bill_customer_sk customer_sk + ,cs_item_sk item_sk +from catalog_sales,date_dim +where cs_sold_date_sk = d_date_sk + and d_month_seq between 1211 and 1211 + 11 +group by cs_bill_customer_sk + ,cs_item_sk) + select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only + ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only + ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog +from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk + and ssci.item_sk = csci.item_sk) + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpcds/q98.sql b/benchmarks/tpc/queries/tpcds/q98.sql new file mode 100644 index 0000000000..cfaf54de3d --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q98.sql @@ -0,0 +1,34 @@ +-- CometBench-DS query 98 derived from TPC-DS query 98 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ss_ext_sales_price) as itemrevenue + ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over + (partition by i_class) as revenueratio +from + store_sales + ,item + ,date_dim +where + ss_item_sk = i_item_sk + and i_category in ('Shoes', 'Music', 'Men') + and ss_sold_date_sk = d_date_sk + and d_date between cast('2000-01-05' as date) + and (cast('2000-01-05' as date) + INTERVAL '30 DAYS') +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio; + diff --git a/benchmarks/tpc/queries/tpcds/q99.sql b/benchmarks/tpc/queries/tpcds/q99.sql new file mode 100644 index 0000000000..a72305b0b7 --- /dev/null +++ b/benchmarks/tpc/queries/tpcds/q99.sql @@ -0,0 +1,36 @@ +-- CometBench-DS query 99 derived from TPC-DS query 99 under the terms of the TPC Fair Use Policy. +-- TPC-DS queries are Copyright 2021 Transaction Processing Performance Council. +-- This query was generated at scale factor 1. +select + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as `30 days` + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and + (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as `31-60 days` + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and + (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as `61-90 days` + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and + (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as `91-120 days` + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as `>120 days` +from + catalog_sales + ,warehouse + ,ship_mode + ,call_center + ,date_dim +where + d_month_seq between 1188 and 1188 + 11 +and cs_ship_date_sk = d_date_sk +and cs_warehouse_sk = w_warehouse_sk +and cs_ship_mode_sk = sm_ship_mode_sk +and cs_call_center_sk = cc_call_center_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name + LIMIT 100; + diff --git a/benchmarks/tpc/queries/tpch/q1.sql b/benchmarks/tpc/queries/tpch/q1.sql new file mode 100644 index 0000000000..f24b0e0615 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q1.sql @@ -0,0 +1,23 @@ +-- CometBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order +from + lineitem +where + l_shipdate <= date '1998-12-01' - interval '68 days' +group by + l_returnflag, + l_linestatus +order by + l_returnflag, + l_linestatus; diff --git a/benchmarks/tpc/queries/tpch/q10.sql b/benchmarks/tpc/queries/tpch/q10.sql new file mode 100644 index 0000000000..8475ae5fa2 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q10.sql @@ -0,0 +1,33 @@ +-- CometBench-H query 10 derived from TPC-H query 10 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_custkey, + c_name, + sum(l_extendedprice * (1 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment +from + customer, + orders, + lineitem, + nation +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1993-07-01' + and o_orderdate < date '1993-07-01' + interval '3' month + and l_returnflag = 'R' + and c_nationkey = n_nationkey +group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment +order by + revenue desc limit 20; diff --git a/benchmarks/tpc/queries/tpch/q11.sql b/benchmarks/tpc/queries/tpch/q11.sql new file mode 100644 index 0000000000..314b1e6fc7 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q11.sql @@ -0,0 +1,29 @@ +-- CometBench-H query 11 derived from TPC-H query 11 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value +from + partsupp, + supplier, + nation +where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ALGERIA' +group by + ps_partkey having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001000000 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ALGERIA' + ) +order by + value desc; diff --git a/benchmarks/tpc/queries/tpch/q12.sql b/benchmarks/tpc/queries/tpch/q12.sql new file mode 100644 index 0000000000..f1abd806aa --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q12.sql @@ -0,0 +1,30 @@ +-- CometBench-H query 12 derived from TPC-H query 12 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count +from + orders, + lineitem +where + o_orderkey = l_orderkey + and l_shipmode in ('FOB', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1995-01-01' + and l_receiptdate < date '1995-01-01' + interval '1' year +group by + l_shipmode +order by + l_shipmode; diff --git a/benchmarks/tpc/queries/tpch/q13.sql b/benchmarks/tpc/queries/tpch/q13.sql new file mode 100644 index 0000000000..03c3f05818 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q13.sql @@ -0,0 +1,22 @@ +-- CometBench-H query 13 derived from TPC-H query 13 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_count, + count(*) as custdist +from + ( + select + c_custkey, + count(o_orderkey) + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%express%requests%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) +group by + c_count +order by + custdist desc, + c_count desc; diff --git a/benchmarks/tpc/queries/tpch/q14.sql b/benchmarks/tpc/queries/tpch/q14.sql new file mode 100644 index 0000000000..9f7b52e548 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q14.sql @@ -0,0 +1,15 @@ +-- CometBench-H query 14 derived from TPC-H query 14 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue +from + lineitem, + part +where + l_partkey = p_partkey + and l_shipdate >= date '1995-02-01' + and l_shipdate < date '1995-02-01' + interval '1' month; diff --git a/benchmarks/tpc/queries/tpch/q15.sql b/benchmarks/tpc/queries/tpch/q15.sql new file mode 100644 index 0000000000..dda91adf7b --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q15.sql @@ -0,0 +1,33 @@ +-- CometBench-H query 15 derived from TPC-H query 15 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +create view revenue0 (supplier_no, total_revenue) as + select + l_suppkey, + sum(l_extendedprice * (1 - l_discount)) + from + lineitem + where + l_shipdate >= date '1996-08-01' + and l_shipdate < date '1996-08-01' + interval '3' month + group by + l_suppkey; +select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue +from + supplier, + revenue0 +where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) +order by + s_suppkey; +drop view revenue0; diff --git a/benchmarks/tpc/queries/tpch/q16.sql b/benchmarks/tpc/queries/tpch/q16.sql new file mode 100644 index 0000000000..6f6b023acb --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q16.sql @@ -0,0 +1,32 @@ +-- CometBench-H query 16 derived from TPC-H query 16 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt +from + partsupp, + part +where + p_partkey = ps_partkey + and p_brand <> 'Brand#14' + and p_type not like 'SMALL PLATED%' + and p_size in (14, 6, 5, 31, 49, 15, 41, 47) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' + ) +group by + p_brand, + p_type, + p_size +order by + supplier_cnt desc, + p_brand, + p_type, + p_size; diff --git a/benchmarks/tpc/queries/tpch/q17.sql b/benchmarks/tpc/queries/tpch/q17.sql new file mode 100644 index 0000000000..8b90e18002 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q17.sql @@ -0,0 +1,19 @@ +-- CometBench-H query 17 derived from TPC-H query 17 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice) / 7.0 as avg_yearly +from + lineitem, + part +where + p_partkey = l_partkey + and p_brand = 'Brand#42' + and p_container = 'LG BAG' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey + ); diff --git a/benchmarks/tpc/queries/tpch/q18.sql b/benchmarks/tpc/queries/tpch/q18.sql new file mode 100644 index 0000000000..e0f733e088 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q18.sql @@ -0,0 +1,34 @@ +-- CometBench-H query 18 derived from TPC-H query 18 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) +from + customer, + orders, + lineitem +where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey having + sum(l_quantity) > 313 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey +group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice +order by + o_totalprice desc, + o_orderdate limit 100; diff --git a/benchmarks/tpc/queries/tpch/q19.sql b/benchmarks/tpc/queries/tpch/q19.sql new file mode 100644 index 0000000000..8777b2c887 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q19.sql @@ -0,0 +1,37 @@ +-- CometBench-H query 19 derived from TPC-H query 19 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice* (1 - l_discount)) as revenue +from + lineitem, + part +where + ( + p_partkey = l_partkey + and p_brand = 'Brand#21' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 8 and l_quantity <= 8 + 10 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#13' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 20 and l_quantity <= 20 + 10 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#52' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 30 and l_quantity <= 30 + 10 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); diff --git a/benchmarks/tpc/queries/tpch/q2.sql b/benchmarks/tpc/queries/tpch/q2.sql new file mode 100644 index 0000000000..947413fcf5 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q2.sql @@ -0,0 +1,45 @@ +-- CometBench-H query 2 derived from TPC-H query 2 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment +from + part, + supplier, + partsupp, + nation, + region +where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 48 + and p_type like '%TIN' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + ) +order by + s_acctbal desc, + n_name, + s_name, + p_partkey limit 100; diff --git a/benchmarks/tpc/queries/tpch/q20.sql b/benchmarks/tpc/queries/tpch/q20.sql new file mode 100644 index 0000000000..d83bfcc696 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q20.sql @@ -0,0 +1,39 @@ +-- CometBench-H query 20 derived from TPC-H query 20 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_name, + s_address +from + supplier, + nation +where + s_suppkey in ( + select + ps_suppkey + from + partsupp + where + ps_partkey in ( + select + p_partkey + from + part + where + p_name like 'blanched%' + ) + and ps_availqty > ( + select + 0.5 * sum(l_quantity) + from + lineitem + where + l_partkey = ps_partkey + and l_suppkey = ps_suppkey + and l_shipdate >= date '1993-01-01' + and l_shipdate < date '1993-01-01' + interval '1' year + ) + ) + and s_nationkey = n_nationkey + and n_name = 'KENYA' +order by + s_name; diff --git a/benchmarks/tpc/queries/tpch/q21.sql b/benchmarks/tpc/queries/tpch/q21.sql new file mode 100644 index 0000000000..ce6ade49a3 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q21.sql @@ -0,0 +1,41 @@ +-- CometBench-H query 21 derived from TPC-H query 21 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + s_name, + count(*) as numwait +from + supplier, + lineitem l1, + orders, + nation +where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' +group by + s_name +order by + numwait desc, + s_name limit 100; diff --git a/benchmarks/tpc/queries/tpch/q22.sql b/benchmarks/tpc/queries/tpch/q22.sql new file mode 100644 index 0000000000..012f1f698a --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q22.sql @@ -0,0 +1,39 @@ +-- CometBench-H query 22 derived from TPC-H query 22 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal +from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('24', '34', '16', '30', '33', '14', '13') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00 + and substring(c_phone from 1 for 2) in + ('24', '34', '16', '30', '33', '14', '13') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale +group by + cntrycode +order by + cntrycode; diff --git a/benchmarks/tpc/queries/tpch/q3.sql b/benchmarks/tpc/queries/tpch/q3.sql new file mode 100644 index 0000000000..e277402d88 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q3.sql @@ -0,0 +1,24 @@ +-- CometBench-H query 3 derived from TPC-H query 3 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority +from + customer, + orders, + lineitem +where + c_mktsegment = 'BUILDING' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-15' + and l_shipdate > date '1995-03-15' +group by + l_orderkey, + o_orderdate, + o_shippriority +order by + revenue desc, + o_orderdate limit 10; diff --git a/benchmarks/tpc/queries/tpch/q4.sql b/benchmarks/tpc/queries/tpch/q4.sql new file mode 100644 index 0000000000..814ea8a2ec --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q4.sql @@ -0,0 +1,23 @@ +-- CometBench-H query 4 derived from TPC-H query 4 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + o_orderpriority, + count(*) as order_count +from + orders +where + o_orderdate >= date '1995-04-01' + and o_orderdate < date '1995-04-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) +group by + o_orderpriority +order by + o_orderpriority; diff --git a/benchmarks/tpc/queries/tpch/q5.sql b/benchmarks/tpc/queries/tpch/q5.sql new file mode 100644 index 0000000000..5c462c397b --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q5.sql @@ -0,0 +1,26 @@ +-- CometBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue +from + customer, + orders, + lineitem, + supplier, + nation, + region +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year +group by + n_name +order by + revenue desc; diff --git a/benchmarks/tpc/queries/tpch/q6.sql b/benchmarks/tpc/queries/tpch/q6.sql new file mode 100644 index 0000000000..22b92f4e91 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q6.sql @@ -0,0 +1,11 @@ +-- CometBench-H query 6 derived from TPC-H query 6 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + sum(l_extendedprice * l_discount) as revenue +from + lineitem +where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + and l_discount between 0.04 - 0.01 and 0.04 + 0.01 + and l_quantity < 24; diff --git a/benchmarks/tpc/queries/tpch/q7.sql b/benchmarks/tpc/queries/tpch/q7.sql new file mode 100644 index 0000000000..26b4078ffc --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q7.sql @@ -0,0 +1,41 @@ +-- CometBench-H query 7 derived from TPC-H query 7 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue +from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'GERMANY' and n2.n_name = 'IRAQ') + or (n1.n_name = 'IRAQ' and n2.n_name = 'GERMANY') + ) + and l_shipdate between date '1995-01-01' and date '1996-12-31' + ) as shipping +group by + supp_nation, + cust_nation, + l_year +order by + supp_nation, + cust_nation, + l_year; diff --git a/benchmarks/tpc/queries/tpch/q8.sql b/benchmarks/tpc/queries/tpch/q8.sql new file mode 100644 index 0000000000..c0ac2ecf16 --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q8.sql @@ -0,0 +1,39 @@ +-- CometBench-H query 8 derived from TPC-H query 8 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + o_year, + sum(case + when nation = 'IRAQ' then volume + else 0 + end) / sum(volume) as mkt_share +from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'MIDDLE EAST' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'LARGE PLATED STEEL' + ) as all_nations +group by + o_year +order by + o_year; diff --git a/benchmarks/tpc/queries/tpch/q9.sql b/benchmarks/tpc/queries/tpch/q9.sql new file mode 100644 index 0000000000..ab525c371e --- /dev/null +++ b/benchmarks/tpc/queries/tpch/q9.sql @@ -0,0 +1,34 @@ +-- CometBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. +-- TPC-H queries are Copyright 1993-2022 Transaction Processing Performance Council. +select + nation, + o_year, + sum(amount) as sum_profit +from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%moccasin%' + ) as profit +group by + nation, + o_year +order by + nation, + o_year desc; diff --git a/benchmarks/tpc/run.py b/benchmarks/tpc/run.py new file mode 100755 index 0000000000..38b0ed500b --- /dev/null +++ b/benchmarks/tpc/run.py @@ -0,0 +1,397 @@ +#!/usr/bin/env python3 +# +# 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. + +"""Consolidated TPC benchmark runner for Spark-based engines. + +Usage: + python3 run.py --engine comet --benchmark tpch + python3 run.py --engine comet --benchmark tpcds --iterations 3 + python3 run.py --engine comet-iceberg --benchmark tpch + python3 run.py --engine comet --benchmark tpch --dry-run + python3 run.py --engine spark --benchmark tpch --no-restart +""" + +import argparse +import os +import re +import subprocess +import sys + +# --------------------------------------------------------------------------- +# TOML loading – prefer stdlib tomllib (3.11+), else minimal fallback +# --------------------------------------------------------------------------- + +try: + import tomllib # Python 3.11+ + + def load_toml(path): + with open(path, "rb") as f: + return tomllib.load(f) + +except ModuleNotFoundError: + + def _parse_toml(text): + """Minimal TOML parser supporting tables, quoted-key strings, plain + strings, arrays of strings, booleans, and comments. Sufficient for + the engine config files used by this runner.""" + root = {} + current = root + for line in text.splitlines(): + line = line.strip() + if not line or line.startswith("#"): + continue + # Table header: [env.defaults] or [spark_conf] + m = re.match(r"^\[([^\]]+)\]$", line) + if m: + keys = m.group(1).split(".") + current = root + for k in keys: + current = current.setdefault(k, {}) + continue + # Key = value + m = re.match(r'^("(?:[^"\\]|\\.)*"|[A-Za-z0-9_.]+)\s*=\s*(.+)$', line) + if not m: + continue + raw_key, raw_val = m.group(1), m.group(2).strip() + key = raw_key.strip('"') + val = _parse_value(raw_val) + current[key] = val + return root + + def _parse_value(raw): + if raw == "true": + return True + if raw == "false": + return False + if raw.startswith('"') and raw.endswith('"'): + return raw[1:-1] + if raw.startswith("["): + # Simple array of strings + items = [] + for m in re.finditer(r'"((?:[^"\\]|\\.)*)"', raw): + items.append(m.group(1)) + return items + if raw.startswith("{"): + # Inline table: { KEY = "VAL", ... } + tbl = {} + for m in re.finditer(r'([A-Za-z0-9_]+)\s*=\s*"((?:[^"\\]|\\.)*)"', raw): + tbl[m.group(1)] = m.group(2) + return tbl + return raw + + def load_toml(path): + with open(path, "r") as f: + return _parse_toml(f.read()) + + +# --------------------------------------------------------------------------- +# Common Spark configuration (shared across all engines) +# --------------------------------------------------------------------------- + +COMMON_SPARK_CONF = { + "spark.driver.memory": "8G", + "spark.executor.memory": "16g", + "spark.memory.offHeap.enabled": "true", + "spark.memory.offHeap.size": "16g", + "spark.eventLog.enabled": "true", + "spark.eventLog.dir": os.environ.get("SPARK_EVENT_LOG_DIR", "/tmp/spark-events"), + "spark.hadoop.fs.s3a.impl": "org.apache.hadoop.fs.s3a.S3AFileSystem", + "spark.hadoop.fs.s3a.aws.credentials.provider": "com.amazonaws.auth.DefaultAWSCredentialsProviderChain", +} + +# --------------------------------------------------------------------------- +# Benchmark profiles +# --------------------------------------------------------------------------- + +BENCHMARK_PROFILES = { + "tpch": { + "executor_instances": "2", + "executor_cores": "8", + "max_cores": "16", + "data_env": "TPCH_DATA", + "format": "parquet", + }, + "tpcds": { + "executor_instances": "2", + "executor_cores": "8", + "max_cores": "16", + "data_env": "TPCDS_DATA", + "format": None, # omit --format for TPC-DS + }, +} + +# --------------------------------------------------------------------------- +# Helpers +# --------------------------------------------------------------------------- + + +def resolve_env(value): + """Expand $VAR and ${VAR} references using os.environ.""" + if not isinstance(value, str): + return value + return re.sub( + r"\$\{([^}]+)\}|\$([A-Za-z_][A-Za-z0-9_]*)", + lambda m: os.environ.get(m.group(1) or m.group(2), ""), + value, + ) + + +def resolve_env_in_list(lst): + return [resolve_env(v) for v in lst] + + +def load_engine_config(engine_name): + """Load and return the TOML config for the given engine.""" + script_dir = os.path.dirname(os.path.abspath(__file__)) + config_path = os.path.join(script_dir, "engines", f"{engine_name}.toml") + if not os.path.exists(config_path): + available = sorted( + f.removesuffix(".toml") + for f in os.listdir(os.path.join(script_dir, "engines")) + if f.endswith(".toml") + ) + print(f"Error: Unknown engine '{engine_name}'", file=sys.stderr) + print(f"Available engines: {', '.join(available)}", file=sys.stderr) + sys.exit(1) + return load_toml(config_path) + + +def apply_env_defaults(config): + """Set environment variable defaults from [env.defaults].""" + defaults = config.get("env", {}).get("defaults", {}) + for key, val in defaults.items(): + if key not in os.environ: + os.environ[key] = val + + +def apply_env_exports(config): + """Export environment variables from [env.exports].""" + exports = config.get("env", {}).get("exports", {}) + for key, val in exports.items(): + os.environ[key] = val + + +def check_required_env(config): + """Validate that required environment variables are set.""" + required = config.get("env", {}).get("required", []) + missing = [v for v in required if not os.environ.get(v)] + if missing: + print( + f"Error: Required environment variable(s) not set: {', '.join(missing)}", + file=sys.stderr, + ) + sys.exit(1) + + +def check_common_env(): + """Validate SPARK_HOME and SPARK_MASTER are set.""" + for var in ("SPARK_HOME", "SPARK_MASTER"): + if not os.environ.get(var): + print(f"Error: {var} is not set", file=sys.stderr) + sys.exit(1) + + +def check_benchmark_env(config, benchmark): + """Validate benchmark-specific environment variables.""" + profile = BENCHMARK_PROFILES[benchmark] + use_iceberg = config.get("tpcbench_args", {}).get("use_iceberg", False) + + required = [] + if not use_iceberg: + required.append(profile["data_env"]) + + missing = [v for v in required if not os.environ.get(v)] + if missing: + print( + f"Error: Required environment variable(s) not set for " + f"{benchmark}: {', '.join(missing)}", + file=sys.stderr, + ) + sys.exit(1) + + # Default ICEBERG_DATABASE to the benchmark name if not already set + if use_iceberg and not os.environ.get("ICEBERG_DATABASE"): + os.environ["ICEBERG_DATABASE"] = benchmark + + +def build_spark_submit_cmd(config, benchmark, args): + """Build the spark-submit command list.""" + spark_home = os.environ["SPARK_HOME"] + spark_master = os.environ["SPARK_MASTER"] + profile = BENCHMARK_PROFILES[benchmark] + + cmd = [os.path.join(spark_home, "bin", "spark-submit")] + cmd += ["--master", spark_master] + + # --jars + jars = config.get("spark_submit", {}).get("jars", []) + if jars: + cmd += ["--jars", ",".join(resolve_env_in_list(jars))] + + # --driver-class-path + driver_cp = config.get("spark_submit", {}).get("driver_class_path", []) + if driver_cp: + cmd += ["--driver-class-path", ":".join(resolve_env_in_list(driver_cp))] + + # Merge spark confs: common + benchmark profile + engine overrides + conf = dict(COMMON_SPARK_CONF) + conf["spark.executor.instances"] = profile["executor_instances"] + conf["spark.executor.cores"] = profile["executor_cores"] + conf["spark.cores.max"] = profile["max_cores"] + + engine_conf = config.get("spark_conf", {}) + for key, val in engine_conf.items(): + if isinstance(val, bool): + val = "true" if val else "false" + conf[resolve_env(key)] = resolve_env(str(val)) + + for key, val in sorted(conf.items()): + cmd += ["--conf", f"{key}={val}"] + + # tpcbench.py path + cmd.append("tpcbench.py") + + # tpcbench args + engine_name = config.get("engine", {}).get("name", args.engine) + cmd += ["--name", engine_name] + cmd += ["--benchmark", benchmark] + + use_iceberg = config.get("tpcbench_args", {}).get("use_iceberg", False) + if use_iceberg: + cmd += ["--catalog", resolve_env("${ICEBERG_CATALOG}")] + cmd += ["--database", resolve_env("${ICEBERG_DATABASE}")] + else: + data_var = profile["data_env"] + data_val = os.environ.get(data_var, "") + cmd += ["--data", data_val] + + cmd += ["--output", args.output] + cmd += ["--iterations", str(args.iterations)] + + if args.query is not None: + cmd += ["--query", str(args.query)] + + if profile["format"] and not use_iceberg: + cmd += ["--format", profile["format"]] + + return cmd + + +def restart_spark(): + """Stop and start Spark master and worker.""" + spark_home = os.environ["SPARK_HOME"] + sbin = os.path.join(spark_home, "sbin") + spark_master = os.environ["SPARK_MASTER"] + + # Stop (ignore errors) + subprocess.run( + [os.path.join(sbin, "stop-master.sh")], + stdout=subprocess.DEVNULL, + stderr=subprocess.DEVNULL, + ) + subprocess.run( + [os.path.join(sbin, "stop-worker.sh")], + stdout=subprocess.DEVNULL, + stderr=subprocess.DEVNULL, + ) + + # Start (check errors) + r = subprocess.run([os.path.join(sbin, "start-master.sh")]) + if r.returncode != 0: + print("Error: Failed to start Spark master", file=sys.stderr) + sys.exit(1) + + r = subprocess.run([os.path.join(sbin, "start-worker.sh"), spark_master]) + if r.returncode != 0: + print("Error: Failed to start Spark worker", file=sys.stderr) + sys.exit(1) + + +def main(): + parser = argparse.ArgumentParser( + description="Consolidated TPC benchmark runner for Spark-based engines." + ) + parser.add_argument( + "--engine", + required=True, + help="Engine name (matches a TOML file in engines/)", + ) + parser.add_argument( + "--benchmark", + required=True, + choices=["tpch", "tpcds"], + help="Benchmark to run", + ) + parser.add_argument( + "--iterations", type=int, default=1, help="Number of iterations (default: 1)" + ) + parser.add_argument( + "--output", default=".", help="Output directory (default: .)" + ) + parser.add_argument( + "--query", type=int, default=None, help="Run a single query number" + ) + parser.add_argument( + "--no-restart", + action="store_true", + help="Skip Spark master/worker restart", + ) + parser.add_argument( + "--dry-run", + action="store_true", + help="Print the spark-submit command without executing", + ) + args = parser.parse_args() + + config = load_engine_config(args.engine) + + # Apply env defaults and exports before validation + apply_env_defaults(config) + apply_env_exports(config) + + check_common_env() + check_required_env(config) + check_benchmark_env(config, args.benchmark) + + # Restart Spark unless --no-restart or --dry-run + if not args.no_restart and not args.dry_run: + restart_spark() + + cmd = build_spark_submit_cmd(config, args.benchmark, args) + + if args.dry_run: + # Group paired arguments (e.g. --conf key=value) on one line + parts = [] + i = 0 + while i < len(cmd): + token = cmd[i] + if token.startswith("--") and i + 1 < len(cmd) and not cmd[i + 1].startswith("--"): + parts.append(f"{token} {cmd[i + 1]}") + i += 2 + else: + parts.append(token) + i += 1 + print(" \\\n ".join(parts)) + else: + r = subprocess.run(cmd) + sys.exit(r.returncode) + + +if __name__ == "__main__": + main() diff --git a/dev/benchmarks/tpcbench.py b/benchmarks/tpc/tpcbench.py similarity index 87% rename from dev/benchmarks/tpcbench.py rename to benchmarks/tpc/tpcbench.py index 400ccd175a..036d7b0e9a 100644 --- a/dev/benchmarks/tpcbench.py +++ b/benchmarks/tpc/tpcbench.py @@ -25,7 +25,9 @@ import argparse from datetime import datetime +import hashlib import json +import os from pyspark.sql import SparkSession import time from typing import Dict @@ -45,23 +47,35 @@ def dedup_columns(df): return df.toDF(*new_cols) +def result_hash(rows): + """Compute a deterministic MD5 hash from collected rows.""" + sorted_rows = sorted(rows, key=lambda r: str(r)) + h = hashlib.md5() + for row in sorted_rows: + h.update(str(row).encode("utf-8")) + return h.hexdigest() + + def main( benchmark: str, data_path: str, catalog: str, database: str, - query_path: str, iterations: int, output: str, name: str, format: str, query_num: int = None, write_path: str = None, - options: Dict[str, str] = None + options: Dict[str, str] = None, ): if options is None: options = {} + query_path = os.path.join( + os.path.dirname(os.path.abspath(__file__)), "queries", benchmark + ) + spark = SparkSession.builder \ .appName(f"{name} benchmark derived from {benchmark}") \ .getOrCreate() @@ -94,7 +108,10 @@ def main( print(f"Registering table {table} from {source}") df = spark.table(source) else: + # Support both "customer/" and "customer.parquet/" layouts source = f"{data_path}/{table}.{format}" + if not os.path.exists(source): + source = f"{data_path}/{table}" print(f"Registering table {table} from {source}") df = spark.read.format(format).options(**options).load(source) df.createOrReplaceTempView(table) @@ -104,7 +121,6 @@ def main( results = { 'engine': 'datafusion-comet', 'benchmark': benchmark, - 'query_path': query_path, 'spark_conf': conf_dict, } if using_iceberg: @@ -156,14 +172,19 @@ def main( print(f"Results written to {output_path}") else: rows = df.collect() - print(f"Query {query} returned {len(rows)} rows") + row_count = len(rows) + row_hash = result_hash(rows) + print(f"Query {query} returned {row_count} rows, hash={row_hash}") end_time = time.time() elapsed = end_time - start_time print(f"Query {query} took {elapsed:.2f} seconds") - query_timings = results.setdefault(query, []) - query_timings.append(elapsed) + query_result = results.setdefault(query, {"durations": []}) + query_result["durations"].append(round(elapsed, 3)) + if "row_count" not in query_result and not write_path: + query_result["row_count"] = row_count + query_result["result_hash"] = row_hash iter_end_time = time.time() print(f"\nIteration {iteration + 1} took {iter_end_time - iter_start_time:.2f} seconds") @@ -215,10 +236,6 @@ def main( help="Database containing TPC tables (only used with --catalog)" ) - parser.add_argument( - "--queries", required=True, - help="Path to query SQL files" - ) parser.add_argument( "--iterations", type=int, default=1, help="Number of iterations" @@ -246,12 +263,11 @@ def main( args.data, args.catalog, args.database, - args.queries, args.iterations, args.output, args.name, args.format, args.query, args.write, - args.options + args.options, ) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index f1ee0e1868..2ffaf9834d 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -78,11 +78,11 @@ object CometConf extends ShimCometConf { val COMET_PREFIX = "spark.comet"; - val COMET_EXEC_CONFIG_PREFIX: String = s"$COMET_PREFIX.exec"; + val COMET_EXEC_CONFIG_PREFIX: String = s"$COMET_PREFIX.exec" - val COMET_EXPR_CONFIG_PREFIX: String = s"$COMET_PREFIX.expression"; + val COMET_EXPR_CONFIG_PREFIX: String = s"$COMET_PREFIX.expression" - val COMET_OPERATOR_CONFIG_PREFIX: String = s"$COMET_PREFIX.operator"; + val COMET_OPERATOR_CONFIG_PREFIX: String = s"$COMET_PREFIX.operator" val COMET_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.enabled") .category(CATEGORY_EXEC) @@ -112,7 +112,7 @@ object CometConf extends ShimCometConf { "feature is highly experimental and only partially implemented. It should not " + "be used in production.") .booleanConf - .createWithDefault(false) + .createWithEnvVarOrDefault("ENABLE_COMET_WRITE", false) // Deprecated: native_comet uses mutable buffers incompatible with Arrow FFI best practices // and does not support complex types. Use native_iceberg_compat or auto instead. @@ -125,16 +125,14 @@ object CometConf extends ShimCometConf { val SCAN_AUTO = "auto" val COMET_NATIVE_SCAN_IMPL: ConfigEntry[String] = conf("spark.comet.scan.impl") - .category(CATEGORY_SCAN) + .category(CATEGORY_PARQUET) .doc( - "The implementation of Comet Native Scan to use. Available modes are " + + "The implementation of Comet's Parquet scan to use. Available scans are " + s"`$SCAN_NATIVE_DATAFUSION`, and `$SCAN_NATIVE_ICEBERG_COMPAT`. " + - s"`$SCAN_NATIVE_DATAFUSION` is a fully native implementation of scan based on " + - "DataFusion. " + - s"`$SCAN_NATIVE_ICEBERG_COMPAT` is the recommended native implementation that " + - "exposes apis to read parquet columns natively and supports complex types. " + - s"`$SCAN_AUTO` (default) chooses the best scan.") - .internal() + s"`$SCAN_NATIVE_DATAFUSION` is a fully native implementation, and " + + s"`$SCAN_NATIVE_ICEBERG_COMPAT` is a hybrid implementation that supports some " + + "additional features, such as row indexes and field ids. " + + s"`$SCAN_AUTO` (default) chooses the best available scan based on the scan schema.") .stringConf .transform(_.toLowerCase(Locale.ROOT)) .checkValues(Set(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT, SCAN_AUTO)) @@ -162,6 +160,17 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_ICEBERG_DATA_FILE_CONCURRENCY_LIMIT: ConfigEntry[Int] = + conf("spark.comet.scan.icebergNative.dataFileConcurrencyLimit") + .category(CATEGORY_SCAN) + .doc( + "The number of Iceberg data files to read concurrently within a single task. " + + "Higher values improve throughput for tables with many small files by overlapping " + + "I/O latency, but increase memory usage. Values between 2 and 8 are suggested.") + .intConf + .checkValue(v => v > 0, "Data file concurrency limit must be positive") + .createWithDefault(1) + val COMET_CSV_V2_NATIVE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.csv.v2.enabled") .category(CATEGORY_TESTING) @@ -502,6 +511,13 @@ object CometConf extends ShimCometConf { "Ensure that Comet shuffle memory overhead factor is a double greater than 0") .createWithDefault(1.0) + val COMET_BATCH_SIZE: ConfigEntry[Int] = conf("spark.comet.batchSize") + .category(CATEGORY_TUNING) + .doc("The columnar batch size, i.e., the maximum number of rows that a batch can contain.") + .intConf + .checkValue(v => v > 0, "Batch size must be positive") + .createWithDefault(8192) + val COMET_COLUMNAR_SHUFFLE_BATCH_SIZE: ConfigEntry[Int] = conf("spark.comet.columnar.shuffle.batch.size") .category(CATEGORY_SHUFFLE) @@ -509,6 +525,9 @@ object CometConf extends ShimCometConf { "this should not be larger than batch size (i.e., `spark.comet.batchSize`). Otherwise " + "it will produce larger batches than expected in the native operator after shuffle.") .intConf + .checkValue( + v => v <= COMET_BATCH_SIZE.get(), + "Should not be larger than batch size `spark.comet.batchSize`") .createWithDefault(8192) val COMET_SHUFFLE_WRITE_BUFFER_SIZE: ConfigEntry[Long] = @@ -564,6 +583,7 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + // Used on native side. Check spark_config.rs how the config is used val COMET_DEBUG_MEMORY_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_PREFIX.debug.memory") .category(CATEGORY_TESTING) @@ -622,12 +642,6 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) - val COMET_BATCH_SIZE: ConfigEntry[Int] = conf("spark.comet.batchSize") - .category(CATEGORY_TUNING) - .doc("The columnar batch size, i.e., the maximum number of rows that a batch can contain.") - .intConf - .createWithDefault(8192) - val COMET_PARQUET_ENABLE_DIRECT_BUFFER: ConfigEntry[Boolean] = conf("spark.comet.parquet.enable.directBuffer") .category(CATEGORY_PARQUET) @@ -807,14 +821,6 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) - val COMET_REGEXP_ALLOW_INCOMPATIBLE: ConfigEntry[Boolean] = - conf("spark.comet.regexp.allowIncompatible") - .category(CATEGORY_EXEC) - .doc("Comet is not currently fully compatible with Spark for all regular expressions. " + - s"Set this config to true to allow them anyway. $COMPAT_GUIDE.") - .booleanConf - .createWithDefault(false) - val COMET_METRICS_UPDATE_INTERVAL: ConfigEntry[Long] = conf("spark.comet.metrics.updateInterval") .category(CATEGORY_EXEC) @@ -833,6 +839,7 @@ object CometConf extends ShimCometConf { .stringConf .createOptional + // Used on native side. Check spark_config.rs how the config is used val COMET_MAX_TEMP_DIRECTORY_SIZE: ConfigEntry[Long] = conf("spark.comet.maxTempDirectorySize") .category(CATEGORY_EXEC) @@ -857,6 +864,9 @@ object CometConf extends ShimCometConf { .booleanConf .createWithEnvVarOrDefault("ENABLE_COMET_STRICT_TESTING", false) + val COMET_OPERATOR_DATA_WRITING_COMMAND_ALLOW_INCOMPAT: ConfigEntry[Boolean] = + createOperatorIncompatConfig("DataWritingCommandExec") + /** Create a config to enable a specific operator */ private def createExecEnabledConfig( exec: String, @@ -872,6 +882,25 @@ object CometConf extends ShimCometConf { .createWithDefault(defaultValue) } + /** + * Converts a config key to a valid environment variable name. Example: + * "spark.comet.operator.DataWritingCommandExec.allowIncompatible" -> + * "SPARK_COMET_OPERATOR_DATAWRITINGCOMMANDEXEC_ALLOWINCOMPATIBLE" + */ + private def configKeyToEnvVar(configKey: String): String = + configKey.toUpperCase(Locale.ROOT).replace('.', '_') + + private def createOperatorIncompatConfig(name: String): ConfigEntry[Boolean] = { + val configKey = getOperatorAllowIncompatConfigKey(name) + val envVar = configKeyToEnvVar(configKey) + conf(configKey) + .category(CATEGORY_EXEC) + .doc(s"Whether to allow incompatibility for operator: $name. " + + s"False by default. Can be overridden with $envVar env variable") + .booleanConf + .createWithEnvVarOrDefault(envVar, false) + } + def isExprEnabled(name: String, conf: SQLConf = SQLConf.get): Boolean = { getBooleanConf(getExprEnabledConfigKey(name), defaultValue = true, conf) } diff --git a/dev/benchmarks/README.md b/dev/benchmarks/README.md deleted file mode 100644 index b3ea674199..0000000000 --- a/dev/benchmarks/README.md +++ /dev/null @@ -1,151 +0,0 @@ - - -# Comet Benchmarking Scripts - -This directory contains scripts used for generating benchmark results that are published in this repository and in -the Comet documentation. - -For full instructions on running these benchmarks on an EC2 instance, see the [Comet Benchmarking on EC2 Guide]. - -[Comet Benchmarking on EC2 Guide]: https://datafusion.apache.org/comet/contributor-guide/benchmarking_aws_ec2.html - -## Example usage - -Set Spark environment variables: - -```shell -export SPARK_HOME=/opt/spark-3.5.3-bin-hadoop3/ -export SPARK_MASTER=spark://yourhostname:7077 -``` - -Set path to queries and data: - -```shell -export TPCH_QUERIES=/mnt/bigdata/tpch/queries/ -export TPCH_DATA=/mnt/bigdata/tpch/sf100/ -``` - -Run Spark benchmark: - -```shell -export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 -sudo ./drop-caches.sh -./spark-tpch.sh -``` - -Run Comet benchmark: - -```shell -export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 -export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar -sudo ./drop-caches.sh -./comet-tpch.sh -``` - -Run Gluten benchmark: - -```shell -export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 -export GLUTEN_JAR=/opt/gluten/gluten-velox-bundle-spark3.5_2.12-linux_amd64-1.4.0.jar -sudo ./drop-caches.sh -./gluten-tpch.sh -``` - -Generating charts: - -```shell -python3 generate-comparison.py --benchmark tpch --labels "Spark 3.5.3" "Comet 0.9.0" "Gluten 1.4.0" --title "TPC-H @ 100 GB (single executor, 8 cores, local Parquet files)" spark-tpch-1752338506381.json comet-tpch-1752337818039.json gluten-tpch-1752337474344.json -``` - -## Iceberg Benchmarking - -Comet includes native Iceberg support via iceberg-rust integration. This enables benchmarking TPC-H queries -against Iceberg tables with native scan acceleration. - -### Prerequisites - -Download the Iceberg Spark runtime JAR (required for running the benchmark): - -```shell -wget https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/iceberg-spark-runtime-3.5_2.12-1.8.1.jar -export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar -``` - -Note: Table creation uses `--packages` which auto-downloads the dependency. - -### Create Iceberg TPC-H tables - -Convert existing Parquet TPC-H data to Iceberg format: - -```shell -export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse -export ICEBERG_CATALOG=${ICEBERG_CATALOG:-local} - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ - create-iceberg-tpch.py \ - --parquet-path $TPCH_DATA \ - --catalog $ICEBERG_CATALOG \ - --database tpch -``` - -### Run Iceberg benchmark - -```shell -export JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 -export COMET_JAR=/opt/comet/comet-spark-spark3.5_2.12-0.10.0.jar -export ICEBERG_JAR=/path/to/iceberg-spark-runtime-3.5_2.12-1.8.1.jar -export ICEBERG_WAREHOUSE=/mnt/bigdata/iceberg-warehouse -export TPCH_QUERIES=/mnt/bigdata/tpch/queries/ -sudo ./drop-caches.sh -./comet-tpch-iceberg.sh -``` - -The benchmark uses `spark.comet.scan.icebergNative.enabled=true` to enable Comet's native iceberg-rust -integration. Verify native scanning is active by checking for `CometIcebergNativeScanExec` in the -physical plan output. - -### Iceberg-specific options - -| Environment Variable | Default | Description | -| -------------------- | ---------- | ----------------------------------- | -| `ICEBERG_CATALOG` | `local` | Iceberg catalog name | -| `ICEBERG_DATABASE` | `tpch` | Database containing TPC-H tables | -| `ICEBERG_WAREHOUSE` | (required) | Path to Iceberg warehouse directory | - -### Comparing Parquet vs Iceberg performance - -Run both benchmarks and compare: - -```shell -python3 generate-comparison.py --benchmark tpch \ - --labels "Comet (Parquet)" "Comet (Iceberg)" \ - --title "TPC-H @ 100 GB: Parquet vs Iceberg" \ - comet-tpch-*.json comet-iceberg-tpch-*.json -``` diff --git a/dev/benchmarks/blaze-tpcds.sh b/dev/benchmarks/blaze-tpcds.sh deleted file mode 100755 index 90a4a48468..0000000000 --- a/dev/benchmarks/blaze-tpcds.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# 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. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $BLAZE_JAR \ - --driver-class-path $BLAZE_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=2 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=16 \ - --conf spark.executor.memory=16g \ - --conf spark.executor.memoryOverhead=16g \ - --conf spark.memory.offHeap.enabled=false \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$BLAZE_JAR \ - --conf spark.executor.extraClassPath=$BLAZE_JAR \ - --conf spark.sql.extensions=org.apache.spark.sql.blaze.BlazeSparkSessionExtension \ - --conf spark.shuffle.manager=org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager \ - --conf spark.blaze.enable=true \ - --conf spark.blaze.forceShuffledHashJoin=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name blaze \ - --benchmark tpcds \ - --data $TPCDS_DATA \ - --queries $TPCDS_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/blaze-tpch.sh b/dev/benchmarks/blaze-tpch.sh deleted file mode 100755 index 2c6878737d..0000000000 --- a/dev/benchmarks/blaze-tpch.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# 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. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $BLAZE_JAR \ - --driver-class-path $BLAZE_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.executor.memoryOverhead=16g \ - --conf spark.memory.offHeap.enabled=false \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$BLAZE_JAR \ - --conf spark.executor.extraClassPath=$BLAZE_JAR \ - --conf spark.sql.extensions=org.apache.spark.sql.blaze.BlazeSparkSessionExtension \ - --conf spark.shuffle.manager=org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleManager \ - --conf spark.blaze.enable=true \ - --conf spark.blaze.forceShuffledHashJoin=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name blaze \ - --benchmark tpch \ - --data $TPCH_DATA \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/comet-tpcds.sh b/dev/benchmarks/comet-tpcds.sh deleted file mode 100755 index b55b27188c..0000000000 --- a/dev/benchmarks/comet-tpcds.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# 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. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $COMET_JAR \ - --driver-class-path $COMET_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=2 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=16 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$COMET_JAR \ - --conf spark.executor.extraClassPath=$COMET_JAR \ - --conf spark.plugins=org.apache.spark.CometPlugin \ - --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.scan.impl=native_datafusion \ - --conf spark.comet.expression.Cast.allowIncompatible=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name comet \ - --benchmark tpcds \ - --data $TPCDS_DATA \ - --queries $TPCDS_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/comet-tpch-iceberg.sh b/dev/benchmarks/comet-tpch-iceberg.sh deleted file mode 100755 index 7907125c82..0000000000 --- a/dev/benchmarks/comet-tpch-iceberg.sh +++ /dev/null @@ -1,114 +0,0 @@ -#!/bin/bash -# -# 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. -# - -# TPC-H benchmark using Iceberg tables with Comet's native iceberg-rust integration. -# -# Required environment variables: -# SPARK_HOME - Path to Spark installation -# SPARK_MASTER - Spark master URL (e.g., spark://localhost:7077) -# COMET_JAR - Path to Comet JAR -# ICEBERG_JAR - Path to Iceberg Spark runtime JAR -# ICEBERG_WAREHOUSE - Path to Iceberg warehouse directory -# TPCH_QUERIES - Path to TPC-H query files -# -# Optional: -# ICEBERG_CATALOG - Catalog name (default: local) -# ICEBERG_DATABASE - Database name (default: tpch) -# -# Setup (run once to create Iceberg tables from Parquet): -# $SPARK_HOME/bin/spark-submit \ -# --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ -# --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ -# --conf spark.sql.catalog.local.type=hadoop \ -# --conf spark.sql.catalog.local.warehouse=$ICEBERG_WAREHOUSE \ -# create-iceberg-tpch.py \ -# --parquet-path $TPCH_DATA \ -# --catalog local \ -# --database tpch - -set -e - -# Defaults -ICEBERG_CATALOG=${ICEBERG_CATALOG:-local} -ICEBERG_DATABASE=${ICEBERG_DATABASE:-tpch} - -# Validate required variables -if [ -z "$SPARK_HOME" ]; then - echo "Error: SPARK_HOME is not set" - exit 1 -fi -if [ -z "$COMET_JAR" ]; then - echo "Error: COMET_JAR is not set" - exit 1 -fi -if [ -z "$ICEBERG_JAR" ]; then - echo "Error: ICEBERG_JAR is not set" - echo "Download from: https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/1.8.1/" - exit 1 -fi -if [ -z "$ICEBERG_WAREHOUSE" ]; then - echo "Error: ICEBERG_WAREHOUSE is not set" - exit 1 -fi -if [ -z "$TPCH_QUERIES" ]; then - echo "Error: TPCH_QUERIES is not set" - exit 1 -fi - -$SPARK_HOME/sbin/stop-master.sh 2>/dev/null || true -$SPARK_HOME/sbin/stop-worker.sh 2>/dev/null || true - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $COMET_JAR,$ICEBERG_JAR \ - --driver-class-path $COMET_JAR:$ICEBERG_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$COMET_JAR:$ICEBERG_JAR \ - --conf spark.executor.extraClassPath=$COMET_JAR:$ICEBERG_JAR \ - --conf spark.plugins=org.apache.spark.CometPlugin \ - --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.exec.replaceSortMergeJoin=true \ - --conf spark.comet.expression.Cast.allowIncompatible=true \ - --conf spark.comet.enabled=true \ - --conf spark.comet.exec.enabled=true \ - --conf spark.comet.scan.icebergNative.enabled=true \ - --conf spark.comet.explainFallback.enabled=true \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.type=hadoop \ - --conf spark.sql.catalog.${ICEBERG_CATALOG}.warehouse=$ICEBERG_WAREHOUSE \ - --conf spark.sql.defaultCatalog=${ICEBERG_CATALOG} \ - tpcbench.py \ - --name comet-iceberg \ - --benchmark tpch \ - --catalog $ICEBERG_CATALOG \ - --database $ICEBERG_DATABASE \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh deleted file mode 100755 index a748a02319..0000000000 --- a/dev/benchmarks/comet-tpch.sh +++ /dev/null @@ -1,55 +0,0 @@ -#!/bin/bash -# -# 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. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --jars $COMET_JAR \ - --driver-class-path $COMET_JAR \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.driver.extraClassPath=$COMET_JAR \ - --conf spark.executor.extraClassPath=$COMET_JAR \ - --conf spark.plugins=org.apache.spark.CometPlugin \ - --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.comet.scan.impl=native_datafusion \ - --conf spark.comet.exec.replaceSortMergeJoin=true \ - --conf spark.comet.expression.Cast.allowIncompatible=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name comet \ - --benchmark tpch \ - --data $TPCH_DATA \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 \ - --format parquet diff --git a/dev/benchmarks/create-iceberg-tpch.py b/dev/benchmarks/create-iceberg-tpch.py deleted file mode 100644 index 44f0f63a2e..0000000000 --- a/dev/benchmarks/create-iceberg-tpch.py +++ /dev/null @@ -1,88 +0,0 @@ -# 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. - -""" -Convert TPC-H Parquet data to Iceberg tables. - -Usage: - spark-submit \ - --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.8.1 \ - --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ - --conf spark.sql.catalog.local.type=hadoop \ - --conf spark.sql.catalog.local.warehouse=/path/to/iceberg-warehouse \ - create-iceberg-tpch.py \ - --parquet-path /path/to/tpch/parquet \ - --catalog local \ - --database tpch -""" - -import argparse -from pyspark.sql import SparkSession -import time - - -def main(parquet_path: str, catalog: str, database: str): - spark = SparkSession.builder \ - .appName("Create Iceberg TPC-H Tables") \ - .getOrCreate() - - table_names = [ - "customer", - "lineitem", - "nation", - "orders", - "part", - "partsupp", - "region", - "supplier" - ] - - # Create database if it doesn't exist - spark.sql(f"CREATE DATABASE IF NOT EXISTS {catalog}.{database}") - - for table in table_names: - parquet_table_path = f"{parquet_path}/{table}.parquet" - iceberg_table = f"{catalog}.{database}.{table}" - - print(f"Converting {parquet_table_path} -> {iceberg_table}") - start_time = time.time() - - # Drop table if exists to allow re-running - spark.sql(f"DROP TABLE IF EXISTS {iceberg_table}") - - # Read parquet and write as Iceberg - df = spark.read.parquet(parquet_table_path) - df.writeTo(iceberg_table).using("iceberg").create() - - row_count = spark.table(iceberg_table).count() - elapsed = time.time() - start_time - print(f" Created {iceberg_table} with {row_count} rows in {elapsed:.2f}s") - - print("\nAll TPC-H tables created successfully!") - print(f"Tables available at: {catalog}.{database}.*") - - spark.stop() - - -if __name__ == "__main__": - parser = argparse.ArgumentParser(description="Convert TPC-H Parquet data to Iceberg tables") - parser.add_argument("--parquet-path", required=True, help="Path to TPC-H Parquet data directory") - parser.add_argument("--catalog", required=True, help="Iceberg catalog name (e.g., 'local')") - parser.add_argument("--database", default="tpch", help="Database name to create tables in") - args = parser.parse_args() - - main(args.parquet_path, args.catalog, args.database) diff --git a/dev/benchmarks/gluten-tpcds.sh b/dev/benchmarks/gluten-tpcds.sh deleted file mode 100755 index 7c475c79c0..0000000000 --- a/dev/benchmarks/gluten-tpcds.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# 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. -# -export TZ=UTC - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=2 \ - --conf spark.executor.memory=16G \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=16 \ - --conf spark.eventLog.enabled=true \ - --jars $GLUTEN_JAR \ - --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.driver.extraClassPath=${GLUTEN_JAR} \ - --conf spark.executor.extraClassPath=${GLUTEN_JAR} \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.gluten.sql.columnar.forceShuffledHashJoin=true \ - --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager \ - --conf spark.sql.session.timeZone=UTC \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name gluten \ - --benchmark tpcds \ - --data $TPCDS_DATA \ - --queries $TPCDS_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/gluten-tpch.sh b/dev/benchmarks/gluten-tpch.sh deleted file mode 100755 index 46c3ed7527..0000000000 --- a/dev/benchmarks/gluten-tpch.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/bin/bash -# -# 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. -# -export TZ=UTC - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.memory=16G \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.eventLog.enabled=true \ - --jars $GLUTEN_JAR \ - --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.driver.extraClassPath=${GLUTEN_JAR} \ - --conf spark.executor.extraClassPath=${GLUTEN_JAR} \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.gluten.sql.columnar.forceShuffledHashJoin=true \ - --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager \ - --conf spark.sql.session.timeZone=UTC \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name gluten \ - --benchmark tpch \ - --data $TPCH_DATA \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/spark-tpcds.sh b/dev/benchmarks/spark-tpcds.sh deleted file mode 100755 index dad079ba23..0000000000 --- a/dev/benchmarks/spark-tpcds.sh +++ /dev/null @@ -1,45 +0,0 @@ -#!/bin/bash -# -# 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. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=2 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=16 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name spark \ - --benchmark tpcds \ - --data $TPCDS_DATA \ - --queries $TPCDS_QUERIES \ - --output . \ - --iterations 1 diff --git a/dev/benchmarks/spark-tpch.sh b/dev/benchmarks/spark-tpch.sh deleted file mode 100755 index ae359f049f..0000000000 --- a/dev/benchmarks/spark-tpch.sh +++ /dev/null @@ -1,46 +0,0 @@ -#!/bin/bash -# -# 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. -# - -$SPARK_HOME/sbin/stop-master.sh -$SPARK_HOME/sbin/stop-worker.sh - -$SPARK_HOME/sbin/start-master.sh -$SPARK_HOME/sbin/start-worker.sh $SPARK_MASTER - -$SPARK_HOME/bin/spark-submit \ - --master $SPARK_MASTER \ - --conf spark.driver.memory=8G \ - --conf spark.executor.instances=1 \ - --conf spark.executor.cores=8 \ - --conf spark.cores.max=8 \ - --conf spark.executor.memory=16g \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=16g \ - --conf spark.eventLog.enabled=true \ - --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem \ - --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ - tpcbench.py \ - --name spark \ - --benchmark tpch \ - --data $TPCH_DATA \ - --queries $TPCH_QUERIES \ - --output . \ - --iterations 1 \ - --format parquet diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 0205888433..5713cacea5 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1009,7 +1009,7 @@ index 18123a4d6ec..fbe4c766eee 100644 - regexp_extract($"a", "(\\d+)-(\\d+)", 1)), - Row("num-num", "300", "100") :: Row("num-num", "400", "100") :: - Row("num-num", "400-400", "100") :: Nil) -+ withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { ++ withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { + val df = Seq( + ("100-200", "(\\d+)-(\\d+)", "300"), + ("100-200", "(\\d+)-(\\d+)", "400"), diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index d2d72e9d68..3aaecdecb1 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -985,7 +985,7 @@ index fa1a64460fc..1d2e215d6a3 100644 - ("100-200", "(\\d+)-(\\d+)", "300"), - ("100-200", "(\\d+)-(\\d+)", "400"), - ("100-200", "(\\d+)", "400")).toDF("a", "b", "c") -+ withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { ++ withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { + val df = Seq( + ("100-200", "(\\d+)-(\\d+)", "300"), + ("100-200", "(\\d+)-(\\d+)", "400"), diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index d6694e827f..a1b2506554 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -1165,7 +1165,7 @@ index 0df7f806272..52d33d67328 100644 - ("100-200", "(\\d+)-(\\d+)", "300"), - ("100-200", "(\\d+)-(\\d+)", "400"), - ("100-200", "(\\d+)", "400")).toDF("a", "b", "c") -+ withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { ++ withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { + val df = Seq( + ("100-200", "(\\d+)-(\\d+)", "300"), + ("100-200", "(\\d+)-(\\d+)", "400"), diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt index 4ea10c1dff..3511e1483e 100644 --- a/dev/release/rat_exclude_files.txt +++ b/dev/release/rat_exclude_files.txt @@ -28,3 +28,4 @@ spark/src/test/resources/tpch-extended/q*.sql spark/src/test/resources/test-data/*.csv spark/src/test/resources/test-data/*.ndjson spark/inspections/CometTPC*results.txt +benchmarks/tpc/queries/**/*.sql diff --git a/docs/source/about/gluten_comparison.md b/docs/source/about/gluten_comparison.md index 492479bb9d..40c6c2741a 100644 --- a/docs/source/about/gluten_comparison.md +++ b/docs/source/about/gluten_comparison.md @@ -86,7 +86,7 @@ on your existing Spark jobs. ![tpch_allqueries_comet_gluten.png](/_static/images/tpch_allqueries_comet_gluten.png) -The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks). +The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc). ## Ease of Development & Contributing diff --git a/docs/source/contributor-guide/benchmark-results/tpc-ds.md b/docs/source/contributor-guide/benchmark-results/tpc-ds.md index 66ff2e51a7..bea254cc08 100644 --- a/docs/source/contributor-guide/benchmark-results/tpc-ds.md +++ b/docs/source/contributor-guide/benchmark-results/tpc-ds.md @@ -46,4 +46,4 @@ The raw results of these benchmarks in JSON format is available here: - [Spark](spark-3.5.3-tpcds.json) - [Comet](comet-0.11.0-tpcds.json) -The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks). +The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc). diff --git a/docs/source/contributor-guide/benchmark-results/tpc-h.md b/docs/source/contributor-guide/benchmark-results/tpc-h.md index 4424d9eac7..2170426c05 100644 --- a/docs/source/contributor-guide/benchmark-results/tpc-h.md +++ b/docs/source/contributor-guide/benchmark-results/tpc-h.md @@ -46,4 +46,4 @@ The raw results of these benchmarks in JSON format is available here: - [Spark](spark-3.5.3-tpch.json) - [Comet](comet-0.11.0-tpch.json) -The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks). +The scripts that were used to generate these results can be found [here](https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc). diff --git a/docs/source/contributor-guide/benchmarking.md b/docs/source/contributor-guide/benchmarking.md index 768089c955..49af73376f 100644 --- a/docs/source/contributor-guide/benchmarking.md +++ b/docs/source/contributor-guide/benchmarking.md @@ -21,7 +21,7 @@ under the License. To track progress on performance, we regularly run benchmarks derived from TPC-H and TPC-DS. -The benchmarking scripts are contained at [https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks](https://github.com/apache/datafusion-comet/tree/main/dev/benchmarks). +The benchmarking scripts are contained [here](https://github.com/apache/datafusion-comet/tree/main/benchmarks/tpc). Data generation scripts are available in the [DataFusion Benchmarks](https://github.com/apache/datafusion-benchmarks) GitHub repository. diff --git a/docs/source/contributor-guide/benchmarking_aws_ec2.md b/docs/source/contributor-guide/benchmarking_aws_ec2.md index 922b0379fe..81f15d64ea 100644 --- a/docs/source/contributor-guide/benchmarking_aws_ec2.md +++ b/docs/source/contributor-guide/benchmarking_aws_ec2.md @@ -109,23 +109,23 @@ export COMET_JAR=/home/ec2-user/datafusion-comet/spark/target/comet-spark-spark3 ## Run Benchmarks -Use the scripts in `dev/benchmarks` in the Comet repository. +Use the scripts in `benchmarks/tpc` in the Comet repository. ```shell -cd dev/benchmarks +cd benchmarks/tpc export TPCH_QUERIES=/home/ec2-user/datafusion-benchmarks/tpch/queries/ ``` Run Spark benchmark: ```shell -./spark-tpch.sh +python3 run.py --engine spark --benchmark tpch ``` Run Comet benchmark: ```shell -./comet-tpch.sh +python3 run.py --engine comet --benchmark tpch ``` ## Running Benchmarks with S3 @@ -164,4 +164,9 @@ Modify the scripts to add the following configurations. --conf spark.hadoop.fs.s3a.aws.credentials.provider=com.amazonaws.auth.DefaultAWSCredentialsProviderChain \ ``` -Now run the `spark-tpch.sh` and `comet-tpch.sh` scripts. +Now run the benchmarks: + +```shell +python3 run.py --engine spark --benchmark tpch +python3 run.py --engine comet --benchmark tpch +``` diff --git a/docs/source/contributor-guide/development.md b/docs/source/contributor-guide/development.md index 14a67ff848..b83f3174d8 100644 --- a/docs/source/contributor-guide/development.md +++ b/docs/source/contributor-guide/development.md @@ -28,6 +28,79 @@ under the License. ├── spark <- Spark integration ``` +## Threading Architecture + +Comet's native execution runs on a shared tokio multi-threaded runtime. Understanding this +architecture is important because it affects how you write native operators and JVM callbacks. + +### How execution works + +Spark calls into native code via JNI from an **executor task thread**. There are two execution +paths depending on whether the plan reads data from the JVM: + +**Async I/O path (no JVM data sources, e.g. Iceberg scans):** The DataFusion stream is spawned +onto a tokio worker thread and batches are delivered to the executor thread via an `mpsc` channel. +The executor thread parks in `blocking_recv()` until the next batch is ready. This avoids +busy-polling on I/O-bound workloads. + +**JVM data source path (ScanExec present):** The executor thread calls `block_on()` and polls the +DataFusion stream directly, interleaving `pull_input_batches()` calls on `Poll::Pending` to feed +data from the JVM into ScanExec operators. + +In both cases, DataFusion operators execute on **tokio worker threads**, not on the Spark executor +task thread. All Spark tasks on an executor share one tokio runtime. + +### Rules for native code + +**Do not use `thread_local!` or assume thread identity.** Tokio may run your operator's `poll` +method on any worker thread, and may move it between threads across polls. Any state must live +in the operator struct or be shared via `Arc`. + +**JNI calls work from any thread, but have overhead.** `JVMClasses::get_env()` calls +`AttachCurrentThread`, which acquires JVM internal locks. The `AttachGuard` detaches the thread +when dropped. Repeated attach/detach cycles on tokio workers add overhead, so avoid calling +into the JVM on hot paths during stream execution. + +**Do not call `TaskContext.get()` from JVM callbacks during execution.** Spark's `TaskContext` is +a `ThreadLocal` on the executor task thread. JVM methods invoked from tokio worker threads will +see `null`. If you need task metadata, capture it at construction time (in `createPlan` or +operator setup) and store it in the operator. See `CometTaskMemoryManager` for an example — it +captures `TaskContext.get().taskMemoryManager()` in its constructor and uses the stored reference +thereafter. + +**Memory pool operations call into the JVM.** `CometUnifiedMemoryPool` and `CometFairMemoryPool` +call `acquireMemory()` / `releaseMemory()` via JNI whenever DataFusion operators grow or shrink +memory reservations. This happens on whatever thread the operator is executing on. These calls +are thread-safe (they use stored `GlobalRef`s, not thread-locals), but they do trigger +`AttachCurrentThread`. + +**Scalar subqueries call into the JVM.** `Subquery::evaluate()` calls static methods on +`CometScalarSubquery` via JNI. These use a static `HashMap`, not thread-locals, so they are +safe from any thread. + +**Parquet encryption calls into the JVM.** `CometKeyRetriever::retrieve_key()` calls the JVM +to unwrap decryption keys during Parquet reads. It uses a stored `GlobalRef` and a cached +`JMethodID`, so it is safe from any thread. + +### The tokio runtime + +The runtime is created once per executor JVM in a `Lazy` static: + +- **Worker threads:** `num_cpus` by default, configurable via `COMET_WORKER_THREADS` +- **Max blocking threads:** 512 by default, configurable via `COMET_MAX_BLOCKING_THREADS` +- All async I/O (S3, HTTP, Parquet reads) runs on worker threads as non-blocking futures + +### Summary of what is safe and what is not + +| Pattern | Safe? | Notes | +| ----------------------------------------- | ------ | ---------------------------------------- | +| `Arc` shared across operators | Yes | Standard Rust thread safety | +| `JVMClasses::get_env()` from tokio worker | Yes | Attaches thread to JVM automatically | +| `thread_local!` in operator code | **No** | Tokio moves tasks between threads | +| `TaskContext.get()` in JVM callback | **No** | Returns `null` on non-executor threads | +| Storing `JNIEnv` in an operator | **No** | `JNIEnv` is thread-specific | +| Capturing state at plan creation time | Yes | Runs on executor thread, store in struct | + ## Development Setup 1. Make sure `JAVA_HOME` is set and point to JDK using [support matrix](../user-guide/latest/installation.md) diff --git a/docs/source/contributor-guide/ffi.md b/docs/source/contributor-guide/ffi.md index b1a51ecb2a..c40c189e99 100644 --- a/docs/source/contributor-guide/ffi.md +++ b/docs/source/contributor-guide/ffi.md @@ -177,9 +177,10 @@ message Scan { #### When ownership is NOT transferred to native: -If the data originates from `native_comet` scan (deprecated, will be removed in a future release) or from -`native_iceberg_compat` in some cases, then ownership is not transferred to native and the JVM may re-use the -underlying buffers in the future. +If the data originates from a scan that uses mutable buffers (such as Iceberg scans using the [hybrid Iceberg reader]), +then ownership is not transferred to native and the JVM may re-use the underlying buffers in the future. + +[hybrid Iceberg reader]: https://datafusion.apache.org/comet/user-guide/latest/iceberg.html#hybrid-reader It is critical that the native code performs a deep copy of the arrays if the arrays are to be buffered by operators such as `SortExec` or `ShuffleWriterExec`, otherwise data corruption is likely to occur. diff --git a/docs/source/contributor-guide/parquet_scans.md b/docs/source/contributor-guide/parquet_scans.md index bbacff4d93..7df9394882 100644 --- a/docs/source/contributor-guide/parquet_scans.md +++ b/docs/source/contributor-guide/parquet_scans.md @@ -19,71 +19,60 @@ under the License. # Comet Parquet Scan Implementations -Comet currently has three distinct implementations of the Parquet scan operator. The configuration property -`spark.comet.scan.impl` is used to select an implementation. The default setting is `spark.comet.scan.impl=auto`, and -Comet will choose the most appropriate implementation based on the Parquet schema and other Comet configuration -settings. Most users should not need to change this setting. However, it is possible to force Comet to try and use -a particular implementation for all scan operations by setting this configuration property to one of the following -implementations. - -| Implementation | Description | -| ----------------------- | ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| `native_comet` | **Deprecated.** This implementation provides strong compatibility with Spark but does not support complex types. This is the original scan implementation in Comet and will be removed in a future release. | -| `native_iceberg_compat` | This implementation delegates to DataFusion's `DataSourceExec` but uses a hybrid approach of JVM and native code. This scan is designed to be integrated with Iceberg in the future. | -| `native_datafusion` | This experimental implementation delegates to DataFusion's `DataSourceExec` for full native execution. There are known compatibility issues when using this scan. | - -The `native_datafusion` and `native_iceberg_compat` scans provide the following benefits over the `native_comet` -implementation: - -- Leverages the DataFusion community's ongoing improvements to `DataSourceExec` -- Provides support for reading complex types (structs, arrays, and maps) -- Delegates Parquet decoding to native Rust code rather than JVM-side decoding -- Improves performance - -> **Note on mutable buffers:** Both `native_comet` and `native_iceberg_compat` use reusable mutable buffers -> when transferring data from JVM to native code via Arrow FFI. The `native_iceberg_compat` implementation uses DataFusion's native Parquet reader for data columns, bypassing Comet's mutable buffer infrastructure entirely. However, partition columns still use `ConstantColumnReader`, which relies on Comet's mutable buffers that are reused across batches. This means native operators that buffer data (such as `SortExec` or `ShuffleWriterExec`) must perform deep copies to avoid data corruption. -> See the [FFI documentation](ffi.md) for details on the `arrow_ffi_safe` flag and ownership semantics. - -The `native_datafusion` and `native_iceberg_compat` scans share the following limitations: - -- When reading Parquet files written by systems other than Spark that contain columns with the logical type `UINT_8` - (unsigned 8-bit integers), Comet may produce different results than Spark. Spark maps `UINT_8` to `ShortType`, but - Comet's Arrow-based readers respect the unsigned type and read the data as unsigned rather than signed. Since Comet - cannot distinguish `ShortType` columns that came from `UINT_8` versus signed `INT16`, by default Comet falls back to - Spark when scanning Parquet files containing `ShortType` columns. This behavior can be disabled by setting - `spark.comet.scan.unsignedSmallIntSafetyCheck=false`. Note that `ByteType` columns are always safe because they can - only come from signed `INT8`, where truncation preserves the signed value. -- No support for default values that are nested types (e.g., maps, arrays, structs). Literal default values are supported. -- No support for datetime rebasing detection or the `spark.comet.exceptionOnDatetimeRebase` configuration. When reading - Parquet files containing dates or timestamps written before Spark 3.0 (which used a hybrid Julian/Gregorian calendar), - the `native_comet` implementation can detect these legacy values and either throw an exception or read them without - rebasing. The DataFusion-based implementations do not have this detection capability and will read all dates/timestamps - as if they were written using the Proleptic Gregorian calendar. This may produce incorrect results for dates before - October 15, 1582. -- No support for Spark's Datasource V2 API. When `spark.sql.sources.useV1SourceList` does not include `parquet`, - Spark uses the V2 API for Parquet scans. The DataFusion-based implementations only support the V1 API, so Comet - will fall back to `native_comet` when V2 is enabled. - -The `native_datafusion` scan has some additional limitations: +Comet currently has two distinct implementations of the Parquet scan operator. + +| Scan Implementation | Notes | +| ----------------------- | ---------------------- | +| `native_datafusion` | Fully native scan | +| `native_iceberg_compat` | Hybrid JVM/native scan | + +The configuration property +`spark.comet.scan.impl` is used to select an implementation. The default setting is `spark.comet.scan.impl=auto`, which +currently always uses the `native_iceberg_compat` implementation. Most users should not need to change this setting. +However, it is possible to force Comet to use a particular implementation for all scan operations by setting +this configuration property to one of the following implementations. For example: `--conf spark.comet.scan.impl=native_datafusion`. + +The following features are not supported by either scan implementation, and Comet will fall back to Spark in these scenarios: + +- `ShortType` columns, by default. When reading Parquet files written by systems other than Spark that contain + columns with the logical type `UINT_8` (unsigned 8-bit integers), Comet may produce different results than Spark. + Spark maps `UINT_8` to `ShortType`, but Comet's Arrow-based readers respect the unsigned type and read the data as + unsigned rather than signed. Since Comet cannot distinguish `ShortType` columns that came from `UINT_8` versus + signed `INT16`, by default Comet falls back to Spark when scanning Parquet files containing `ShortType` columns. + This behavior can be disabled by setting `spark.comet.scan.unsignedSmallIntSafetyCheck=false`. Note that `ByteType` + columns are always safe because they can only come from signed `INT8`, where truncation preserves the signed value. +- Default values that are nested types (e.g., maps, arrays, structs). Literal default values are supported. +- Spark's Datasource V2 API. When `spark.sql.sources.useV1SourceList` does not include `parquet`, Spark uses the + V2 API for Parquet scans. The DataFusion-based implementations only support the V1 API. +- Spark metadata columns (e.g., `_metadata.file_path`) +- No support for Dynamic Partition Pruning (DPP) + +The following shared limitation may produce incorrect results without falling back to Spark: + +- No support for datetime rebasing detection or the `spark.comet.exceptionOnDatetimeRebase` configuration. When + reading Parquet files containing dates or timestamps written before Spark 3.0 (which used a hybrid + Julian/Gregorian calendar), dates/timestamps will be read as if they were written using the Proleptic Gregorian + calendar. This may produce incorrect results for dates before October 15, 1582. + +The `native_datafusion` scan has some additional limitations, mostly related to Parquet metadata. All of these +cause Comet to fall back to Spark. - No support for row indexes -- `PARQUET_FIELD_ID_READ_ENABLED` is not respected [#1758] -- There are failures in the Spark SQL test suite [#1545] -- Setting Spark configs `ignoreMissingFiles` or `ignoreCorruptFiles` to `true` is not compatible with Spark +- No support for reading Parquet field IDs +- No support for `input_file_name()`, `input_file_block_start()`, or `input_file_block_length()` SQL functions. + The `native_datafusion` scan does not use Spark's `FileScanRDD`, so these functions cannot populate their values. +- No support for `ignoreMissingFiles` or `ignoreCorruptFiles` being set to `true` -## S3 Support - -There are some differences in S3 support between the scan implementations. - -### `native_comet` (Deprecated) +The `native_iceberg_compat` scan has the following additional limitation that may produce incorrect results +without falling back to Spark: -> **Note:** The `native_comet` scan implementation is deprecated and will be removed in a future release. +- Some Spark configuration values are hard-coded to their defaults rather than respecting user-specified values. + This may produce incorrect results when non-default values are set. The affected configurations are + `spark.sql.parquet.binaryAsString`, `spark.sql.parquet.int96AsTimestamp`, `spark.sql.caseSensitive`, + `spark.sql.parquet.inferTimestampNTZ.enabled`, and `spark.sql.legacy.parquet.nanosAsLong`. See + [issue #1816](https://github.com/apache/datafusion-comet/issues/1816) for more details. -The `native_comet` Parquet scan implementation reads data from S3 using the [Hadoop-AWS module](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html), which -is identical to the approach commonly used with vanilla Spark. AWS credential configuration and other Hadoop S3A -configurations works the same way as in vanilla Spark. - -### `native_datafusion` and `native_iceberg_compat` +## S3 Support The `native_datafusion` and `native_iceberg_compat` Parquet scan implementations completely offload data loading to native code. They use the [`object_store` crate](https://crates.io/crates/object_store) to read data from S3 and @@ -95,7 +84,8 @@ continue to work as long as the configurations are supported and can be translat #### Additional S3 Configuration Options -Beyond credential providers, the `native_datafusion` implementation supports additional S3 configuration options: +Beyond credential providers, the `native_datafusion` and `native_iceberg_compat` implementations support additional +S3 configuration options: | Option | Description | | ------------------------------- | -------------------------------------------------------------------------------------------------- | @@ -108,7 +98,8 @@ All configuration options support bucket-specific overrides using the pattern `f #### Examples -The following examples demonstrate how to configure S3 access with the `native_datafusion` Parquet scan implementation using different authentication methods. +The following examples demonstrate how to configure S3 access with the `native_datafusion` and `native_iceberg_compat` +Parquet scan implementations using different authentication methods. **Example 1: Simple Credentials** @@ -140,11 +131,8 @@ $SPARK_HOME/bin/spark-shell \ #### Limitations -The S3 support of `native_datafusion` has the following limitations: +The S3 support of `native_datafusion` and `native_iceberg_compat` has the following limitations: 1. **Partial Hadoop S3A configuration support**: Not all Hadoop S3A configurations are currently supported. Only the configurations listed in the tables above are translated and applied to the underlying `object_store` crate. 2. **Custom credential providers**: Custom implementations of AWS credential providers are not supported. The implementation only supports the standard credential providers listed in the table above. We are planning to add support for custom credential providers through a JNI-based adapter that will allow calling Java credential providers from native code. See [issue #1829](https://github.com/apache/datafusion-comet/issues/1829) for more details. - -[#1545]: https://github.com/apache/datafusion-comet/issues/1545 -[#1758]: https://github.com/apache/datafusion-comet/issues/1758 diff --git a/docs/source/contributor-guide/roadmap.md b/docs/source/contributor-guide/roadmap.md index 3abe6f3f13..6d99ee5456 100644 --- a/docs/source/contributor-guide/roadmap.md +++ b/docs/source/contributor-guide/roadmap.md @@ -26,11 +26,11 @@ helpful to have a roadmap for some of the major items that require coordination ### Iceberg Integration -Iceberg integration is still a work-in-progress ([#2060]), with major improvements expected in the next few -releases. The default `auto` scan mode now uses `native_iceberg_compat` instead of `native_comet`, enabling -support for complex types. +Iceberg tables reads are now fully native, powered by a scan operator backed by Iceberg-rust ([#2528]). We anticipate +major improvements expected in the next few releases, including bringing Iceberg table format V3 features (_e.g._, +encryption) to the reader. -[#2060]: https://github.com/apache/datafusion-comet/issues/2060 +[#2528]: https://github.com/apache/datafusion-comet/pull/2528 ### Spark 4.0 Support @@ -40,19 +40,16 @@ more Spark SQL tests and fully implementing ANSI support ([#313]) for all suppor [#313]: https://github.com/apache/datafusion-comet/issues/313 [#1637]: https://github.com/apache/datafusion-comet/issues/1637 -### Removing the native_comet scan implementation +### Dynamic Partition Pruning -The `native_comet` scan implementation is now deprecated and will be removed in a future release ([#2186], [#2177]). -This is the original scan implementation that uses mutable buffers (which is incompatible with best practices around -Arrow FFI) and does not support complex types. +Iceberg table scans support Dynamic Partition Pruning (DPP) filters generated by Spark's `PlanDynamicPruningFilters` +optimizer rule ([#3349]). However, we still need to bring this functionality to our Parquet reader. Furthermore, +Spark's `PlanAdaptiveDynamicPruningFilters` optimizer rule runs after Comet's rules, so DPP with Adaptive Query +Execution requires a redesign of Comet's plan translation. We are focused on implementing DPP to keep Comet competitive +with benchmarks that benefit from this feature like TPC-DS. This effort can be tracked at [#3510]. -Now that the default `auto` scan mode uses `native_iceberg_compat` (which is based on DataFusion's `DataSourceExec`), -we can proceed with removing the `native_comet` scan implementation, and then improve the efficiency of our use of -Arrow FFI ([#2171]). - -[#2186]: https://github.com/apache/datafusion-comet/issues/2186 -[#2171]: https://github.com/apache/datafusion-comet/issues/2171 -[#2177]: https://github.com/apache/datafusion-comet/issues/2177 +[#3349]: https://github.com/apache/datafusion-comet/pull/3349 +[#3510]: https://github.com/apache/datafusion-comet/issues/3510 ## Ongoing Improvements diff --git a/docs/source/user-guide/latest/compatibility.md b/docs/source/user-guide/latest/compatibility.md index c09f6a61e6..21695bdf57 100644 --- a/docs/source/user-guide/latest/compatibility.md +++ b/docs/source/user-guide/latest/compatibility.md @@ -62,7 +62,7 @@ the [Comet Supported Expressions Guide](expressions.md) for more information on Comet uses the Rust regexp crate for evaluating regular expressions, and this has different behavior from Java's regular expression engine. Comet will fall back to Spark for patterns that are known to produce different results, but -this can be overridden by setting `spark.comet.regexp.allowIncompatible=true`. +this can be overridden by setting `spark.comet.expression.regexp.allowIncompatible=true`. ## Window Functions diff --git a/docs/spark_expressions_support.md b/docs/spark_expressions_support.md index 2c18cbd08d..5474894108 100644 --- a/docs/spark_expressions_support.md +++ b/docs/spark_expressions_support.md @@ -234,7 +234,7 @@ ### hash_funcs -- [ ] crc32 +- [x] crc32 - [ ] hash - [x] md5 - [ ] sha diff --git a/native/Cargo.lock b/native/Cargo.lock index 41a8f07f9d..bb1da27ce0 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -391,9 +391,13 @@ checksum = "b0f477b951e452a0b6b4a10b53ccd569042d1d01729b519e02074a9c0958a063" [[package]] name = "assertables" -version = "9.8.4" +version = "9.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dcd1f7f2f608b9a888a851f234086946c2ca1dfeadf1431c5082fee0942eeb6" +checksum = "098fbfe90e8af520c4a968eaad7a4031908e473394f16c99cd9cce6369328a68" +dependencies = [ + "regex", + "walkdir", +] [[package]] name = "async-channel" @@ -477,7 +481,7 @@ dependencies = [ "futures-lite", "parking", "polling", - "rustix 1.1.3", + "rustix 1.1.4", "slab", "windows-sys 0.61.2", ] @@ -559,9 +563,9 @@ checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "aws-config" -version = "1.8.13" +version = "1.8.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c456581cb3c77fafcc8c67204a70680d40b61112d6da78c77bd31d945b65f1b5" +checksum = "8a8fc176d53d6fe85017f230405e3255cedb4a02221cb55ed6d76dccbbb099b2" dependencies = [ "aws-credential-types", "aws-runtime", @@ -589,9 +593,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.12" +version = "1.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e26bbf46abc608f2dc61fd6cb3b7b0665497cc259a21520151ed98f8b37d2c79" +checksum = "6d203b0bf2626dcba8665f5cd0871d7c2c0930223d6b6be9097592fea21242d0" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -623,9 +627,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.6.0" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c635c2dc792cb4a11ce1a4f392a925340d1bdf499289b5ec1ec6810954eb43f5" +checksum = "b0f92058d22a46adf53ec57a6a96f34447daf02bff52e8fb956c66bcd5c6ac12" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -636,6 +640,7 @@ dependencies = [ "aws-smithy-types", "aws-types", "bytes", + "bytes-utils", "fastrand", "http 1.4.0", "http-body 1.0.1", @@ -647,9 +652,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.93.0" +version = "1.94.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9dcb38bb33fc0a11f1ffc3e3e85669e0a11a37690b86f77e75306d8f369146a0" +checksum = "699da1961a289b23842d88fe2984c6ff68735fdf9bdcbc69ceaeb2491c9bf434" dependencies = [ "aws-credential-types", "aws-runtime", @@ -671,9 +676,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.95.0" +version = "1.96.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ada8ffbea7bd1be1f53df1dadb0f8fdb04badb13185b3321b929d1ee3caad09" +checksum = "e3e3a4cb3b124833eafea9afd1a6cc5f8ddf3efefffc6651ef76a03cbc6b4981" dependencies = [ "aws-credential-types", "aws-runtime", @@ -695,9 +700,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.97.0" +version = "1.98.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6443ccadc777095d5ed13e21f5c364878c9f5bad4e35187a6cdbd863b0afcad" +checksum = "89c4f19655ab0856375e169865c91264de965bd74c407c7f1e403184b1049409" dependencies = [ "aws-credential-types", "aws-runtime", @@ -720,9 +725,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "1.3.8" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efa49f3c607b92daae0c078d48a4571f599f966dce3caee5f1ea55c4d9073f99" +checksum = "68f6ae9b71597dc5fd115d52849d7a5556ad9265885ad3492ea8d73b93bbc46e" dependencies = [ "aws-credential-types", "aws-smithy-http", @@ -742,9 +747,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.2.12" +version = "1.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3cba48474f1d6807384d06fec085b909f5807e16653c5af5c45dfe89539f0b70" +checksum = "5cc50d0f63e714784b84223abd7abbc8577de8c35d699e0edd19f0a88a08ae13" dependencies = [ "futures-util", "pin-project-lite", @@ -753,9 +758,9 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.63.3" +version = "0.63.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "630e67f2a31094ffa51b210ae030855cb8f3b7ee1329bdd8d085aaf61e8b97fc" +checksum = "af4a8a5fe3e4ac7ee871237c340bbce13e982d37543b65700f4419e039f5d78e" dependencies = [ "aws-smithy-runtime-api", "aws-smithy-types", @@ -774,9 +779,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.9" +version = "1.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12fb0abf49ff0cab20fd31ac1215ed7ce0ea92286ba09e2854b42ba5cabe7525" +checksum = "0709f0083aa19b704132684bc26d3c868e06bd428ccc4373b0b55c3e8748a58b" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -798,27 +803,27 @@ dependencies = [ [[package]] name = "aws-smithy-json" -version = "0.62.3" +version = "0.62.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3cb96aa208d62ee94104645f7b2ecaf77bf27edf161590b6224bfbac2832f979" +checksum = "27b3a779093e18cad88bbae08dc4261e1d95018c4c5b9356a52bcae7c0b6e9bb" dependencies = [ "aws-smithy-types", ] [[package]] name = "aws-smithy-observability" -version = "0.2.4" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0a46543fbc94621080b3cf553eb4cbbdc41dd9780a30c4756400f0139440a1d" +checksum = "4d3f39d5bb871aaf461d59144557f16d5927a5248a983a40654d9cf3b9ba183b" dependencies = [ "aws-smithy-runtime-api", ] [[package]] name = "aws-smithy-query" -version = "0.60.13" +version = "0.60.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cebbddb6f3a5bd81553643e9c7daf3cc3dc5b0b5f398ac668630e8a84e6fff0" +checksum = "05f76a580e3d8f8961e5d48763214025a2af65c2fa4cd1fb7f270a0e107a71b0" dependencies = [ "aws-smithy-types", "urlencoding", @@ -826,9 +831,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.10.0" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3df87c14f0127a0d77eb261c3bc45d5b4833e2a1f63583ebfb728e4852134ee" +checksum = "8fd3dfc18c1ce097cf81fced7192731e63809829c6cbf933c1ec47452d08e1aa" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -851,9 +856,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.11.4" +version = "1.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c55e0837e9b8526f49e0b9bfa9ee18ddee70e853f5bc09c5d11ebceddcb0fec" +checksum = "b4af6e5def28be846479bbeac55aa4603d6f7986fc5da4601ba324dd5d377516" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -868,9 +873,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.4.4" +version = "1.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "576b0d6991c9c32bc14fc340582ef148311f924d41815f641a308b5d11e8e7cd" +checksum = "8ca2734c16913a45343b37313605d84e7d8b34a4611598ce1d25b35860a2bed3" dependencies = [ "base64-simd", "bytes", @@ -891,18 +896,18 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.60.13" +version = "0.60.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11b2f670422ff42bf7065031e72b45bc52a3508bd089f743ea90731ca2b6ea57" +checksum = "b53543b4b86ed43f051644f704a98c7291b3618b67adf057ee77a366fa52fcaa" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "1.3.11" +version = "1.3.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d980627d2dd7bfc32a3c025685a033eeab8d365cc840c631ef59d1b8f428164" +checksum = "6c50f3cdf47caa8d01f2be4a6663ea02418e892f9bbfd82c7b9a3a37eaccdd3a" dependencies = [ "aws-credential-types", "aws-smithy-async", @@ -1296,9 +1301,9 @@ dependencies = [ [[package]] name = "chrono" -version = "0.4.43" +version = "0.4.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fac4744fb15ae8337dc853fee7fb3f4e48c0fbaa23d0afe49c447b4fab126118" +checksum = "c673075a2e0e5f4a1dde27ce9dee1ea4558c7ffe648f576438a20ca1d2acc4b0" dependencies = [ "iana-time-zone", "js-sys", @@ -2822,9 +2827,9 @@ checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" [[package]] name = "futures" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65bc07b1a8bc7c85c5f2e110c476c7389b4554ba72af57d8445ea63a576b0876" +checksum = "8b147ee9d1f6d097cef9ce628cd2ee62288d963e16fb287bd9286455b241382d" dependencies = [ "futures-channel", "futures-core", @@ -2837,9 +2842,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dff15bf788c671c1934e366d07e30c1814a8ef514e1af724a602e8a2fbe1b10" +checksum = "07bbe89c50d7a535e539b8c17bc0b49bdb77747034daa8087407d655f3f7cc1d" dependencies = [ "futures-core", "futures-sink", @@ -2847,15 +2852,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" +checksum = "7e3450815272ef58cec6d564423f6e755e25379b217b0bc688e295ba24df6b1d" [[package]] name = "futures-executor" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e28d1d997f585e54aebc3f97d39e72338912123a67330d723fdbb564d646c9f" +checksum = "baf29c38818342a3b26b5b923639e7b1f4a61fc5e76102d4b1981c6dc7a7579d" dependencies = [ "futures-core", "futures-task", @@ -2864,9 +2869,9 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" +checksum = "cecba35d7ad927e23624b22ad55235f2239cfa44fd10428eecbeba6d6a717718" [[package]] name = "futures-lite" @@ -2883,9 +2888,9 @@ dependencies = [ [[package]] name = "futures-macro" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" +checksum = "e835b70203e41293343137df5c0664546da5745f82ec9b84d40be8336958447b" dependencies = [ "proc-macro2", "quote", @@ -2894,15 +2899,15 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e575fab7d1e0dcb8d0c7bcf9a63ee213816ab51902e6d244a95819acacf1d4f7" +checksum = "c39754e157331b013978ec91992bde1ac089843443c49cbc7f46150b0fad0893" [[package]] name = "futures-task" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" +checksum = "037711b3d59c33004d3856fbdc83b99d4ff37a24768fa1be9ce3538a1cde4393" [[package]] name = "futures-timer" @@ -2912,9 +2917,9 @@ checksum = "f288b0a4f20f9a56b5d1da57e2227c661b7b16168e2f72365f57b63326e29b24" [[package]] name = "futures-util" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" +checksum = "389ca41296e6190b48053de0321d02a77f32f8a5d2461dd38762c0593805c6d6" dependencies = [ "futures-channel", "futures-core", @@ -2924,7 +2929,6 @@ dependencies = [ "futures-task", "memchr", "pin-project-lite", - "pin-utils", "slab", ] @@ -3784,9 +3788,9 @@ checksum = "d26c52dbd32dccf2d10cac7725f8eae5296885fb5703b261f7d0a0739ec807ab" [[package]] name = "linux-raw-sys" -version = "0.11.0" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" +checksum = "32a66949e030da00e8c7d4434b251670a91556f4144941d37452769c25d58a53" [[package]] name = "litemap" @@ -4261,9 +4265,9 @@ dependencies = [ [[package]] name = "parquet" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f6a2926a30477c0b95fea6c28c3072712b139337a242c2cc64817bdc20a8854" +checksum = "6ee96b29972a257b855ff2341b37e61af5f12d6af1158b6dcdb5b31ea07bb3cb" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -4302,9 +4306,9 @@ dependencies = [ [[package]] name = "parquet-variant" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c254fac16af78ad96aa442290cb6504951c4d484fdfcfe58f4588033d30e4c8f" +checksum = "a6c31f8f9bfefb9dbf67b0807e00fd918676954a7477c889be971ac904103184" dependencies = [ "arrow-schema", "chrono", @@ -4316,9 +4320,9 @@ dependencies = [ [[package]] name = "parquet-variant-compute" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2178772f1c5ad7e5da8b569d986d3f5cbb4a4cee915925f28fdc700dbb2e80cf" +checksum = "196cd9f7178fed3ac8d5e6d2b51193818e896bbc3640aea3fde3440114a8f39c" dependencies = [ "arrow", "arrow-schema", @@ -4332,9 +4336,9 @@ dependencies = [ [[package]] name = "parquet-variant-json" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a1510daa121c04848368f9c38d0be425b9418c70be610ecc0aa8071738c0ef3" +checksum = "ed23d7acc90ef60f7fdbcc473fa2fdaefa33542ed15b84388959346d52c839be" dependencies = [ "arrow-schema", "base64", @@ -4479,7 +4483,7 @@ dependencies = [ "concurrent-queue", "hermit-abi", "pin-project-lite", - "rustix 1.1.3", + "rustix 1.1.4", "windows-sys 0.61.2", ] @@ -4582,7 +4586,7 @@ dependencies = [ "chrono", "flate2", "procfs-core", - "rustix 1.1.3", + "rustix 1.1.4", ] [[package]] @@ -5184,14 +5188,14 @@ dependencies = [ [[package]] name = "rustix" -version = "1.1.3" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "146c9e247ccc180c1f61615433868c99f3de3ae256a30a43b49f67c2d9171f34" +checksum = "b6fe4565b9518b83ef4f91bb47ce29620ca828bd32cb7e408f0062e9930ba190" dependencies = [ "bitflags 2.10.0", "errno", "libc", - "linux-raw-sys 0.11.0", + "linux-raw-sys 0.12.1", "windows-sys 0.61.2", ] @@ -5722,14 +5726,14 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "tempfile" -version = "3.25.0" +version = "3.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0136791f7c95b1f6dd99f9cc786b91bb81c3800b639b3478e561ddb7be95e5f1" +checksum = "82a72c767771b47409d2345987fda8628641887d5466101319899796367354a0" dependencies = [ "fastrand", "getrandom 0.4.1", "once_cell", - "rustix 1.1.3", + "rustix 1.1.4", "windows-sys 0.61.2", ] @@ -6178,11 +6182,11 @@ checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" [[package]] name = "uuid" -version = "1.20.0" +version = "1.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee48d38b119b0cd71fe4141b30f5ba9c7c5d9f4e7a3a8b4a674e4b6ef789976f" +checksum = "b672338555252d43fd2240c714dc444b8c6fb0a5c5335e65a07bba7742735ddb" dependencies = [ - "getrandom 0.3.4", + "getrandom 0.4.1", "js-sys", "serde_core", "wasm-bindgen", diff --git a/native/Cargo.toml b/native/Cargo.toml index 2be5f1785c..7c666796df 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -37,7 +37,7 @@ rust-version = "1.88" arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.11.1" } -parquet = { version = "57.2.0", default-features = false, features = ["experimental"] } +parquet = { version = "57.3.0", default-features = false, features = ["experimental"] } datafusion = { version = "51.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-datasource = { version = "51.0.0" } datafusion-spark = { version = "51.0.0" } @@ -45,15 +45,15 @@ datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.10" } -futures = "0.3.28" +futures = "0.3.32" num = "0.4" rand = "0.10" regex = "1.12.3" thiserror = "2" object_store = { version = "0.12.3", features = ["gcp", "azure", "aws", "http"] } url = "2.2" -aws-config = "1.8.13" -aws-credential-types = "1.2.12" +aws-config = "1.8.14" +aws-credential-types = "1.2.13" iceberg = { git = "https://github.com/apache/iceberg-rust", rev = "ee21563" } [profile.release] diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 5d4fa283c1..c7077c2960 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -56,7 +56,7 @@ zstd = "0.13.3" rand = { workspace = true } num = { workspace = true } bytes = { workspace = true } -tempfile = "3.25.0" +tempfile = "3.26.0" itertools = "0.14.0" paste = "1.0.14" datafusion = { workspace = true, features = ["parquet_encryption", "sql"] } @@ -81,7 +81,7 @@ 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" +uuid = "1.21.0" [target.'cfg(target_os = "linux")'.dependencies] procfs = "0.18.0" diff --git a/native/core/src/execution/columnar_to_row.rs b/native/core/src/execution/columnar_to_row.rs index 66b53af2bd..495253c832 100644 --- a/native/core/src/execution/columnar_to_row.rs +++ b/native/core/src/execution/columnar_to_row.rs @@ -1068,7 +1068,19 @@ impl ColumnarToRowContext { })?; Ok(Arc::new(decimal_array)) } - _ => Ok(Arc::clone(array)), + _ => { + // For any other type mismatch, attempt an Arrow cast. + // This handles cases like Int32 → Date32 (which can happen when Spark + // generates default column values using the physical storage type rather + // than the logical type). + let options = CastOptions::default(); + cast_with_options(array, schema_type, &options).map_err(|e| { + CometError::Internal(format!( + "Failed to cast array from {:?} to {:?}: {}", + actual_type, schema_type, e + )) + }) + } } } diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index c2cb1d9102..0193f3012c 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -29,7 +29,7 @@ use crate::{ use arrow::array::{Array, RecordBatch, UInt32Array}; use arrow::compute::{take, TakeOptions}; use arrow::datatypes::DataType as ArrowDataType; -use datafusion::common::ScalarValue; +use datafusion::common::{Result as DataFusionResult, ScalarValue}; use datafusion::execution::disk_manager::DiskManagerMode; use datafusion::execution::memory_pool::MemoryPool; use datafusion::execution::runtime_env::RuntimeEnvBuilder; @@ -46,6 +46,7 @@ use datafusion_spark::function::datetime::date_add::SparkDateAdd; use datafusion_spark::function::datetime::date_sub::SparkDateSub; use datafusion_spark::function::datetime::last_day::SparkLastDay; use datafusion_spark::function::datetime::next_day::SparkNextDay; +use datafusion_spark::function::hash::crc32::SparkCrc32; use datafusion_spark::function::hash::sha1::SparkSha1; use datafusion_spark::function::hash::sha2::SparkSha2; use datafusion_spark::function::map::map_from_entries::MapFromEntries; @@ -72,6 +73,7 @@ use std::path::PathBuf; use std::time::{Duration, Instant}; use std::{sync::Arc, task::Poll}; use tokio::runtime::Runtime; +use tokio::sync::mpsc; use crate::execution::memory_pools::{ create_memory_pool, handle_task_shared_pool_release, parse_memory_pool_config, MemoryPoolConfig, @@ -85,19 +87,31 @@ use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_t use crate::execution::memory_pools::logging_pool::LoggingMemoryPool; use crate::execution::spark_config::{ SparkConfig, COMET_DEBUG_ENABLED, COMET_DEBUG_MEMORY, COMET_EXPLAIN_NATIVE_ENABLED, - COMET_MAX_TEMP_DIRECTORY_SIZE, COMET_TRACING_ENABLED, + COMET_MAX_TEMP_DIRECTORY_SIZE, COMET_TRACING_ENABLED, SPARK_EXECUTOR_CORES, }; use crate::parquet::encryption_support::{CometEncryptionFactory, ENCRYPTION_FACTORY_ID}; use datafusion_comet_proto::spark_operator::operator::OpStruct; use log::info; -use once_cell::sync::Lazy; +use std::sync::OnceLock; #[cfg(feature = "jemalloc")] use tikv_jemalloc_ctl::{epoch, stats}; -static TOKIO_RUNTIME: Lazy = Lazy::new(|| { +static TOKIO_RUNTIME: OnceLock = OnceLock::new(); + +fn parse_usize_env_var(name: &str) -> Option { + std::env::var_os(name).and_then(|n| n.to_str().and_then(|s| s.parse::().ok())) +} + +fn build_runtime(default_worker_threads: Option) -> Runtime { let mut builder = tokio::runtime::Builder::new_multi_thread(); if let Some(n) = parse_usize_env_var("COMET_WORKER_THREADS") { + info!("Comet tokio runtime: using COMET_WORKER_THREADS={n}"); + builder.worker_threads(n); + } else if let Some(n) = default_worker_threads { + info!("Comet tokio runtime: using spark.executor.cores={n} worker threads"); builder.worker_threads(n); + } else { + info!("Comet tokio runtime: using default thread count"); } if let Some(n) = parse_usize_env_var("COMET_MAX_BLOCKING_THREADS") { builder.max_blocking_threads(n); @@ -106,15 +120,17 @@ static TOKIO_RUNTIME: Lazy = Lazy::new(|| { .enable_all() .build() .expect("Failed to create Tokio runtime") -}); +} -fn parse_usize_env_var(name: &str) -> Option { - std::env::var_os(name).and_then(|n| n.to_str().and_then(|s| s.parse::().ok())) +/// Initialize the global Tokio runtime with the given default worker thread count. +/// If the runtime is already initialized, this is a no-op. +pub fn init_runtime(default_worker_threads: usize) { + TOKIO_RUNTIME.get_or_init(|| build_runtime(Some(default_worker_threads))); } /// Function to get a handle to the global Tokio runtime pub fn get_runtime() -> &'static Runtime { - &TOKIO_RUNTIME + TOKIO_RUNTIME.get_or_init(|| build_runtime(None)) } /// Comet native execution context. Kept alive across JNI calls. @@ -135,6 +151,8 @@ struct ExecutionContext { pub input_sources: Vec>, /// The record batch stream to pull results from pub stream: Option, + /// Receives batches from a spawned tokio task (async I/O path) + pub batch_receiver: Option>>, /// Native metrics pub metrics: Arc, // The interval in milliseconds to update metrics @@ -188,6 +206,11 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( let spark_configs = serde::deserialize_config(bytes.as_slice())?; let spark_config: HashMap = spark_configs.entries.into_iter().collect(); + // Initialize the tokio runtime with spark.executor.cores as the default + // worker thread count, falling back to 1 if not set. + let executor_cores = spark_config.get_usize(SPARK_EXECUTOR_CORES, 1); + init_runtime(executor_cores); + // Access Comet configs let debug_native = spark_config.get_bool(COMET_DEBUG_ENABLED); let explain_native = spark_config.get_bool(COMET_EXPLAIN_NATIVE_ENABLED); @@ -286,6 +309,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan( scans: vec![], input_sources, stream: None, + batch_receiver: None, metrics, metrics_update_interval, metrics_last_update_time: Instant::now(), @@ -375,6 +399,7 @@ fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkHex::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkWidthBucket::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(MapFromEntries::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkCrc32::default())); } /// Prepares arrow arrays for output. @@ -528,21 +553,62 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( // Each Comet native execution corresponds to a single Spark partition, // so we should always execute partition 0. let stream = root_op.native_plan.execute(0, task_ctx)?; - exec_context.stream = Some(stream); + + if exec_context.scans.is_empty() { + // No JVM data sources — spawn onto tokio so the executor + // thread parks in blocking_recv instead of busy-polling. + // + // Channel capacity of 2 allows the producer to work one batch + // ahead while the consumer processes the current one via JNI, + // without buffering excessive memory. Increasing this would + // trade memory for latency hiding if JNI/FFI overhead dominates; + // decreasing to 1 would serialize production and consumption. + let (tx, rx) = mpsc::channel(2); + let mut stream = stream; + get_runtime().spawn(async move { + while let Some(batch) = stream.next().await { + if tx.send(batch).await.is_err() { + break; + } + } + }); + exec_context.batch_receiver = Some(rx); + } else { + exec_context.stream = Some(stream); + } } else { // Pull input batches pull_input_batches(exec_context)?; } - // Enter the runtime once for the entire polling loop to avoid repeated - // Runtime::enter() overhead + if let Some(rx) = &mut exec_context.batch_receiver { + match rx.blocking_recv() { + Some(Ok(batch)) => { + update_metrics(&mut env, exec_context)?; + return prepare_output( + &mut env, + array_addrs, + schema_addrs, + batch, + exec_context.debug_native, + ); + } + Some(Err(e)) => { + return Err(e.into()); + } + None => { + log_plan_metrics(exec_context, stage_id, partition); + return Ok(-1); + } + } + } + + // ScanExec path: busy-poll to interleave JVM batch pulls with stream polling get_runtime().block_on(async { loop { - // Polling the stream. let next_item = exec_context.stream.as_mut().unwrap().next(); let poll_output = poll!(next_item); - // update metrics at interval // Only check time every 100 polls to reduce syscall overhead if let Some(interval) = exec_context.metrics_update_interval { exec_context.poll_count_since_metrics_check += 1; @@ -558,7 +624,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( match poll_output { Poll::Ready(Some(output)) => { - // prepare output for FFI transfer return prepare_output( &mut env, array_addrs, @@ -568,43 +633,14 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan( ); } Poll::Ready(None) => { - // Reaches EOF of output. - if exec_context.explain_native { - if let Some(plan) = &exec_context.root_op { - let formatted_plan_str = DisplayableExecutionPlan::with_metrics( - plan.native_plan.as_ref(), - ) - .indent(true); - info!( - "Comet native query plan with metrics (Plan #{} Stage {} Partition {}):\ - \n plan creation took {:?}:\ - \n{formatted_plan_str:}", - plan.plan_id, stage_id, partition, exec_context.plan_creation_time - ); - } - } + log_plan_metrics(exec_context, stage_id, partition); return Ok(-1); } - // A poll pending means the stream is not ready yet. Poll::Pending => { - if exec_context.scans.is_empty() { - // Pure async I/O (e.g., IcebergScanExec, DataSourceExec) - // Yield to let the executor drive I/O instead of busy-polling - tokio::task::yield_now().await; - } else { - // Has ScanExec operators - // Busy-poll to pull batches from JVM - // TODO: Investigate if JNI calls are safe without block_in_place. - // block_in_place prevents Tokio from migrating this task to another thread, - // which is necessary because JNI env is thread-local. If we can guarantee - // thread safety another way, we could remove this wrapper for better perf. - tokio::task::block_in_place(|| { - pull_input_batches(exec_context) - })?; - } - - // Output not ready yet - continue; + // JNI call to pull batches from JVM into ScanExec operators. + // block_in_place lets tokio move other tasks off this worker + // while we wait for JVM data. + tokio::task::block_in_place(|| pull_input_batches(exec_context))?; } } } @@ -646,6 +682,21 @@ fn update_metrics(env: &mut JNIEnv, exec_context: &mut ExecutionContext) -> Come } } +fn log_plan_metrics(exec_context: &ExecutionContext, stage_id: jint, partition: jint) { + if exec_context.explain_native { + if let Some(plan) = &exec_context.root_op { + let formatted_plan_str = + DisplayableExecutionPlan::with_metrics(plan.native_plan.as_ref()).indent(true); + info!( + "Comet native query plan with metrics (Plan #{} Stage {} Partition {}):\ + \n plan creation took {:?}:\ + \n{formatted_plan_str:}", + plan.plan_id, stage_id, partition, exec_context.plan_creation_time + ); + } + } +} + fn convert_datatype_arrays( env: &'_ mut JNIEnv<'_>, serialized_datatypes: JObjectArray, diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index bc20592e90..39ce25002b 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -61,6 +61,8 @@ pub struct IcebergScanExec { catalog_properties: HashMap, /// Pre-planned file scan tasks tasks: Vec, + /// Number of data files to read concurrently + data_file_concurrency_limit: usize, /// Metrics metrics: ExecutionPlanMetricsSet, } @@ -71,6 +73,7 @@ impl IcebergScanExec { schema: SchemaRef, catalog_properties: HashMap, tasks: Vec, + data_file_concurrency_limit: usize, ) -> Result { let output_schema = schema; let plan_properties = Self::compute_properties(Arc::clone(&output_schema), 1); @@ -83,6 +86,7 @@ impl IcebergScanExec { plan_properties, catalog_properties, tasks, + data_file_concurrency_limit, metrics, }) } @@ -158,7 +162,7 @@ impl IcebergScanExec { let reader = iceberg::arrow::ArrowReaderBuilder::new(file_io) .with_batch_size(batch_size) - .with_data_file_concurrency_limit(context.session_config().target_partitions()) + .with_data_file_concurrency_limit(self.data_file_concurrency_limit) .with_row_selection_enabled(true) .build(); diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 92a9cb23e2..ef81cdfbfa 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1176,12 +1176,14 @@ impl PhysicalPlanner { .collect(); let metadata_location = common.metadata_location.clone(); let tasks = parse_file_scan_tasks_from_common(common, &scan.file_scan_tasks)?; + let data_file_concurrency_limit = common.data_file_concurrency_limit as usize; let iceberg_scan = IcebergScanExec::new( metadata_location, required_schema, catalog_properties, tasks, + data_file_concurrency_limit, )?; Ok(( @@ -3905,134 +3907,6 @@ mod tests { }); } - #[test] - fn test_array_repeat() { - let session_ctx = SessionContext::new(); - let task_ctx = session_ctx.task_ctx(); - let planner = PhysicalPlanner::new(Arc::from(session_ctx), 0); - - // Mock scan operator with 3 INT32 columns - let op_scan = Operator { - plan_id: 0, - children: vec![], - op_struct: Some(OpStruct::Scan(spark_operator::Scan { - fields: vec![ - spark_expression::DataType { - type_id: 3, // Int32 - type_info: None, - }, - spark_expression::DataType { - type_id: 3, // Int32 - type_info: None, - }, - spark_expression::DataType { - type_id: 3, // Int32 - type_info: None, - }, - ], - source: "".to_string(), - arrow_ffi_safe: false, - })), - }; - - // Mock expression to read a INT32 column with position 0 - let array_col = spark_expression::Expr { - expr_struct: Some(Bound(spark_expression::BoundReference { - index: 0, - datatype: Some(spark_expression::DataType { - type_id: 3, - type_info: None, - }), - })), - }; - - // Mock expression to read a INT32 column with position 1 - let array_col_1 = spark_expression::Expr { - expr_struct: Some(Bound(spark_expression::BoundReference { - index: 1, - datatype: Some(spark_expression::DataType { - type_id: 3, - type_info: None, - }), - })), - }; - - // Make a projection operator with array_repeat(array_col, array_col_1) - let projection = Operator { - children: vec![op_scan], - plan_id: 0, - op_struct: Some(OpStruct::Projection(spark_operator::Projection { - project_list: vec![spark_expression::Expr { - expr_struct: Some(ExprStruct::ScalarFunc(spark_expression::ScalarFunc { - func: "array_repeat".to_string(), - args: vec![array_col, array_col_1], - return_type: None, - fail_on_error: false, - })), - }], - })), - }; - - // Create a physical plan - let (mut scans, datafusion_plan) = - planner.create_plan(&projection, &mut vec![], 1).unwrap(); - - // Start executing the plan in a separate thread - // The plan waits for incoming batches and emitting result as input comes - let mut stream = datafusion_plan.native_plan.execute(0, task_ctx).unwrap(); - - let runtime = tokio::runtime::Runtime::new().unwrap(); - // create async channel - let (tx, mut rx) = mpsc::channel(1); - - // Send data as input to the plan being executed in a separate thread - runtime.spawn(async move { - // create data batch - // 0, 1, 2 - // 3, 4, 5 - // 6, null, null - let a = Int32Array::from(vec![Some(0), Some(3), Some(6)]); - let b = Int32Array::from(vec![Some(1), Some(4), None]); - let c = Int32Array::from(vec![Some(2), Some(5), None]); - let input_batch1 = InputBatch::Batch(vec![Arc::new(a), Arc::new(b), Arc::new(c)], 3); - let input_batch2 = InputBatch::EOF; - - let batches = vec![input_batch1, input_batch2]; - - for batch in batches.into_iter() { - tx.send(batch).await.unwrap(); - } - }); - - // Wait for the plan to finish executing and assert the result - runtime.block_on(async move { - loop { - let batch = rx.recv().await.unwrap(); - scans[0].set_input_batch(batch); - match poll!(stream.next()) { - Poll::Ready(Some(batch)) => { - assert!(batch.is_ok(), "got error {}", batch.unwrap_err()); - let batch = batch.unwrap(); - let expected = [ - "+--------------+", - "| col_0 |", - "+--------------+", - "| [0] |", - "| [3, 3, 3, 3] |", - "| |", - "+--------------+", - ]; - assert_batches_eq!(expected, &[batch]); - } - Poll::Ready(None) => { - break; - } - _ => {} - } - } - }); - } - /// Executes a `test_data_query` SQL query /// and saves the result into a temp folder using parquet format /// Read the file back to the memory using a custom schema diff --git a/native/core/src/execution/spark_config.rs b/native/core/src/execution/spark_config.rs index b257a5ba68..277c0eb43b 100644 --- a/native/core/src/execution/spark_config.rs +++ b/native/core/src/execution/spark_config.rs @@ -22,10 +22,12 @@ pub(crate) const COMET_DEBUG_ENABLED: &str = "spark.comet.debug.enabled"; pub(crate) const COMET_EXPLAIN_NATIVE_ENABLED: &str = "spark.comet.explain.native.enabled"; pub(crate) const COMET_MAX_TEMP_DIRECTORY_SIZE: &str = "spark.comet.maxTempDirectorySize"; pub(crate) const COMET_DEBUG_MEMORY: &str = "spark.comet.debug.memory"; +pub(crate) const SPARK_EXECUTOR_CORES: &str = "spark.executor.cores"; pub(crate) trait SparkConfig { fn get_bool(&self, name: &str) -> bool; fn get_u64(&self, name: &str, default_value: u64) -> u64; + fn get_usize(&self, name: &str, default_value: usize) -> usize; } impl SparkConfig for HashMap { @@ -40,4 +42,10 @@ impl SparkConfig for HashMap { .and_then(|str_val| str_val.parse::().ok()) .unwrap_or(default_value) } + + fn get_usize(&self, name: &str, default_value: usize) -> usize { + self.get(name) + .and_then(|str_val| str_val.parse::().ok()) + .unwrap_or(default_value) + } } diff --git a/native/fs-hdfs/Cargo.toml b/native/fs-hdfs/Cargo.toml index 4d2fe0605d..d7271f2951 100644 --- a/native/fs-hdfs/Cargo.toml +++ b/native/fs-hdfs/Cargo.toml @@ -52,5 +52,5 @@ url = "^2.2" log = "^0.4" [dev-dependencies] -uuid = {version = "^1.20", features = ["v4"]} -tempfile = "^3.25" +uuid = {version = "^1.21", features = ["v4"]} +tempfile = "^3.26" diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 93872b462c..bf2752bdd0 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -175,6 +175,9 @@ message IcebergScanCommon { repeated PartitionData partition_data_pool = 9; repeated DeleteFileList delete_files_pool = 10; repeated spark.spark_expression.Expr residual_pool = 11; + + // Number of data files to read concurrently within a single task + uint32 data_file_concurrency_limit = 12; } message IcebergScan { diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index fd0a211b29..e7c238f7eb 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -92,6 +92,14 @@ harness = false name = "to_csv" harness = false +[[bench]] +name = "cast_int_to_timestamp" +harness = false + [[test]] name = "test_udf_registration" path = "tests/spark_expr_reg.rs" + +[[bench]] +name = "cast_from_boolean" +harness = false diff --git a/native/spark-expr/benches/cast_from_boolean.rs b/native/spark-expr/benches/cast_from_boolean.rs new file mode 100644 index 0000000000..dbb986df91 --- /dev/null +++ b/native/spark-expr/benches/cast_from_boolean.rs @@ -0,0 +1,89 @@ +// 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. + +use arrow::array::{BooleanBuilder, RecordBatch}; +use arrow::datatypes::{DataType, Field, Schema}; +use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion::physical_expr::expressions::Column; +use datafusion::physical_expr::PhysicalExpr; +use datafusion_comet_spark_expr::{Cast, EvalMode, SparkCastOptions}; +use std::sync::Arc; + +fn criterion_benchmark(c: &mut Criterion) { + let expr = Arc::new(Column::new("a", 0)); + let boolean_batch = create_boolean_batch(); + let spark_cast_options = SparkCastOptions::new(EvalMode::Legacy, "UTC", false); + let cast_to_i8 = Cast::new(expr.clone(), DataType::Int8, spark_cast_options.clone()); + let cast_to_i16 = Cast::new(expr.clone(), DataType::Int16, spark_cast_options.clone()); + let cast_to_i32 = Cast::new(expr.clone(), DataType::Int32, spark_cast_options.clone()); + let cast_to_i64 = Cast::new(expr.clone(), DataType::Int64, spark_cast_options.clone()); + let cast_to_f32 = Cast::new(expr.clone(), DataType::Float32, spark_cast_options.clone()); + let cast_to_f64 = Cast::new(expr.clone(), DataType::Float64, spark_cast_options.clone()); + let cast_to_str = Cast::new(expr.clone(), DataType::Utf8, spark_cast_options.clone()); + let cast_to_decimal = Cast::new(expr, DataType::Decimal128(10, 4), spark_cast_options); + + let mut group = c.benchmark_group("cast_bool".to_string()); + group.bench_function("i8", |b| { + b.iter(|| cast_to_i8.evaluate(&boolean_batch).unwrap()); + }); + group.bench_function("i16", |b| { + b.iter(|| cast_to_i16.evaluate(&boolean_batch).unwrap()); + }); + group.bench_function("i32", |b| { + b.iter(|| cast_to_i32.evaluate(&boolean_batch).unwrap()); + }); + group.bench_function("i64", |b| { + b.iter(|| cast_to_i64.evaluate(&boolean_batch).unwrap()); + }); + group.bench_function("f32", |b| { + b.iter(|| cast_to_f32.evaluate(&boolean_batch).unwrap()); + }); + group.bench_function("f64", |b| { + b.iter(|| cast_to_f64.evaluate(&boolean_batch).unwrap()); + }); + group.bench_function("str", |b| { + b.iter(|| cast_to_str.evaluate(&boolean_batch).unwrap()); + }); + group.bench_function("decimal", |b| { + b.iter(|| cast_to_decimal.evaluate(&boolean_batch).unwrap()); + }); +} + +fn create_boolean_batch() -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Boolean, true)])); + let mut b = BooleanBuilder::with_capacity(1000); + for i in 0..1000 { + if i % 10 == 0 { + b.append_null(); + } else { + b.append_value(rand::random::()); + } + } + let array = b.finish(); + RecordBatch::try_new(schema, vec![Arc::new(array)]).unwrap() +} + +fn config() -> Criterion { + Criterion::default() +} + +criterion_group! { + name = benches; + config = config(); + targets = criterion_benchmark +} +criterion_main!(benches); diff --git a/native/spark-expr/benches/cast_int_to_timestamp.rs b/native/spark-expr/benches/cast_int_to_timestamp.rs new file mode 100644 index 0000000000..20143d2b0e --- /dev/null +++ b/native/spark-expr/benches/cast_int_to_timestamp.rs @@ -0,0 +1,131 @@ +// 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. + +use arrow::array::builder::{Int16Builder, Int32Builder, Int64Builder, Int8Builder}; +use arrow::array::RecordBatch; +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion::physical_expr::{expressions::Column, PhysicalExpr}; +use datafusion_comet_spark_expr::{Cast, EvalMode, SparkCastOptions}; +use std::sync::Arc; + +const BATCH_SIZE: usize = 8192; + +fn criterion_benchmark(c: &mut Criterion) { + // Test with UTC timezone + let spark_cast_options = SparkCastOptions::new(EvalMode::Legacy, "UTC", false); + let timestamp_type = DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())); + + let mut group = c.benchmark_group("cast_int_to_timestamp"); + + // Int8 -> Timestamp + let batch_i8 = create_int8_batch(); + let expr_i8 = Arc::new(Column::new("a", 0)); + let cast_i8_to_ts = Cast::new(expr_i8, timestamp_type.clone(), spark_cast_options.clone()); + group.bench_function("cast_i8_to_timestamp", |b| { + b.iter(|| cast_i8_to_ts.evaluate(&batch_i8).unwrap()); + }); + + // Int16 -> Timestamp + let batch_i16 = create_int16_batch(); + let expr_i16 = Arc::new(Column::new("a", 0)); + let cast_i16_to_ts = Cast::new(expr_i16, timestamp_type.clone(), spark_cast_options.clone()); + group.bench_function("cast_i16_to_timestamp", |b| { + b.iter(|| cast_i16_to_ts.evaluate(&batch_i16).unwrap()); + }); + + // Int32 -> Timestamp + let batch_i32 = create_int32_batch(); + let expr_i32 = Arc::new(Column::new("a", 0)); + let cast_i32_to_ts = Cast::new(expr_i32, timestamp_type.clone(), spark_cast_options.clone()); + group.bench_function("cast_i32_to_timestamp", |b| { + b.iter(|| cast_i32_to_ts.evaluate(&batch_i32).unwrap()); + }); + + // Int64 -> Timestamp + let batch_i64 = create_int64_batch(); + let expr_i64 = Arc::new(Column::new("a", 0)); + let cast_i64_to_ts = Cast::new(expr_i64, timestamp_type.clone(), spark_cast_options.clone()); + group.bench_function("cast_i64_to_timestamp", |b| { + b.iter(|| cast_i64_to_ts.evaluate(&batch_i64).unwrap()); + }); + + group.finish(); +} + +fn create_int8_batch() -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int8, true)])); + let mut b = Int8Builder::with_capacity(BATCH_SIZE); + for i in 0..BATCH_SIZE { + if i % 10 == 0 { + b.append_null(); + } else { + b.append_value(rand::random::()); + } + } + RecordBatch::try_new(schema, vec![Arc::new(b.finish())]).unwrap() +} + +fn create_int16_batch() -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int16, true)])); + let mut b = Int16Builder::with_capacity(BATCH_SIZE); + for i in 0..BATCH_SIZE { + if i % 10 == 0 { + b.append_null(); + } else { + b.append_value(rand::random::()); + } + } + RecordBatch::try_new(schema, vec![Arc::new(b.finish())]).unwrap() +} + +fn create_int32_batch() -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); + let mut b = Int32Builder::with_capacity(BATCH_SIZE); + for i in 0..BATCH_SIZE { + if i % 10 == 0 { + b.append_null(); + } else { + b.append_value(rand::random::()); + } + } + RecordBatch::try_new(schema, vec![Arc::new(b.finish())]).unwrap() +} + +fn create_int64_batch() -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, true)])); + let mut b = Int64Builder::with_capacity(BATCH_SIZE); + for i in 0..BATCH_SIZE { + if i % 10 == 0 { + b.append_null(); + } else { + b.append_value(rand::random::()); + } + } + RecordBatch::try_new(schema, vec![Arc::new(b.finish())]).unwrap() +} + +fn config() -> Criterion { + Criterion::default() +} + +criterion_group! { + name = benches; + config = config(); + targets = criterion_benchmark +} +criterion_main!(benches); diff --git a/native/spark-expr/src/array_funcs/array_repeat.rs b/native/spark-expr/src/array_funcs/array_repeat.rs deleted file mode 100644 index 7ba8f0b910..0000000000 --- a/native/spark-expr/src/array_funcs/array_repeat.rs +++ /dev/null @@ -1,216 +0,0 @@ -// 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. - -use arrow::array::{ - new_null_array, Array, ArrayRef, Capacities, GenericListArray, ListArray, MutableArrayData, - NullBufferBuilder, OffsetSizeTrait, UInt64Array, -}; -use arrow::buffer::OffsetBuffer; -use arrow::compute; -use arrow::compute::cast; -use arrow::datatypes::DataType::{LargeList, List}; -use arrow::datatypes::{DataType, Field}; -use datafusion::common::cast::{as_large_list_array, as_list_array, as_uint64_array}; -use datafusion::common::{exec_err, DataFusionError, ScalarValue}; -use datafusion::logical_expr::ColumnarValue; -use std::sync::Arc; - -pub fn make_scalar_function( - inner: F, -) -> impl Fn(&[ColumnarValue]) -> Result -where - F: Fn(&[ArrayRef]) -> Result, -{ - move |args: &[ColumnarValue]| { - // first, identify if any of the arguments is an Array. If yes, store its `len`, - // as any scalar will need to be converted to an array of len `len`. - let len = args - .iter() - .fold(Option::::None, |acc, arg| match arg { - ColumnarValue::Scalar(_) => acc, - ColumnarValue::Array(a) => Some(a.len()), - }); - - let is_scalar = len.is_none(); - - let args = ColumnarValue::values_to_arrays(args)?; - - let result = (inner)(&args); - - if is_scalar { - // If all inputs are scalar, keeps output as scalar - let result = result.and_then(|arr| ScalarValue::try_from_array(&arr, 0)); - result.map(ColumnarValue::Scalar) - } else { - result.map(ColumnarValue::Array) - } - } -} - -pub fn spark_array_repeat(args: &[ColumnarValue]) -> Result { - make_scalar_function(spark_array_repeat_inner)(args) -} - -/// Array_repeat SQL function -fn spark_array_repeat_inner(args: &[ArrayRef]) -> datafusion::common::Result { - let element = &args[0]; - let count_array = &args[1]; - - let count_array = match count_array.data_type() { - DataType::Int64 => &cast(count_array, &DataType::UInt64)?, - DataType::UInt64 => count_array, - _ => return exec_err!("count must be an integer type"), - }; - - let count_array = as_uint64_array(count_array)?; - - match element.data_type() { - List(_) => { - let list_array = as_list_array(element)?; - general_list_repeat::(list_array, count_array) - } - LargeList(_) => { - let list_array = as_large_list_array(element)?; - general_list_repeat::(list_array, count_array) - } - _ => general_repeat::(element, count_array), - } -} - -/// For each element of `array[i]` repeat `count_array[i]` times. -/// -/// Assumption for the input: -/// 1. `count[i] >= 0` -/// 2. `array.len() == count_array.len()` -/// -/// For example, -/// ```text -/// array_repeat( -/// [1, 2, 3], [2, 0, 1] => [[1, 1], [], [3]] -/// ) -/// ``` -fn general_repeat( - array: &ArrayRef, - count_array: &UInt64Array, -) -> datafusion::common::Result { - let data_type = array.data_type(); - let mut new_values = vec![]; - - let count_vec = count_array - .values() - .to_vec() - .iter() - .map(|x| *x as usize) - .collect::>(); - - let mut nulls = NullBufferBuilder::new(count_array.len()); - - for (row_index, &count) in count_vec.iter().enumerate() { - nulls.append(!count_array.is_null(row_index)); - let repeated_array = if array.is_null(row_index) { - new_null_array(data_type, count) - } else { - let original_data = array.to_data(); - let capacity = Capacities::Array(count); - let mut mutable = - MutableArrayData::with_capacities(vec![&original_data], false, capacity); - - for _ in 0..count { - mutable.extend(0, row_index, row_index + 1); - } - - let data = mutable.freeze(); - arrow::array::make_array(data) - }; - new_values.push(repeated_array); - } - - let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); - let values = compute::concat(&new_values)?; - - Ok(Arc::new(GenericListArray::::try_new( - Arc::new(Field::new_list_field(data_type.to_owned(), true)), - OffsetBuffer::from_lengths(count_vec), - values, - nulls.finish(), - )?)) -} - -/// Handle List version of `general_repeat` -/// -/// For each element of `list_array[i]` repeat `count_array[i]` times. -/// -/// For example, -/// ```text -/// array_repeat( -/// [[1, 2, 3], [4, 5], [6]], [2, 0, 1] => [[[1, 2, 3], [1, 2, 3]], [], [[6]]] -/// ) -/// ``` -fn general_list_repeat( - list_array: &GenericListArray, - count_array: &UInt64Array, -) -> datafusion::common::Result { - let data_type = list_array.data_type(); - let value_type = list_array.value_type(); - let mut new_values = vec![]; - - let count_vec = count_array - .values() - .to_vec() - .iter() - .map(|x| *x as usize) - .collect::>(); - - for (list_array_row, &count) in list_array.iter().zip(count_vec.iter()) { - let list_arr = match list_array_row { - Some(list_array_row) => { - let original_data = list_array_row.to_data(); - let capacity = Capacities::Array(original_data.len() * count); - let mut mutable = - MutableArrayData::with_capacities(vec![&original_data], false, capacity); - - for _ in 0..count { - mutable.extend(0, 0, original_data.len()); - } - - let data = mutable.freeze(); - let repeated_array = arrow::array::make_array(data); - - let list_arr = GenericListArray::::try_new( - Arc::new(Field::new_list_field(value_type.clone(), true)), - OffsetBuffer::::from_lengths(vec![original_data.len(); count]), - repeated_array, - None, - )?; - Arc::new(list_arr) as ArrayRef - } - None => new_null_array(data_type, count), - }; - new_values.push(list_arr); - } - - let lengths = new_values.iter().map(|a| a.len()).collect::>(); - let new_values: Vec<_> = new_values.iter().map(|a| a.as_ref()).collect(); - let values = compute::concat(&new_values)?; - - Ok(Arc::new(ListArray::try_new( - Arc::new(Field::new_list_field(data_type.to_owned(), true)), - OffsetBuffer::::from_lengths(lengths), - values, - None, - )?)) -} diff --git a/native/spark-expr/src/array_funcs/mod.rs b/native/spark-expr/src/array_funcs/mod.rs index 063dd7a5aa..3ef50a252f 100644 --- a/native/spark-expr/src/array_funcs/mod.rs +++ b/native/spark-expr/src/array_funcs/mod.rs @@ -16,13 +16,11 @@ // under the License. mod array_insert; -mod array_repeat; mod get_array_struct_fields; mod list_extract; mod size; pub use array_insert::ArrayInsert; -pub use array_repeat::spark_array_repeat; pub use get_array_struct_fields::GetArrayStructFields; pub use list_extract::ListExtract; pub use size::{spark_size, SparkSizeFunc}; diff --git a/native/spark-expr/src/bitwise_funcs/bitwise_count.rs b/native/spark-expr/src/bitwise_funcs/bitwise_count.rs index 4ab63e532c..b65c507320 100644 --- a/native/spark-expr/src/bitwise_funcs/bitwise_count.rs +++ b/native/spark-expr/src/bitwise_funcs/bitwise_count.rs @@ -16,7 +16,7 @@ // under the License. use arrow::{array::*, datatypes::DataType}; -use datafusion::common::{exec_err, internal_datafusion_err, internal_err, Result}; +use datafusion::common::{exec_err, internal_datafusion_err, Result}; use datafusion::logical_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility}; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; use std::any::Any; @@ -99,15 +99,38 @@ pub fn spark_bit_count(args: [ColumnarValue; 1]) -> Result { DataType::Int16 => compute_op!(array, Int16Array), DataType::Int32 => compute_op!(array, Int32Array), DataType::Int64 => compute_op!(array, Int64Array), - _ => exec_err!("bit_count can't be evaluated because the expression's type is {:?}, not signed int", array.data_type()), + _ => exec_err!("bit_count can't be evaluated because the array's type is {:?}, not signed int/boolean", array.data_type()), }; result.map(ColumnarValue::Array) } - [ColumnarValue::Scalar(_)] => internal_err!("shouldn't go to bitwise count scalar path"), + [ColumnarValue::Scalar(scalar)] => { + use datafusion::common::ScalarValue; + let result = match scalar { + ScalarValue::Int8(Some(v)) => bit_count(v as i64), + ScalarValue::Int16(Some(v)) => bit_count(v as i64), + ScalarValue::Int32(Some(v)) => bit_count(v as i64), + ScalarValue::Int64(Some(v)) => bit_count(v), + ScalarValue::Boolean(Some(v)) => bit_count(if v { 1 } else { 0 }), + ScalarValue::Int8(None) + | ScalarValue::Int16(None) + | ScalarValue::Int32(None) + | ScalarValue::Int64(None) + | ScalarValue::Boolean(None) => { + return Ok(ColumnarValue::Scalar(ScalarValue::Int32(None))) + } + _ => { + return exec_err!( + "bit_count can't be evaluated because the scalar's type is {:?}, not signed int/boolean", + scalar.data_type() + ) + } + }; + Ok(ColumnarValue::Scalar(ScalarValue::Int32(Some(result)))) + } } } -// Here’s the equivalent Rust implementation of the bitCount function (similar to Apache Spark's bitCount for LongType) +// Here’s the equivalent Rust implementation of the bitCount function (similar to Java's bitCount for LongType) fn bit_count(i: i64) -> i32 { let mut u = i as u64; u = u - ((u >> 1) & 0x5555555555555555); @@ -121,7 +144,7 @@ fn bit_count(i: i64) -> i32 { #[cfg(test)] mod tests { - use datafusion::common::{cast::as_int32_array, Result}; + use datafusion::common::{cast::as_int32_array, Result, ScalarValue}; use super::*; @@ -133,8 +156,18 @@ mod tests { Some(12345), Some(89), Some(-3456), + Some(i32::MIN), + Some(i32::MAX), ]))); - let expected = &Int32Array::from(vec![Some(1), None, Some(6), Some(4), Some(54)]); + let expected = &Int32Array::from(vec![ + Some(1), + None, + Some(6), + Some(4), + Some(54), + Some(33), + Some(31), + ]); let ColumnarValue::Array(result) = spark_bit_count([args])? else { unreachable!() @@ -145,4 +178,16 @@ mod tests { Ok(()) } + + #[test] + fn bitwise_count_scalar() { + let args = ColumnarValue::Scalar(ScalarValue::Int64(Some(i64::MAX))); + + match spark_bit_count([args]) { + Ok(ColumnarValue::Scalar(ScalarValue::Int32(Some(actual)))) => { + assert_eq!(actual, 63) + } + _ => unreachable!(), + } + } } diff --git a/native/spark-expr/src/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs index 323a483171..4bfdef7096 100644 --- a/native/spark-expr/src/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -20,9 +20,9 @@ use crate::math_funcs::abs::abs; use crate::math_funcs::checked_arithmetic::{checked_add, checked_div, checked_mul, checked_sub}; use crate::math_funcs::modulo_expr::spark_modulo; use crate::{ - spark_array_repeat, spark_ceil, spark_decimal_div, spark_decimal_integral_div, spark_floor, - spark_isnan, spark_lpad, spark_make_decimal, spark_read_side_padding, spark_round, spark_rpad, - spark_unhex, spark_unscaled_value, EvalMode, SparkBitwiseCount, SparkContains, SparkDateDiff, + spark_ceil, spark_decimal_div, spark_decimal_integral_div, spark_floor, spark_isnan, + spark_lpad, spark_make_decimal, spark_read_side_padding, spark_round, spark_rpad, spark_unhex, + spark_unscaled_value, EvalMode, SparkBitwiseCount, SparkContains, SparkDateDiff, SparkDateTrunc, SparkMakeDate, SparkSizeFunc, SparkStringSpace, }; use arrow::datatypes::DataType; @@ -169,10 +169,6 @@ pub fn create_comet_physical_fun_with_eval_mode( let func = Arc::new(spark_isnan); make_comet_scalar_udf!("isnan", func, without data_type) } - "array_repeat" => { - let func = Arc::new(spark_array_repeat); - make_comet_scalar_udf!("array_repeat", func, without data_type) - } "spark_modulo" => { let func = Arc::new(spark_modulo); make_comet_scalar_udf!("spark_modulo", func, without data_type, fail_on_error) diff --git a/native/spark-expr/src/conversion_funcs/boolean.rs b/native/spark-expr/src/conversion_funcs/boolean.rs new file mode 100644 index 0000000000..db288fa32a --- /dev/null +++ b/native/spark-expr/src/conversion_funcs/boolean.rs @@ -0,0 +1,196 @@ +// 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. + +use crate::SparkResult; +use arrow::array::{ArrayRef, AsArray, Decimal128Array}; +use arrow::datatypes::DataType; +use std::sync::Arc; + +pub fn is_df_cast_from_bool_spark_compatible(to_type: &DataType) -> bool { + use DataType::*; + matches!( + to_type, + Int8 | Int16 | Int32 | Int64 | Float32 | Float64 | Utf8 + ) +} + +// only DF incompatible boolean cast +pub fn cast_boolean_to_decimal( + array: &ArrayRef, + precision: u8, + scale: i8, +) -> SparkResult { + let bool_array = array.as_boolean(); + let scaled_val = 10_i128.pow(scale as u32); + let result: Decimal128Array = bool_array + .iter() + .map(|v| v.map(|b| if b { scaled_val } else { 0 })) + .collect(); + Ok(Arc::new(result.with_precision_and_scale(precision, scale)?)) +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::cast::cast_array; + use crate::{EvalMode, SparkCastOptions}; + use arrow::array::{ + Array, ArrayRef, BooleanArray, Float32Array, Float64Array, Int16Array, Int32Array, + Int64Array, Int8Array, StringArray, + }; + use arrow::datatypes::DataType::Decimal128; + use std::sync::Arc; + + fn test_input_bool_array() -> ArrayRef { + Arc::new(BooleanArray::from(vec![Some(true), Some(false), None])) + } + + fn test_input_spark_opts() -> SparkCastOptions { + SparkCastOptions::new(EvalMode::Legacy, "Asia/Kolkata", false) + } + + #[test] + fn test_is_df_cast_from_bool_spark_compatible() { + assert!(!is_df_cast_from_bool_spark_compatible(&DataType::Boolean)); + assert!(is_df_cast_from_bool_spark_compatible(&DataType::Int8)); + assert!(is_df_cast_from_bool_spark_compatible(&DataType::Int16)); + assert!(is_df_cast_from_bool_spark_compatible(&DataType::Int32)); + assert!(is_df_cast_from_bool_spark_compatible(&DataType::Int64)); + assert!(is_df_cast_from_bool_spark_compatible(&DataType::Float32)); + assert!(is_df_cast_from_bool_spark_compatible(&DataType::Float64)); + assert!(is_df_cast_from_bool_spark_compatible(&DataType::Utf8)); + assert!(!is_df_cast_from_bool_spark_compatible( + &DataType::Decimal128(10, 4) + )); + assert!(!is_df_cast_from_bool_spark_compatible(&DataType::Null)); + } + + #[test] + fn test_bool_to_int8_cast() { + let result = cast_array( + test_input_bool_array(), + &DataType::Int8, + &test_input_spark_opts(), + ) + .unwrap(); + let arr = result.as_any().downcast_ref::().unwrap(); + assert_eq!(arr.value(0), 1); + assert_eq!(arr.value(1), 0); + assert!(arr.is_null(2)); + } + + #[test] + fn test_bool_to_int16_cast() { + let result = cast_array( + test_input_bool_array(), + &DataType::Int16, + &test_input_spark_opts(), + ) + .unwrap(); + let arr = result.as_any().downcast_ref::().unwrap(); + assert_eq!(arr.value(0), 1); + assert_eq!(arr.value(1), 0); + assert!(arr.is_null(2)); + } + + #[test] + fn test_bool_to_int32_cast() { + let result = cast_array( + test_input_bool_array(), + &DataType::Int32, + &test_input_spark_opts(), + ) + .unwrap(); + let arr = result.as_any().downcast_ref::().unwrap(); + assert_eq!(arr.value(0), 1); + assert_eq!(arr.value(1), 0); + assert!(arr.is_null(2)); + } + + #[test] + fn test_bool_to_int64_cast() { + let result = cast_array( + test_input_bool_array(), + &DataType::Int64, + &test_input_spark_opts(), + ) + .unwrap(); + let arr = result.as_any().downcast_ref::().unwrap(); + assert_eq!(arr.value(0), 1); + assert_eq!(arr.value(1), 0); + assert!(arr.is_null(2)); + } + + #[test] + fn test_bool_to_float32_cast() { + let result = cast_array( + test_input_bool_array(), + &DataType::Float32, + &test_input_spark_opts(), + ) + .unwrap(); + let arr = result.as_any().downcast_ref::().unwrap(); + assert_eq!(arr.value(0), 1.0); + assert_eq!(arr.value(1), 0.0); + assert!(arr.is_null(2)); + } + + #[test] + fn test_bool_to_float64_cast() { + let result = cast_array( + test_input_bool_array(), + &DataType::Float64, + &test_input_spark_opts(), + ) + .unwrap(); + let arr = result.as_any().downcast_ref::().unwrap(); + assert_eq!(arr.value(0), 1.0); + assert_eq!(arr.value(1), 0.0); + assert!(arr.is_null(2)); + } + + #[test] + fn test_bool_to_string_cast() { + let result = cast_array( + test_input_bool_array(), + &DataType::Utf8, + &test_input_spark_opts(), + ) + .unwrap(); + let arr = result.as_any().downcast_ref::().unwrap(); + assert_eq!(arr.value(0), "true"); + assert_eq!(arr.value(1), "false"); + assert!(arr.is_null(2)); + } + + #[test] + fn test_bool_to_decimal_cast() { + let result = cast_array( + test_input_bool_array(), + &Decimal128(10, 4), + &test_input_spark_opts(), + ) + .unwrap(); + let expected_arr = Decimal128Array::from(vec![10000_i128, 0_i128]) + .with_precision_and_scale(10, 4) + .unwrap(); + let arr = result.as_any().downcast_ref::().unwrap(); + assert_eq!(arr.value(0), expected_arr.value(0)); + assert_eq!(arr.value(1), expected_arr.value(1)); + assert!(arr.is_null(2)); + } +} diff --git a/native/spark-expr/src/conversion_funcs/cast.rs b/native/spark-expr/src/conversion_funcs/cast.rs index 2809104f26..37604ab4aa 100644 --- a/native/spark-expr/src/conversion_funcs/cast.rs +++ b/native/spark-expr/src/conversion_funcs/cast.rs @@ -15,6 +15,15 @@ // specific language governing permissions and limitations // under the License. +use crate::conversion_funcs::boolean::{ + cast_boolean_to_decimal, is_df_cast_from_bool_spark_compatible, +}; +use crate::conversion_funcs::string::{ + cast_string_to_date, cast_string_to_decimal, cast_string_to_float, cast_string_to_int, + cast_string_to_timestamp, is_df_cast_from_string_spark_compatible, spark_cast_utf8_to_boolean, +}; +use crate::conversion_funcs::utils::cast_overflow; +use crate::conversion_funcs::utils::spark_cast_postprocess; use crate::utils::array_with_timezone; use crate::EvalMode::Legacy; use crate::{timezone, BinaryOutputStyle}; @@ -22,55 +31,45 @@ use crate::{EvalMode, SparkError, SparkResult}; use arrow::array::builder::StringBuilder; use arrow::array::{ BinaryBuilder, BooleanBuilder, Decimal128Builder, DictionaryArray, GenericByteArray, ListArray, - PrimitiveBuilder, StringArray, StructArray, TimestampMicrosecondBuilder, + StringArray, StructArray, TimestampMicrosecondBuilder, }; use arrow::compute::can_cast_types; -use arrow::datatypes::{ - i256, ArrowDictionaryKeyType, ArrowNativeType, DataType, Decimal256Type, GenericBinaryType, - Schema, -}; +use arrow::datatypes::GenericBinaryType; +use arrow::datatypes::{ArrowDictionaryKeyType, ArrowNativeType, DataType, Schema}; +use arrow::error::ArrowError; use arrow::{ array::{ cast::AsArray, types::{Date32Type, Int16Type, Int32Type, Int8Type}, - Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Float64Array, - GenericStringArray, Int16Array, Int32Array, Int64Array, Int8Array, OffsetSizeTrait, - PrimitiveArray, + Array, ArrayRef, Decimal128Array, Float32Array, Float64Array, GenericStringArray, + Int16Array, Int32Array, Int64Array, Int8Array, OffsetSizeTrait, PrimitiveArray, }, - compute::{cast_with_options, take, unary, CastOptions}, + compute::{cast_with_options, take, CastOptions}, datatypes::{ is_validate_decimal_precision, ArrowPrimitiveType, Decimal128Type, Float32Type, - Float64Type, Int64Type, TimestampMicrosecondType, + Float64Type, Int64Type, }, - error::ArrowError, record_batch::RecordBatch, util::display::FormatOptions, }; -use base64::prelude::*; -use chrono::{DateTime, NaiveDate, TimeZone, Timelike}; -use datafusion::common::{ - cast::as_generic_string_array, internal_err, DataFusionError, Result as DataFusionResult, - ScalarValue, -}; +use base64::prelude::BASE64_STANDARD_NO_PAD; +use base64::Engine; +use chrono::{NaiveDate, TimeZone}; +use datafusion::common::{internal_err, DataFusionError, Result as DataFusionResult, ScalarValue}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::ColumnarValue; -use num::{ - cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, ToPrimitive, - Zero, -}; -use regex::Regex; +use num::{cast::AsPrimitive, ToPrimitive, Zero}; use std::str::FromStr; use std::{ any::Any, fmt::{Debug, Display, Formatter}, hash::Hash, - num::Wrapping, sync::Arc, }; static TIMESTAMP_FORMAT: Option<&str> = Some("%Y-%m-%d %H:%M:%S%.f"); -const MICROS_PER_SECOND: i64 = 1000000; +pub(crate) const MICROS_PER_SECOND: i64 = 1000000; static CAST_OPTIONS: CastOptions = CastOptions { safe: true, @@ -79,67 +78,6 @@ static CAST_OPTIONS: CastOptions = CastOptions { .with_timestamp_format(TIMESTAMP_FORMAT), }; -struct TimeStampInfo { - year: i32, - month: u32, - day: u32, - hour: u32, - minute: u32, - second: u32, - microsecond: u32, -} - -impl Default for TimeStampInfo { - fn default() -> Self { - TimeStampInfo { - year: 1, - month: 1, - day: 1, - hour: 0, - minute: 0, - second: 0, - microsecond: 0, - } - } -} - -impl TimeStampInfo { - pub fn with_year(&mut self, year: i32) -> &mut Self { - self.year = year; - self - } - - pub fn with_month(&mut self, month: u32) -> &mut Self { - self.month = month; - self - } - - pub fn with_day(&mut self, day: u32) -> &mut Self { - self.day = day; - self - } - - pub fn with_hour(&mut self, hour: u32) -> &mut Self { - self.hour = hour; - self - } - - pub fn with_minute(&mut self, minute: u32) -> &mut Self { - self.minute = minute; - self - } - - pub fn with_second(&mut self, second: u32) -> &mut Self { - self.second = second; - self - } - - pub fn with_microsecond(&mut self, microsecond: u32) -> &mut Self { - self.microsecond = microsecond; - self - } -} - #[derive(Debug, Eq)] pub struct Cast { pub child: Arc, @@ -163,54 +101,6 @@ impl Hash for Cast { } } -macro_rules! cast_utf8_to_int { - ($array:expr, $array_type:ty, $parse_fn:expr) => {{ - let len = $array.len(); - let mut cast_array = PrimitiveArray::<$array_type>::builder(len); - let parse_fn = $parse_fn; - if $array.null_count() == 0 { - for i in 0..len { - if let Some(cast_value) = parse_fn($array.value(i))? { - cast_array.append_value(cast_value); - } else { - cast_array.append_null() - } - } - } else { - for i in 0..len { - if $array.is_null(i) { - cast_array.append_null() - } else if let Some(cast_value) = parse_fn($array.value(i))? { - cast_array.append_value(cast_value); - } else { - cast_array.append_null() - } - } - } - let result: SparkResult = Ok(Arc::new(cast_array.finish()) as ArrayRef); - result - }}; -} -macro_rules! cast_utf8_to_timestamp { - ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident, $tz:expr) => {{ - let len = $array.len(); - let mut cast_array = PrimitiveArray::<$array_type>::builder(len).with_timezone("UTC"); - for i in 0..len { - if $array.is_null(i) { - cast_array.append_null() - } else if let Ok(Some(cast_value)) = - $cast_method($array.value(i).trim(), $eval_mode, $tz) - { - cast_array.append_value(cast_value); - } else { - cast_array.append_null() - } - } - let result: ArrayRef = Arc::new(cast_array.finish()) as ArrayRef; - result - }}; -} - macro_rules! cast_float_to_string { ($from:expr, $eval_mode:expr, $type:ty, $output_type:ty, $offset_type:ty) => {{ @@ -613,6 +503,23 @@ macro_rules! cast_decimal_to_int32_up { }}; } +macro_rules! cast_int_to_timestamp_impl { + ($array:expr, $builder:expr, $primitive_type:ty) => {{ + let arr = $array.as_primitive::<$primitive_type>(); + for i in 0..arr.len() { + if arr.is_null(i) { + $builder.append_null(); + } else { + // saturating_mul limits to i64::MIN/MAX on overflow instead of panicking, + // which could occur when converting extreme values (e.g., Long.MIN_VALUE) + // matching spark behavior (irrespective of EvalMode) + let micros = (arr.value(i) as i64).saturating_mul(MICROS_PER_SECOND); + $builder.append_value(micros); + } + } + }}; +} + // copied from arrow::dataTypes::Decimal128Type since Decimal128Type::format_decimal can't be called directly fn format_decimal_str(value_str: &str, precision: usize, scale: i8) -> String { let (sign, rest) = match value_str.strip_prefix('-') { @@ -759,7 +666,7 @@ fn dict_from_values( Ok(Arc::new(dict_array)) } -fn cast_array( +pub(crate) fn cast_array( array: ArrayRef, to_type: &DataType, cast_options: &SparkCastOptions, @@ -915,6 +822,7 @@ fn cast_array( (Boolean, Decimal128(precision, scale)) => { cast_boolean_to_decimal(&array, *precision, *scale) } + (Int8 | Int16 | Int32 | Int64, Timestamp(_, tz)) => cast_int_to_timestamp(&array, tz), _ if cast_options.is_adapting_schema || is_datafusion_spark_compatible(from_type, to_type) => { @@ -933,6 +841,29 @@ fn cast_array( Ok(spark_cast_postprocess(cast_result?, from_type, to_type)) } +fn cast_int_to_timestamp( + array_ref: &ArrayRef, + target_tz: &Option>, +) -> SparkResult { + // Input is seconds since epoch, multiply by MICROS_PER_SECOND to get microseconds. + let mut builder = TimestampMicrosecondBuilder::with_capacity(array_ref.len()); + + match array_ref.data_type() { + DataType::Int8 => cast_int_to_timestamp_impl!(array_ref, builder, Int8Type), + DataType::Int16 => cast_int_to_timestamp_impl!(array_ref, builder, Int16Type), + DataType::Int32 => cast_int_to_timestamp_impl!(array_ref, builder, Int32Type), + DataType::Int64 => cast_int_to_timestamp_impl!(array_ref, builder, Int64Type), + dt => { + return Err(SparkError::Internal(format!( + "Unsupported type for cast_int_to_timestamp: {:?}", + dt + ))) + } + } + + Ok(Arc::new(builder.finish().with_timezone_opt(target_tz.clone())) as ArrayRef) +} + fn cast_date_to_timestamp( array_ref: &ArrayRef, cast_options: &SparkCastOptions, @@ -977,164 +908,6 @@ fn cast_date_to_timestamp( )) } -fn cast_boolean_to_decimal(array: &ArrayRef, precision: u8, scale: i8) -> SparkResult { - let bool_array = array.as_boolean(); - let scaled_val = 10_i128.pow(scale as u32); - let result: Decimal128Array = bool_array - .iter() - .map(|v| v.map(|b| if b { scaled_val } else { 0 })) - .collect(); - Ok(Arc::new(result.with_precision_and_scale(precision, scale)?)) -} - -fn cast_string_to_float( - array: &ArrayRef, - to_type: &DataType, - eval_mode: EvalMode, -) -> SparkResult { - match to_type { - DataType::Float32 => cast_string_to_float_impl::(array, eval_mode, "FLOAT"), - DataType::Float64 => cast_string_to_float_impl::(array, eval_mode, "DOUBLE"), - _ => Err(SparkError::Internal(format!( - "Unsupported cast to float type: {:?}", - to_type - ))), - } -} - -fn cast_string_to_float_impl( - array: &ArrayRef, - eval_mode: EvalMode, - type_name: &str, -) -> SparkResult -where - T::Native: FromStr + num::Float, -{ - let arr = array - .as_any() - .downcast_ref::() - .ok_or_else(|| SparkError::Internal("Expected string array".to_string()))?; - - let mut builder = PrimitiveBuilder::::with_capacity(arr.len()); - - for i in 0..arr.len() { - if arr.is_null(i) { - builder.append_null(); - } else { - let str_value = arr.value(i).trim(); - match parse_string_to_float(str_value) { - Some(v) => builder.append_value(v), - None => { - if eval_mode == EvalMode::Ansi { - return Err(invalid_value(arr.value(i), "STRING", type_name)); - } - builder.append_null(); - } - } - } - } - - Ok(Arc::new(builder.finish())) -} - -/// helper to parse floats from string inputs -fn parse_string_to_float(s: &str) -> Option -where - F: FromStr + num::Float, -{ - // Handle +inf / -inf - if s.eq_ignore_ascii_case("inf") - || s.eq_ignore_ascii_case("+inf") - || s.eq_ignore_ascii_case("infinity") - || s.eq_ignore_ascii_case("+infinity") - { - return Some(F::infinity()); - } - if s.eq_ignore_ascii_case("-inf") || s.eq_ignore_ascii_case("-infinity") { - return Some(F::neg_infinity()); - } - if s.eq_ignore_ascii_case("nan") { - return Some(F::nan()); - } - // Remove D/F suffix if present - let pruned_float_str = - if s.ends_with("d") || s.ends_with("D") || s.ends_with('f') || s.ends_with('F') { - &s[..s.len() - 1] - } else { - s - }; - // Rust's parse logic already handles scientific notations so we just rely on it - pruned_float_str.parse::().ok() -} - -fn cast_binary_to_string( - array: &dyn Array, - spark_cast_options: &SparkCastOptions, -) -> Result { - let input = array - .as_any() - .downcast_ref::>>() - .unwrap(); - - fn binary_formatter(value: &[u8], spark_cast_options: &SparkCastOptions) -> String { - match spark_cast_options.binary_output_style { - Some(s) => spark_binary_formatter(value, s), - None => cast_binary_formatter(value), - } - } - - let output_array = input - .iter() - .map(|value| match value { - Some(value) => Ok(Some(binary_formatter(value, spark_cast_options))), - _ => Ok(None), - }) - .collect::, ArrowError>>()?; - Ok(Arc::new(output_array)) -} - -/// This function mimics the [BinaryFormatter]: https://github.com/apache/spark/blob/v4.0.0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala#L449-L468 -/// used by SparkSQL's ToPrettyString expression. -/// The BinaryFormatter was [introduced]: https://issues.apache.org/jira/browse/SPARK-47911 in Spark 4.0.0 -/// Before Spark 4.0.0, the default is SPACE_DELIMITED_UPPERCASE_HEX -fn spark_binary_formatter(value: &[u8], binary_output_style: BinaryOutputStyle) -> String { - match binary_output_style { - BinaryOutputStyle::Utf8 => String::from_utf8(value.to_vec()).unwrap(), - BinaryOutputStyle::Basic => { - format!( - "{:?}", - value - .iter() - .map(|v| i8::from_ne_bytes([*v])) - .collect::>() - ) - } - BinaryOutputStyle::Base64 => BASE64_STANDARD_NO_PAD.encode(value), - BinaryOutputStyle::Hex => value - .iter() - .map(|v| hex::encode_upper([*v])) - .collect::(), - BinaryOutputStyle::HexDiscrete => { - // Spark's default SPACE_DELIMITED_UPPERCASE_HEX - format!( - "[{}]", - value - .iter() - .map(|v| hex::encode_upper([*v])) - .collect::>() - .join(" ") - ) - } - } -} - -fn cast_binary_formatter(value: &[u8]) -> String { - match String::from_utf8(value.to_vec()) { - Ok(value) => value, - Err(_) => unsafe { String::from_utf8_unchecked(value.to_vec()) }, - } -} - /// Determines if DataFusion supports the given cast in a way that is /// compatible with Spark fn is_datafusion_spark_compatible(from_type: &DataType, to_type: &DataType) -> bool { @@ -1145,16 +918,7 @@ fn is_datafusion_spark_compatible(from_type: &DataType, to_type: &DataType) -> b DataType::Null => { matches!(to_type, DataType::List(_)) } - DataType::Boolean => matches!( - to_type, - DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::Float32 - | DataType::Float64 - | DataType::Utf8 - ), + DataType::Boolean => is_df_cast_from_bool_spark_compatible(to_type), DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => { matches!( to_type, @@ -1190,7 +954,7 @@ fn is_datafusion_spark_compatible(from_type: &DataType, to_type: &DataType) -> b | DataType::Decimal256(_, _) | DataType::Utf8 // note that there can be formatting differences ), - DataType::Utf8 => matches!(to_type, DataType::Binary), + DataType::Utf8 => is_df_cast_from_string_spark_compatible(to_type), DataType::Date32 => matches!(to_type, DataType::Int32 | DataType::Utf8), DataType::Timestamp(_, _) => { matches!( @@ -1337,148 +1101,6 @@ fn casts_struct_to_string( Ok(Arc::new(builder.finish())) } -fn cast_string_to_int( - to_type: &DataType, - array: &ArrayRef, - eval_mode: EvalMode, -) -> SparkResult { - let string_array = array - .as_any() - .downcast_ref::>() - .expect("cast_string_to_int expected a string array"); - - // Select parse function once per batch based on eval_mode - let cast_array: ArrayRef = - match (to_type, eval_mode) { - (DataType::Int8, EvalMode::Legacy) => { - cast_utf8_to_int!(string_array, Int8Type, parse_string_to_i8_legacy)? - } - (DataType::Int8, EvalMode::Ansi) => { - cast_utf8_to_int!(string_array, Int8Type, parse_string_to_i8_ansi)? - } - (DataType::Int8, EvalMode::Try) => { - cast_utf8_to_int!(string_array, Int8Type, parse_string_to_i8_try)? - } - (DataType::Int16, EvalMode::Legacy) => { - cast_utf8_to_int!(string_array, Int16Type, parse_string_to_i16_legacy)? - } - (DataType::Int16, EvalMode::Ansi) => { - cast_utf8_to_int!(string_array, Int16Type, parse_string_to_i16_ansi)? - } - (DataType::Int16, EvalMode::Try) => { - cast_utf8_to_int!(string_array, Int16Type, parse_string_to_i16_try)? - } - (DataType::Int32, EvalMode::Legacy) => cast_utf8_to_int!( - string_array, - Int32Type, - |s| do_parse_string_to_int_legacy::(s, i32::MIN) - )?, - (DataType::Int32, EvalMode::Ansi) => { - cast_utf8_to_int!(string_array, Int32Type, |s| do_parse_string_to_int_ansi::< - i32, - >( - s, "INT", i32::MIN - ))? - } - (DataType::Int32, EvalMode::Try) => { - cast_utf8_to_int!( - string_array, - Int32Type, - |s| do_parse_string_to_int_try::(s, i32::MIN) - )? - } - (DataType::Int64, EvalMode::Legacy) => cast_utf8_to_int!( - string_array, - Int64Type, - |s| do_parse_string_to_int_legacy::(s, i64::MIN) - )?, - (DataType::Int64, EvalMode::Ansi) => { - cast_utf8_to_int!(string_array, Int64Type, |s| do_parse_string_to_int_ansi::< - i64, - >( - s, "BIGINT", i64::MIN - ))? - } - (DataType::Int64, EvalMode::Try) => { - cast_utf8_to_int!( - string_array, - Int64Type, - |s| do_parse_string_to_int_try::(s, i64::MIN) - )? - } - (dt, _) => unreachable!( - "{}", - format!("invalid integer type {dt} in cast from string") - ), - }; - Ok(cast_array) -} - -fn cast_string_to_date( - array: &ArrayRef, - to_type: &DataType, - eval_mode: EvalMode, -) -> SparkResult { - let string_array = array - .as_any() - .downcast_ref::>() - .expect("Expected a string array"); - - if to_type != &DataType::Date32 { - unreachable!("Invalid data type {:?} in cast from string", to_type); - } - - let len = string_array.len(); - let mut cast_array = PrimitiveArray::::builder(len); - - for i in 0..len { - let value = if string_array.is_null(i) { - None - } else { - match date_parser(string_array.value(i), eval_mode) { - Ok(Some(cast_value)) => Some(cast_value), - Ok(None) => None, - Err(e) => return Err(e), - } - }; - - match value { - Some(cast_value) => cast_array.append_value(cast_value), - None => cast_array.append_null(), - } - } - - Ok(Arc::new(cast_array.finish()) as ArrayRef) -} - -fn cast_string_to_timestamp( - array: &ArrayRef, - to_type: &DataType, - eval_mode: EvalMode, - timezone_str: &str, -) -> SparkResult { - let string_array = array - .as_any() - .downcast_ref::>() - .expect("Expected a string array"); - - let tz = &timezone::Tz::from_str(timezone_str).unwrap(); - - let cast_array: ArrayRef = match to_type { - DataType::Timestamp(_, _) => { - cast_utf8_to_timestamp!( - string_array, - eval_mode, - TimestampMicrosecondType, - timestamp_parser, - tz - ) - } - _ => unreachable!("Invalid data type {:?} in cast from string", to_type), - }; - Ok(cast_array) -} - fn cast_float64_to_decimal128( array: &dyn Array, precision: u8, @@ -1697,38 +1319,6 @@ fn spark_cast_int_to_int( } } -fn spark_cast_utf8_to_boolean( - from: &dyn Array, - eval_mode: EvalMode, -) -> SparkResult -where - OffsetSize: OffsetSizeTrait, -{ - let array = from - .as_any() - .downcast_ref::>() - .unwrap(); - - let output_array = array - .iter() - .map(|value| match value { - Some(value) => match value.to_ascii_lowercase().trim() { - "t" | "true" | "y" | "yes" | "1" => Ok(Some(true)), - "f" | "false" | "n" | "no" | "0" => Ok(Some(false)), - _ if eval_mode == EvalMode::Ansi => Err(SparkError::CastInvalidValue { - value: value.to_string(), - from_type: "STRING".to_string(), - to_type: "BOOLEAN".to_string(), - }), - _ => Ok(None), - }, - _ => Ok(None), - }) - .collect::>()?; - - Ok(Arc::new(output_array)) -} - fn spark_cast_decimal_to_boolean(array: &dyn Array) -> SparkResult { let decimal_array = array.as_primitive::(); let mut result = BooleanBuilder::with_capacity(decimal_array.len()); @@ -1882,551 +1472,19 @@ fn spark_cast_nonintegral_numeric_to_integral( } } -fn parse_string_to_i8_legacy(str: &str) -> SparkResult> { - match do_parse_string_to_int_legacy::(str, i32::MIN)? { - Some(v) if v >= i8::MIN as i32 && v <= i8::MAX as i32 => Ok(Some(v as i8)), - _ => Ok(None), +impl Display for Cast { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Cast [data_type: {}, timezone: {}, child: {}, eval_mode: {:?}]", + self.data_type, self.cast_options.timezone, self.child, &self.cast_options.eval_mode + ) } } -fn parse_string_to_i8_ansi(str: &str) -> SparkResult> { - match do_parse_string_to_int_ansi::(str, "TINYINT", i32::MIN)? { - Some(v) if v >= i8::MIN as i32 && v <= i8::MAX as i32 => Ok(Some(v as i8)), - _ => Err(invalid_value(str, "STRING", "TINYINT")), - } -} - -fn parse_string_to_i8_try(str: &str) -> SparkResult> { - match do_parse_string_to_int_try::(str, i32::MIN)? { - Some(v) if v >= i8::MIN as i32 && v <= i8::MAX as i32 => Ok(Some(v as i8)), - _ => Ok(None), - } -} - -fn parse_string_to_i16_legacy(str: &str) -> SparkResult> { - match do_parse_string_to_int_legacy::(str, i32::MIN)? { - Some(v) if v >= i16::MIN as i32 && v <= i16::MAX as i32 => Ok(Some(v as i16)), - _ => Ok(None), - } -} - -fn parse_string_to_i16_ansi(str: &str) -> SparkResult> { - match do_parse_string_to_int_ansi::(str, "SMALLINT", i32::MIN)? { - Some(v) if v >= i16::MIN as i32 && v <= i16::MAX as i32 => Ok(Some(v as i16)), - _ => Err(invalid_value(str, "STRING", "SMALLINT")), - } -} - -fn parse_string_to_i16_try(str: &str) -> SparkResult> { - match do_parse_string_to_int_try::(str, i32::MIN)? { - Some(v) if v >= i16::MIN as i32 && v <= i16::MAX as i32 => Ok(Some(v as i16)), - _ => Ok(None), - } -} - -/// Parses sign and returns (is_negative, remaining_bytes after sign) -/// Returns None if invalid (empty input, or just "+" or "-") -fn parse_sign(bytes: &[u8]) -> Option<(bool, &[u8])> { - let (&first, rest) = bytes.split_first()?; - match first { - b'-' if !rest.is_empty() => Some((true, rest)), - b'+' if !rest.is_empty() => Some((false, rest)), - _ => Some((false, bytes)), - } -} - -/// Finalizes the result by applying the sign. Returns None if overflow would occur. -fn finalize_int_result(result: T, negative: bool) -> Option { - if negative { - Some(result) - } else { - result.checked_neg().filter(|&n| n >= T::zero()) - } -} - -/// Equivalent to -/// - org.apache.spark.unsafe.types.UTF8String.toInt(IntWrapper intWrapper, boolean allowDecimal) -/// - org.apache.spark.unsafe.types.UTF8String.toLong(LongWrapper longWrapper, boolean allowDecimal) -fn do_parse_string_to_int_legacy + Copy>( - str: &str, - min_value: T, -) -> SparkResult> { - let trimmed_bytes = str.as_bytes().trim_ascii(); - - let (negative, digits) = match parse_sign(trimmed_bytes) { - Some(result) => result, - None => return Ok(None), - }; - - let mut result: T = T::zero(); - let radix = T::from(10_u8); - let stop_value = min_value / radix; - - let mut iter = digits.iter(); - - // Parse integer portion until '.' or end - for &ch in iter.by_ref() { - if ch == b'.' { - break; - } - - if !ch.is_ascii_digit() { - return Ok(None); - } - - if result < stop_value { - return Ok(None); - } - let v = result * radix; - let digit: T = T::from(ch - b'0'); - match v.checked_sub(&digit) { - Some(x) if x <= T::zero() => result = x, - _ => return Ok(None), - } - } - - // Validate decimal portion (digits only, values ignored) - for &ch in iter { - if !ch.is_ascii_digit() { - return Ok(None); - } - } - - Ok(finalize_int_result(result, negative)) -} - -fn do_parse_string_to_int_ansi + Copy>( - str: &str, - type_name: &str, - min_value: T, -) -> SparkResult> { - let error = || Err(invalid_value(str, "STRING", type_name)); - - let trimmed_bytes = str.as_bytes().trim_ascii(); - - let (negative, digits) = match parse_sign(trimmed_bytes) { - Some(result) => result, - None => return error(), - }; - - let mut result: T = T::zero(); - let radix = T::from(10_u8); - let stop_value = min_value / radix; - - for &ch in digits { - if ch == b'.' || !ch.is_ascii_digit() { - return error(); - } - - if result < stop_value { - return error(); - } - let v = result * radix; - let digit: T = T::from(ch - b'0'); - match v.checked_sub(&digit) { - Some(x) if x <= T::zero() => result = x, - _ => return error(), - } - } - - finalize_int_result(result, negative) - .map(Some) - .ok_or_else(|| invalid_value(str, "STRING", type_name)) -} - -fn do_parse_string_to_int_try + Copy>( - str: &str, - min_value: T, -) -> SparkResult> { - let trimmed_bytes = str.as_bytes().trim_ascii(); - - let (negative, digits) = match parse_sign(trimmed_bytes) { - Some(result) => result, - None => return Ok(None), - }; - - let mut result: T = T::zero(); - let radix = T::from(10_u8); - let stop_value = min_value / radix; - - for &ch in digits { - if ch == b'.' || !ch.is_ascii_digit() { - return Ok(None); - } - - if result < stop_value { - return Ok(None); - } - let v = result * radix; - let digit: T = T::from(ch - b'0'); - match v.checked_sub(&digit) { - Some(x) if x <= T::zero() => result = x, - _ => return Ok(None), - } - } - - Ok(finalize_int_result(result, negative)) -} - -fn cast_string_to_decimal( - array: &ArrayRef, - to_type: &DataType, - precision: &u8, - scale: &i8, - eval_mode: EvalMode, -) -> SparkResult { - match to_type { - DataType::Decimal128(_, _) => { - cast_string_to_decimal128_impl(array, eval_mode, *precision, *scale) - } - DataType::Decimal256(_, _) => { - cast_string_to_decimal256_impl(array, eval_mode, *precision, *scale) - } - _ => Err(SparkError::Internal(format!( - "Unexpected type in cast_string_to_decimal: {:?}", - to_type - ))), - } -} - -fn cast_string_to_decimal128_impl( - array: &ArrayRef, - eval_mode: EvalMode, - precision: u8, - scale: i8, -) -> SparkResult { - let string_array = array - .as_any() - .downcast_ref::() - .ok_or_else(|| SparkError::Internal("Expected string array".to_string()))?; - - let mut decimal_builder = Decimal128Builder::with_capacity(string_array.len()); - - for i in 0..string_array.len() { - if string_array.is_null(i) { - decimal_builder.append_null(); - } else { - let str_value = string_array.value(i); - match parse_string_to_decimal(str_value, precision, scale) { - Ok(Some(decimal_value)) => { - decimal_builder.append_value(decimal_value); - } - Ok(None) => { - if eval_mode == EvalMode::Ansi { - return Err(invalid_value( - string_array.value(i), - "STRING", - &format!("DECIMAL({},{})", precision, scale), - )); - } - decimal_builder.append_null(); - } - Err(e) => { - if eval_mode == EvalMode::Ansi { - return Err(e); - } - decimal_builder.append_null(); - } - } - } - } - - Ok(Arc::new( - decimal_builder - .with_precision_and_scale(precision, scale)? - .finish(), - )) -} - -fn cast_string_to_decimal256_impl( - array: &ArrayRef, - eval_mode: EvalMode, - precision: u8, - scale: i8, -) -> SparkResult { - let string_array = array - .as_any() - .downcast_ref::() - .ok_or_else(|| SparkError::Internal("Expected string array".to_string()))?; - - let mut decimal_builder = PrimitiveBuilder::::with_capacity(string_array.len()); - - for i in 0..string_array.len() { - if string_array.is_null(i) { - decimal_builder.append_null(); - } else { - let str_value = string_array.value(i); - match parse_string_to_decimal(str_value, precision, scale) { - Ok(Some(decimal_value)) => { - // Convert i128 to i256 - let i256_value = i256::from_i128(decimal_value); - decimal_builder.append_value(i256_value); - } - Ok(None) => { - if eval_mode == EvalMode::Ansi { - return Err(invalid_value( - str_value, - "STRING", - &format!("DECIMAL({},{})", precision, scale), - )); - } - decimal_builder.append_null(); - } - Err(e) => { - if eval_mode == EvalMode::Ansi { - return Err(e); - } - decimal_builder.append_null(); - } - } - } - } - - Ok(Arc::new( - decimal_builder - .with_precision_and_scale(precision, scale)? - .finish(), - )) -} - -/// Parse a string to decimal following Spark's behavior -fn parse_string_to_decimal(input_str: &str, precision: u8, scale: i8) -> SparkResult> { - let string_bytes = input_str.as_bytes(); - let mut start = 0; - let mut end = string_bytes.len(); - - // trim whitespaces - while start < end && string_bytes[start].is_ascii_whitespace() { - start += 1; - } - while end > start && string_bytes[end - 1].is_ascii_whitespace() { - end -= 1; - } - - let trimmed = &input_str[start..end]; - - if trimmed.is_empty() { - return Ok(None); - } - // Handle special values (inf, nan, etc.) - if trimmed.eq_ignore_ascii_case("inf") - || trimmed.eq_ignore_ascii_case("+inf") - || trimmed.eq_ignore_ascii_case("infinity") - || trimmed.eq_ignore_ascii_case("+infinity") - || trimmed.eq_ignore_ascii_case("-inf") - || trimmed.eq_ignore_ascii_case("-infinity") - || trimmed.eq_ignore_ascii_case("nan") - { - return Ok(None); - } - - // validate and parse mantissa and exponent or bubble up the error - let (mantissa, exponent) = parse_decimal_str(trimmed, input_str, precision, scale)?; - - // Early return mantissa 0, Spark checks if it fits digits and throw error in ansi - if mantissa == 0 { - if exponent < -37 { - return Err(SparkError::NumericOutOfRange { - value: input_str.to_string(), - }); - } - return Ok(Some(0)); - } - - // scale adjustment - let target_scale = scale as i32; - let scale_adjustment = target_scale - exponent; - - let scaled_value = if scale_adjustment >= 0 { - // Need to multiply (increase scale) but return None if scale is too high to fit i128 - if scale_adjustment > 38 { - return Ok(None); - } - mantissa.checked_mul(10_i128.pow(scale_adjustment as u32)) - } else { - // Need to divide (decrease scale) - let abs_scale_adjustment = (-scale_adjustment) as u32; - if abs_scale_adjustment > 38 { - return Ok(Some(0)); - } - - let divisor = 10_i128.pow(abs_scale_adjustment); - let quotient_opt = mantissa.checked_div(divisor); - // Check if divisor is 0 - if quotient_opt.is_none() { - return Ok(None); - } - let quotient = quotient_opt.unwrap(); - let remainder = mantissa % divisor; - - // Round half up: if abs(remainder) >= divisor/2, round away from zero - let half_divisor = divisor / 2; - let rounded = if remainder.abs() >= half_divisor { - if mantissa >= 0 { - quotient + 1 - } else { - quotient - 1 - } - } else { - quotient - }; - Some(rounded) - }; - - match scaled_value { - Some(value) => { - if is_validate_decimal_precision(value, precision) { - Ok(Some(value)) - } else { - // Value ok but exceeds precision mentioned . THrow error - Err(SparkError::NumericValueOutOfRange { - value: trimmed.to_string(), - precision, - scale, - }) - } - } - None => { - // Overflow when scaling raise exception - Err(SparkError::NumericValueOutOfRange { - value: trimmed.to_string(), - precision, - scale, - }) - } - } -} - -fn invalid_decimal_cast(value: &str, precision: u8, scale: i8) -> SparkError { - invalid_value( - value, - "STRING", - &format!("DECIMAL({},{})", precision, scale), - ) -} - -/// Parse a decimal string into mantissa and scale -/// e.g., "123.45" -> (12345, 2), "-0.001" -> (-1, 3) , 0e50 -> (0,50) etc -fn parse_decimal_str( - s: &str, - original_str: &str, - precision: u8, - scale: i8, -) -> SparkResult<(i128, i32)> { - if s.is_empty() { - return Err(invalid_decimal_cast(original_str, precision, scale)); - } - - let (mantissa_str, exponent) = if let Some(e_pos) = s.find(|c| ['e', 'E'].contains(&c)) { - let mantissa_part = &s[..e_pos]; - let exponent_part = &s[e_pos + 1..]; - // Parse exponent - let exp: i32 = exponent_part - .parse() - .map_err(|_| invalid_decimal_cast(original_str, precision, scale))?; - - (mantissa_part, exp) - } else { - (s, 0) - }; - - let negative = mantissa_str.starts_with('-'); - let mantissa_str = if negative || mantissa_str.starts_with('+') { - &mantissa_str[1..] - } else { - mantissa_str - }; - - if mantissa_str.starts_with('+') || mantissa_str.starts_with('-') { - return Err(invalid_decimal_cast(original_str, precision, scale)); - } - - let (integral_part, fractional_part) = match mantissa_str.find('.') { - Some(dot_pos) => { - if mantissa_str[dot_pos + 1..].contains('.') { - return Err(invalid_decimal_cast(original_str, precision, scale)); - } - (&mantissa_str[..dot_pos], &mantissa_str[dot_pos + 1..]) - } - None => (mantissa_str, ""), - }; - - if integral_part.is_empty() && fractional_part.is_empty() { - return Err(invalid_decimal_cast(original_str, precision, scale)); - } - - if !integral_part.is_empty() && !integral_part.bytes().all(|b| b.is_ascii_digit()) { - return Err(invalid_decimal_cast(original_str, precision, scale)); - } - - if !fractional_part.is_empty() && !fractional_part.bytes().all(|b| b.is_ascii_digit()) { - return Err(invalid_decimal_cast(original_str, precision, scale)); - } - - // Parse integral part - let integral_value: i128 = if integral_part.is_empty() { - // Empty integral part is valid (e.g., ".5" or "-.7e9") - 0 - } else { - integral_part - .parse() - .map_err(|_| invalid_decimal_cast(original_str, precision, scale))? - }; - - // Parse fractional part - let fractional_scale = fractional_part.len() as i32; - let fractional_value: i128 = if fractional_part.is_empty() { - 0 - } else { - fractional_part - .parse() - .map_err(|_| invalid_decimal_cast(original_str, precision, scale))? - }; - - // Combine: value = integral * 10^fractional_scale + fractional - let mantissa = integral_value - .checked_mul(10_i128.pow(fractional_scale as u32)) - .and_then(|v| v.checked_add(fractional_value)) - .ok_or_else(|| invalid_decimal_cast(original_str, precision, scale))?; - - let final_mantissa = if negative { -mantissa } else { mantissa }; - // final scale = fractional_scale - exponent - // For example : "1.23E-5" has fractional_scale=2, exponent=-5, so scale = 2 - (-5) = 7 - let final_scale = fractional_scale - exponent; - Ok((final_mantissa, final_scale)) -} - -#[inline] -fn invalid_value(value: &str, from_type: &str, to_type: &str) -> SparkError { - SparkError::CastInvalidValue { - value: value.to_string(), - from_type: from_type.to_string(), - to_type: to_type.to_string(), - } -} - -#[inline] -fn cast_overflow(value: &str, from_type: &str, to_type: &str) -> SparkError { - SparkError::CastOverFlow { - value: value.to_string(), - from_type: from_type.to_string(), - to_type: to_type.to_string(), - } -} - -impl Display for Cast { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!( - f, - "Cast [data_type: {}, timezone: {}, child: {}, eval_mode: {:?}]", - self.data_type, self.cast_options.timezone, self.child, &self.cast_options.eval_mode - ) - } -} - -impl PhysicalExpr for Cast { - fn as_any(&self) -> &dyn Any { - self +impl PhysicalExpr for Cast { + fn as_any(&self) -> &dyn Any { + self } fn fmt_sql(&self, _: &mut Formatter<'_>) -> std::fmt::Result { @@ -2465,792 +1523,81 @@ impl PhysicalExpr for Cast { } } -fn timestamp_parser( - value: &str, - eval_mode: EvalMode, - tz: &T, -) -> SparkResult> { - let value = value.trim(); - if value.is_empty() { - return Ok(None); - } - // Define regex patterns and corresponding parsing functions - let patterns = &[ - ( - Regex::new(r"^\d{4,5}$").unwrap(), - parse_str_to_year_timestamp as fn(&str, &T) -> SparkResult>, - ), - ( - Regex::new(r"^\d{4,5}-\d{2}$").unwrap(), - parse_str_to_month_timestamp, - ), - ( - Regex::new(r"^\d{4,5}-\d{2}-\d{2}$").unwrap(), - parse_str_to_day_timestamp, - ), - ( - Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{1,2}$").unwrap(), - parse_str_to_hour_timestamp, - ), - ( - Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}$").unwrap(), - parse_str_to_minute_timestamp, - ), - ( - Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}$").unwrap(), - parse_str_to_second_timestamp, - ), - ( - Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d{1,6}$").unwrap(), - parse_str_to_microsecond_timestamp, - ), - ( - Regex::new(r"^T\d{1,2}$").unwrap(), - parse_str_to_time_only_timestamp, - ), - ]; - - let mut timestamp = None; - - // Iterate through patterns and try matching - for (pattern, parse_func) in patterns { - if pattern.is_match(value) { - timestamp = parse_func(value, tz)?; - break; - } - } - - if timestamp.is_none() { - return if eval_mode == EvalMode::Ansi { - Err(SparkError::CastInvalidValue { - value: value.to_string(), - from_type: "STRING".to_string(), - to_type: "TIMESTAMP".to_string(), - }) - } else { - Ok(None) - }; - } - - match timestamp { - Some(ts) => Ok(Some(ts)), - None => Err(SparkError::Internal( - "Failed to parse timestamp".to_string(), - )), - } -} - -fn parse_timestamp_to_micros( - timestamp_info: &TimeStampInfo, - tz: &T, -) -> SparkResult> { - let datetime = tz.with_ymd_and_hms( - timestamp_info.year, - timestamp_info.month, - timestamp_info.day, - timestamp_info.hour, - timestamp_info.minute, - timestamp_info.second, - ); - - // Check if datetime is not None - let tz_datetime = match datetime.single() { - Some(dt) => dt - .with_timezone(tz) - .with_nanosecond(timestamp_info.microsecond * 1000), - None => { - return Err(SparkError::Internal( - "Failed to parse timestamp".to_string(), - )); - } - }; - - let result = match tz_datetime { - Some(dt) => dt.timestamp_micros(), - None => { - return Err(SparkError::Internal( - "Failed to parse timestamp".to_string(), - )); - } - }; - - Ok(Some(result)) -} - -fn get_timestamp_values( - value: &str, - timestamp_type: &str, - tz: &T, -) -> SparkResult> { - let values: Vec<_> = value.split(['T', '-', ':', '.']).collect(); - let year = values[0].parse::().unwrap_or_default(); - let month = values.get(1).map_or(1, |m| m.parse::().unwrap_or(1)); - let day = values.get(2).map_or(1, |d| d.parse::().unwrap_or(1)); - let hour = values.get(3).map_or(0, |h| h.parse::().unwrap_or(0)); - let minute = values.get(4).map_or(0, |m| m.parse::().unwrap_or(0)); - let second = values.get(5).map_or(0, |s| s.parse::().unwrap_or(0)); - let microsecond = values.get(6).map_or(0, |ms| ms.parse::().unwrap_or(0)); - - let mut timestamp_info = TimeStampInfo::default(); - - let timestamp_info = match timestamp_type { - "year" => timestamp_info.with_year(year), - "month" => timestamp_info.with_year(year).with_month(month), - "day" => timestamp_info - .with_year(year) - .with_month(month) - .with_day(day), - "hour" => timestamp_info - .with_year(year) - .with_month(month) - .with_day(day) - .with_hour(hour), - "minute" => timestamp_info - .with_year(year) - .with_month(month) - .with_day(day) - .with_hour(hour) - .with_minute(minute), - "second" => timestamp_info - .with_year(year) - .with_month(month) - .with_day(day) - .with_hour(hour) - .with_minute(minute) - .with_second(second), - "microsecond" => timestamp_info - .with_year(year) - .with_month(month) - .with_day(day) - .with_hour(hour) - .with_minute(minute) - .with_second(second) - .with_microsecond(microsecond), - _ => { - return Err(SparkError::CastInvalidValue { - value: value.to_string(), - from_type: "STRING".to_string(), - to_type: "TIMESTAMP".to_string(), - }) - } - }; - - parse_timestamp_to_micros(timestamp_info, tz) -} - -fn parse_str_to_year_timestamp(value: &str, tz: &T) -> SparkResult> { - get_timestamp_values(value, "year", tz) -} - -fn parse_str_to_month_timestamp(value: &str, tz: &T) -> SparkResult> { - get_timestamp_values(value, "month", tz) -} - -fn parse_str_to_day_timestamp(value: &str, tz: &T) -> SparkResult> { - get_timestamp_values(value, "day", tz) -} - -fn parse_str_to_hour_timestamp(value: &str, tz: &T) -> SparkResult> { - get_timestamp_values(value, "hour", tz) -} - -fn parse_str_to_minute_timestamp(value: &str, tz: &T) -> SparkResult> { - get_timestamp_values(value, "minute", tz) -} - -fn parse_str_to_second_timestamp(value: &str, tz: &T) -> SparkResult> { - get_timestamp_values(value, "second", tz) -} - -fn parse_str_to_microsecond_timestamp( - value: &str, - tz: &T, -) -> SparkResult> { - get_timestamp_values(value, "microsecond", tz) -} - -fn parse_str_to_time_only_timestamp(value: &str, tz: &T) -> SparkResult> { - let values: Vec<&str> = value.split('T').collect(); - let time_values: Vec = values[1] - .split(':') - .map(|v| v.parse::().unwrap_or(0)) - .collect(); - - let datetime = tz.from_utc_datetime(&chrono::Utc::now().naive_utc()); - let timestamp = datetime - .with_timezone(tz) - .with_hour(time_values.first().copied().unwrap_or_default()) - .and_then(|dt| dt.with_minute(*time_values.get(1).unwrap_or(&0))) - .and_then(|dt| dt.with_second(*time_values.get(2).unwrap_or(&0))) - .and_then(|dt| dt.with_nanosecond(*time_values.get(3).unwrap_or(&0) * 1_000)) - .map(|dt| dt.timestamp_micros()) - .unwrap_or_default(); - - Ok(Some(timestamp)) -} - -//a string to date parser - port of spark's SparkDateTimeUtils#stringToDate. -fn date_parser(date_str: &str, eval_mode: EvalMode) -> SparkResult> { - // local functions - fn get_trimmed_start(bytes: &[u8]) -> usize { - let mut start = 0; - while start < bytes.len() && is_whitespace_or_iso_control(bytes[start]) { - start += 1; - } - start - } - - fn get_trimmed_end(start: usize, bytes: &[u8]) -> usize { - let mut end = bytes.len() - 1; - while end > start && is_whitespace_or_iso_control(bytes[end]) { - end -= 1; - } - end + 1 - } - - fn is_whitespace_or_iso_control(byte: u8) -> bool { - byte.is_ascii_whitespace() || byte.is_ascii_control() - } - - fn is_valid_digits(segment: i32, digits: usize) -> bool { - // An integer is able to represent a date within [+-]5 million years. - let max_digits_year = 7; - //year (segment 0) can be between 4 to 7 digits, - //month and day (segment 1 and 2) can be between 1 to 2 digits - (segment == 0 && digits >= 4 && digits <= max_digits_year) - || (segment != 0 && digits > 0 && digits <= 2) - } - - fn return_result(date_str: &str, eval_mode: EvalMode) -> SparkResult> { - if eval_mode == EvalMode::Ansi { - Err(SparkError::CastInvalidValue { - value: date_str.to_string(), - from_type: "STRING".to_string(), - to_type: "DATE".to_string(), - }) - } else { - Ok(None) - } - } - // end local functions - - if date_str.is_empty() { - return return_result(date_str, eval_mode); - } - - //values of date segments year, month and day defaulting to 1 - let mut date_segments = [1, 1, 1]; - let mut sign = 1; - let mut current_segment = 0; - let mut current_segment_value = Wrapping(0); - let mut current_segment_digits = 0; - let bytes = date_str.as_bytes(); - - let mut j = get_trimmed_start(bytes); - let str_end_trimmed = get_trimmed_end(j, bytes); - - if j == str_end_trimmed { - return return_result(date_str, eval_mode); - } - - //assign a sign to the date - if bytes[j] == b'-' || bytes[j] == b'+' { - sign = if bytes[j] == b'-' { -1 } else { 1 }; - j += 1; - } +fn cast_binary_to_string( + array: &dyn Array, + spark_cast_options: &SparkCastOptions, +) -> Result { + let input = array + .as_any() + .downcast_ref::>>() + .unwrap(); - //loop to the end of string until we have processed 3 segments, - //exit loop on encountering any space ' ' or 'T' after the 3rd segment - while j < str_end_trimmed && (current_segment < 3 && !(bytes[j] == b' ' || bytes[j] == b'T')) { - let b = bytes[j]; - if current_segment < 2 && b == b'-' { - //check for validity of year and month segments if current byte is separator - if !is_valid_digits(current_segment, current_segment_digits) { - return return_result(date_str, eval_mode); - } - //if valid update corresponding segment with the current segment value. - date_segments[current_segment as usize] = current_segment_value.0; - current_segment_value = Wrapping(0); - current_segment_digits = 0; - current_segment += 1; - } else if !b.is_ascii_digit() { - return return_result(date_str, eval_mode); - } else { - //increment value of current segment by the next digit - let parsed_value = Wrapping((b - b'0') as i32); - current_segment_value = current_segment_value * Wrapping(10) + parsed_value; - current_segment_digits += 1; + fn binary_formatter(value: &[u8], spark_cast_options: &SparkCastOptions) -> String { + match spark_cast_options.binary_output_style { + Some(s) => spark_binary_formatter(value, s), + None => cast_binary_formatter(value), } - j += 1; - } - - //check for validity of last segment - if !is_valid_digits(current_segment, current_segment_digits) { - return return_result(date_str, eval_mode); } - if current_segment < 2 && j < str_end_trimmed { - // For the `yyyy` and `yyyy-[m]m` formats, entire input must be consumed. - return return_result(date_str, eval_mode); - } - - date_segments[current_segment as usize] = current_segment_value.0; - - match NaiveDate::from_ymd_opt( - sign * date_segments[0], - date_segments[1] as u32, - date_segments[2] as u32, - ) { - Some(date) => { - let duration_since_epoch = date - .signed_duration_since(DateTime::UNIX_EPOCH.naive_utc().date()) - .num_days(); - Ok(Some(duration_since_epoch.to_i32().unwrap())) - } - None => Ok(None), - } + let output_array = input + .iter() + .map(|value| match value { + Some(value) => Ok(Some(binary_formatter(value, spark_cast_options))), + _ => Ok(None), + }) + .collect::, ArrowError>>()?; + Ok(Arc::new(output_array)) } -/// This takes for special casting cases of Spark. E.g., Timestamp to Long. -/// This function runs as a post process of the DataFusion cast(). By the time it arrives here, -/// Dictionary arrays are already unpacked by the DataFusion cast() since Spark cannot specify -/// Dictionary as to_type. The from_type is taken before the DataFusion cast() runs in -/// expressions/cast.rs, so it can be still Dictionary. -fn spark_cast_postprocess(array: ArrayRef, from_type: &DataType, to_type: &DataType) -> ArrayRef { - match (from_type, to_type) { - (DataType::Timestamp(_, _), DataType::Int64) => { - // See Spark's `Cast` expression - unary_dyn::<_, Int64Type>(&array, |v| div_floor(v, MICROS_PER_SECOND)).unwrap() - } - (DataType::Dictionary(_, value_type), DataType::Int64) - if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => - { - // See Spark's `Cast` expression - unary_dyn::<_, Int64Type>(&array, |v| div_floor(v, MICROS_PER_SECOND)).unwrap() - } - (DataType::Timestamp(_, _), DataType::Utf8) => remove_trailing_zeroes(array), - (DataType::Dictionary(_, value_type), DataType::Utf8) - if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => - { - remove_trailing_zeroes(array) +/// This function mimics the [BinaryFormatter]: https://github.com/apache/spark/blob/v4.0.0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala#L449-L468 +/// used by SparkSQL's ToPrettyString expression. +/// The BinaryFormatter was [introduced]: https://issues.apache.org/jira/browse/SPARK-47911 in Spark 4.0.0 +/// Before Spark 4.0.0, the default is SPACE_DELIMITED_UPPERCASE_HEX +fn spark_binary_formatter(value: &[u8], binary_output_style: BinaryOutputStyle) -> String { + match binary_output_style { + BinaryOutputStyle::Utf8 => String::from_utf8(value.to_vec()).unwrap(), + BinaryOutputStyle::Basic => { + format!( + "{:?}", + value + .iter() + .map(|v| i8::from_ne_bytes([*v])) + .collect::>() + ) } - _ => array, - } -} - -/// A fork & modified version of Arrow's `unary_dyn` which is being deprecated -fn unary_dyn(array: &ArrayRef, op: F) -> Result -where - T: ArrowPrimitiveType, - F: Fn(T::Native) -> T::Native, -{ - if let Some(d) = array.as_any_dictionary_opt() { - let new_values = unary_dyn::(d.values(), op)?; - return Ok(Arc::new(d.with_values(Arc::new(new_values)))); - } - - match array.as_primitive_opt::() { - Some(a) if PrimitiveArray::::is_compatible(a.data_type()) => { - Ok(Arc::new(unary::( - array.as_any().downcast_ref::>().unwrap(), - op, - ))) + BinaryOutputStyle::Base64 => BASE64_STANDARD_NO_PAD.encode(value), + BinaryOutputStyle::Hex => value + .iter() + .map(|v| hex::encode_upper([*v])) + .collect::(), + BinaryOutputStyle::HexDiscrete => { + // Spark's default SPACE_DELIMITED_UPPERCASE_HEX + format!( + "[{}]", + value + .iter() + .map(|v| hex::encode_upper([*v])) + .collect::>() + .join(" ") + ) } - _ => Err(ArrowError::NotYetImplemented(format!( - "Cannot perform unary operation of type {} on array of type {}", - T::DATA_TYPE, - array.data_type() - ))), } } -/// Remove any trailing zeroes in the string if they occur after in the fractional seconds, -/// to match Spark behavior -/// example: -/// "1970-01-01 05:29:59.900" => "1970-01-01 05:29:59.9" -/// "1970-01-01 05:29:59.990" => "1970-01-01 05:29:59.99" -/// "1970-01-01 05:29:59.999" => "1970-01-01 05:29:59.999" -/// "1970-01-01 05:30:00" => "1970-01-01 05:30:00" -/// "1970-01-01 05:30:00.001" => "1970-01-01 05:30:00.001" -fn remove_trailing_zeroes(array: ArrayRef) -> ArrayRef { - let string_array = as_generic_string_array::(&array).unwrap(); - let result = string_array - .iter() - .map(|s| s.map(trim_end)) - .collect::>(); - Arc::new(result) as ArrayRef -} - -fn trim_end(s: &str) -> &str { - if s.rfind('.').is_some() { - s.trim_end_matches('0') - } else { - s +fn cast_binary_formatter(value: &[u8]) -> String { + match String::from_utf8(value.to_vec()) { + Ok(value) => value, + Err(_) => unsafe { String::from_utf8_unchecked(value.to_vec()) }, } } #[cfg(test)] mod tests { + use super::*; use arrow::array::StringArray; use arrow::datatypes::TimestampMicrosecondType; use arrow::datatypes::{Field, Fields, TimeUnit}; use core::f64; - use std::str::FromStr; - - use super::*; - - /// Test helper that wraps the mode-specific parse functions - fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> SparkResult> { - match eval_mode { - EvalMode::Legacy => parse_string_to_i8_legacy(str), - EvalMode::Ansi => parse_string_to_i8_ansi(str), - EvalMode::Try => parse_string_to_i8_try(str), - } - } - - #[test] - #[cfg_attr(miri, ignore)] // test takes too long with miri - fn timestamp_parser_test() { - let tz = &timezone::Tz::from_str("UTC").unwrap(); - // write for all formats - assert_eq!( - timestamp_parser("2020", EvalMode::Legacy, tz).unwrap(), - Some(1577836800000000) // this is in milliseconds - ); - assert_eq!( - timestamp_parser("2020-01", EvalMode::Legacy, tz).unwrap(), - Some(1577836800000000) - ); - assert_eq!( - timestamp_parser("2020-01-01", EvalMode::Legacy, tz).unwrap(), - Some(1577836800000000) - ); - assert_eq!( - timestamp_parser("2020-01-01T12", EvalMode::Legacy, tz).unwrap(), - Some(1577880000000000) - ); - assert_eq!( - timestamp_parser("2020-01-01T12:34", EvalMode::Legacy, tz).unwrap(), - Some(1577882040000000) - ); - assert_eq!( - timestamp_parser("2020-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(), - Some(1577882096000000) - ); - assert_eq!( - timestamp_parser("2020-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(), - Some(1577882096123456) - ); - assert_eq!( - timestamp_parser("0100", EvalMode::Legacy, tz).unwrap(), - Some(-59011459200000000) - ); - assert_eq!( - timestamp_parser("0100-01", EvalMode::Legacy, tz).unwrap(), - Some(-59011459200000000) - ); - assert_eq!( - timestamp_parser("0100-01-01", EvalMode::Legacy, tz).unwrap(), - Some(-59011459200000000) - ); - assert_eq!( - timestamp_parser("0100-01-01T12", EvalMode::Legacy, tz).unwrap(), - Some(-59011416000000000) - ); - assert_eq!( - timestamp_parser("0100-01-01T12:34", EvalMode::Legacy, tz).unwrap(), - Some(-59011413960000000) - ); - assert_eq!( - timestamp_parser("0100-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(), - Some(-59011413904000000) - ); - assert_eq!( - timestamp_parser("0100-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(), - Some(-59011413903876544) - ); - assert_eq!( - timestamp_parser("10000", EvalMode::Legacy, tz).unwrap(), - Some(253402300800000000) - ); - assert_eq!( - timestamp_parser("10000-01", EvalMode::Legacy, tz).unwrap(), - Some(253402300800000000) - ); - assert_eq!( - timestamp_parser("10000-01-01", EvalMode::Legacy, tz).unwrap(), - Some(253402300800000000) - ); - assert_eq!( - timestamp_parser("10000-01-01T12", EvalMode::Legacy, tz).unwrap(), - Some(253402344000000000) - ); - assert_eq!( - timestamp_parser("10000-01-01T12:34", EvalMode::Legacy, tz).unwrap(), - Some(253402346040000000) - ); - assert_eq!( - timestamp_parser("10000-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(), - Some(253402346096000000) - ); - assert_eq!( - timestamp_parser("10000-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(), - Some(253402346096123456) - ); - // assert_eq!( - // timestamp_parser("T2", EvalMode::Legacy).unwrap(), - // Some(1714356000000000) // this value needs to change everyday. - // ); - } - - #[test] - #[cfg_attr(miri, ignore)] // test takes too long with miri - fn test_cast_string_to_timestamp() { - let array: ArrayRef = Arc::new(StringArray::from(vec![ - Some("2020-01-01T12:34:56.123456"), - Some("T2"), - Some("0100-01-01T12:34:56.123456"), - Some("10000-01-01T12:34:56.123456"), - ])); - let tz = &timezone::Tz::from_str("UTC").unwrap(); - - let string_array = array - .as_any() - .downcast_ref::>() - .expect("Expected a string array"); - - let eval_mode = EvalMode::Legacy; - let result = cast_utf8_to_timestamp!( - &string_array, - eval_mode, - TimestampMicrosecondType, - timestamp_parser, - tz - ); - - assert_eq!( - result.data_type(), - &DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())) - ); - assert_eq!(result.len(), 4); - } - - #[test] - fn test_cast_dict_string_to_timestamp() -> DataFusionResult<()> { - // prepare input data - let keys = Int32Array::from(vec![0, 1]); - let values: ArrayRef = Arc::new(StringArray::from(vec![ - Some("2020-01-01T12:34:56.123456"), - Some("T2"), - ])); - let dict_array = Arc::new(DictionaryArray::new(keys, values)); - - let timezone = "UTC".to_string(); - // test casting string dictionary array to timestamp array - let cast_options = SparkCastOptions::new(EvalMode::Legacy, &timezone, false); - let result = cast_array( - dict_array, - &DataType::Timestamp(TimeUnit::Microsecond, Some(timezone.clone().into())), - &cast_options, - )?; - assert_eq!( - *result.data_type(), - DataType::Timestamp(TimeUnit::Microsecond, Some(timezone.into())) - ); - assert_eq!(result.len(), 2); - - Ok(()) - } - - #[test] - fn date_parser_test() { - for date in &[ - "2020", - "2020-01", - "2020-01-01", - "02020-01-01", - "002020-01-01", - "0002020-01-01", - "2020-1-1", - "2020-01-01 ", - "2020-01-01T", - ] { - for eval_mode in &[EvalMode::Legacy, EvalMode::Ansi, EvalMode::Try] { - assert_eq!(date_parser(date, *eval_mode).unwrap(), Some(18262)); - } - } - - //dates in invalid formats - for date in &[ - "abc", - "", - "not_a_date", - "3/", - "3/12", - "3/12/2020", - "3/12/2002 T", - "202", - "2020-010-01", - "2020-10-010", - "2020-10-010T", - "--262143-12-31", - "--262143-12-31 ", - ] { - for eval_mode in &[EvalMode::Legacy, EvalMode::Try] { - assert_eq!(date_parser(date, *eval_mode).unwrap(), None); - } - assert!(date_parser(date, EvalMode::Ansi).is_err()); - } - - for date in &["-3638-5"] { - for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { - assert_eq!(date_parser(date, *eval_mode).unwrap(), Some(-2048160)); - } - } - - //Naive Date only supports years 262142 AD to 262143 BC - //returns None for dates out of range supported by Naive Date. - for date in &[ - "-262144-1-1", - "262143-01-1", - "262143-1-1", - "262143-01-1 ", - "262143-01-01T ", - "262143-1-01T 1234", - "-0973250", - ] { - for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { - assert_eq!(date_parser(date, *eval_mode).unwrap(), None); - } - } - } - - #[test] - fn test_cast_string_to_date() { - let array: ArrayRef = Arc::new(StringArray::from(vec![ - Some("2020"), - Some("2020-01"), - Some("2020-01-01"), - Some("2020-01-01T"), - ])); - - let result = cast_string_to_date(&array, &DataType::Date32, EvalMode::Legacy).unwrap(); - - let date32_array = result - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(date32_array.len(), 4); - date32_array - .iter() - .for_each(|v| assert_eq!(v.unwrap(), 18262)); - } - - #[test] - fn test_cast_string_array_with_valid_dates() { - let array_with_invalid_date: ArrayRef = Arc::new(StringArray::from(vec![ - Some("-262143-12-31"), - Some("\n -262143-12-31 "), - Some("-262143-12-31T \t\n"), - Some("\n\t-262143-12-31T\r"), - Some("-262143-12-31T 123123123"), - Some("\r\n-262143-12-31T \r123123123"), - Some("\n -262143-12-31T \n\t"), - ])); - - for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { - let result = - cast_string_to_date(&array_with_invalid_date, &DataType::Date32, *eval_mode) - .unwrap(); - - let date32_array = result - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(result.len(), 7); - date32_array - .iter() - .for_each(|v| assert_eq!(v.unwrap(), -96464928)); - } - } - - #[test] - fn test_cast_string_array_with_invalid_dates() { - let array_with_invalid_date: ArrayRef = Arc::new(StringArray::from(vec![ - Some("2020"), - Some("2020-01"), - Some("2020-01-01"), - //4 invalid dates - Some("2020-010-01T"), - Some("202"), - Some(" 202 "), - Some("\n 2020-\r8 "), - Some("2020-01-01T"), - // Overflows i32 - Some("-4607172990231812908"), - ])); - - for eval_mode in &[EvalMode::Legacy, EvalMode::Try] { - let result = - cast_string_to_date(&array_with_invalid_date, &DataType::Date32, *eval_mode) - .unwrap(); - - let date32_array = result - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!( - date32_array.iter().collect::>(), - vec![ - Some(18262), - Some(18262), - Some(18262), - None, - None, - None, - None, - Some(18262), - None - ] - ); - } - - let result = - cast_string_to_date(&array_with_invalid_date, &DataType::Date32, EvalMode::Ansi); - match result { - Err(e) => assert!( - e.to_string().contains( - "[CAST_INVALID_INPUT] The value '2020-010-01T' of the type \"STRING\" cannot be cast to \"DATE\" because it is malformed") - ), - _ => panic!("Expected error"), - } - } - - #[test] - fn test_cast_string_as_i8() { - // basic - assert_eq!( - cast_string_to_i8("127", EvalMode::Legacy).unwrap(), - Some(127_i8) - ); - assert_eq!(cast_string_to_i8("128", EvalMode::Legacy).unwrap(), None); - assert!(cast_string_to_i8("128", EvalMode::Ansi).is_err()); - // decimals - assert_eq!( - cast_string_to_i8("0.2", EvalMode::Legacy).unwrap(), - Some(0_i8) - ); - assert_eq!( - cast_string_to_i8(".", EvalMode::Legacy).unwrap(), - Some(0_i8) - ); - // TRY should always return null for decimals - assert_eq!(cast_string_to_i8("0.2", EvalMode::Try).unwrap(), None); - assert_eq!(cast_string_to_i8(".", EvalMode::Try).unwrap(), None); - // ANSI mode should throw error on decimal - assert!(cast_string_to_i8("0.2", EvalMode::Ansi).is_err()); - assert!(cast_string_to_i8(".", EvalMode::Ansi).is_err()); - } #[test] fn test_cast_unsupported_timestamp_to_date() { @@ -3519,4 +1866,94 @@ mod tests { assert_eq!(r#"[null]"#, string_array.value(2)); assert_eq!(r#"[]"#, string_array.value(3)); } + + #[test] + fn test_cast_int_to_timestamp() { + let timezones: [Option>; 6] = [ + Some(Arc::from("UTC")), + Some(Arc::from("America/New_York")), + Some(Arc::from("America/Los_Angeles")), + Some(Arc::from("Europe/London")), + Some(Arc::from("Asia/Tokyo")), + Some(Arc::from("Australia/Sydney")), + ]; + + for tz in &timezones { + let int8_array: ArrayRef = Arc::new(Int8Array::from(vec![ + Some(0), + Some(1), + Some(-1), + Some(127), + Some(-128), + None, + ])); + + let result = cast_int_to_timestamp(&int8_array, tz).unwrap(); + let ts_array = result.as_primitive::(); + + assert_eq!(ts_array.value(0), 0); + assert_eq!(ts_array.value(1), 1_000_000); + assert_eq!(ts_array.value(2), -1_000_000); + assert_eq!(ts_array.value(3), 127_000_000); + assert_eq!(ts_array.value(4), -128_000_000); + assert!(ts_array.is_null(5)); + assert_eq!(ts_array.timezone(), tz.as_ref().map(|s| s.as_ref())); + + let int16_array: ArrayRef = Arc::new(Int16Array::from(vec![ + Some(0), + Some(1), + Some(-1), + Some(32767), + Some(-32768), + None, + ])); + + let result = cast_int_to_timestamp(&int16_array, tz).unwrap(); + let ts_array = result.as_primitive::(); + + assert_eq!(ts_array.value(0), 0); + assert_eq!(ts_array.value(1), 1_000_000); + assert_eq!(ts_array.value(2), -1_000_000); + assert_eq!(ts_array.value(3), 32_767_000_000_i64); + assert_eq!(ts_array.value(4), -32_768_000_000_i64); + assert!(ts_array.is_null(5)); + assert_eq!(ts_array.timezone(), tz.as_ref().map(|s| s.as_ref())); + + let int32_array: ArrayRef = Arc::new(Int32Array::from(vec![ + Some(0), + Some(1), + Some(-1), + Some(1704067200), + None, + ])); + + let result = cast_int_to_timestamp(&int32_array, tz).unwrap(); + let ts_array = result.as_primitive::(); + + assert_eq!(ts_array.value(0), 0); + assert_eq!(ts_array.value(1), 1_000_000); + assert_eq!(ts_array.value(2), -1_000_000); + assert_eq!(ts_array.value(3), 1_704_067_200_000_000_i64); + assert!(ts_array.is_null(4)); + assert_eq!(ts_array.timezone(), tz.as_ref().map(|s| s.as_ref())); + + let int64_array: ArrayRef = Arc::new(Int64Array::from(vec![ + Some(0), + Some(1), + Some(-1), + Some(i64::MAX), + Some(i64::MIN), + ])); + + let result = cast_int_to_timestamp(&int64_array, tz).unwrap(); + let ts_array = result.as_primitive::(); + + assert_eq!(ts_array.value(0), 0); + assert_eq!(ts_array.value(1), 1_000_000_i64); + assert_eq!(ts_array.value(2), -1_000_000_i64); + assert_eq!(ts_array.value(3), i64::MAX); + assert_eq!(ts_array.value(4), i64::MIN); + assert_eq!(ts_array.timezone(), tz.as_ref().map(|s| s.as_ref())); + } + } } diff --git a/native/spark-expr/src/conversion_funcs/mod.rs b/native/spark-expr/src/conversion_funcs/mod.rs index f2c6f7ca36..33d7a8e211 100644 --- a/native/spark-expr/src/conversion_funcs/mod.rs +++ b/native/spark-expr/src/conversion_funcs/mod.rs @@ -15,4 +15,7 @@ // specific language governing permissions and limitations // under the License. +mod boolean; pub mod cast; +mod string; +mod utils; diff --git a/native/spark-expr/src/conversion_funcs/string.rs b/native/spark-expr/src/conversion_funcs/string.rs new file mode 100644 index 0000000000..531d334d15 --- /dev/null +++ b/native/spark-expr/src/conversion_funcs/string.rs @@ -0,0 +1,1632 @@ +// 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. + +use crate::{timezone, EvalMode, SparkError, SparkResult}; +use arrow::array::{ + Array, ArrayRef, ArrowPrimitiveType, BooleanArray, Decimal128Builder, GenericStringArray, + OffsetSizeTrait, PrimitiveArray, PrimitiveBuilder, StringArray, +}; +use arrow::compute::DecimalCast; +use arrow::datatypes::{ + i256, is_validate_decimal_precision, DataType, Date32Type, Decimal256Type, Float32Type, + Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, TimestampMicrosecondType, +}; +use chrono::{DateTime, NaiveDate, TimeZone, Timelike}; +use num::traits::CheckedNeg; +use num::{CheckedSub, Integer}; +use regex::Regex; +use std::num::Wrapping; +use std::str::FromStr; +use std::sync::Arc; + +macro_rules! cast_utf8_to_timestamp { + ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident, $tz:expr) => {{ + let len = $array.len(); + let mut cast_array = PrimitiveArray::<$array_type>::builder(len).with_timezone("UTC"); + for i in 0..len { + if $array.is_null(i) { + cast_array.append_null() + } else if let Ok(Some(cast_value)) = + $cast_method($array.value(i).trim(), $eval_mode, $tz) + { + cast_array.append_value(cast_value); + } else { + cast_array.append_null() + } + } + let result: ArrayRef = Arc::new(cast_array.finish()) as ArrayRef; + result + }}; +} + +macro_rules! cast_utf8_to_int { + ($array:expr, $array_type:ty, $parse_fn:expr) => {{ + let len = $array.len(); + let mut cast_array = PrimitiveArray::<$array_type>::builder(len); + let parse_fn = $parse_fn; + if $array.null_count() == 0 { + for i in 0..len { + if let Some(cast_value) = parse_fn($array.value(i))? { + cast_array.append_value(cast_value); + } else { + cast_array.append_null() + } + } + } else { + for i in 0..len { + if $array.is_null(i) { + cast_array.append_null() + } else if let Some(cast_value) = parse_fn($array.value(i))? { + cast_array.append_value(cast_value); + } else { + cast_array.append_null() + } + } + } + let result: SparkResult = Ok(Arc::new(cast_array.finish()) as ArrayRef); + result + }}; +} + +struct TimeStampInfo { + year: i32, + month: u32, + day: u32, + hour: u32, + minute: u32, + second: u32, + microsecond: u32, +} + +impl Default for TimeStampInfo { + fn default() -> Self { + TimeStampInfo { + year: 1, + month: 1, + day: 1, + hour: 0, + minute: 0, + second: 0, + microsecond: 0, + } + } +} + +impl TimeStampInfo { + fn with_year(&mut self, year: i32) -> &mut Self { + self.year = year; + self + } + + fn with_month(&mut self, month: u32) -> &mut Self { + self.month = month; + self + } + + fn with_day(&mut self, day: u32) -> &mut Self { + self.day = day; + self + } + + fn with_hour(&mut self, hour: u32) -> &mut Self { + self.hour = hour; + self + } + + fn with_minute(&mut self, minute: u32) -> &mut Self { + self.minute = minute; + self + } + + fn with_second(&mut self, second: u32) -> &mut Self { + self.second = second; + self + } + + fn with_microsecond(&mut self, microsecond: u32) -> &mut Self { + self.microsecond = microsecond; + self + } +} + +pub(crate) fn is_df_cast_from_string_spark_compatible(to_type: &DataType) -> bool { + matches!(to_type, DataType::Binary) +} + +pub(crate) fn cast_string_to_float( + array: &ArrayRef, + to_type: &DataType, + eval_mode: EvalMode, +) -> SparkResult { + match to_type { + DataType::Float32 => cast_string_to_float_impl::(array, eval_mode, "FLOAT"), + DataType::Float64 => cast_string_to_float_impl::(array, eval_mode, "DOUBLE"), + _ => Err(SparkError::Internal(format!( + "Unsupported cast to float type: {:?}", + to_type + ))), + } +} + +fn cast_string_to_float_impl( + array: &ArrayRef, + eval_mode: EvalMode, + type_name: &str, +) -> SparkResult +where + T::Native: FromStr + num::Float, +{ + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| SparkError::Internal("Expected string array".to_string()))?; + + let mut builder = PrimitiveBuilder::::with_capacity(arr.len()); + + for i in 0..arr.len() { + if arr.is_null(i) { + builder.append_null(); + } else { + let str_value = arr.value(i).trim(); + match parse_string_to_float(str_value) { + Some(v) => builder.append_value(v), + None => { + if eval_mode == EvalMode::Ansi { + return Err(invalid_value(arr.value(i), "STRING", type_name)); + } + builder.append_null(); + } + } + } + } + + Ok(Arc::new(builder.finish())) +} + +/// helper to parse floats from string inputs +fn parse_string_to_float(s: &str) -> Option +where + F: FromStr + num::Float, +{ + // Handle +inf / -inf + if s.eq_ignore_ascii_case("inf") + || s.eq_ignore_ascii_case("+inf") + || s.eq_ignore_ascii_case("infinity") + || s.eq_ignore_ascii_case("+infinity") + { + return Some(F::infinity()); + } + if s.eq_ignore_ascii_case("-inf") || s.eq_ignore_ascii_case("-infinity") { + return Some(F::neg_infinity()); + } + if s.eq_ignore_ascii_case("nan") { + return Some(F::nan()); + } + // Remove D/F suffix if present + let pruned_float_str = + if s.ends_with("d") || s.ends_with("D") || s.ends_with('f') || s.ends_with('F') { + &s[..s.len() - 1] + } else { + s + }; + // Rust's parse logic already handles scientific notations so we just rely on it + pruned_float_str.parse::().ok() +} + +pub(crate) fn spark_cast_utf8_to_boolean( + from: &dyn Array, + eval_mode: EvalMode, +) -> SparkResult +where + OffsetSize: OffsetSizeTrait, +{ + let array = from + .as_any() + .downcast_ref::>() + .unwrap(); + + let output_array = array + .iter() + .map(|value| match value { + Some(value) => match value.to_ascii_lowercase().trim() { + "t" | "true" | "y" | "yes" | "1" => Ok(Some(true)), + "f" | "false" | "n" | "no" | "0" => Ok(Some(false)), + _ if eval_mode == EvalMode::Ansi => Err(SparkError::CastInvalidValue { + value: value.to_string(), + from_type: "STRING".to_string(), + to_type: "BOOLEAN".to_string(), + }), + _ => Ok(None), + }, + _ => Ok(None), + }) + .collect::>()?; + + Ok(Arc::new(output_array)) +} + +pub(crate) fn cast_string_to_decimal( + array: &ArrayRef, + to_type: &DataType, + precision: &u8, + scale: &i8, + eval_mode: EvalMode, +) -> SparkResult { + match to_type { + DataType::Decimal128(_, _) => { + cast_string_to_decimal128_impl(array, eval_mode, *precision, *scale) + } + DataType::Decimal256(_, _) => { + cast_string_to_decimal256_impl(array, eval_mode, *precision, *scale) + } + _ => Err(SparkError::Internal(format!( + "Unexpected type in cast_string_to_decimal: {:?}", + to_type + ))), + } +} + +fn cast_string_to_decimal128_impl( + array: &ArrayRef, + eval_mode: EvalMode, + precision: u8, + scale: i8, +) -> SparkResult { + let string_array = array + .as_any() + .downcast_ref::() + .ok_or_else(|| SparkError::Internal("Expected string array".to_string()))?; + + let mut decimal_builder = Decimal128Builder::with_capacity(string_array.len()); + + for i in 0..string_array.len() { + if string_array.is_null(i) { + decimal_builder.append_null(); + } else { + let str_value = string_array.value(i); + match parse_string_to_decimal(str_value, precision, scale) { + Ok(Some(decimal_value)) => { + decimal_builder.append_value(decimal_value); + } + Ok(None) => { + if eval_mode == EvalMode::Ansi { + return Err(invalid_value( + string_array.value(i), + "STRING", + &format!("DECIMAL({},{})", precision, scale), + )); + } + decimal_builder.append_null(); + } + Err(e) => { + if eval_mode == EvalMode::Ansi { + return Err(e); + } + decimal_builder.append_null(); + } + } + } + } + + Ok(Arc::new( + decimal_builder + .with_precision_and_scale(precision, scale)? + .finish(), + )) +} + +fn cast_string_to_decimal256_impl( + array: &ArrayRef, + eval_mode: EvalMode, + precision: u8, + scale: i8, +) -> SparkResult { + let string_array = array + .as_any() + .downcast_ref::() + .ok_or_else(|| SparkError::Internal("Expected string array".to_string()))?; + + let mut decimal_builder = PrimitiveBuilder::::with_capacity(string_array.len()); + + for i in 0..string_array.len() { + if string_array.is_null(i) { + decimal_builder.append_null(); + } else { + let str_value = string_array.value(i); + match parse_string_to_decimal(str_value, precision, scale) { + Ok(Some(decimal_value)) => { + // Convert i128 to i256 + let i256_value = i256::from_i128(decimal_value); + decimal_builder.append_value(i256_value); + } + Ok(None) => { + if eval_mode == EvalMode::Ansi { + return Err(invalid_value( + str_value, + "STRING", + &format!("DECIMAL({},{})", precision, scale), + )); + } + decimal_builder.append_null(); + } + Err(e) => { + if eval_mode == EvalMode::Ansi { + return Err(e); + } + decimal_builder.append_null(); + } + } + } + } + + Ok(Arc::new( + decimal_builder + .with_precision_and_scale(precision, scale)? + .finish(), + )) +} + +/// Parse a decimal string into mantissa and scale +/// e.g., "123.45" -> (12345, 2), "-0.001" -> (-1, 3) , 0e50 -> (0,50) etc +/// Parse a string to decimal following Spark's behavior +fn parse_string_to_decimal(input_str: &str, precision: u8, scale: i8) -> SparkResult> { + let string_bytes = input_str.as_bytes(); + let mut start = 0; + let mut end = string_bytes.len(); + + // trim whitespaces + while start < end && string_bytes[start].is_ascii_whitespace() { + start += 1; + } + while end > start && string_bytes[end - 1].is_ascii_whitespace() { + end -= 1; + } + + let trimmed = &input_str[start..end]; + + if trimmed.is_empty() { + return Ok(None); + } + // Handle special values (inf, nan, etc.) + if trimmed.eq_ignore_ascii_case("inf") + || trimmed.eq_ignore_ascii_case("+inf") + || trimmed.eq_ignore_ascii_case("infinity") + || trimmed.eq_ignore_ascii_case("+infinity") + || trimmed.eq_ignore_ascii_case("-inf") + || trimmed.eq_ignore_ascii_case("-infinity") + || trimmed.eq_ignore_ascii_case("nan") + { + return Ok(None); + } + + // validate and parse mantissa and exponent or bubble up the error + let (mantissa, exponent) = parse_decimal_str(trimmed, input_str, precision, scale)?; + + // Early return mantissa 0, Spark checks if it fits digits and throw error in ansi + if mantissa == 0 { + if exponent < -37 { + return Err(SparkError::NumericOutOfRange { + value: input_str.to_string(), + }); + } + return Ok(Some(0)); + } + + // scale adjustment + let target_scale = scale as i32; + let scale_adjustment = target_scale - exponent; + + let scaled_value = if scale_adjustment >= 0 { + // Need to multiply (increase scale) but return None if scale is too high to fit i128 + if scale_adjustment > 38 { + return Ok(None); + } + mantissa.checked_mul(10_i128.pow(scale_adjustment as u32)) + } else { + // Need to divide (decrease scale) + let abs_scale_adjustment = (-scale_adjustment) as u32; + if abs_scale_adjustment > 38 { + return Ok(Some(0)); + } + + let divisor = 10_i128.pow(abs_scale_adjustment); + let quotient_opt = mantissa.checked_div(divisor); + // Check if divisor is 0 + if quotient_opt.is_none() { + return Ok(None); + } + let quotient = quotient_opt.unwrap(); + let remainder = mantissa % divisor; + + // Round half up: if abs(remainder) >= divisor/2, round away from zero + let half_divisor = divisor / 2; + let rounded = if remainder.abs() >= half_divisor { + if mantissa >= 0 { + quotient + 1 + } else { + quotient - 1 + } + } else { + quotient + }; + Some(rounded) + }; + + match scaled_value { + Some(value) => { + if is_validate_decimal_precision(value, precision) { + Ok(Some(value)) + } else { + // Value ok but exceeds precision mentioned . THrow error + Err(SparkError::NumericValueOutOfRange { + value: trimmed.to_string(), + precision, + scale, + }) + } + } + None => { + // Overflow when scaling raise exception + Err(SparkError::NumericValueOutOfRange { + value: trimmed.to_string(), + precision, + scale, + }) + } + } +} + +fn invalid_decimal_cast(value: &str, precision: u8, scale: i8) -> SparkError { + invalid_value( + value, + "STRING", + &format!("DECIMAL({},{})", precision, scale), + ) +} + +/// Parse a decimal string into mantissa and scale +/// e.g., "123.45" -> (12345, 2), "-0.001" -> (-1, 3) , 0e50 -> (0,50) etc +fn parse_decimal_str( + s: &str, + original_str: &str, + precision: u8, + scale: i8, +) -> SparkResult<(i128, i32)> { + if s.is_empty() { + return Err(invalid_decimal_cast(original_str, precision, scale)); + } + + let (mantissa_str, exponent) = if let Some(e_pos) = s.find(|c| ['e', 'E'].contains(&c)) { + let mantissa_part = &s[..e_pos]; + let exponent_part = &s[e_pos + 1..]; + // Parse exponent + let exp: i32 = exponent_part + .parse() + .map_err(|_| invalid_decimal_cast(original_str, precision, scale))?; + + (mantissa_part, exp) + } else { + (s, 0) + }; + + let negative = mantissa_str.starts_with('-'); + let mantissa_str = if negative || mantissa_str.starts_with('+') { + &mantissa_str[1..] + } else { + mantissa_str + }; + + if mantissa_str.starts_with('+') || mantissa_str.starts_with('-') { + return Err(invalid_decimal_cast(original_str, precision, scale)); + } + + let (integral_part, fractional_part) = match mantissa_str.find('.') { + Some(dot_pos) => { + if mantissa_str[dot_pos + 1..].contains('.') { + return Err(invalid_decimal_cast(original_str, precision, scale)); + } + (&mantissa_str[..dot_pos], &mantissa_str[dot_pos + 1..]) + } + None => (mantissa_str, ""), + }; + + if integral_part.is_empty() && fractional_part.is_empty() { + return Err(invalid_decimal_cast(original_str, precision, scale)); + } + + if !integral_part.is_empty() && !integral_part.bytes().all(|b| b.is_ascii_digit()) { + return Err(invalid_decimal_cast(original_str, precision, scale)); + } + + if !fractional_part.is_empty() && !fractional_part.bytes().all(|b| b.is_ascii_digit()) { + return Err(invalid_decimal_cast(original_str, precision, scale)); + } + + // Parse integral part + let integral_value: i128 = if integral_part.is_empty() { + // Empty integral part is valid (e.g., ".5" or "-.7e9") + 0 + } else { + integral_part + .parse() + .map_err(|_| invalid_decimal_cast(original_str, precision, scale))? + }; + + // Parse fractional part + let fractional_scale = fractional_part.len() as i32; + let fractional_value: i128 = if fractional_part.is_empty() { + 0 + } else { + fractional_part + .parse() + .map_err(|_| invalid_decimal_cast(original_str, precision, scale))? + }; + + // Combine: value = integral * 10^fractional_scale + fractional + let mantissa = integral_value + .checked_mul(10_i128.pow(fractional_scale as u32)) + .and_then(|v| v.checked_add(fractional_value)) + .ok_or_else(|| invalid_decimal_cast(original_str, precision, scale))?; + + let final_mantissa = if negative { -mantissa } else { mantissa }; + // final scale = fractional_scale - exponent + // For example : "1.23E-5" has fractional_scale=2, exponent=-5, so scale = 2 - (-5) = 7 + let final_scale = fractional_scale - exponent; + Ok((final_mantissa, final_scale)) +} + +pub(crate) fn cast_string_to_date( + array: &ArrayRef, + to_type: &DataType, + eval_mode: EvalMode, +) -> SparkResult { + let string_array = array + .as_any() + .downcast_ref::>() + .expect("Expected a string array"); + + if to_type != &DataType::Date32 { + unreachable!("Invalid data type {:?} in cast from string", to_type); + } + + let len = string_array.len(); + let mut cast_array = PrimitiveArray::::builder(len); + + for i in 0..len { + let value = if string_array.is_null(i) { + None + } else { + match date_parser(string_array.value(i), eval_mode) { + Ok(Some(cast_value)) => Some(cast_value), + Ok(None) => None, + Err(e) => return Err(e), + } + }; + + match value { + Some(cast_value) => cast_array.append_value(cast_value), + None => cast_array.append_null(), + } + } + + Ok(Arc::new(cast_array.finish()) as ArrayRef) +} + +pub(crate) fn cast_string_to_timestamp( + array: &ArrayRef, + to_type: &DataType, + eval_mode: EvalMode, + timezone_str: &str, +) -> SparkResult { + let string_array = array + .as_any() + .downcast_ref::>() + .expect("Expected a string array"); + + let tz = &timezone::Tz::from_str(timezone_str).unwrap(); + + let cast_array: ArrayRef = match to_type { + DataType::Timestamp(_, _) => { + cast_utf8_to_timestamp!( + string_array, + eval_mode, + TimestampMicrosecondType, + timestamp_parser, + tz + ) + } + _ => unreachable!("Invalid data type {:?} in cast from string", to_type), + }; + Ok(cast_array) +} + +pub(crate) fn cast_string_to_int( + to_type: &DataType, + array: &ArrayRef, + eval_mode: EvalMode, +) -> SparkResult { + let string_array = array + .as_any() + .downcast_ref::>() + .expect("cast_string_to_int expected a string array"); + + // Select parse function once per batch based on eval_mode + let cast_array: ArrayRef = + match (to_type, eval_mode) { + (DataType::Int8, EvalMode::Legacy) => { + cast_utf8_to_int!(string_array, Int8Type, parse_string_to_i8_legacy)? + } + (DataType::Int8, EvalMode::Ansi) => { + cast_utf8_to_int!(string_array, Int8Type, parse_string_to_i8_ansi)? + } + (DataType::Int8, EvalMode::Try) => { + cast_utf8_to_int!(string_array, Int8Type, parse_string_to_i8_try)? + } + (DataType::Int16, EvalMode::Legacy) => { + cast_utf8_to_int!(string_array, Int16Type, parse_string_to_i16_legacy)? + } + (DataType::Int16, EvalMode::Ansi) => { + cast_utf8_to_int!(string_array, Int16Type, parse_string_to_i16_ansi)? + } + (DataType::Int16, EvalMode::Try) => { + cast_utf8_to_int!(string_array, Int16Type, parse_string_to_i16_try)? + } + (DataType::Int32, EvalMode::Legacy) => cast_utf8_to_int!( + string_array, + Int32Type, + |s| do_parse_string_to_int_legacy::(s, i32::MIN) + )?, + (DataType::Int32, EvalMode::Ansi) => { + cast_utf8_to_int!(string_array, Int32Type, |s| do_parse_string_to_int_ansi::< + i32, + >( + s, "INT", i32::MIN + ))? + } + (DataType::Int32, EvalMode::Try) => { + cast_utf8_to_int!( + string_array, + Int32Type, + |s| do_parse_string_to_int_try::(s, i32::MIN) + )? + } + (DataType::Int64, EvalMode::Legacy) => cast_utf8_to_int!( + string_array, + Int64Type, + |s| do_parse_string_to_int_legacy::(s, i64::MIN) + )?, + (DataType::Int64, EvalMode::Ansi) => { + cast_utf8_to_int!(string_array, Int64Type, |s| do_parse_string_to_int_ansi::< + i64, + >( + s, "BIGINT", i64::MIN + ))? + } + (DataType::Int64, EvalMode::Try) => { + cast_utf8_to_int!( + string_array, + Int64Type, + |s| do_parse_string_to_int_try::(s, i64::MIN) + )? + } + (dt, _) => unreachable!( + "{}", + format!("invalid integer type {dt} in cast from string") + ), + }; + Ok(cast_array) +} + +/// Finalizes the result by applying the sign. Returns None if overflow would occur. +fn finalize_int_result(result: T, negative: bool) -> Option { + if negative { + Some(result) + } else { + result.checked_neg().filter(|&n| n >= T::zero()) + } +} + +/// Equivalent to +/// - org.apache.spark.unsafe.types.UTF8String.toInt(IntWrapper intWrapper, boolean allowDecimal) +/// - org.apache.spark.unsafe.types.UTF8String.toLong(LongWrapper longWrapper, boolean allowDecimal) +fn do_parse_string_to_int_legacy + Copy>( + str: &str, + min_value: T, +) -> SparkResult> { + let trimmed_bytes = str.as_bytes().trim_ascii(); + + let (negative, digits) = match parse_sign(trimmed_bytes) { + Some(result) => result, + None => return Ok(None), + }; + + let mut result: T = T::zero(); + let radix = T::from(10_u8); + let stop_value = min_value / radix; + + let mut iter = digits.iter(); + + // Parse integer portion until '.' or end + for &ch in iter.by_ref() { + if ch == b'.' { + break; + } + + if !ch.is_ascii_digit() { + return Ok(None); + } + + if result < stop_value { + return Ok(None); + } + let v = result * radix; + let digit: T = T::from(ch - b'0'); + match v.checked_sub(&digit) { + Some(x) if x <= T::zero() => result = x, + _ => return Ok(None), + } + } + + // Validate decimal portion (digits only, values ignored) + for &ch in iter { + if !ch.is_ascii_digit() { + return Ok(None); + } + } + + Ok(finalize_int_result(result, negative)) +} + +fn do_parse_string_to_int_ansi + Copy>( + str: &str, + type_name: &str, + min_value: T, +) -> SparkResult> { + let error = || Err(invalid_value(str, "STRING", type_name)); + + let trimmed_bytes = str.as_bytes().trim_ascii(); + + let (negative, digits) = match parse_sign(trimmed_bytes) { + Some(result) => result, + None => return error(), + }; + + let mut result: T = T::zero(); + let radix = T::from(10_u8); + let stop_value = min_value / radix; + + for &ch in digits { + if ch == b'.' || !ch.is_ascii_digit() { + return error(); + } + + if result < stop_value { + return error(); + } + let v = result * radix; + let digit: T = T::from(ch - b'0'); + match v.checked_sub(&digit) { + Some(x) if x <= T::zero() => result = x, + _ => return error(), + } + } + + finalize_int_result(result, negative) + .map(Some) + .ok_or_else(|| invalid_value(str, "STRING", type_name)) +} + +fn do_parse_string_to_int_try + Copy>( + str: &str, + min_value: T, +) -> SparkResult> { + let trimmed_bytes = str.as_bytes().trim_ascii(); + + let (negative, digits) = match parse_sign(trimmed_bytes) { + Some(result) => result, + None => return Ok(None), + }; + + let mut result: T = T::zero(); + let radix = T::from(10_u8); + let stop_value = min_value / radix; + + for &ch in digits { + if ch == b'.' || !ch.is_ascii_digit() { + return Ok(None); + } + + if result < stop_value { + return Ok(None); + } + let v = result * radix; + let digit: T = T::from(ch - b'0'); + match v.checked_sub(&digit) { + Some(x) if x <= T::zero() => result = x, + _ => return Ok(None), + } + } + + Ok(finalize_int_result(result, negative)) +} + +fn parse_string_to_i8_legacy(str: &str) -> SparkResult> { + match do_parse_string_to_int_legacy::(str, i32::MIN)? { + Some(v) if v >= i8::MIN as i32 && v <= i8::MAX as i32 => Ok(Some(v as i8)), + _ => Ok(None), + } +} + +fn parse_string_to_i8_ansi(str: &str) -> SparkResult> { + match do_parse_string_to_int_ansi::(str, "TINYINT", i32::MIN)? { + Some(v) if v >= i8::MIN as i32 && v <= i8::MAX as i32 => Ok(Some(v as i8)), + _ => Err(invalid_value(str, "STRING", "TINYINT")), + } +} + +fn parse_string_to_i8_try(str: &str) -> SparkResult> { + match do_parse_string_to_int_try::(str, i32::MIN)? { + Some(v) if v >= i8::MIN as i32 && v <= i8::MAX as i32 => Ok(Some(v as i8)), + _ => Ok(None), + } +} + +fn parse_string_to_i16_legacy(str: &str) -> SparkResult> { + match do_parse_string_to_int_legacy::(str, i32::MIN)? { + Some(v) if v >= i16::MIN as i32 && v <= i16::MAX as i32 => Ok(Some(v as i16)), + _ => Ok(None), + } +} + +fn parse_string_to_i16_ansi(str: &str) -> SparkResult> { + match do_parse_string_to_int_ansi::(str, "SMALLINT", i32::MIN)? { + Some(v) if v >= i16::MIN as i32 && v <= i16::MAX as i32 => Ok(Some(v as i16)), + _ => Err(invalid_value(str, "STRING", "SMALLINT")), + } +} + +fn parse_string_to_i16_try(str: &str) -> SparkResult> { + match do_parse_string_to_int_try::(str, i32::MIN)? { + Some(v) if v >= i16::MIN as i32 && v <= i16::MAX as i32 => Ok(Some(v as i16)), + _ => Ok(None), + } +} + +/// Parses sign and returns (is_negative, remaining_bytes after sign) +/// Returns None if invalid (empty input, or just "+" or "-") +fn parse_sign(bytes: &[u8]) -> Option<(bool, &[u8])> { + let (&first, rest) = bytes.split_first()?; + match first { + b'-' if !rest.is_empty() => Some((true, rest)), + b'+' if !rest.is_empty() => Some((false, rest)), + _ => Some((false, bytes)), + } +} + +#[inline] +pub fn invalid_value(value: &str, from_type: &str, to_type: &str) -> SparkError { + SparkError::CastInvalidValue { + value: value.to_string(), + from_type: from_type.to_string(), + to_type: to_type.to_string(), + } +} + +fn get_timestamp_values( + value: &str, + timestamp_type: &str, + tz: &T, +) -> SparkResult> { + let values: Vec<_> = value.split(['T', '-', ':', '.']).collect(); + let year = values[0].parse::().unwrap_or_default(); + let month = values.get(1).map_or(1, |m| m.parse::().unwrap_or(1)); + let day = values.get(2).map_or(1, |d| d.parse::().unwrap_or(1)); + let hour = values.get(3).map_or(0, |h| h.parse::().unwrap_or(0)); + let minute = values.get(4).map_or(0, |m| m.parse::().unwrap_or(0)); + let second = values.get(5).map_or(0, |s| s.parse::().unwrap_or(0)); + let microsecond = values.get(6).map_or(0, |ms| ms.parse::().unwrap_or(0)); + + let mut timestamp_info = TimeStampInfo::default(); + + let timestamp_info = match timestamp_type { + "year" => timestamp_info.with_year(year), + "month" => timestamp_info.with_year(year).with_month(month), + "day" => timestamp_info + .with_year(year) + .with_month(month) + .with_day(day), + "hour" => timestamp_info + .with_year(year) + .with_month(month) + .with_day(day) + .with_hour(hour), + "minute" => timestamp_info + .with_year(year) + .with_month(month) + .with_day(day) + .with_hour(hour) + .with_minute(minute), + "second" => timestamp_info + .with_year(year) + .with_month(month) + .with_day(day) + .with_hour(hour) + .with_minute(minute) + .with_second(second), + "microsecond" => timestamp_info + .with_year(year) + .with_month(month) + .with_day(day) + .with_hour(hour) + .with_minute(minute) + .with_second(second) + .with_microsecond(microsecond), + _ => { + return Err(SparkError::CastInvalidValue { + value: value.to_string(), + from_type: "STRING".to_string(), + to_type: "TIMESTAMP".to_string(), + }) + } + }; + parse_timestamp_to_micros(timestamp_info, tz) +} + +fn parse_timestamp_to_micros( + timestamp_info: &TimeStampInfo, + tz: &T, +) -> SparkResult> { + let datetime = tz.with_ymd_and_hms( + timestamp_info.year, + timestamp_info.month, + timestamp_info.day, + timestamp_info.hour, + timestamp_info.minute, + timestamp_info.second, + ); + + // Check if datetime is not None + let tz_datetime = match datetime.single() { + Some(dt) => dt + .with_timezone(tz) + .with_nanosecond(timestamp_info.microsecond * 1000), + None => { + return Err(SparkError::Internal( + "Failed to parse timestamp".to_string(), + )); + } + }; + + let result = match tz_datetime { + Some(dt) => dt.timestamp_micros(), + None => { + return Err(SparkError::Internal( + "Failed to parse timestamp".to_string(), + )); + } + }; + + Ok(Some(result)) +} + +fn parse_str_to_year_timestamp(value: &str, tz: &T) -> SparkResult> { + get_timestamp_values(value, "year", tz) +} + +fn parse_str_to_month_timestamp(value: &str, tz: &T) -> SparkResult> { + get_timestamp_values(value, "month", tz) +} + +fn parse_str_to_day_timestamp(value: &str, tz: &T) -> SparkResult> { + get_timestamp_values(value, "day", tz) +} + +fn parse_str_to_hour_timestamp(value: &str, tz: &T) -> SparkResult> { + get_timestamp_values(value, "hour", tz) +} + +fn parse_str_to_minute_timestamp(value: &str, tz: &T) -> SparkResult> { + get_timestamp_values(value, "minute", tz) +} + +fn parse_str_to_second_timestamp(value: &str, tz: &T) -> SparkResult> { + get_timestamp_values(value, "second", tz) +} + +fn parse_str_to_microsecond_timestamp( + value: &str, + tz: &T, +) -> SparkResult> { + get_timestamp_values(value, "microsecond", tz) +} + +// used in tests only +fn timestamp_parser( + value: &str, + eval_mode: EvalMode, + tz: &T, +) -> SparkResult> { + let value = value.trim(); + if value.is_empty() { + return Ok(None); + } + // Define regex patterns and corresponding parsing functions + let patterns = &[ + ( + Regex::new(r"^\d{4,5}$").unwrap(), + parse_str_to_year_timestamp as fn(&str, &T) -> SparkResult>, + ), + ( + Regex::new(r"^\d{4,5}-\d{2}$").unwrap(), + parse_str_to_month_timestamp, + ), + ( + Regex::new(r"^\d{4,5}-\d{2}-\d{2}$").unwrap(), + parse_str_to_day_timestamp, + ), + ( + Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{1,2}$").unwrap(), + parse_str_to_hour_timestamp, + ), + ( + Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}$").unwrap(), + parse_str_to_minute_timestamp, + ), + ( + Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}$").unwrap(), + parse_str_to_second_timestamp, + ), + ( + Regex::new(r"^\d{4,5}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d{1,6}$").unwrap(), + parse_str_to_microsecond_timestamp, + ), + ( + Regex::new(r"^T\d{1,2}$").unwrap(), + parse_str_to_time_only_timestamp, + ), + ]; + + let mut timestamp = None; + + // Iterate through patterns and try matching + for (pattern, parse_func) in patterns { + if pattern.is_match(value) { + timestamp = parse_func(value, tz)?; + break; + } + } + + if timestamp.is_none() { + return if eval_mode == EvalMode::Ansi { + Err(SparkError::CastInvalidValue { + value: value.to_string(), + from_type: "STRING".to_string(), + to_type: "TIMESTAMP".to_string(), + }) + } else { + Ok(None) + }; + } + + match timestamp { + Some(ts) => Ok(Some(ts)), + None => Err(SparkError::Internal( + "Failed to parse timestamp".to_string(), + )), + } +} + +fn parse_str_to_time_only_timestamp(value: &str, tz: &T) -> SparkResult> { + let values: Vec<&str> = value.split('T').collect(); + let time_values: Vec = values[1] + .split(':') + .map(|v| v.parse::().unwrap_or(0)) + .collect(); + + let datetime = tz.from_utc_datetime(&chrono::Utc::now().naive_utc()); + let timestamp = datetime + .with_timezone(tz) + .with_hour(time_values.first().copied().unwrap_or_default()) + .and_then(|dt| dt.with_minute(*time_values.get(1).unwrap_or(&0))) + .and_then(|dt| dt.with_second(*time_values.get(2).unwrap_or(&0))) + .and_then(|dt| dt.with_nanosecond(*time_values.get(3).unwrap_or(&0) * 1_000)) + .map(|dt| dt.timestamp_micros()) + .unwrap_or_default(); + + Ok(Some(timestamp)) +} + +//a string to date parser - port of spark's SparkDateTimeUtils#stringToDate. +fn date_parser(date_str: &str, eval_mode: EvalMode) -> SparkResult> { + // local functions + fn get_trimmed_start(bytes: &[u8]) -> usize { + let mut start = 0; + while start < bytes.len() && is_whitespace_or_iso_control(bytes[start]) { + start += 1; + } + start + } + + fn get_trimmed_end(start: usize, bytes: &[u8]) -> usize { + let mut end = bytes.len() - 1; + while end > start && is_whitespace_or_iso_control(bytes[end]) { + end -= 1; + } + end + 1 + } + + fn is_whitespace_or_iso_control(byte: u8) -> bool { + byte.is_ascii_whitespace() || byte.is_ascii_control() + } + + fn is_valid_digits(segment: i32, digits: usize) -> bool { + // An integer is able to represent a date within [+-]5 million years. + let max_digits_year = 7; + //year (segment 0) can be between 4 to 7 digits, + //month and day (segment 1 and 2) can be between 1 to 2 digits + (segment == 0 && digits >= 4 && digits <= max_digits_year) + || (segment != 0 && digits > 0 && digits <= 2) + } + + fn return_result(date_str: &str, eval_mode: EvalMode) -> SparkResult> { + if eval_mode == EvalMode::Ansi { + Err(SparkError::CastInvalidValue { + value: date_str.to_string(), + from_type: "STRING".to_string(), + to_type: "DATE".to_string(), + }) + } else { + Ok(None) + } + } + // end local functions + + if date_str.is_empty() { + return return_result(date_str, eval_mode); + } + + //values of date segments year, month and day defaulting to 1 + let mut date_segments = [1, 1, 1]; + let mut sign = 1; + let mut current_segment = 0; + let mut current_segment_value = Wrapping(0); + let mut current_segment_digits = 0; + let bytes = date_str.as_bytes(); + + let mut j = get_trimmed_start(bytes); + let str_end_trimmed = get_trimmed_end(j, bytes); + + if j == str_end_trimmed { + return return_result(date_str, eval_mode); + } + + //assign a sign to the date + if bytes[j] == b'-' || bytes[j] == b'+' { + sign = if bytes[j] == b'-' { -1 } else { 1 }; + j += 1; + } + + //loop to the end of string until we have processed 3 segments, + //exit loop on encountering any space ' ' or 'T' after the 3rd segment + while j < str_end_trimmed && (current_segment < 3 && !(bytes[j] == b' ' || bytes[j] == b'T')) { + let b = bytes[j]; + if current_segment < 2 && b == b'-' { + //check for validity of year and month segments if current byte is separator + if !is_valid_digits(current_segment, current_segment_digits) { + return return_result(date_str, eval_mode); + } + //if valid update corresponding segment with the current segment value. + date_segments[current_segment as usize] = current_segment_value.0; + current_segment_value = Wrapping(0); + current_segment_digits = 0; + current_segment += 1; + } else if !b.is_ascii_digit() { + return return_result(date_str, eval_mode); + } else { + //increment value of current segment by the next digit + let parsed_value = Wrapping((b - b'0') as i32); + current_segment_value = current_segment_value * Wrapping(10) + parsed_value; + current_segment_digits += 1; + } + j += 1; + } + + //check for validity of last segment + if !is_valid_digits(current_segment, current_segment_digits) { + return return_result(date_str, eval_mode); + } + + if current_segment < 2 && j < str_end_trimmed { + // For the `yyyy` and `yyyy-[m]m` formats, entire input must be consumed. + return return_result(date_str, eval_mode); + } + + date_segments[current_segment as usize] = current_segment_value.0; + + match NaiveDate::from_ymd_opt( + sign * date_segments[0], + date_segments[1] as u32, + date_segments[2] as u32, + ) { + Some(date) => { + let duration_since_epoch = date + .signed_duration_since(DateTime::UNIX_EPOCH.naive_utc().date()) + .num_days(); + Ok(Some(duration_since_epoch.to_i32().unwrap())) + } + None => Ok(None), + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::cast::cast_array; + use crate::SparkCastOptions; + use arrow::array::{DictionaryArray, Int32Array, StringArray}; + use arrow::datatypes::TimeUnit; + use datafusion::common::Result as DataFusionResult; + + /// Test helper that wraps the mode-specific parse functions + fn cast_string_to_i8(str: &str, eval_mode: EvalMode) -> SparkResult> { + match eval_mode { + EvalMode::Legacy => parse_string_to_i8_legacy(str), + EvalMode::Ansi => parse_string_to_i8_ansi(str), + EvalMode::Try => parse_string_to_i8_try(str), + } + } + + #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri + fn test_cast_string_to_timestamp() { + let array: ArrayRef = Arc::new(StringArray::from(vec![ + Some("2020-01-01T12:34:56.123456"), + Some("T2"), + Some("0100-01-01T12:34:56.123456"), + Some("10000-01-01T12:34:56.123456"), + ])); + let tz = &timezone::Tz::from_str("UTC").unwrap(); + + let string_array = array + .as_any() + .downcast_ref::>() + .expect("Expected a string array"); + + let eval_mode = EvalMode::Legacy; + let result = cast_utf8_to_timestamp!( + &string_array, + eval_mode, + TimestampMicrosecondType, + timestamp_parser, + tz + ); + + assert_eq!( + result.data_type(), + &DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())) + ); + assert_eq!(result.len(), 4); + } + + #[test] + fn test_cast_dict_string_to_timestamp() -> DataFusionResult<()> { + // prepare input data + let keys = Int32Array::from(vec![0, 1]); + let values: ArrayRef = Arc::new(StringArray::from(vec![ + Some("2020-01-01T12:34:56.123456"), + Some("T2"), + ])); + let dict_array = Arc::new(DictionaryArray::new(keys, values)); + + let timezone = "UTC".to_string(); + // test casting string dictionary array to timestamp array + let cast_options = SparkCastOptions::new(EvalMode::Legacy, &timezone, false); + let result = cast_array( + dict_array, + &DataType::Timestamp(TimeUnit::Microsecond, Some(timezone.clone().into())), + &cast_options, + )?; + assert_eq!( + *result.data_type(), + DataType::Timestamp(TimeUnit::Microsecond, Some(timezone.into())) + ); + assert_eq!(result.len(), 2); + + Ok(()) + } + + #[test] + #[cfg_attr(miri, ignore)] // test takes too long with miri + fn timestamp_parser_test() { + let tz = &timezone::Tz::from_str("UTC").unwrap(); + // write for all formats + assert_eq!( + timestamp_parser("2020", EvalMode::Legacy, tz).unwrap(), + Some(1577836800000000) // this is in milliseconds + ); + assert_eq!( + timestamp_parser("2020-01", EvalMode::Legacy, tz).unwrap(), + Some(1577836800000000) + ); + assert_eq!( + timestamp_parser("2020-01-01", EvalMode::Legacy, tz).unwrap(), + Some(1577836800000000) + ); + assert_eq!( + timestamp_parser("2020-01-01T12", EvalMode::Legacy, tz).unwrap(), + Some(1577880000000000) + ); + assert_eq!( + timestamp_parser("2020-01-01T12:34", EvalMode::Legacy, tz).unwrap(), + Some(1577882040000000) + ); + assert_eq!( + timestamp_parser("2020-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(), + Some(1577882096000000) + ); + assert_eq!( + timestamp_parser("2020-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(), + Some(1577882096123456) + ); + assert_eq!( + timestamp_parser("0100", EvalMode::Legacy, tz).unwrap(), + Some(-59011459200000000) + ); + assert_eq!( + timestamp_parser("0100-01", EvalMode::Legacy, tz).unwrap(), + Some(-59011459200000000) + ); + assert_eq!( + timestamp_parser("0100-01-01", EvalMode::Legacy, tz).unwrap(), + Some(-59011459200000000) + ); + assert_eq!( + timestamp_parser("0100-01-01T12", EvalMode::Legacy, tz).unwrap(), + Some(-59011416000000000) + ); + assert_eq!( + timestamp_parser("0100-01-01T12:34", EvalMode::Legacy, tz).unwrap(), + Some(-59011413960000000) + ); + assert_eq!( + timestamp_parser("0100-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(), + Some(-59011413904000000) + ); + assert_eq!( + timestamp_parser("0100-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(), + Some(-59011413903876544) + ); + assert_eq!( + timestamp_parser("10000", EvalMode::Legacy, tz).unwrap(), + Some(253402300800000000) + ); + assert_eq!( + timestamp_parser("10000-01", EvalMode::Legacy, tz).unwrap(), + Some(253402300800000000) + ); + assert_eq!( + timestamp_parser("10000-01-01", EvalMode::Legacy, tz).unwrap(), + Some(253402300800000000) + ); + assert_eq!( + timestamp_parser("10000-01-01T12", EvalMode::Legacy, tz).unwrap(), + Some(253402344000000000) + ); + assert_eq!( + timestamp_parser("10000-01-01T12:34", EvalMode::Legacy, tz).unwrap(), + Some(253402346040000000) + ); + assert_eq!( + timestamp_parser("10000-01-01T12:34:56", EvalMode::Legacy, tz).unwrap(), + Some(253402346096000000) + ); + assert_eq!( + timestamp_parser("10000-01-01T12:34:56.123456", EvalMode::Legacy, tz).unwrap(), + Some(253402346096123456) + ); + } + + #[test] + fn date_parser_test() { + for date in &[ + "2020", + "2020-01", + "2020-01-01", + "02020-01-01", + "002020-01-01", + "0002020-01-01", + "2020-1-1", + "2020-01-01 ", + "2020-01-01T", + ] { + for eval_mode in &[EvalMode::Legacy, EvalMode::Ansi, EvalMode::Try] { + assert_eq!(date_parser(date, *eval_mode).unwrap(), Some(18262)); + } + } + + //dates in invalid formats + for date in &[ + "abc", + "", + "not_a_date", + "3/", + "3/12", + "3/12/2020", + "3/12/2002 T", + "202", + "2020-010-01", + "2020-10-010", + "2020-10-010T", + "--262143-12-31", + "--262143-12-31 ", + ] { + for eval_mode in &[EvalMode::Legacy, EvalMode::Try] { + assert_eq!(date_parser(date, *eval_mode).unwrap(), None); + } + assert!(date_parser(date, EvalMode::Ansi).is_err()); + } + + for date in &["-3638-5"] { + for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { + assert_eq!(date_parser(date, *eval_mode).unwrap(), Some(-2048160)); + } + } + + //Naive Date only supports years 262142 AD to 262143 BC + //returns None for dates out of range supported by Naive Date. + for date in &[ + "-262144-1-1", + "262143-01-1", + "262143-1-1", + "262143-01-1 ", + "262143-01-01T ", + "262143-1-01T 1234", + "-0973250", + ] { + for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { + assert_eq!(date_parser(date, *eval_mode).unwrap(), None); + } + } + } + + #[test] + fn test_cast_string_to_date() { + let array: ArrayRef = Arc::new(StringArray::from(vec![ + Some("2020"), + Some("2020-01"), + Some("2020-01-01"), + Some("2020-01-01T"), + ])); + + let result = cast_string_to_date(&array, &DataType::Date32, EvalMode::Legacy).unwrap(); + + let date32_array = result + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(date32_array.len(), 4); + date32_array + .iter() + .for_each(|v| assert_eq!(v.unwrap(), 18262)); + } + + #[test] + fn test_cast_string_array_with_valid_dates() { + let array_with_invalid_date: ArrayRef = Arc::new(StringArray::from(vec![ + Some("-262143-12-31"), + Some("\n -262143-12-31 "), + Some("-262143-12-31T \t\n"), + Some("\n\t-262143-12-31T\r"), + Some("-262143-12-31T 123123123"), + Some("\r\n-262143-12-31T \r123123123"), + Some("\n -262143-12-31T \n\t"), + ])); + + for eval_mode in &[EvalMode::Legacy, EvalMode::Try, EvalMode::Ansi] { + let result = + cast_string_to_date(&array_with_invalid_date, &DataType::Date32, *eval_mode) + .unwrap(); + + let date32_array = result + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(result.len(), 7); + date32_array + .iter() + .for_each(|v| assert_eq!(v.unwrap(), -96464928)); + } + } + + #[test] + fn test_cast_string_array_with_invalid_dates() { + let array_with_invalid_date: ArrayRef = Arc::new(StringArray::from(vec![ + Some("2020"), + Some("2020-01"), + Some("2020-01-01"), + //4 invalid dates + Some("2020-010-01T"), + Some("202"), + Some(" 202 "), + Some("\n 2020-\r8 "), + Some("2020-01-01T"), + // Overflows i32 + Some("-4607172990231812908"), + ])); + + for eval_mode in &[EvalMode::Legacy, EvalMode::Try] { + let result = + cast_string_to_date(&array_with_invalid_date, &DataType::Date32, *eval_mode) + .unwrap(); + + let date32_array = result + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + date32_array.iter().collect::>(), + vec![ + Some(18262), + Some(18262), + Some(18262), + None, + None, + None, + None, + Some(18262), + None + ] + ); + } + + let result = + cast_string_to_date(&array_with_invalid_date, &DataType::Date32, EvalMode::Ansi); + match result { + Err(e) => assert!( + e.to_string().contains( + "[CAST_INVALID_INPUT] The value '2020-010-01T' of the type \"STRING\" cannot be cast to \"DATE\" because it is malformed") + ), + _ => panic!("Expected error"), + } + } + + #[test] + fn test_cast_string_as_i8() { + // basic + assert_eq!( + cast_string_to_i8("127", EvalMode::Legacy).unwrap(), + Some(127_i8) + ); + assert_eq!(cast_string_to_i8("128", EvalMode::Legacy).unwrap(), None); + assert!(cast_string_to_i8("128", EvalMode::Ansi).is_err()); + // decimals + assert_eq!( + cast_string_to_i8("0.2", EvalMode::Legacy).unwrap(), + Some(0_i8) + ); + assert_eq!( + cast_string_to_i8(".", EvalMode::Legacy).unwrap(), + Some(0_i8) + ); + // TRY should always return null for decimals + assert_eq!(cast_string_to_i8("0.2", EvalMode::Try).unwrap(), None); + assert_eq!(cast_string_to_i8(".", EvalMode::Try).unwrap(), None); + // ANSI mode should throw error on decimal + assert!(cast_string_to_i8("0.2", EvalMode::Ansi).is_err()); + assert!(cast_string_to_i8(".", EvalMode::Ansi).is_err()); + } +} diff --git a/native/spark-expr/src/conversion_funcs/utils.rs b/native/spark-expr/src/conversion_funcs/utils.rs new file mode 100644 index 0000000000..bac080a968 --- /dev/null +++ b/native/spark-expr/src/conversion_funcs/utils.rs @@ -0,0 +1,119 @@ +// 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. + +use crate::cast::MICROS_PER_SECOND; +use crate::SparkError; +use arrow::array::{ + Array, ArrayRef, ArrowPrimitiveType, AsArray, GenericStringArray, PrimitiveArray, +}; +use arrow::compute::unary; +use arrow::datatypes::{DataType, Int64Type}; +use arrow::error::ArrowError; +use datafusion::common::cast::as_generic_string_array; +use num::integer::div_floor; +use std::sync::Arc; + +/// A fork & modified version of Arrow's `unary_dyn` which is being deprecated +pub fn unary_dyn(array: &ArrayRef, op: F) -> Result +where + T: ArrowPrimitiveType, + F: Fn(T::Native) -> T::Native, +{ + if let Some(d) = array.as_any_dictionary_opt() { + let new_values = unary_dyn::(d.values(), op)?; + return Ok(Arc::new(d.with_values(Arc::new(new_values)))); + } + + match array.as_primitive_opt::() { + Some(a) if PrimitiveArray::::is_compatible(a.data_type()) => { + Ok(Arc::new(unary::( + array.as_any().downcast_ref::>().unwrap(), + op, + ))) + } + _ => Err(ArrowError::NotYetImplemented(format!( + "Cannot perform unary operation of type {} on array of type {}", + T::DATA_TYPE, + array.data_type() + ))), + } +} + +/// This takes for special casting cases of Spark. E.g., Timestamp to Long. +/// This function runs as a post process of the DataFusion cast(). By the time it arrives here, +/// Dictionary arrays are already unpacked by the DataFusion cast() since Spark cannot specify +/// Dictionary as to_type. The from_type is taken before the DataFusion cast() runs in +/// expressions/cast.rs, so it can be still Dictionary. +pub fn spark_cast_postprocess( + array: ArrayRef, + from_type: &DataType, + to_type: &DataType, +) -> ArrayRef { + match (from_type, to_type) { + (DataType::Timestamp(_, _), DataType::Int64) => { + // See Spark's `Cast` expression + unary_dyn::<_, Int64Type>(&array, |v| div_floor(v, MICROS_PER_SECOND)).unwrap() + } + (DataType::Dictionary(_, value_type), DataType::Int64) + if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => + { + // See Spark's `Cast` expression + unary_dyn::<_, Int64Type>(&array, |v| div_floor(v, MICROS_PER_SECOND)).unwrap() + } + (DataType::Timestamp(_, _), DataType::Utf8) => remove_trailing_zeroes(array), + (DataType::Dictionary(_, value_type), DataType::Utf8) + if matches!(value_type.as_ref(), &DataType::Timestamp(_, _)) => + { + remove_trailing_zeroes(array) + } + _ => array, + } +} + +/// Remove any trailing zeroes in the string if they occur after in the fractional seconds, +/// to match Spark behavior +/// example: +/// "1970-01-01 05:29:59.900" => "1970-01-01 05:29:59.9" +/// "1970-01-01 05:29:59.990" => "1970-01-01 05:29:59.99" +/// "1970-01-01 05:29:59.999" => "1970-01-01 05:29:59.999" +/// "1970-01-01 05:30:00" => "1970-01-01 05:30:00" +/// "1970-01-01 05:30:00.001" => "1970-01-01 05:30:00.001" +fn remove_trailing_zeroes(array: ArrayRef) -> ArrayRef { + let string_array = as_generic_string_array::(&array).unwrap(); + let result = string_array + .iter() + .map(|s| s.map(trim_end)) + .collect::>(); + Arc::new(result) as ArrayRef +} + +fn trim_end(s: &str) -> &str { + if s.rfind('.').is_some() { + s.trim_end_matches('0') + } else { + s + } +} + +#[inline] +pub fn cast_overflow(value: &str, from_type: &str, to_type: &str) -> SparkError { + SparkError::CastOverFlow { + value: value.to_string(), + from_type: from_type.to_string(), + to_type: to_type.to_string(), + } +} diff --git a/pom.xml b/pom.xml index 09081050ee..cde42655a3 100644 --- a/pom.xml +++ b/pom.xml @@ -477,12 +477,43 @@ under the License. ${testcontainers.version} test + software.amazon.awssdk s3 ${amazon-awssdk-v2.version} test + + software.amazon.awssdk + sts + ${amazon-awssdk-v2.version} + test + + + software.amazon.awssdk + dynamodb + ${amazon-awssdk-v2.version} + test + + + software.amazon.awssdk + glue + ${amazon-awssdk-v2.version} + test + + + software.amazon.awssdk + kms + ${amazon-awssdk-v2.version} + test + org.codehaus.jackson @@ -1069,6 +1100,7 @@ under the License. dev/release/rat_exclude_files.txt dev/release/requirements.txt native/proto/src/generated/** + benchmarks/tpc/queries/** diff --git a/spark/pom.xml b/spark/pom.xml index 8e3dd2095b..0b3a74f360 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -169,10 +169,27 @@ under the License. org.testcontainers minio + software.amazon.awssdk s3 + + software.amazon.awssdk + sts + + + software.amazon.awssdk + dynamodb + + + software.amazon.awssdk + glue + + + software.amazon.awssdk + kms + diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index d27f88b496..f17d8f4f72 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -278,6 +278,11 @@ object CometExecIterator extends Logging { builder.putEntries(k, v) } } + // Inject the resolved executor cores so the native side can use it + // for tokio runtime thread count + val executorCores = numDriverOrExecutorCores(SparkEnv.get.conf) + builder.putEntries("spark.executor.cores", executorCores.toString) + builder.build().toByteArray } diff --git a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala index 000cc5fd4f..15dfcb2d7c 100644 --- a/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala +++ b/spark/src/main/scala/org/apache/comet/expressions/CometCast.scala @@ -67,16 +67,36 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { case _: Literal => exprToProtoInternal(Literal.create(cast.eval(), cast.dataType), inputs, binding) case _ => - val childExpr = exprToProtoInternal(cast.child, inputs, binding) - if (childExpr.isDefined) { - castToProto(cast, cast.timeZoneId, cast.dataType, childExpr.get, evalMode(cast)) + if (isAlwaysCastToNull(cast.child.dataType, cast.dataType, evalMode(cast))) { + exprToProtoInternal(Literal.create(null, cast.dataType), inputs, binding) } else { - withInfo(cast, cast.child) - None + val childExpr = exprToProtoInternal(cast.child, inputs, binding) + if (childExpr.isDefined) { + castToProto(cast, cast.timeZoneId, cast.dataType, childExpr.get, evalMode(cast)) + } else { + withInfo(cast, cast.child) + None + } } } } +// Some casts like date -> int/byte / long are always null. Terminate early in planning + private def isAlwaysCastToNull( + fromType: DataType, + toType: DataType, + evalMode: CometEvalMode.Value): Boolean = { + (fromType, toType) match { + case ( + DataTypes.DateType, + DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | + DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | + DataTypes.DoubleType | _: DecimalType) if evalMode == CometEvalMode.LEGACY => + true + case _ => false + } + } + /** * Wrap an already serialized expression in a cast. */ @@ -168,7 +188,7 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { } } Compatible() - case (DataTypes.DateType, toType) => canCastFromDate(toType) + case (DataTypes.DateType, toType) => canCastFromDate(toType, evalMode) case _ => unsupported(fromType, toType) } } @@ -279,6 +299,8 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { Compatible() case DataTypes.BinaryType if (evalMode == CometEvalMode.LEGACY) => Compatible() + case DataTypes.TimestampType => + Compatible() case _ => unsupported(DataTypes.ByteType, toType) } @@ -293,6 +315,8 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { Compatible() case DataTypes.BinaryType if (evalMode == CometEvalMode.LEGACY) => Compatible() + case DataTypes.TimestampType => + Compatible() case _ => unsupported(DataTypes.ShortType, toType) } @@ -308,6 +332,8 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { case _: DecimalType => Compatible() case DataTypes.BinaryType if (evalMode == CometEvalMode.LEGACY) => Compatible() + case DataTypes.TimestampType => + Compatible() case _ => unsupported(DataTypes.IntegerType, toType) } @@ -323,6 +349,8 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { case _: DecimalType => Compatible() case DataTypes.BinaryType if (evalMode == CometEvalMode.LEGACY) => Compatible() + case DataTypes.TimestampType => + Compatible() case _ => unsupported(DataTypes.LongType, toType) } @@ -355,11 +383,16 @@ object CometCast extends CometExpressionSerde[Cast] with CometExprShim { case _ => Unsupported(Some(s"Cast from DecimalType to $toType is not supported")) } - private def canCastFromDate(toType: DataType): SupportLevel = toType match { - case DataTypes.TimestampType => - Compatible() - case _ => Unsupported(Some(s"Cast from DateType to $toType is not supported")) - } + private def canCastFromDate(toType: DataType, evalMode: CometEvalMode.Value): SupportLevel = + toType match { + case DataTypes.TimestampType => + Compatible() + case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType | + DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | + DataTypes.DoubleType | _: DecimalType if evalMode == CometEvalMode.LEGACY => + Compatible() + case _ => Unsupported(Some(s"Cast from DateType to $toType is not supported")) + } private def unsupported(fromType: DataType, toType: DataType): Unsupported = { Unsupported(Some(s"Cast from $fromType to $toType is not supported")) 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 1922a3ec6a..3443e6db2b 100644 --- a/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala +++ b/spark/src/main/scala/org/apache/comet/iceberg/IcebergReflection.scala @@ -257,6 +257,32 @@ object IcebergReflection extends Logging { } } + /** + * Gets storage properties from an Iceberg table's FileIO. + * + * This extracts credentials from the FileIO implementation, which is critical for REST catalog + * credential vending. The REST catalog returns temporary S3 credentials per-table via the + * loadTable response, stored in the table's FileIO (typically ResolvingFileIO). + * + * The properties() method is not on the FileIO interface -- it exists on specific + * implementations like ResolvingFileIO and S3FileIO. Returns None gracefully when unavailable. + */ + def getFileIOProperties(table: Any): Option[Map[String, String]] = { + import scala.jdk.CollectionConverters._ + getFileIO(table).flatMap { fileIO => + findMethodInHierarchy(fileIO.getClass, "properties").flatMap { propsMethod => + propsMethod.invoke(fileIO) match { + case javaMap: java.util.Map[_, _] => + val scalaMap = javaMap.asScala.collect { case (k: String, v: String) => + k -> v + }.toMap + if (scalaMap.nonEmpty) Some(scalaMap) else None + case _ => None + } + } + } + } + /** * Gets the schema from an Iceberg table. */ 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 e015c0d39a..42669bb5ee 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -49,7 +49,7 @@ import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflecti import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.{Native, SupportsComet} import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} -import org.apache.comet.serde.operator.CometNativeScan +import org.apache.comet.serde.operator.{CometIcebergNativeScan, CometNativeScan} import org.apache.comet.shims.{CometTypeShim, ShimFileFormat, ShimSubqueryBroadcast} /** @@ -391,9 +391,18 @@ case class CometScanRule(session: SparkSession) val hadoopS3Options = NativeConfig.extractObjectStoreOptions(hadoopConf, effectiveUri) - val catalogProperties = - org.apache.comet.serde.operator.CometIcebergNativeScan - .hadoopToIcebergS3Properties(hadoopS3Options) + val hadoopDerivedProperties = + CometIcebergNativeScan.hadoopToIcebergS3Properties(hadoopS3Options) + + // Extract vended credentials from FileIO (REST catalog credential vending). + // FileIO properties take precedence over Hadoop-derived properties because + // they contain per-table credentials vended by the REST catalog. + val fileIOProperties = tableOpt + .flatMap(IcebergReflection.getFileIOProperties) + .map(CometIcebergNativeScan.filterStorageProperties) + .getOrElse(Map.empty) + + val catalogProperties = hadoopDerivedProperties ++ fileIOProperties val result = CometIcebergNativeScanMetadata .extract(scanExec.scan, effectiveLocation, catalogProperties) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index f627b0c465..9d13ccd9ed 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -139,6 +139,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[StructsToCsv] -> CometStructsToCsv) private val hashExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( + classOf[Crc32] -> CometScalarFunction("crc32"), classOf[Md5] -> CometScalarFunction("md5"), classOf[Murmur3Hash] -> CometMurmur3Hash, classOf[Sha2] -> CometSha2, @@ -149,7 +150,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[Ascii] -> CometScalarFunction("ascii"), classOf[BitLength] -> CometScalarFunction("bit_length"), classOf[Chr] -> CometScalarFunction("char"), - classOf[ConcatWs] -> CometScalarFunction("concat_ws"), + classOf[ConcatWs] -> CometConcatWs, classOf[Concat] -> CometConcat, classOf[Contains] -> CometScalarFunction("contains"), classOf[EndsWith] -> CometScalarFunction("ends_with"), diff --git a/spark/src/main/scala/org/apache/comet/serde/arrays.scala b/spark/src/main/scala/org/apache/comet/serde/arrays.scala index cdaf3d5f88..b7ebb9ba7b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/arrays.scala +++ b/spark/src/main/scala/org/apache/comet/serde/arrays.scala @@ -247,18 +247,41 @@ object CometArraysOverlap extends CometExpressionSerde[ArraysOverlap] { object CometArrayRepeat extends CometExpressionSerde[ArrayRepeat] { - override def getSupportLevel(expr: ArrayRepeat): SupportLevel = Incompatible(None) - override def convert( expr: ArrayRepeat, inputs: Seq[Attribute], binding: Boolean): Option[ExprOuterClass.Expr] = { - val leftArrayExprProto = exprToProto(expr.children.head, inputs, binding) - val rightArrayExprProto = exprToProto(expr.children(1), inputs, binding) + val elementProto = exprToProto(expr.left, inputs, binding) + val countProto = exprToProto(expr.right, inputs, binding) + val returnType = ArrayType(elementType = expr.left.dataType) + for { + countIsNotNullExpr <- countIsNotNullExpr(expr, inputs, binding) + arrayRepeatExprProto <- scalarFunctionExprToProto("array_repeat", elementProto, countProto) + nullLiteralExprProto <- exprToProtoInternal(Literal(null, returnType), inputs, binding) + } yield { + val caseWhenProto = ExprOuterClass.CaseWhen + .newBuilder() + .addWhen(countIsNotNullExpr) + .addThen(arrayRepeatExprProto) + .setElseExpr(nullLiteralExprProto) + .build() + ExprOuterClass.Expr + .newBuilder() + .setCaseWhen(caseWhenProto) + .build() + } + } - val arraysRepeatScalarExpr = - scalarFunctionExprToProto("array_repeat", leftArrayExprProto, rightArrayExprProto) - optExprWithInfo(arraysRepeatScalarExpr, expr, expr.children: _*) + private def countIsNotNullExpr( + expr: ArrayRepeat, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + createUnaryExpr( + expr, + expr.right, + inputs, + binding, + (builder, countExpr) => builder.setIsNotNull(countExpr)) } } diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala index 31575138f8..69b9bd5f85 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala @@ -60,7 +60,7 @@ object CometDataWritingCommand extends CometOperatorSerde[DataWritingCommandExec case _: ParquetFileFormat => if (!cmd.outputPath.toString.startsWith("file:") && !cmd.outputPath.toString .startsWith("hdfs:")) { - return Unsupported(Some("Only HDFS/local filesystems output paths are supported")) + return Unsupported(Some("Supported output filesystems: local, HDFS")) } if (cmd.bucketSpec.isDefined) { diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala index 957f621032..9f1a015996 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometIcebergNativeScan.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeExec} import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceRDD, DataSourceRDDPartition} import org.apache.spark.sql.types._ -import org.apache.comet.ConfigEntry +import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflection} import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.ExprOuterClass.Expr @@ -488,6 +488,21 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit } } + /** Storage-related property prefixes passed through to native FileIO. */ + private val storagePropertyPrefixes = + Seq("s3.", "gcs.", "adls.", "client.") + + /** + * Filters a properties map to only include storage-related keys. FileIO.properties() may + * contain catalog URIs, bearer tokens, and other non-storage settings that should not be passed + * to the native FileIO builder. + */ + def filterStorageProperties(props: Map[String, String]): Map[String, String] = { + props.filter { case (key, _) => + storagePropertyPrefixes.exists(prefix => key.startsWith(prefix)) + } + } + /** * Transforms Hadoop S3A configuration keys to Iceberg FileIO property keys. * @@ -742,6 +757,8 @@ object CometIcebergNativeScan extends CometOperatorSerde[CometBatchScanExec] wit var totalTasks = 0 commonBuilder.setMetadataLocation(metadata.metadataLocation) + commonBuilder.setDataFileConcurrencyLimit( + CometConf.COMET_ICEBERG_DATA_FILE_CONCURRENCY_LIMIT.get()) metadata.catalogProperties.foreach { case (key, value) => commonBuilder.putCatalogProperties(key, value) } diff --git a/spark/src/main/scala/org/apache/comet/serde/strings.scala b/spark/src/main/scala/org/apache/comet/serde/strings.scala index db60709007..64ba644048 100644 --- a/spark/src/main/scala/org/apache/comet/serde/strings.scala +++ b/spark/src/main/scala/org/apache/comet/serde/strings.scala @@ -21,7 +21,7 @@ package org.apache.comet.serde import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, Expression, If, InitCap, IsNull, Left, Length, Like, Literal, Lower, RegExpReplace, Right, RLike, StringLPad, StringRepeat, StringRPad, StringSplit, Substring, Upper} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, ConcatWs, Expression, If, InitCap, IsNull, Left, Length, Like, Literal, Lower, RegExpReplace, Right, RLike, StringLPad, StringRepeat, StringRPad, StringSplit, Substring, Upper} import org.apache.spark.sql.types.{BinaryType, DataTypes, LongType, StringType} import org.apache.spark.unsafe.types.UTF8String @@ -199,6 +199,27 @@ object CometConcat extends CometScalarFunction[Concat]("concat") { } } +object CometConcatWs extends CometExpressionSerde[ConcatWs] { + + override def convert(expr: ConcatWs, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = { + expr.children.headOption match { + // Match Spark behavior: when the separator is NULL, the result of concat_ws is NULL. + case Some(Literal(null, _)) => + val nullLiteral = Literal.create(null, expr.dataType) + exprToProtoInternal(nullLiteral, inputs, binding) + + case _ if expr.children.forall(_.foldable) => + // Fall back to Spark for all-literal args so ConstantFolding can handle it + withInfo(expr, "all arguments are foldable") + None + + case _ => + // For all other cases, use the generic scalar function implementation. + CometScalarFunction[ConcatWs]("concat_ws").convert(expr, inputs, binding) + } + } +} + object CometLike extends CometExpressionSerde[Like] { override def convert(expr: Like, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = { @@ -223,11 +244,11 @@ object CometRLike extends CometExpressionSerde[RLike] { expr.right match { case Literal(pattern, DataTypes.StringType) => if (!RegExp.isSupportedPattern(pattern.toString) && - !CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.get()) { + !CometConf.isExprAllowIncompat("regexp")) { withInfo( expr, s"Regexp pattern $pattern is not compatible with Spark. " + - s"Set ${CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key}=true " + + s"Set ${CometConf.getExprAllowIncompatConfigKey("regexp")}=true " + "to allow it anyway.") None } else { @@ -298,11 +319,11 @@ object CometStringLPad extends CometExpressionSerde[StringLPad] { object CometRegExpReplace extends CometExpressionSerde[RegExpReplace] { override def getSupportLevel(expr: RegExpReplace): SupportLevel = { if (!RegExp.isSupportedPattern(expr.regexp.toString) && - !CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.get()) { + !CometConf.isExprAllowIncompat("regexp")) { withInfo( expr, s"Regexp pattern ${expr.regexp} is not compatible with Spark. " + - s"Set ${CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key}=true " + + s"Set ${CometConf.getExprAllowIncompatConfigKey("regexp")}=true " + "to allow it anyway.") return Incompatible() } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeColumnarToRowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeColumnarToRowExec.scala index a520098ed1..2345386143 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeColumnarToRowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeColumnarToRowExec.scala @@ -232,11 +232,6 @@ case class CometNativeColumnarToRowExec(child: SparkPlan) object CometNativeColumnarToRowExec { - /** - * Checks if native columnar to row conversion is enabled. - */ - def isEnabled: Boolean = CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.get() - /** * Checks if the given schema is supported by native columnar to row conversion. * diff --git a/spark/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/spark/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 7d5d6ce6b2..7b04110d3d 100644 --- a/spark/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/spark/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -98,6 +98,14 @@ public class RESTCatalogAdapter implements RESTClient { private final SupportsNamespaces asNamespaceCatalog; private final ViewCatalog asViewCatalog; + // Optional credentials to inject into loadTable responses, simulating REST catalog + // credential vending. When non-empty, these are added to LoadTableResponse.config(). + private Map vendedCredentials = ImmutableMap.of(); + + public void setVendedCredentials(Map credentials) { + this.vendedCredentials = credentials; + } + public RESTCatalogAdapter(Catalog catalog) { this.catalog = catalog; this.asNamespaceCatalog = @@ -279,6 +287,26 @@ private static OAuthTokenResponse handleOAuthRequest(Object body) { @SuppressWarnings({"MethodLength", "checkstyle:CyclomaticComplexity"}) public T handleRequest( Route route, Map vars, Object body, Class responseType) { + T response = doHandleRequest(route, vars, body, responseType); + // Inject vended credentials into any LoadTableResponse, simulating REST catalog + // credential vending. This covers CREATE_TABLE, LOAD_TABLE, UPDATE_TABLE, etc. + if (!vendedCredentials.isEmpty() && response instanceof LoadTableResponse) { + LoadTableResponse original = (LoadTableResponse) response; + @SuppressWarnings("unchecked") + T withCreds = + (T) + LoadTableResponse.builder() + .withTableMetadata(original.tableMetadata()) + .addAllConfig(original.config()) + .addAllConfig(vendedCredentials) + .build(); + return withCreds; + } + return response; + } + + private T doHandleRequest( + Route route, Map vars, Object body, Class responseType) { switch (route) { case TOKENS: return castResponse(responseType, handleOAuthRequest(body)); diff --git a/spark/src/test/resources/sql-tests/expressions/array/array_repeat.sql b/spark/src/test/resources/sql-tests/expressions/array/array_repeat.sql index 926b1141ef..f9db5ec1ec 100644 --- a/spark/src/test/resources/sql-tests/expressions/array/array_repeat.sql +++ b/spark/src/test/resources/sql-tests/expressions/array/array_repeat.sql @@ -23,17 +23,17 @@ CREATE TABLE test_array_repeat(val int, cnt int) USING parquet statement INSERT INTO test_array_repeat VALUES (1, 3), (NULL, 3), (1, 0), (1, -1), (1, NULL) -query spark_answer_only +query SELECT array_repeat(val, cnt) FROM test_array_repeat -- column + literal -query spark_answer_only +query SELECT array_repeat(val, 3) FROM test_array_repeat -- literal + column -query spark_answer_only +query SELECT array_repeat(1, cnt) FROM test_array_repeat -- literal + literal -query spark_answer_only +query SELECT array_repeat(1, 3), array_repeat(NULL, 3), array_repeat(1, 0), array_repeat(1, -1) diff --git a/spark/src/test/resources/sql-tests/expressions/bitwise/bitwise.sql b/spark/src/test/resources/sql-tests/expressions/bitwise/bitwise.sql index 640aa1e990..74a971f368 100644 --- a/spark/src/test/resources/sql-tests/expressions/bitwise/bitwise.sql +++ b/spark/src/test/resources/sql-tests/expressions/bitwise/bitwise.sql @@ -73,7 +73,7 @@ SELECT bit_get(i, pos) FROM test_bit_get query SELECT 1111 & 2, 1111 | 2, 1111 ^ 2 -query ignore(https://github.com/apache/datafusion-comet/issues/3341) +query SELECT bit_count(0), bit_count(7), bit_count(-1) query spark_answer_only diff --git a/spark/src/test/resources/sql-tests/expressions/hash/crc32.sql b/spark/src/test/resources/sql-tests/expressions/hash/crc32.sql new file mode 100644 index 0000000000..64b401c98c --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/hash/crc32.sql @@ -0,0 +1,32 @@ +-- 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. + +-- ConfigMatrix: parquet.enable.dictionary=false,true + +-- crc32 function +statement +CREATE TABLE test(col string, a int, b float) USING parquet + +statement +INSERT INTO test VALUES ('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999), ('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) + +query +SELECT crc32(col), crc32(cast(a as string)), crc32(cast(b as string)) FROM test + +-- literal arguments +query +SELECT crc32('Spark SQL') diff --git a/spark/src/test/resources/sql-tests/expressions/string/concat_ws.sql b/spark/src/test/resources/sql-tests/expressions/string/concat_ws.sql index 4a3df68965..81ccfb0f36 100644 --- a/spark/src/test/resources/sql-tests/expressions/string/concat_ws.sql +++ b/spark/src/test/resources/sql-tests/expressions/string/concat_ws.sql @@ -42,6 +42,6 @@ INSERT INTO names VALUES(1, 'James', 'B', 'Taylor'), (2, 'Smith', 'C', 'Davis'), query SELECT concat_ws(' ', first_name, middle_initial, last_name) FROM names --- literal + literal + literal -query ignore(https://github.com/apache/datafusion-comet/issues/3339) +-- literal + literal + literal (falls back to Spark when all args are foldable) +query spark_answer_only SELECT concat_ws(',', 'hello', 'world'), concat_ws(',', '', ''), concat_ws(',', NULL, 'b', 'c'), concat_ws(NULL, 'a', 'b') diff --git a/spark/src/test/resources/sql-tests/expressions/string/regexp_replace_enabled.sql b/spark/src/test/resources/sql-tests/expressions/string/regexp_replace_enabled.sql index 7ea13a18a2..bf8a544515 100644 --- a/spark/src/test/resources/sql-tests/expressions/string/regexp_replace_enabled.sql +++ b/spark/src/test/resources/sql-tests/expressions/string/regexp_replace_enabled.sql @@ -16,7 +16,7 @@ -- under the License. -- Test regexp_replace() with regexp allowIncompatible enabled (happy path) --- Config: spark.comet.regexp.allowIncompatible=true +-- Config: spark.comet.expression.regexp.allowIncompatible=true -- ConfigMatrix: parquet.enable.dictionary=false,true statement diff --git a/spark/src/test/resources/sql-tests/expressions/string/rlike_enabled.sql b/spark/src/test/resources/sql-tests/expressions/string/rlike_enabled.sql index 968a2f22f7..822fb3ddb8 100644 --- a/spark/src/test/resources/sql-tests/expressions/string/rlike_enabled.sql +++ b/spark/src/test/resources/sql-tests/expressions/string/rlike_enabled.sql @@ -16,7 +16,7 @@ -- under the License. -- Test RLIKE with regexp allowIncompatible enabled (happy path) --- Config: spark.comet.regexp.allowIncompatible=true +-- Config: spark.comet.expression.regexp.allowIncompatible=true -- ConfigMatrix: parquet.enable.dictionary=false,true statement diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 9fc9a16579..72c2390d71 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -65,6 +65,23 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { lazy val usingParquetExecWithIncompatTypes: Boolean = hasUnsignedSmallIntSafetyCheck(conf) + // Timezone list to check temporal type casts + private val compatibleTimezones = Seq( + "UTC", + "America/New_York", + "America/Chicago", + "America/Denver", + "America/Los_Angeles", + "Europe/London", + "Europe/Paris", + "Europe/Berlin", + "Asia/Tokyo", + "Asia/Shanghai", + "Asia/Singapore", + "Asia/Kolkata", + "Australia/Sydney", + "Pacific/Auckland") + test("all valid cast combinations covered") { val names = testNames @@ -223,12 +240,15 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { testTry = false) } - ignore("cast ByteType to TimestampType") { - // input: -1, expected: 1969-12-31 15:59:59.0, actual: 1969-12-31 15:59:59.999999 - castTest( - generateBytes(), - DataTypes.TimestampType, - hasIncompatibleType = usingParquetExecWithIncompatTypes) + test("cast ByteType to TimestampType") { + compatibleTimezones.foreach { tz => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + castTest( + generateBytes(), + DataTypes.TimestampType, + hasIncompatibleType = usingParquetExecWithIncompatTypes) + } + } } // CAST from ShortType @@ -300,12 +320,15 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { testTry = false) } - ignore("cast ShortType to TimestampType") { - // input: -1003, expected: 1969-12-31 15:43:17.0, actual: 1969-12-31 15:59:59.998997 - castTest( - generateShorts(), - DataTypes.TimestampType, - hasIncompatibleType = usingParquetExecWithIncompatTypes) + test("cast ShortType to TimestampType") { + compatibleTimezones.foreach { tz => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + castTest( + generateShorts(), + DataTypes.TimestampType, + hasIncompatibleType = usingParquetExecWithIncompatTypes) + } + } } // CAST from integer @@ -363,9 +386,12 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateInts(), DataTypes.BinaryType, testAnsi = false, testTry = false) } - ignore("cast IntegerType to TimestampType") { - // input: -1000479329, expected: 1938-04-19 01:04:31.0, actual: 1969-12-31 15:43:19.520671 - castTest(generateInts(), DataTypes.TimestampType) + test("cast IntegerType to TimestampType") { + compatibleTimezones.foreach { tz => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + castTest(generateInts(), DataTypes.TimestampType) + } + } } // CAST from LongType @@ -410,9 +436,17 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { castTest(generateLongs(), DataTypes.BinaryType, testAnsi = false, testTry = false) } - ignore("cast LongType to TimestampType") { - // java.lang.ArithmeticException: long overflow - castTest(generateLongs(), DataTypes.TimestampType) + test("cast LongType to TimestampType") { + // Cast back to long avoids java.sql.Timestamp overflow during collect() for extreme values + compatibleTimezones.foreach { tz => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + withTable("t1") { + generateLongs().write.saveAsTable("t1") + val df = spark.sql("select a, cast(cast(a as timestamp) as long) from t1") + checkSparkAnswerAndOperator(df) + } + } + } } // CAST from FloatType @@ -960,44 +994,44 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { // CAST from DateType - ignore("cast DateType to BooleanType") { - // Arrow error: Cast error: Casting from Date32 to Boolean not supported - castTest(generateDates(), DataTypes.BooleanType) + // Date to Boolean/Byte/Short/Int/Long/Float/Double/Decimal casts always return NULL + // in LEGACY mode. In ANSI and TRY mode, Spark throws AnalysisException at + // query parsing time. Hence, ANSI and Try mode are disabled in tests + + test("cast DateType to BooleanType") { + castTest(generateDates(), DataTypes.BooleanType, testAnsi = false, testTry = false) } - ignore("cast DateType to ByteType") { - // Arrow error: Cast error: Casting from Date32 to Int8 not supported - castTest(generateDates(), DataTypes.ByteType) + test("cast DateType to ByteType") { + castTest(generateDates(), DataTypes.ByteType, testAnsi = false, testTry = false) } - ignore("cast DateType to ShortType") { - // Arrow error: Cast error: Casting from Date32 to Int16 not supported - castTest(generateDates(), DataTypes.ShortType) + test("cast DateType to ShortType") { + castTest(generateDates(), DataTypes.ShortType, testAnsi = false, testTry = false) } - ignore("cast DateType to IntegerType") { - // input: 2345-01-01, expected: null, actual: 3789391 - castTest(generateDates(), DataTypes.IntegerType) + test("cast DateType to IntegerType") { + castTest(generateDates(), DataTypes.IntegerType, testAnsi = false, testTry = false) } - ignore("cast DateType to LongType") { - // input: 2024-01-01, expected: null, actual: 19723 - castTest(generateDates(), DataTypes.LongType) + test("cast DateType to LongType") { + castTest(generateDates(), DataTypes.LongType, testAnsi = false, testTry = false) } - ignore("cast DateType to FloatType") { - // Arrow error: Cast error: Casting from Date32 to Float32 not supported - castTest(generateDates(), DataTypes.FloatType) + test("cast DateType to FloatType") { + castTest(generateDates(), DataTypes.FloatType, testAnsi = false, testTry = false) } - ignore("cast DateType to DoubleType") { - // Arrow error: Cast error: Casting from Date32 to Float64 not supported - castTest(generateDates(), DataTypes.DoubleType) + test("cast DateType to DoubleType") { + castTest(generateDates(), DataTypes.DoubleType, testAnsi = false, testTry = false) } - ignore("cast DateType to DecimalType(10,2)") { - // Arrow error: Cast error: Casting from Date32 to Decimal128(10, 2) not supported - castTest(generateDates(), DataTypes.createDecimalType(10, 2)) + test("cast DateType to DecimalType(10,2)") { + castTest( + generateDates(), + DataTypes.createDecimalType(10, 2), + testAnsi = false, + testTry = false) } test("cast DateType to StringType") { @@ -1042,13 +1076,13 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { ignore("cast TimestampType to ShortType") { // https://github.com/apache/datafusion-comet/issues/352 - // input: 2023-12-31 10:00:00.0, expected: -21472, actual: null] + // input: 2023-12-31 10:00:00.0, expected: -21472, actual: null castTest(generateTimestamps(), DataTypes.ShortType) } ignore("cast TimestampType to IntegerType") { // https://github.com/apache/datafusion-comet/issues/352 - // input: 2023-12-31 10:00:00.0, expected: 1704045600, actual: null] + // input: 2023-12-31 10:00:00.0, expected: 1704045600, actual: null castTest(generateTimestamps(), DataTypes.IntegerType) } diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 1ab8d54fd2..f0f022868f 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -958,7 +958,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { // add repetitive data to trigger dictionary encoding Range(0, 100).map(_ => "John Smith") withParquetFile(data.zipWithIndex, withDictionary) { file => - withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { spark.read.parquet(file).createOrReplaceTempView(table) val query = sql(s"select _2 as id, _1 rlike 'R[a-z]+s [Rr]ose' from $table") checkSparkAnswerAndOperator(query) @@ -996,7 +996,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { withTable(table) { sql(s"create table $table(id int, name varchar(20)) using parquet") sql(s"insert into $table values(1,'James Smith')") - withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { val query2 = sql(s"select id from $table where name rlike name") val (_, cometPlan) = checkSparkAnswer(query2) val explain = new ExtendedExplainInfo().generateExtendedInfo(cometPlan) @@ -1030,7 +1030,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { // "Smith$", "Smith\\Z", "Smith\\z") - withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { patterns.foreach { pattern => val query2 = sql(s"select name, '$pattern', name rlike '$pattern' from $table") checkSparkAnswerAndOperator(query2) @@ -1090,7 +1090,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { "\\V") val qualifiers = Seq("", "+", "*", "?", "{1,}") - withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { // testing every possible combination takes too long, so we pick some // random combinations for (_ <- 0 until 100) { @@ -1987,6 +1987,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { |md5(col), md5(cast(a as string)), md5(cast(b as string)), |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), |xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col), + |crc32(col), crc32(cast(a as string)), crc32(cast(b as string)), |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1), |sha1(col), sha1(cast(a as string)), sha1(cast(b as string)) |from test @@ -2097,6 +2098,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { |md5(col), md5(cast(a as string)), --md5(cast(b as string)), |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), |xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col), + |crc32(col), crc32(cast(a as string)), |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128), sha2(col, -1), |sha1(col), sha1(cast(a as string)) |from test diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala index 0a188f6cce..f37d997c41 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzIcebergSuite.scala @@ -133,7 +133,7 @@ class CometFuzzIcebergSuite extends CometFuzzIcebergBase { } test("regexp_replace") { - withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { val df = spark.table(icebergTableName) // We want to make sure that the schema generator wasn't modified to accidentally omit // StringType, since then this test would not run any queries and silently pass. diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestBase.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestBase.scala index 0e179b40fc..6b6f02f9fd 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestBase.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestBase.scala @@ -111,14 +111,16 @@ class CometFuzzTestBase extends CometTestBase with AdaptiveSparkPlanHelper { override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { - Seq("native", "jvm").foreach { shuffleMode => - super.test(testName + s" ($shuffleMode shuffle)", testTags: _*) { - withSQLConf( - CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "false", - CometConf.COMET_SHUFFLE_MODE.key -> shuffleMode) { - testFun + Seq(("native", "false"), ("jvm", "true"), ("jvm", "false")).foreach { + case (shuffleMode, nativeC2R) => + super.test(testName + s" ($shuffleMode shuffle, nativeC2R=$nativeC2R)", testTags: _*) { + withSQLConf( + CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.key -> nativeC2R, + CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "false", + CometConf.COMET_SHUFFLE_MODE.key -> shuffleMode) { + testFun + } } - } } } diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index 02d13c841d..fe6032414e 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -255,7 +255,7 @@ class CometFuzzTestSuite extends CometFuzzTestBase { } test("regexp_replace") { - withSQLConf(CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true") { + withSQLConf(CometConf.getExprAllowIncompatConfigKey("regexp") -> "true") { val df = spark.read.parquet(filename) df.createOrReplaceTempView("t1") // We want to make sure that the schema generator wasn't modified to accidentally omit diff --git a/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala b/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala index 00955e6291..c1c90adfa3 100644 --- a/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala +++ b/spark/src/test/scala/org/apache/comet/IcebergReadFromS3Suite.scala @@ -23,7 +23,9 @@ import org.apache.spark.SparkConf import org.apache.spark.sql.comet.CometIcebergNativeScanExec import org.apache.spark.sql.execution.SparkPlan -class IcebergReadFromS3Suite extends CometS3TestBase { +import org.apache.comet.iceberg.RESTCatalogHelper + +class IcebergReadFromS3Suite extends CometS3TestBase with RESTCatalogHelper { override protected val testBucketName = "test-iceberg-bucket" @@ -227,4 +229,74 @@ class IcebergReadFromS3Suite extends CometS3TestBase { spark.sql("DROP TABLE s3_catalog.db.mor_delete_test") } + + test("REST catalog credential vending rejects wrong credentials") { + assume(icebergAvailable, "Iceberg not available in classpath") + + val wrongCreds = Map( + "s3.access-key-id" -> "WRONG_ACCESS_KEY", + "s3.secret-access-key" -> "WRONG_SECRET_KEY", + "s3.endpoint" -> minioContainer.getS3URL, + "s3.path-style-access" -> "true") + val warehouse = s"s3a://$testBucketName/warehouse-bad-creds" + + withRESTCatalog(vendedCredentials = wrongCreds, warehouseLocation = Some(warehouse)) { + (restUri, _, _) => + withSQLConf( + "spark.sql.catalog.bad_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.bad_cat.catalog-impl" -> "org.apache.iceberg.rest.RESTCatalog", + "spark.sql.catalog.bad_cat.uri" -> restUri, + "spark.sql.catalog.bad_cat.warehouse" -> warehouse) { + + spark.sql("CREATE NAMESPACE bad_cat.db") + + // CREATE TABLE succeeds (metadata only, no S3 access needed) + spark.sql("CREATE TABLE bad_cat.db.test (id INT) USING iceberg") + + // INSERT fails because S3FileIO uses the wrong vended credentials + val e = intercept[Exception] { + spark.sql("INSERT INTO bad_cat.db.test VALUES (1)") + } + assert(e.getMessage.contains("403"), s"Expected S3 403 error but got: ${e.getMessage}") + } + } + } + + test("REST catalog credential vending with native Iceberg scan on S3") { + assume(icebergAvailable, "Iceberg not available in classpath") + + val vendedCreds = Map( + "s3.access-key-id" -> userName, + "s3.secret-access-key" -> password, + "s3.endpoint" -> minioContainer.getS3URL, + "s3.path-style-access" -> "true") + val warehouse = s"s3a://$testBucketName/warehouse-vending" + + withRESTCatalog(vendedCredentials = vendedCreds, warehouseLocation = Some(warehouse)) { + (restUri, _, _) => + withSQLConf( + "spark.sql.catalog.vend_cat" -> "org.apache.iceberg.spark.SparkCatalog", + "spark.sql.catalog.vend_cat.catalog-impl" -> "org.apache.iceberg.rest.RESTCatalog", + "spark.sql.catalog.vend_cat.uri" -> restUri, + "spark.sql.catalog.vend_cat.warehouse" -> warehouse, + CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true") { + + spark.sql("CREATE NAMESPACE vend_cat.db") + + spark.sql(""" + CREATE TABLE vend_cat.db.simple ( + id INT, name STRING, value DOUBLE + ) USING iceberg + """) + spark.sql(""" + INSERT INTO vend_cat.db.simple + VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) + """) + checkIcebergNativeScan("SELECT * FROM vend_cat.db.simple ORDER BY id") + + spark.sql("DROP TABLE vend_cat.db.simple") + spark.sql("DROP NAMESPACE vend_cat.db") + } + } + } } diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index e4c405c003..815f03f213 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -48,7 +48,7 @@ class CometParquetWriterSuite extends CometTestBase { withSQLConf( CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", SQLConf.SESSION_LOCAL_TIMEZONE.key -> "America/Halifax", - CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true", + CometConf.COMET_OPERATOR_DATA_WRITING_COMMAND_ALLOW_INCOMPAT.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true") { writeWithCometNativeWriteExec(inputPath, outputPath) @@ -70,7 +70,7 @@ class CometParquetWriterSuite extends CometTestBase { withSQLConf( CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", SQLConf.SESSION_LOCAL_TIMEZONE.key -> "America/Halifax", - CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true", + CometConf.COMET_OPERATOR_DATA_WRITING_COMMAND_ALLOW_INCOMPAT.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true") { withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> "native_datafusion") { @@ -310,55 +310,6 @@ class CometParquetWriterSuite extends CometTestBase { } } - // ignored: native_comet scan is no longer supported - ignore("native write falls back when scan produces non-Arrow data") { - // This test verifies that when a native scan (like native_comet) doesn't support - // certain data types (complex types), the native write correctly falls back to Spark - // instead of failing at runtime with "Comet execution only takes Arrow Arrays" error. - withTempPath { dir => - val inputPath = new File(dir, "input.parquet").getAbsolutePath - val outputPath = new File(dir, "output.parquet").getAbsolutePath - - // Create data with complex types and write without Comet - withSQLConf(CometConf.COMET_ENABLED.key -> "false") { - val df = Seq((1, Seq(1, 2, 3)), (2, Seq(4, 5)), (3, Seq(6, 7, 8, 9))) - .toDF("id", "values") - df.write.parquet(inputPath) - } - - // With native Parquet write enabled but using native_comet scan which doesn't - // support complex types, the scan falls back to Spark. The native write should - // detect this and also fall back to Spark instead of failing at runtime. - withSQLConf( - CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true", - // Use native_comet which doesn't support complex types - CometConf.COMET_NATIVE_SCAN_IMPL.key -> "native_comet") { - - val plan = - captureWritePlan(path => spark.read.parquet(inputPath).write.parquet(path), outputPath) - - // Verify NO CometNativeWriteExec in the plan (should have fallen back to Spark) - val hasNativeWrite = plan.exists { - case _: CometNativeWriteExec => true - case d: DataWritingCommandExec => - d.child.exists(_.isInstanceOf[CometNativeWriteExec]) - case _ => false - } - - assert( - !hasNativeWrite, - "Expected fallback to Spark write (no CometNativeWriteExec), but found native write " + - s"in plan:\n${plan.treeString}") - - // Verify the data was written correctly - val result = spark.read.parquet(outputPath).collect() - assert(result.length == 3, "Expected 3 rows to be written") - } - } - } - test("parquet write complex types fuzz test") { withTempPath { dir => val outputPath = new File(dir, "output.parquet").getAbsolutePath @@ -517,7 +468,7 @@ class CometParquetWriterSuite extends CometTestBase { withSQLConf( CometConf.COMET_EXEC_ENABLED.key -> "true", // enable experimental native writes - CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true", + CometConf.COMET_OPERATOR_DATA_WRITING_COMMAND_ALLOW_INCOMPAT.key -> "true", CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", // explicitly set scan impl to override CI defaults CometConf.COMET_NATIVE_SCAN_IMPL.key -> "auto", diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSMicroBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSMicroBenchmark.scala index d9c49bc596..5982460a87 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSMicroBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometTPCDSMicroBenchmark.scala @@ -114,7 +114,7 @@ object CometTPCDSMicroBenchmark extends CometTPCQueryBenchmarkBase { withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_REGEXP_ALLOW_INCOMPATIBLE.key -> "true", + CometConf.getExprAllowIncompatConfigKey("regexp") -> "true", // enabling COMET_EXPLAIN_NATIVE_ENABLED may add overhead but is useful for debugging CometConf.COMET_EXPLAIN_NATIVE_ENABLED.key -> "false") { cometSpark.sql(queryString).noop() diff --git a/spark/src/test/spark-3.x/org/apache/comet/iceberg/RESTCatalogHelper.scala b/spark/src/test/spark-3.x/org/apache/comet/iceberg/RESTCatalogHelper.scala index 6230ee33e1..856700c2cf 100644 --- a/spark/src/test/spark-3.x/org/apache/comet/iceberg/RESTCatalogHelper.scala +++ b/spark/src/test/spark-3.x/org/apache/comet/iceberg/RESTCatalogHelper.scala @@ -26,7 +26,22 @@ import java.nio.file.Files trait RESTCatalogHelper { /** Helper to set up REST catalog with embedded Jetty server (Spark 3.x / Jetty 9.4) */ - def withRESTCatalog(f: (String, org.eclipse.jetty.server.Server, File) => Unit): Unit = { + def withRESTCatalog(f: (String, org.eclipse.jetty.server.Server, File) => Unit): Unit = + withRESTCatalog()(f) + + /** + * Helper to set up REST catalog with optional credential vending. + * + * @param vendedCredentials + * Storage credentials to inject into loadTable responses, simulating REST catalog credential + * vending. When non-empty, these are added to every LoadTableResponse.config(). + * @param warehouseLocation + * Override the warehouse location (e.g., for S3). Defaults to a local temp directory. + */ + def withRESTCatalog( + vendedCredentials: Map[String, String] = Map.empty, + warehouseLocation: Option[String] = None)( + f: (String, org.eclipse.jetty.server.Server, File) => Unit): Unit = { import org.apache.iceberg.inmemory.InMemoryCatalog import org.apache.iceberg.CatalogProperties import org.apache.iceberg.rest.{RESTCatalogAdapter, RESTCatalogServlet} @@ -35,12 +50,18 @@ trait RESTCatalogHelper { import org.eclipse.jetty.server.handler.gzip.GzipHandler val warehouseDir = Files.createTempDirectory("comet-rest-catalog-test").toFile + val effectiveWarehouse = warehouseLocation.getOrElse(warehouseDir.getAbsolutePath) + val backendCatalog = new InMemoryCatalog() backendCatalog.initialize( "in-memory", - java.util.Map.of(CatalogProperties.WAREHOUSE_LOCATION, warehouseDir.getAbsolutePath)) + java.util.Map.of(CatalogProperties.WAREHOUSE_LOCATION, effectiveWarehouse)) val adapter = new RESTCatalogAdapter(backendCatalog) + if (vendedCredentials.nonEmpty) { + import scala.jdk.CollectionConverters._ + adapter.setVendedCredentials(vendedCredentials.asJava) + } val servlet = new RESTCatalogServlet(adapter) val servletContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) diff --git a/spark/src/test/spark-4.0/org/apache/comet/iceberg/RESTCatalogHelper.scala b/spark/src/test/spark-4.0/org/apache/comet/iceberg/RESTCatalogHelper.scala index ccd03c544d..bd53804b8d 100644 --- a/spark/src/test/spark-4.0/org/apache/comet/iceberg/RESTCatalogHelper.scala +++ b/spark/src/test/spark-4.0/org/apache/comet/iceberg/RESTCatalogHelper.scala @@ -26,7 +26,22 @@ import java.nio.file.Files trait RESTCatalogHelper { /** Helper to set up REST catalog with embedded Jetty server (Spark 4.0 / Jetty 11) */ - def withRESTCatalog(f: (String, org.eclipse.jetty.server.Server, File) => Unit): Unit = { + def withRESTCatalog(f: (String, org.eclipse.jetty.server.Server, File) => Unit): Unit = + withRESTCatalog()(f) + + /** + * Helper to set up REST catalog with optional credential vending. + * + * @param vendedCredentials + * Storage credentials to inject into loadTable responses, simulating REST catalog credential + * vending. When non-empty, these are added to every LoadTableResponse.config(). + * @param warehouseLocation + * Override the warehouse location (e.g., for S3). Defaults to a local temp directory. + */ + def withRESTCatalog( + vendedCredentials: Map[String, String] = Map.empty, + warehouseLocation: Option[String] = None)( + f: (String, org.eclipse.jetty.server.Server, File) => Unit): Unit = { import org.apache.iceberg.inmemory.InMemoryCatalog import org.apache.iceberg.CatalogProperties import org.apache.iceberg.rest.{RESTCatalogAdapter, RESTCatalogServlet} @@ -35,12 +50,18 @@ trait RESTCatalogHelper { import org.eclipse.jetty.server.handler.gzip.GzipHandler val warehouseDir = Files.createTempDirectory("comet-rest-catalog-test").toFile + val effectiveWarehouse = warehouseLocation.getOrElse(warehouseDir.getAbsolutePath) + val backendCatalog = new InMemoryCatalog() backendCatalog.initialize( "in-memory", - java.util.Map.of(CatalogProperties.WAREHOUSE_LOCATION, warehouseDir.getAbsolutePath)) + java.util.Map.of(CatalogProperties.WAREHOUSE_LOCATION, effectiveWarehouse)) val adapter = new RESTCatalogAdapter(backendCatalog) + if (vendedCredentials.nonEmpty) { + import scala.jdk.CollectionConverters._ + adapter.setVendedCredentials(vendedCredentials.asJava) + } val servlet = new RESTCatalogServlet(adapter) val servletContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) From 1baacaab721c9d13802d154584fab777d8fba2b0 Mon Sep 17 00:00:00 2001 From: shekharrajak Date: Wed, 25 Feb 2026 11:33:57 +0530 Subject: [PATCH 15/15] fix(build): add enforcer ignores for iceberg-spark-runtime shaded classes --- pom.xml | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index cde42655a3..a4ded40644 100644 --- a/pom.xml +++ b/pom.xml @@ -1167,13 +1167,26 @@ under the License. com.google.thirdparty.publicsuffix.PublicSuffixType - + org.apache.iceberg * org.jetbrains.annotations.* org.intellij.lang.annotations.* + + org.apache.spark.sql.execution.datasources.v2.ReplaceDataExec + org.apache.spark.sql.execution.datasources.v2.ReplaceDataExec$ + + org.apache.yetus.audience.* + + + + + org.apache.yetus + audience-annotations + + org.apache.yetus.audience.*