Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@
#include <Storages/ObjectStorage/DataLakes/Iceberg/Constant.h>
#include <Storages/ObjectStorage/DataLakes/Iceberg/Utils.h>
#include <Storages/MergeTree/MergeTreeDataWriter.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
Expand Down Expand Up @@ -116,7 +115,7 @@ ChunkPartitioner::partitionChunk(const Chunk & chunk)
arguments.push_back(ColumnWithTypeAndName(const_column->clone(), type, "PartitioningTimezone"));
}
auto result
= functions[transform_ind]->build(arguments)->execute(arguments, std::make_shared<DataTypeString>(), chunk.getNumRows(), false);
= functions[transform_ind]->build(arguments)->execute(arguments, result_data_types[transform_ind], chunk.getNumRows(), false);
functions_columns.push_back(result);
raw_columns.push_back(result.get());
for (size_t i = 0; i < chunk.getNumRows(); ++i)
Expand Down
3 changes: 3 additions & 0 deletions src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -551,7 +551,10 @@ Poco::Dynamic::Var getAvroType(DataTypePtr type)
{
switch (type->getTypeId())
{
case TypeIndex::UInt8:
case TypeIndex::Int8:
case TypeIndex::UInt16:
case TypeIndex::Int16:
case TypeIndex::UInt32:
case TypeIndex::Int32:
case TypeIndex::Date:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,4 +36,30 @@ def test_writes_create_partitioned_table(started_cluster_iceberg_with_spark, for
f.write(b"2")

df = spark.read.format("iceberg").load(f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}").collect()
assert len(df) == 1
assert len(df) == 1


@pytest.mark.parametrize("storage_type", ["s3", "local"])
@pytest.mark.parametrize("partition_type", ["toYearNumSinceEpoch(d)", "toMonthNumSinceEpoch(d)", "toRelativeDayNum(d)"])
def test_writes_date_column_with_time_transforms(started_cluster_iceberg_with_spark, storage_type, partition_type):
"""Test that Date columns work with year/month/day partition transforms (issue #86337)."""
instance = started_cluster_iceberg_with_spark.instances["node1"]
spark = started_cluster_iceberg_with_spark.spark_session
TABLE_NAME = "test_writes_date_time_transforms_" + storage_type + "_" + get_uuid_str()

create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster_iceberg_with_spark, "(id Int64, d Date)", 2, partition_type)

assert instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY ALL") == ''

instance.query(f"INSERT INTO {TABLE_NAME} VALUES (1, '2025-08-28');", settings={"allow_insert_into_iceberg": 1})
assert instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY ALL") == '1\t2025-08-28\n'

default_download_directory(
started_cluster_iceberg_with_spark,
storage_type,
f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}/",
f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}/",
)

df = spark.read.format("iceberg").load(f"/var/lib/clickhouse/user_files/iceberg_data/default/{TABLE_NAME}").collect()
assert len(df) == 1
Original file line number Diff line number Diff line change
Expand Up @@ -117,3 +117,41 @@ def test_writes_with_partitioned_table_count_partitions(started_cluster_iceberg_
num_pq_files += 1

assert num_pq_files == 16


@pytest.mark.parametrize("storage_type", ["s3", "local"])
@pytest.mark.parametrize("partition_transform", ["year", "month", "day"])
def test_writes_spark_date_partition_by_time_transform(started_cluster_iceberg_with_spark, storage_type, partition_transform):
"""Regression test for issue #86337: inserting into Spark-created Iceberg table
partitioned by year/month/day on a Date column caused a bad cast exception."""
instance = started_cluster_iceberg_with_spark.instances["node1"]
spark = started_cluster_iceberg_with_spark.spark_session
TABLE_NAME = "test_date_partition_" + partition_transform + "_" + storage_type + "_" + get_uuid_str()

def execute_spark_query(query: str):
spark.sql(query)
default_upload_directory(
started_cluster_iceberg_with_spark,
storage_type,
f"/iceberg_data/default/{TABLE_NAME}/",
f"/iceberg_data/default/{TABLE_NAME}/",
)

execute_spark_query(
f"""
CREATE TABLE {TABLE_NAME} (
c0 DATE
)
USING iceberg
PARTITIONED BY ({partition_transform}(c0))
OPTIONS('format-version'='2')
"""
)
create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster_iceberg_with_spark)

instance.query(
f"INSERT INTO {TABLE_NAME} (c0) VALUES ('2025-08-28');",
settings={"allow_insert_into_iceberg": 1}
)

assert instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY ALL") == '2025-08-28\n'
Loading