khjoshi94 opened a new issue, #15427:
URL: https://github.com/apache/iceberg/issues/15427

   ### Apache Iceberg version
   
   1.10.1 (latest release)
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   **Summary**
   When an Iceberg table is partitioned by a string column (e.g., batch_date), 
and the stored partition value contains trailing or leading whitespace (e.g., 
"20240201 "), Spark filters such as:
   ```scala
   df.filter(col("batch_date") === "20240201")
   ```
   return zero rows, even though the partition contains valid data.
   
   This leads to inconsistent behavior across engines (Spark vs Athena) and can 
silently propagate into empty joins, empty transformations, and empty writes.
   
   ----
   **Impact**
   
   - This issue can cause:
   - Spark reads to return empty DataFrames
   - Spark joins to return no matches
   - downstream transformations to operate on empty datasets
   - Iceberg writes to produce 0 output rows with no warnings
   - Athena to require TRIM(batch_date) to return rows
   - Extreme difficult debugging because partition columns are not stored in 
data files
   
   This is a data correctness and usability issue.
   
   ----
   **Expected Behavior**
   Iceberg should:
   
   - normalize or validate string partition values,
   - or warn when whitespace is detected,
   - or provide a table property to enforce trimming,
   - or at minimum surface a warning when partition pruning eliminates all 
files due to string mismatch.
   
   ----
   **Actual Behavior**
   
   - Iceberg stores the partition value exactly as written (e.g., "20240201 ").
   - Spark performs exact string matching and prunes all files.
   - Spark returns an empty DataFrame with no indication of the underlying 
issue.
   - Athena returns rows only when using TRIM(batch_date).
   - Writes based on these reads produce empty output.
   
   ----
   Minimal Reproducible Spark code
   ```scala
   import org.apache.spark.sql.functions._
   import org.apache.spark.sql.SparkSession
   
   val spark = SparkSession.builder()
     .appName("iceberg-whitespace-repro")
     .config("spark.sql.catalog.demo", "org.apache.iceberg.spark.SparkCatalog")
     .config("spark.sql.catalog.demo.type", "hadoop")
     .config("spark.sql.catalog.demo.warehouse", "/tmp/warehouse")
     .getOrCreate()
   
   // 1. Create table partitioned by string column
   spark.sql("""
     CREATE TABLE demo.db.example (
       id BIGINT,
       value STRING
     )
     USING iceberg
     PARTITIONED BY (batch_date)
   """)
   
   // 2. Write data with trailing whitespace in the partition value
   import spark.implicits._
   Seq((1L, "a", "20240201 ")).toDF("id", "value", "batch_date")
     .writeTo("demo.db.example")
     .append()
   
   // 3. Read and filter in Spark
   val df = spark.table("demo.db.example")
   println("Spark count with exact match: " +
     df.filter(col("batch_date") === "20240201").count())
   
   println("Spark count with TRIM: " +
     df.filter(trim(col("batch_date")) === "20240201").count())
   ```
   ----
   **Test Output**
    - Spark output
   ```scala
   Spark count with exact match: 0
   Spark count with TRIM: 1
   ```
    - Athena output
   ```sql
   SELECT * FROM example WHERE batch_date = '20240201';
   -- returns 0 rows
   
   SELECT * FROM example WHERE TRIM(batch_date) = '20240201';
   -- returns expected rows
   ```
   
    - SHOW PARTITIONS (Athena)
   ```sql
   batch_date=20240201%20
   (%20 indicates a trailing space)
   ```
   
   **Environment**
   - Iceberg version: 1.10.1
   - Spark version: 3.5.5
   - Catalog: AWS Glue 
   - Table format: Iceberg v2 
   
   **Proposals for fixes**
   
    - validating or normalizing string partition values,
    - or providing a table property to enforce trimming,
    - or surfacing a warning when whitespace is detected in partition values,
    - or surfacing a warning when partition pruning eliminates all files due to 
string mismatch.
   
   This would significantly improve correctness and help developers in 
troubleshooting with string‑based partition columns.
   
   
   ### Willingness to contribute
   
   - [ ] I can contribute a fix for this bug independently
   - [x] I would be willing to contribute a fix for this bug with guidance from 
the Iceberg community
   - [ ] I cannot contribute a fix for this bug at this time


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to