diff --git a/be/src/vec/sink/writer/iceberg/viceberg_table_writer.cpp b/be/src/vec/sink/writer/iceberg/viceberg_table_writer.cpp index 55c8451d8320a7..451313053ea0c2 100644 --- a/be/src/vec/sink/writer/iceberg/viceberg_table_writer.cpp +++ b/be/src/vec/sink/writer/iceberg/viceberg_table_writer.cpp @@ -48,6 +48,14 @@ Status VIcebergTableWriter::init_properties(ObjectPool* pool) { Status VIcebergTableWriter::open(RuntimeState* state, RuntimeProfile* profile) { _state = state; + // Get target file size from query options + // If value is 0 or not set, use config::iceberg_sink_max_file_size + _target_file_size_bytes = config::iceberg_sink_max_file_size; + if (state->query_options().__isset.iceberg_write_target_file_size_bytes && + state->query_options().iceberg_write_target_file_size_bytes > 0) { + _target_file_size_bytes = state->query_options().iceberg_write_target_file_size_bytes; + } + // add all counter _written_rows_counter = ADD_COUNTER(_operator_profile, "WrittenRows", TUnit::UNIT); _send_data_timer = ADD_TIMER(_operator_profile, "SendDataTime"); @@ -135,7 +143,7 @@ Status VIcebergTableWriter::write(RuntimeState* state, vectorized::Block& block) _partitions_to_writers.insert({"", writer}); RETURN_IF_ERROR(writer->open(_state, _operator_profile)); } else { - if (writer_iter->second->written_len() > config::iceberg_sink_max_file_size) { + if (writer_iter->second->written_len() > _target_file_size_bytes) { std::string file_name(writer_iter->second->file_name()); int file_name_index = writer_iter->second->file_name_index(); { @@ -214,7 +222,7 @@ Status VIcebergTableWriter::write(RuntimeState* state, vectorized::Block& block) RETURN_IF_ERROR(create_and_open_writer(partition_name, i, nullptr, 0, writer)); } else { std::shared_ptr writer; - if (writer_iter->second->written_len() > config::iceberg_sink_max_file_size) { + if (writer_iter->second->written_len() > _target_file_size_bytes) { std::string file_name(writer_iter->second->file_name()); int file_name_index = writer_iter->second->file_name_index(); { diff --git a/be/src/vec/sink/writer/iceberg/viceberg_table_writer.h b/be/src/vec/sink/writer/iceberg/viceberg_table_writer.h index 1749b9bf044025..4ea3e2135bb9eb 100644 --- a/be/src/vec/sink/writer/iceberg/viceberg_table_writer.h +++ b/be/src/vec/sink/writer/iceberg/viceberg_table_writer.h @@ -116,6 +116,9 @@ class VIcebergTableWriter final : public AsyncResultWriter { TDataSink _t_sink; RuntimeState* _state = nullptr; + // Target file size in bytes for controlling when to split files + int64_t _target_file_size_bytes = 0; + std::shared_ptr _schema; std::unique_ptr _partition_spec; diff --git a/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run21.sql b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run21.sql new file mode 100644 index 00000000000000..e7873d468adae0 --- /dev/null +++ b/docker/thirdparties/docker-compose/iceberg/scripts/create_preinstalled_scripts/iceberg/run21.sql @@ -0,0 +1,152 @@ +create database if not exists demo.test_db; +use demo.test_db; + +-- ===================================================================================== +-- Tables for testing rewrite_data_files with WHERE conditions +-- These tables are created with Spark SQL to ensure min/max metadata is properly +-- generated, which is required for WHERE condition filtering to work correctly. +-- ===================================================================================== + +-- Table 1: For baseline test (rewrite without WHERE condition) +CREATE TABLE IF NOT EXISTS test_rewrite_where_conditions_baseline ( + id BIGINT, + name STRING, + age INT, + salary DOUBLE +) USING iceberg; + +-- Insert data in multiple batches to create multiple files +-- First batch: id 1-10 +INSERT INTO test_rewrite_where_conditions_baseline VALUES +(1, 'Alice', 25, 50000.0), +(2, 'Bob', 30, 60000.0), +(3, 'Charlie', 35, 70000.0), +(4, 'David', 28, 55000.0), +(5, 'Eve', 32, 65000.0), +(6, 'Frank', 27, 52000.0), +(7, 'Grace', 29, 58000.0), +(8, 'Henry', 33, 72000.0), +(9, 'Ivy', 26, 48000.0), +(10, 'Jack', 31, 68000.0); + +-- Second batch: id 11-20 +INSERT INTO test_rewrite_where_conditions_baseline VALUES +(11, 'Kate', 34, 75000.0), +(12, 'Liam', 36, 80000.0), +(13, 'Mia', 38, 82000.0), +(14, 'Noah', 31, 71000.0), +(15, 'Olivia', 35, 76000.0), +(16, 'Peter', 37, 79000.0), +(17, 'Quinn', 32, 73000.0), +(18, 'Rachel', 39, 84000.0), +(19, 'Sam', 33, 74000.0), +(20, 'Tina', 36, 81000.0); + +-- Third batch: id 21-30 +INSERT INTO test_rewrite_where_conditions_baseline VALUES +(21, 'Uma', 41, 90000.0), +(22, 'Victor', 43, 92000.0), +(23, 'Wendy', 45, 95000.0), +(24, 'Xavier', 42, 91000.0), +(25, 'Yara', 44, 93000.0), +(26, 'Zoe', 46, 96000.0), +(27, 'Alex', 41, 89000.0), +(28, 'Blake', 48, 98000.0), +(29, 'Cameron', 47, 97000.0), +(30, 'Dana', 49, 99000.0); + +-- Table 2: For test with WHERE condition matching subset (id >= 11 AND id <= 20) +CREATE TABLE IF NOT EXISTS test_rewrite_where_conditions_with_where ( + id BIGINT, + name STRING, + age INT, + salary DOUBLE +) USING iceberg; + +-- Insert data in multiple batches to create multiple files +-- First batch: id 1-10 +INSERT INTO test_rewrite_where_conditions_with_where VALUES +(1, 'Alice', 25, 50000.0), +(2, 'Bob', 30, 60000.0), +(3, 'Charlie', 35, 70000.0), +(4, 'David', 28, 55000.0), +(5, 'Eve', 32, 65000.0), +(6, 'Frank', 27, 52000.0), +(7, 'Grace', 29, 58000.0), +(8, 'Henry', 33, 72000.0), +(9, 'Ivy', 26, 48000.0), +(10, 'Jack', 31, 68000.0); + +-- Second batch: id 11-20 +INSERT INTO test_rewrite_where_conditions_with_where VALUES +(11, 'Kate', 34, 75000.0), +(12, 'Liam', 36, 80000.0), +(13, 'Mia', 38, 82000.0), +(14, 'Noah', 31, 71000.0), +(15, 'Olivia', 35, 76000.0), +(16, 'Peter', 37, 79000.0), +(17, 'Quinn', 32, 73000.0), +(18, 'Rachel', 39, 84000.0), +(19, 'Sam', 33, 74000.0), +(20, 'Tina', 36, 81000.0); + +-- Third batch: id 21-30 +INSERT INTO test_rewrite_where_conditions_with_where VALUES +(21, 'Uma', 41, 90000.0), +(22, 'Victor', 43, 92000.0), +(23, 'Wendy', 45, 95000.0), +(24, 'Xavier', 42, 91000.0), +(25, 'Yara', 44, 93000.0), +(26, 'Zoe', 46, 96000.0), +(27, 'Alex', 41, 89000.0), +(28, 'Blake', 48, 98000.0), +(29, 'Cameron', 47, 97000.0), +(30, 'Dana', 49, 99000.0); + +-- Table 3: For test with WHERE condition matching no data (id = 99999) +CREATE TABLE IF NOT EXISTS test_rewrite_where_conditions_no_match ( + id BIGINT, + name STRING, + age INT, + salary DOUBLE +) USING iceberg; + +-- Insert data in multiple batches to create multiple files +-- First batch: id 1-10 +INSERT INTO test_rewrite_where_conditions_no_match VALUES +(1, 'Alice', 25, 50000.0), +(2, 'Bob', 30, 60000.0), +(3, 'Charlie', 35, 70000.0), +(4, 'David', 28, 55000.0), +(5, 'Eve', 32, 65000.0), +(6, 'Frank', 27, 52000.0), +(7, 'Grace', 29, 58000.0), +(8, 'Henry', 33, 72000.0), +(9, 'Ivy', 26, 48000.0), +(10, 'Jack', 31, 68000.0); + +-- Second batch: id 11-20 +INSERT INTO test_rewrite_where_conditions_no_match VALUES +(11, 'Kate', 34, 75000.0), +(12, 'Liam', 36, 80000.0), +(13, 'Mia', 38, 82000.0), +(14, 'Noah', 31, 71000.0), +(15, 'Olivia', 35, 76000.0), +(16, 'Peter', 37, 79000.0), +(17, 'Quinn', 32, 73000.0), +(18, 'Rachel', 39, 84000.0), +(19, 'Sam', 33, 74000.0), +(20, 'Tina', 36, 81000.0); + +-- Third batch: id 21-30 +INSERT INTO test_rewrite_where_conditions_no_match VALUES +(21, 'Uma', 41, 90000.0), +(22, 'Victor', 43, 92000.0), +(23, 'Wendy', 45, 95000.0), +(24, 'Xavier', 42, 91000.0), +(25, 'Yara', 44, 93000.0), +(26, 'Zoe', 46, 96000.0), +(27, 'Alex', 41, 89000.0), +(28, 'Blake', 48, 98000.0), +(29, 'Cameron', 47, 97000.0), +(30, 'Dana', 49, 99000.0); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/NamedArguments.java b/fe/fe-core/src/main/java/org/apache/doris/common/NamedArguments.java index b30e5bb34abd01..2c58fd32aeebdb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/NamedArguments.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/NamedArguments.java @@ -224,6 +224,7 @@ public Boolean getBoolean(String name) { * @return The typed value, or null if not set * @throws ClassCastException If the stored value cannot be cast to T */ + @SuppressWarnings("unchecked") public T getValue(String name) { return (T) parsedValues.get(name); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergNereidsUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergNereidsUtils.java new file mode 100644 index 00000000000000..693f7390e90fa3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergNereidsUtils.java @@ -0,0 +1,478 @@ +// 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.doris.datasource.iceberg; + +import org.apache.doris.common.UserException; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.Between; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.GreaterThan; +import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; +import org.apache.doris.nereids.trees.expressions.InPredicate; +import org.apache.doris.nereids.trees.expressions.IsNull; +import org.apache.doris.nereids.trees.expressions.LessThan; +import org.apache.doris.nereids.trees.expressions.LessThanEqual; +import org.apache.doris.nereids.trees.expressions.Not; +import org.apache.doris.nereids.trees.expressions.Or; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DateLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DecimalLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DecimalV3Literal; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.util.DateUtils; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.TimestampType; + +import java.math.BigDecimal; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.List; +import java.util.function.BiFunction; + +/** + * Utility class for converting Nereids expressions to Iceberg expressions. + */ +public class IcebergNereidsUtils { + + /** + * Convert Nereids Expression to Iceberg Expression + */ + public static org.apache.iceberg.expressions.Expression convertNereidsToIcebergExpression( + Expression nereidsExpr, Schema schema) throws UserException { + if (nereidsExpr == null) { + throw new UserException("Nereids expression is null"); + } + + // Handle logical operators + if (nereidsExpr instanceof And) { + And andExpr = (And) nereidsExpr; + org.apache.iceberg.expressions.Expression left = convertNereidsToIcebergExpression(andExpr.child(0), + schema); + org.apache.iceberg.expressions.Expression right = convertNereidsToIcebergExpression(andExpr.child(1), + schema); + if (left != null && right != null) { + return Expressions.and(left, right); + } + throw new UserException("Failed to convert AND expression: one or both children are unsupported"); + } + + if (nereidsExpr instanceof Or) { + Or orExpr = (Or) nereidsExpr; + org.apache.iceberg.expressions.Expression left = convertNereidsToIcebergExpression(orExpr.child(0), + schema); + org.apache.iceberg.expressions.Expression right = convertNereidsToIcebergExpression(orExpr.child(1), + schema); + if (left != null && right != null) { + return Expressions.or(left, right); + } + throw new UserException("Failed to convert OR expression: one or both children are unsupported"); + } + + if (nereidsExpr instanceof Not) { + Not notExpr = (Not) nereidsExpr; + org.apache.iceberg.expressions.Expression child = convertNereidsToIcebergExpression(notExpr.child(), + schema); + if (child != null) { + return Expressions.not(child); + } + throw new UserException("Failed to convert NOT expression: child is unsupported"); + } + + // Handle comparison operators + if (nereidsExpr instanceof EqualTo) { + return convertNereidsBinaryPredicate((EqualTo) nereidsExpr, + schema, Expressions::equal); + } + + if (nereidsExpr instanceof GreaterThan) { + return convertNereidsBinaryPredicate( + (GreaterThan) nereidsExpr, schema, + Expressions::greaterThan); + } + + if (nereidsExpr instanceof GreaterThanEqual) { + return convertNereidsBinaryPredicate( + (GreaterThanEqual) nereidsExpr, schema, + Expressions::greaterThanOrEqual); + } + + if (nereidsExpr instanceof LessThan) { + return convertNereidsBinaryPredicate((LessThan) nereidsExpr, + schema, Expressions::lessThan); + } + + if (nereidsExpr instanceof LessThanEqual) { + return convertNereidsBinaryPredicate( + (LessThanEqual) nereidsExpr, schema, + Expressions::lessThanOrEqual); + } + + // Handle IN predicates + if (nereidsExpr instanceof InPredicate) { + return convertNereidsInPredicate((InPredicate) nereidsExpr, + schema); + } + + // Handle IS NULL + if (nereidsExpr instanceof IsNull) { + Expression child = ((IsNull) nereidsExpr).child(); + if (child instanceof Slot) { + String colName = extractColumnName((Slot) child); + NestedField nestedField = schema.caseInsensitiveFindField(colName); + if (nestedField == null) { + throw new UserException("Column not found in Iceberg schema: " + colName); + } + return Expressions.isNull(nestedField.name()); + } + throw new UserException("IS NULL requires a column reference"); + } + + // Handle BETWEEN predicates + if (nereidsExpr instanceof Between) { + return convertNereidsBetween((Between) nereidsExpr, + schema); + } + + throw new UserException("Unsupported expression type: " + nereidsExpr.getClass().getName()); + } + + /** + * Convert Nereids binary predicate (comparison operators) + */ + private static org.apache.iceberg.expressions.Expression convertNereidsBinaryPredicate( + Expression nereidsExpr, Schema schema, + BiFunction converter) throws UserException { + + // Extract slot and literal from the binary predicate + Slot slot = null; + Literal literal = null; + + if (nereidsExpr.children().size() == 2) { + Expression left = nereidsExpr.child(0); + Expression right = nereidsExpr.child(1); + + if (left instanceof Slot && right instanceof Literal) { + slot = (Slot) left; + literal = (Literal) right; + } else if (left instanceof Literal && right instanceof Slot) { + slot = (Slot) right; + literal = (Literal) left; + } + } + + if (slot == null || literal == null) { + throw new UserException("Binary predicate must be between a column and a literal"); + } + + String colName = extractColumnName(slot); + NestedField nestedField = schema.caseInsensitiveFindField(colName); + if (nestedField == null) { + throw new UserException("Column not found in Iceberg schema: " + colName); + } + + colName = nestedField.name(); + Object value = extractNereidsLiteralValue(literal, nestedField.type()); + + if (value == null) { + if (literal instanceof NullLiteral) { + return Expressions.isNull(colName); + } + throw new UserException("Unsupported or null literal value for column: " + colName); + } + + return converter.apply(colName, value); + } + + /** + * Convert Nereids IN predicate + */ + private static org.apache.iceberg.expressions.Expression convertNereidsInPredicate( + InPredicate inPredicate, Schema schema) throws UserException { + if (inPredicate.children().size() < 2) { + throw new UserException("IN predicate requires at least one value"); + } + + org.apache.doris.nereids.trees.expressions.Expression left = inPredicate.child(0); + if (!(left instanceof Slot)) { + throw new UserException("Left side of IN predicate must be a slot"); + } + + Slot slot = (Slot) left; + String colName = extractColumnName(slot); + NestedField nestedField = schema.caseInsensitiveFindField(colName); + if (nestedField == null) { + throw new UserException("Column not found in Iceberg schema: " + colName); + } + + colName = nestedField.name(); + List values = new ArrayList<>(); + + for (int i = 1; i < inPredicate.children().size(); i++) { + Expression child = inPredicate.child(i); + if (!(child instanceof Literal)) { + throw new UserException("IN predicate values must be literals"); + } + + Object value = extractNereidsLiteralValue( + (Literal) child, nestedField.type()); + if (value == null) { + throw new UserException("Null or unsupported value in IN predicate for column: " + colName); + } + values.add(value); + } + + return Expressions.in(colName, values); + } + + /** + * Convert Nereids BETWEEN predicate + * BETWEEN a AND b is equivalent to: a <= col <= b + */ + private static org.apache.iceberg.expressions.Expression convertNereidsBetween( + Between between, Schema schema) throws UserException { + if (between.children().size() != 3) { + throw new UserException("BETWEEN predicate must have exactly 3 children"); + } + + Expression compareExpr = between.getCompareExpr(); + Expression lowerBound = between.getLowerBound(); + Expression upperBound = between.getUpperBound(); + + // Validate that compareExpr is a slot + if (!(compareExpr instanceof Slot)) { + throw new UserException("Left side of BETWEEN predicate must be a slot"); + } + + // Validate that lowerBound and upperBound are literals + if (!(lowerBound instanceof Literal)) { + throw new UserException("Lower bound of BETWEEN predicate must be a literal"); + } + if (!(upperBound instanceof Literal)) { + throw new UserException("Upper bound of BETWEEN predicate must be a literal"); + } + + Slot slot = (Slot) compareExpr; + String colName = extractColumnName(slot); + NestedField nestedField = schema.caseInsensitiveFindField(colName); + if (nestedField == null) { + throw new UserException("Column not found in Iceberg schema: " + colName); + } + + colName = nestedField.name(); + + // Extract values + Object lowerValue = extractNereidsLiteralValue((Literal) lowerBound, nestedField.type()); + Object upperValue = extractNereidsLiteralValue((Literal) upperBound, nestedField.type()); + + if (lowerValue == null || upperValue == null) { + throw new UserException("BETWEEN predicate bounds cannot be null for column: " + colName); + } + + // BETWEEN a AND b is equivalent to: a <= col AND col <= b + org.apache.iceberg.expressions.Expression lowerBoundExpr = Expressions.greaterThanOrEqual(colName, lowerValue); + org.apache.iceberg.expressions.Expression upperBoundExpr = Expressions.lessThanOrEqual(colName, upperValue); + + return Expressions.and(lowerBoundExpr, upperBoundExpr); + } + + /** + * Extract column name from Slot (SlotReference or UnboundSlot). + * For UnboundSlot, validates that nameParts is a singleton list (single column + * name). + * + * @param slot the slot to extract column name from + * @return the column name + * @throws UserException if UnboundSlot has multiple nameParts or if slot type + * is unsupported + */ + private static String extractColumnName(Slot slot) throws UserException { + if (slot instanceof SlotReference) { + return ((SlotReference) slot).getName(); + } else if (slot instanceof UnboundSlot) { + UnboundSlot unboundSlot = (UnboundSlot) slot; + // Validate that nameParts is a singleton list (simple column name) + if (unboundSlot.getNameParts().size() != 1) { + throw new UserException( + "UnboundSlot must have a single name part, but got: " + unboundSlot.getNameParts()); + } + return unboundSlot.getNameParts().get(0); + } else { + throw new UserException("Unsupported slot type: " + slot.getClass().getName()); + } + } + + /** + * Extract literal value from Nereids Literal expression + */ + private static Object extractNereidsLiteralValue( + Literal literal, + Type icebergType) throws UserException { + try { + Object raw = literal.getValue(); + if (raw == null) { + if (literal instanceof NullLiteral) { + return null; + } + throw new UserException("Literal value is null: " + literal); + } + + switch (icebergType.typeId()) { + case BOOLEAN: + if (literal instanceof BooleanLiteral) { + return ((BooleanLiteral) literal).getValue(); + } + // try to convert to boolean + return Boolean.valueOf(raw.toString()); + case STRING: + return literal.getStringValue(); + case INTEGER: + if (raw instanceof Number) { + return ((Number) raw).intValue(); + } + // try to convert to integer + return Integer.parseInt(literal.getStringValue()); + + case LONG: + case TIME: + if (raw instanceof Number) { + return ((Number) raw).longValue(); + } + // try to convert to long + return Long.parseLong(literal.getStringValue()); + case FLOAT: + if (raw instanceof Number) { + return ((Number) raw).floatValue(); + } + // try to convert to float + return Float.parseFloat(literal.getStringValue()); + case DOUBLE: + if (raw instanceof Number) { + return ((Number) raw).doubleValue(); + } + // try to convert to double + return Double.parseDouble(literal.getStringValue()); + case DECIMAL: + if (literal instanceof DecimalV3Literal) { + return ((DecimalV3Literal) literal) + .getValue(); + } + if (literal instanceof DecimalLiteral) { + return ((DecimalLiteral) literal).getValue(); + } + // try parse from string/number + return new BigDecimal(literal.getStringValue()); + case DATE: + if (literal instanceof DateLiteral) { + return ((DateLiteral) literal) + .getStringValue(); + } + // accept string value for date + return literal.getStringValue(); + case TIMESTAMP: + case TIMESTAMP_NANO: { + // Iceberg expects microseconds since epoch. Honor with/without zone semantics. + if (literal instanceof DateTimeLiteral + || literal instanceof DateLiteral) { + LocalDateTime ldt; + long microSecond = 0L; + if (literal instanceof DateTimeLiteral) { + DateTimeLiteral dt = (DateTimeLiteral) literal; + ldt = dt.toJavaDateType(); + microSecond = dt.getMicroSecond(); + } else { + DateLiteral d = (DateLiteral) literal; + ldt = d.toJavaDateType(); + microSecond = 0L; + } + TimestampType ts = (TimestampType) icebergType; + ZoneId zone = ts.shouldAdjustToUTC() + ? DateUtils.getTimeZone() + : ZoneId.of("UTC"); + long epochMicros = ldt.atZone(zone).toInstant().toEpochMilli() * 1000L + microSecond; + return epochMicros; + } + // String literal: try to parse using Doris's built-in datetime parser + // which supports multiple formats including 'yyyy-MM-dd HH:mm:ss' + if (raw instanceof String) { + String value = (String) raw; + // 1) If numeric, treat as epoch micros directly + try { + return Long.parseLong(value); + } catch (NumberFormatException ignored) { + // not a pure number, fall through to datetime parsing + } + + // 2) Try to parse using Doris's DateLiteral.parseDateTime() which supports + // various formats: 'yyyy-MM-dd', 'yyyy-MM-dd HH:mm:ss', ISO formats, etc. + try { + java.time.temporal.TemporalAccessor temporal = DateLiteral.parseDateTime(value).get(); + TimestampType ts = (TimestampType) icebergType; + ZoneId zone = ts.shouldAdjustToUTC() + ? DateUtils.getTimeZone() + : ZoneId.of("UTC"); + + // Build LocalDateTime from TemporalAccessor using DateUtils helper methods + LocalDateTime ldt = LocalDateTime.of( + DateUtils.getOrDefault(temporal, java.time.temporal.ChronoField.YEAR), + DateUtils.getOrDefault(temporal, java.time.temporal.ChronoField.MONTH_OF_YEAR), + DateUtils.getOrDefault(temporal, java.time.temporal.ChronoField.DAY_OF_MONTH), + DateUtils.getHourOrDefault(temporal), + DateUtils.getOrDefault(temporal, java.time.temporal.ChronoField.MINUTE_OF_HOUR), + DateUtils.getOrDefault(temporal, java.time.temporal.ChronoField.SECOND_OF_MINUTE), + DateUtils.getOrDefault(temporal, java.time.temporal.ChronoField.NANO_OF_SECOND)); + + long microSecond = DateUtils.getOrDefault(temporal, + java.time.temporal.ChronoField.NANO_OF_SECOND) / 1000L; + return ldt.atZone(zone).toInstant().toEpochMilli() * 1000L + microSecond; + } catch (Exception ignored) { + // If Doris parser fails, fall back to passing as string for Iceberg to try + } + + return literal.getStringValue(); + } + if (raw instanceof Number) { + return ((Number) raw).longValue(); + } + throw new UserException("Failed to convert timestamp literal to long: " + raw); + } + case UUID: + case FIXED: + case BINARY: + case GEOMETRY: + case GEOGRAPHY: + // Pass through as bytes/strings where possible + return raw; + default: + throw new UserException("Unsupported literal type: " + icebergType.typeId()); + } + } catch (Exception e) { + throw new UserException("Failed to extract literal value: " + e.getMessage()); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java index 0c4a7908501e28..b1bbcc920cf326 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java @@ -33,11 +33,13 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RewriteFiles; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.io.CloseableIterable; @@ -58,13 +60,17 @@ public class IcebergTransaction implements Transaction { private final IcebergMetadataOps ops; private Table table; - private org.apache.iceberg.Transaction transaction; private final List commitDataList = Lists.newArrayList(); private IcebergInsertCommandContext insertCtx; private String branchName; + // Rewrite operation support + private final List filesToDelete = Lists.newArrayList(); + private final List filesToAdd = Lists.newArrayList(); + private boolean isRewriteMode = false; + public IcebergTransaction(IcebergMetadataOps ops) { this.ops = ops; } @@ -75,6 +81,12 @@ public void updateIcebergCommitData(List commitDataList) { } } + public void updateRewriteFiles(List filesToDelete) { + synchronized (this) { + this.filesToDelete.addAll(filesToDelete); + } + } + public void beginInsert(ExternalTable dorisTable, Optional ctx) throws UserException { ctx.ifPresent(c -> this.insertCtx = (IcebergInsertCommandContext) c); try { @@ -89,8 +101,8 @@ public void beginInsert(ExternalTable dorisTable, Optional throw new RuntimeException(branchName + " is not founded in " + dorisTable.getName()); } else if (!branchRef.isBranch()) { throw new RuntimeException( - branchName - + " is a tag, not a branch. Tags cannot be targets for producing snapshots"); + branchName + + " is a tag, not a branch. Tags cannot be targets for producing snapshots"); } } this.transaction = table.newTransaction(); @@ -102,6 +114,110 @@ public void beginInsert(ExternalTable dorisTable, Optional } + /** + * Begin rewrite transaction for data file rewrite operations + */ + public void beginRewrite(ExternalTable dorisTable) throws UserException { + // For rewrite operations, we work directly on the main table + this.branchName = null; + this.isRewriteMode = true; + + try { + ops.getExecutionAuthenticator().execute(() -> { + // create and start the iceberg transaction + this.table = IcebergUtils.getIcebergTable(dorisTable); + + // For rewrite operations, we work directly on the main table + // No branch information needed + this.transaction = table.newTransaction(); + LOG.info("Started rewrite transaction for table: {} (main table)", + dorisTable.getName()); + return null; + }); + } catch (Exception e) { + throw new UserException("Failed to begin rewrite for iceberg table " + dorisTable.getName() + + " because: " + e.getMessage(), e); + } + } + + /** + * Finish rewrite operation by committing all file changes using RewriteFiles + * API + */ + public void finishRewrite() { + // TODO: refactor IcebergTransaction to make code cleaner + convertCommitDataListToDataFilesToAdd(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Finishing rewrite with {} files to delete and {} files to add", + filesToDelete.size(), filesToAdd.size()); + } + + try { + ops.getExecutionAuthenticator().execute(() -> { + updateManifestAfterRewrite(); + return null; + }); + } catch (Exception e) { + LOG.error("Failed to finish rewrite transaction", e); + throw new RuntimeException(e); + } + } + + private void convertCommitDataListToDataFilesToAdd() { + if (commitDataList.isEmpty()) { + LOG.debug("No commit data to convert for rewrite operation"); + return; + } + + // Get table specification information + PartitionSpec spec = transaction.table().spec(); + FileFormat fileFormat = IcebergUtils.getFileFormat(transaction.table()); + + // Convert commit data to DataFile objects using the same logic as insert + WriteResult writeResult = IcebergWriterHelper.convertToWriterResult(fileFormat, spec, commitDataList); + + // Add the generated DataFiles to filesToAdd list + synchronized (filesToAdd) { + for (DataFile dataFile : writeResult.dataFiles()) { + filesToAdd.add(dataFile); + } + } + + LOG.info("Converted {} commit data entries to {} DataFiles for rewrite operation", + commitDataList.size(), writeResult.dataFiles().length); + } + + private void updateManifestAfterRewrite() { + if (filesToDelete.isEmpty() && filesToAdd.isEmpty()) { + LOG.info("No files to rewrite, skipping commit"); + return; + } + + RewriteFiles rewriteFiles = transaction.newRewrite(); + + // For rewrite operations, we work directly on the main table + rewriteFiles = rewriteFiles.scanManifestsWith(ops.getThreadPoolWithPreAuth()); + + // Add files to delete + for (DataFile dataFile : filesToDelete) { + rewriteFiles.deleteFile(dataFile); + } + + // Add files to add + for (DataFile dataFile : filesToAdd) { + rewriteFiles.addFile(dataFile); + } + + // Commit the rewrite operation + rewriteFiles.commit(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Rewrite committed with {} files deleted and {} files added", + filesToDelete.size(), filesToAdd.size()); + } + } + public void finishInsert(NameMapping nameMapping) { if (LOG.isDebugEnabled()) { LOG.info("iceberg table {} insert table finished!", nameMapping.getFullLocalName()); @@ -154,13 +270,59 @@ public void commit() throws UserException { @Override public void rollback() { - //do nothing + if (isRewriteMode) { + // Clear the collected files for rewrite mode + synchronized (filesToDelete) { + filesToDelete.clear(); + } + synchronized (filesToAdd) { + filesToAdd.clear(); + } + LOG.info("Rewrite transaction rolled back"); + } + // For insert mode, do nothing as original implementation } public long getUpdateCnt() { return commitDataList.stream().mapToLong(TIcebergCommitData::getRowCount).sum(); } + /** + * Get the number of files that will be deleted in rewrite operation + */ + public int getFilesToDeleteCount() { + synchronized (filesToDelete) { + return filesToDelete.size(); + } + } + + /** + * Get the number of files that will be added in rewrite operation + */ + public int getFilesToAddCount() { + synchronized (filesToAdd) { + return filesToAdd.size(); + } + } + + /** + * Get the total size of files to be deleted in rewrite operation + */ + public long getFilesToDeleteSize() { + synchronized (filesToDelete) { + return filesToDelete.stream().mapToLong(DataFile::fileSizeInBytes).sum(); + } + } + + /** + * Get the total size of files to be added in rewrite operation + */ + public long getFilesToAddSize() { + synchronized (filesToAdd) { + return filesToAdd.stream().mapToLong(DataFile::fileSizeInBytes).sum(); + } + } + private void commitAppendTxn(List pendingResults) { // commit append files. AppendFiles appendFiles = transaction.newAppend().scanManifestsWith(ops.getThreadPoolWithPreAuth()); @@ -175,7 +337,6 @@ private void commitAppendTxn(List pendingResults) { appendFiles.commit(); } - private void commitReplaceTxn(List pendingResults) { if (pendingResults.isEmpty()) { // such as : insert overwrite table `dst_tb` select * from `empty_tb` diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/BaseIcebergAction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/BaseIcebergAction.java index ddf8a29b441b0e..b54d7509f0056b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/BaseIcebergAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/BaseIcebergAction.java @@ -34,14 +34,10 @@ */ public abstract class BaseIcebergAction extends BaseExecuteAction { - protected final IcebergExternalTable icebergTable; - protected BaseIcebergAction(String actionType, Map properties, Optional partitionNamesInfo, - Optional whereCondition, - IcebergExternalTable icebergTable) { + Optional whereCondition) { super(actionType, properties, partitionNamesInfo, whereCondition); - this.icebergTable = icebergTable; } @Override @@ -75,10 +71,4 @@ protected void validateIcebergAction() throws UserException { // Default implementation does nothing. } - /** - * Get the underlying Iceberg table instance for procedure execution. - */ - protected IcebergExternalTable getIcebergTable() { - return icebergTable; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergCherrypickSnapshotAction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergCherrypickSnapshotAction.java index 0947b2fb9c0bce..e303f0e9111486 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergCherrypickSnapshotAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergCherrypickSnapshotAction.java @@ -48,9 +48,8 @@ public class IcebergCherrypickSnapshotAction extends BaseIcebergAction { public static final String SNAPSHOT_ID = "snapshot_id"; public IcebergCherrypickSnapshotAction(Map properties, - Optional partitionNamesInfo, Optional whereCondition, - IcebergExternalTable icebergTable) { - super("cherrypick_snapshot", properties, partitionNamesInfo, whereCondition, icebergTable); + Optional partitionNamesInfo, Optional whereCondition) { + super("cherrypick_snapshot", properties, partitionNamesInfo, whereCondition); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergExecuteActionFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergExecuteActionFactory.java index d96934339509ef..94be847700c220 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergExecuteActionFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergExecuteActionFactory.java @@ -61,25 +61,25 @@ public static ExecuteAction createAction(String actionType, Map switch (actionType.toLowerCase()) { case ROLLBACK_TO_SNAPSHOT: return new IcebergRollbackToSnapshotAction(properties, partitionNamesInfo, - whereCondition, table); + whereCondition); case ROLLBACK_TO_TIMESTAMP: return new IcebergRollbackToTimestampAction(properties, partitionNamesInfo, - whereCondition, table); + whereCondition); case SET_CURRENT_SNAPSHOT: return new IcebergSetCurrentSnapshotAction(properties, partitionNamesInfo, - whereCondition, table); + whereCondition); case CHERRYPICK_SNAPSHOT: return new IcebergCherrypickSnapshotAction(properties, partitionNamesInfo, - whereCondition, table); + whereCondition); case FAST_FORWARD: return new IcebergFastForwardAction(properties, partitionNamesInfo, - whereCondition, table); + whereCondition); case EXPIRE_SNAPSHOTS: return new IcebergExpireSnapshotsAction(properties, partitionNamesInfo, - whereCondition, table); + whereCondition); case REWRITE_DATA_FILES: return new IcebergRewriteDataFilesAction(properties, partitionNamesInfo, - whereCondition, table); + whereCondition); default: throw new DdlException("Unsupported Iceberg procedure: " + actionType + ". Supported procedures: " + String.join(", ", getSupportedActions())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergExpireSnapshotsAction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergExpireSnapshotsAction.java index ba247921a08064..77ea784eadc33a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergExpireSnapshotsAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergExpireSnapshotsAction.java @@ -22,7 +22,6 @@ import org.apache.doris.common.ArgumentParsers; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; -import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.info.PartitionNamesInfo; import org.apache.doris.nereids.trees.expressions.Expression; @@ -49,9 +48,8 @@ public class IcebergExpireSnapshotsAction extends BaseIcebergAction { public IcebergExpireSnapshotsAction(Map properties, Optional partitionNamesInfo, - Optional whereCondition, - IcebergExternalTable icebergTable) { - super("expire_snapshots", properties, partitionNamesInfo, whereCondition, icebergTable); + Optional whereCondition) { + super("expire_snapshots", properties, partitionNamesInfo, whereCondition); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergFastForwardAction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergFastForwardAction.java index 6a49713fc89abc..a5560db65520e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergFastForwardAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergFastForwardAction.java @@ -46,9 +46,8 @@ public class IcebergFastForwardAction extends BaseIcebergAction { public IcebergFastForwardAction(Map properties, Optional partitionNamesInfo, - Optional whereCondition, - IcebergExternalTable icebergTable) { - super("fast_forward", properties, partitionNamesInfo, whereCondition, icebergTable); + Optional whereCondition) { + super("fast_forward", properties, partitionNamesInfo, whereCondition); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRewriteDataFilesAction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRewriteDataFilesAction.java index 8bf673b0e6452a..2d9ee4b137753a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRewriteDataFilesAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRewriteDataFilesAction.java @@ -23,21 +23,35 @@ import org.apache.doris.common.ArgumentParsers; import org.apache.doris.common.UserException; import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergUtils; +import org.apache.doris.datasource.iceberg.rewrite.RewriteDataFileExecutor; +import org.apache.doris.datasource.iceberg.rewrite.RewriteDataFilePlanner; +import org.apache.doris.datasource.iceberg.rewrite.RewriteDataGroup; +import org.apache.doris.datasource.iceberg.rewrite.RewriteResult; import org.apache.doris.info.PartitionNamesInfo; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; +import org.apache.iceberg.Table; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.List; import java.util.Map; import java.util.Optional; /** - * Iceberg rewrite data files action implementation. - * This action rewrites data files in Iceberg tables to optimize file sizes - * and improve query performance. + * Action for rewriting Iceberg data files to compact and optimize table data + * + * Execution Flow: + * 1. Validate rewrite parameters and get Iceberg table + * 2. Use RewriteDataFilePlanner to plan and organize file scan tasks into rewrite groups + * 3. Create and start rewrite job concurrently + * 4. Wait for job completion and return result */ public class IcebergRewriteDataFilesAction extends BaseIcebergAction { + private static final Logger LOG = LogManager.getLogger(IcebergRewriteDataFilesAction.class); // File size parameters public static final String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes"; public static final String MIN_FILE_SIZE_BYTES = "min-file-size-bytes"; @@ -55,11 +69,14 @@ public class IcebergRewriteDataFilesAction extends BaseIcebergAction { // Output specification parameter public static final String OUTPUT_SPEC_ID = "output-spec-id"; + // Parameters with special default handling + private long minFileSizeBytes; + private long maxFileSizeBytes; + public IcebergRewriteDataFilesAction(Map properties, Optional partitionNamesInfo, - Optional whereCondition, - IcebergExternalTable icebergTable) { - super("rewrite_data_files", properties, partitionNamesInfo, whereCondition, icebergTable); + Optional whereCondition) { + super("rewrite_data_files", properties, partitionNamesInfo, whereCondition); } /** @@ -132,14 +149,70 @@ protected List getResultSchema() { @Override protected void validateIcebergAction() throws UserException { - // TODO: Implement validation logic for rewrite_data_files parameters + // Validate min and max file size parameters + long targetFileSizeBytes = namedArguments.getLong(TARGET_FILE_SIZE_BYTES); + // min-file-size-bytes default to 75% of target file size + this.minFileSizeBytes = namedArguments.getLong(MIN_FILE_SIZE_BYTES); + if (this.minFileSizeBytes == 0) { + this.minFileSizeBytes = (long) (targetFileSizeBytes * 0.75); + } + // max-file-size-bytes default to 180% of target file size + this.maxFileSizeBytes = namedArguments.getLong(MAX_FILE_SIZE_BYTES); + if (this.maxFileSizeBytes == 0) { + this.maxFileSizeBytes = (long) (targetFileSizeBytes * 1.8); + } + validateNoPartitions(); } @Override protected List executeAction(TableIf table) throws UserException { - // TODO: Implement the logic to rewrite data files in the Iceberg table - // For now, just return dummy values - return Lists.newArrayList("0", "1", "2", "3"); + try { + Table icebergTable = IcebergUtils.getIcebergTable((IcebergExternalTable) table); + + if (icebergTable.currentSnapshot() == null) { + LOG.info("Table {} has no data, skipping rewrite", table.getName()); + // return empty result + return Lists.newArrayList("0", "0", "0", "0"); + } + + RewriteDataFilePlanner.Parameters parameters = buildRewriteParameters(); + + ConnectContext connectContext = ConnectContext.get(); + if (connectContext == null) { + throw new UserException("No active connection context found"); + } + + // Step 1: Plan and organize file scan tasks into groups + RewriteDataFilePlanner fileManager = new RewriteDataFilePlanner(parameters); + Iterable allGroups = fileManager.planAndOrganizeTasks(icebergTable); + + // Step 2: Execute rewrite groups concurrently + List groupsList = Lists.newArrayList(allGroups); + + // Create executor and execute groups concurrently + RewriteDataFileExecutor executor = new RewriteDataFileExecutor( + (IcebergExternalTable) table, connectContext); + long targetFileSizeBytes = namedArguments.getLong(TARGET_FILE_SIZE_BYTES); + RewriteResult totalResult = executor.executeGroupsConcurrently(groupsList, targetFileSizeBytes); + return totalResult.toStringList(); + } catch (Exception e) { + LOG.warn("Failed to rewrite data files for table: " + table.getName(), e); + throw new UserException("Rewrite data files failed: " + e.getMessage()); + } + } + + private RewriteDataFilePlanner.Parameters buildRewriteParameters() { + return new RewriteDataFilePlanner.Parameters( + namedArguments.getLong(TARGET_FILE_SIZE_BYTES), + this.minFileSizeBytes, + this.maxFileSizeBytes, + namedArguments.getInt(MIN_INPUT_FILES), + namedArguments.getBoolean(REWRITE_ALL), + namedArguments.getLong(MAX_FILE_GROUP_SIZE_BYTES), + namedArguments.getInt(DELETE_FILE_THRESHOLD), + namedArguments.getDouble(DELETE_RATIO_THRESHOLD), + namedArguments.getLong(OUTPUT_SPEC_ID), + whereCondition); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRollbackToSnapshotAction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRollbackToSnapshotAction.java index 38e64e6bcbbc7a..8d6b3842a9dc80 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRollbackToSnapshotAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRollbackToSnapshotAction.java @@ -46,9 +46,8 @@ public class IcebergRollbackToSnapshotAction extends BaseIcebergAction { public IcebergRollbackToSnapshotAction(Map properties, Optional partitionNamesInfo, - Optional whereCondition, - IcebergExternalTable icebergTable) { - super("rollback_to_snapshot", properties, partitionNamesInfo, whereCondition, icebergTable); + Optional whereCondition) { + super("rollback_to_snapshot", properties, partitionNamesInfo, whereCondition); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRollbackToTimestampAction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRollbackToTimestampAction.java index e739c1fcd0b3a1..0f50364920c7fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRollbackToTimestampAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergRollbackToTimestampAction.java @@ -48,9 +48,8 @@ public class IcebergRollbackToTimestampAction extends BaseIcebergAction { public IcebergRollbackToTimestampAction(Map properties, Optional partitionNamesInfo, - Optional whereCondition, - IcebergExternalTable icebergTable) { - super("rollback_to_timestamp", properties, partitionNamesInfo, whereCondition, icebergTable); + Optional whereCondition) { + super("rollback_to_timestamp", properties, partitionNamesInfo, whereCondition); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergSetCurrentSnapshotAction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergSetCurrentSnapshotAction.java index 18d81d3dfbde58..44df40f8f492b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergSetCurrentSnapshotAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/action/IcebergSetCurrentSnapshotAction.java @@ -48,9 +48,8 @@ public class IcebergSetCurrentSnapshotAction extends BaseIcebergAction { public IcebergSetCurrentSnapshotAction(Map properties, Optional partitionNamesInfo, - Optional whereCondition, - IcebergExternalTable icebergTable) { - super("set_current_snapshot", properties, partitionNamesInfo, whereCondition, icebergTable); + Optional whereCondition) { + super("set_current_snapshot", properties, partitionNamesInfo, whereCondition); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFileExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFileExecutor.java new file mode 100644 index 00000000000000..c900482710febc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFileExecutor.java @@ -0,0 +1,213 @@ +// 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.doris.datasource.iceberg.rewrite; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.datasource.iceberg.IcebergTransaction; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.scheduler.exception.JobException; +import org.apache.doris.scheduler.executor.TransientTaskExecutor; + +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Executes INSERT-SELECT statements for Iceberg data file rewriting. + */ +public class RewriteDataFileExecutor { + private static final Logger LOG = LogManager.getLogger(RewriteDataFileExecutor.class); + + private final IcebergExternalTable dorisTable; + private final ConnectContext connectContext; + + public RewriteDataFileExecutor(IcebergExternalTable dorisTable, + ConnectContext connectContext) { + this.dorisTable = dorisTable; + this.connectContext = connectContext; + } + + /** + * Execute rewrite for multiple groups concurrently + */ + public RewriteResult executeGroupsConcurrently(List groups, long targetFileSizeBytes) + throws UserException { + // Begin transaction + long transactionId = dorisTable.getCatalog().getTransactionManager().begin(); + IcebergTransaction transaction = (IcebergTransaction) dorisTable.getCatalog().getTransactionManager() + .getTransaction(transactionId); + transaction.beginRewrite(dorisTable); + + // Register files to delete + for (RewriteDataGroup group : groups) { + transaction.updateRewriteFiles(Lists.newArrayList(group.getDataFiles())); + } + + // Create result collector and tasks + List tasks = Lists.newArrayList(); + RewriteResultCollector resultCollector = new RewriteResultCollector(groups.size(), tasks); + + // Create tasks with callbacks + for (RewriteDataGroup group : groups) { + RewriteGroupTask task = new RewriteGroupTask( + group, + transactionId, + dorisTable, + connectContext, + targetFileSizeBytes, + new RewriteGroupTask.RewriteResultCallback() { + @Override + public void onTaskCompleted(Long taskId) { + resultCollector.onTaskCompleted(taskId); + } + + @Override + public void onTaskFailed(Long taskId, Exception error) { + resultCollector.onTaskFailed(taskId, error); + } + }); + tasks.add(task); + } + + // Submit tasks to TransientTaskManager + try { + for (TransientTaskExecutor task : tasks) { + Env.getCurrentEnv().getTransientTaskManager().addMemoryTask(task); + } + } catch (JobException e) { + throw new UserException("Failed to submit rewrite tasks: " + e.getMessage(), e); + } + + // Wait for all tasks to complete + waitForTasksCompletion(resultCollector, groups.size()); + + // Finish rewrite operation + transaction.finishRewrite(); + + // Collect statistics from transaction after all tasks are completed + int rewrittenDataFilesCount = groups.stream().mapToInt(group -> group.getDataFiles().size()).sum(); + // this should after finishRewrite + int addedDataFilesCount = transaction.getFilesToAddCount(); + long rewrittenBytesCount = groups.stream().mapToLong(group -> group.getTotalSize()).sum(); + int removedDeleteFilesCount = groups.stream().mapToInt(group -> group.getDeleteFileCount()).sum(); + + // Commit transaction + transaction.commit(); + + return new RewriteResult(rewrittenDataFilesCount, addedDataFilesCount, + rewrittenBytesCount, removedDeleteFilesCount); + } + + /** + * Wait for all tasks to complete using notification mechanism + */ + private void waitForTasksCompletion(RewriteResultCollector collector, int totalTasks) + throws UserException { + LOG.info("Waiting for {} rewrite tasks to complete using notification mechanism", totalTasks); + + int maxWaitTime = connectContext.getSessionVariable().getInsertTimeoutS(); + + try { + boolean completed = collector.await(maxWaitTime, TimeUnit.SECONDS); + + if (!completed) { + LOG.warn("Rewrite tasks did not complete within timeout of {} seconds", maxWaitTime); + throw new UserException("Rewrite tasks did not complete within timeout"); + } + + // Check if any task failed + if (collector.getFirstError() != null) { + LOG.warn("Rewrite tasks failed: {}", collector.getFirstError().getMessage()); + throw new UserException("Some rewrite tasks failed: " + collector.getFirstError().getMessage(), + collector.getFirstError()); + } + + LOG.info("All {} rewrite tasks completed successfully", totalTasks); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.warn("Wait for tasks completion was interrupted", e); + throw new UserException("Wait for tasks completion was interrupted", e); + } + } + + /** + * Result collector for concurrent rewrite tasks + */ + private static class RewriteResultCollector { + private final int expectedTasks; + private final AtomicInteger completedTasks = new AtomicInteger(0); + private final AtomicInteger failedTasks = new AtomicInteger(0); + private volatile Exception firstError = null; + private final CountDownLatch completionLatch; + private final List allTasks; + + public RewriteResultCollector(int expectedTasks, List tasks) { + this.expectedTasks = expectedTasks; + this.completionLatch = new CountDownLatch(expectedTasks); + this.allTasks = tasks; + } + + public synchronized void onTaskCompleted(Long taskId) { + int completed = completedTasks.incrementAndGet(); + LOG.info("Task {} completed ({}/{})", taskId, completed, expectedTasks); + completionLatch.countDown(); + } + + public synchronized void onTaskFailed(Long taskId, Exception error) { + int failed = failedTasks.incrementAndGet(); + if (firstError == null) { + firstError = error; + + // Cancel all other tasks immediately when first failure occurs + LOG.warn("Task {} failed, cancelling all other tasks", taskId); + cancelAllOtherTasks(taskId); + } + LOG.warn("Task {} failed ({}/{}): {}", taskId, failed, expectedTasks, error.getMessage()); + completionLatch.countDown(); + } + + private void cancelAllOtherTasks(Long failedTaskId) { + for (RewriteGroupTask task : allTasks) { + if (!task.getId().equals(failedTaskId)) { + try { + task.cancel(); + LOG.info("Cancelled task {}", task.getId()); + } catch (Exception e) { + LOG.warn("Failed to cancel task {}: {}", task.getId(), e.getMessage()); + } + } + } + } + + public boolean await(long timeout, TimeUnit unit) throws InterruptedException { + return completionLatch.await(timeout, unit); + } + + public Exception getFirstError() { + return firstError; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFilePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFilePlanner.java new file mode 100644 index 00000000000000..42a969f56251ff --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFilePlanner.java @@ -0,0 +1,362 @@ +// 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.doris.datasource.iceberg.rewrite; + +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.iceberg.IcebergNereidsUtils; +import org.apache.doris.nereids.trees.expressions.Expression; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.util.BinPacking; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * Planner for organizing and filtering file scan tasks into rewrite groups. + */ +public class RewriteDataFilePlanner { + private static final Logger LOG = LogManager.getLogger(RewriteDataFilePlanner.class); + + private final Parameters parameters; + + public RewriteDataFilePlanner(Parameters parameters) { + this.parameters = parameters; + } + + /** + * Plan and organize file scan tasks into rewrite groups + */ + public List planAndOrganizeTasks(Table icebergTable) throws UserException { + try { + // Step 1: Plan FileScanTask from Iceberg table + Iterable allTasks = planFileScanTasks(icebergTable); + + // Step 2: First layer - Group tasks by partition (without filtering files) + Map> filesByPartition = groupTasksByPartition(allTasks); + + // Step 3: Apply binPack grouping strategy within each partition and convert to + // RewriteDataGroup + Map> fileGroupsByPartition = Maps.transformValues( + filesByPartition, this::packGroupsInPartition); + + // Step 4: Flatten all groups from all partitions + return fileGroupsByPartition.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + } catch (Exception e) { + throw new UserException("Failed to plan file scan tasks: " + e.getMessage(), e); + } + } + + /** + * Plan FileScanTask from Iceberg table + */ + private Iterable planFileScanTasks(Table icebergTable) throws UserException { + // Create table scan with optional filters + TableScan tableScan = icebergTable.newScan(); + + // Use current snapshot if available + if (icebergTable.currentSnapshot() != null) { + tableScan = tableScan.useSnapshot(icebergTable.currentSnapshot().snapshotId()); + } + + // Apply WHERE condition if specified + if (parameters.hasWhereCondition()) { + org.apache.iceberg.expressions.Expression icebergExpression = IcebergNereidsUtils + .convertNereidsToIcebergExpression(parameters.getWhereCondition().get(), icebergTable.schema()); + tableScan = tableScan.filter(icebergExpression); + } + + // Ignore residuals to avoid reading data files unnecessarily + tableScan = tableScan.ignoreResiduals(); + + return tableScan.planFiles(); + } + + /** + * Filter files based on rewrite criteria + */ + private Iterable filterFiles(Iterable tasks) { + return Iterables.filter(tasks, this::shouldRewriteFile); + } + + /** + * Check if a file should be rewritten + */ + private boolean shouldRewriteFile(FileScanTask task) { + return outsideDesiredFileSizeRange(task) || tooManyDeletes(task) || tooHighDeleteRatio(task); + } + + /** + * Check if file is outside desired size range + */ + private boolean outsideDesiredFileSizeRange(FileScanTask task) { + long fileSize = task.file().fileSizeInBytes(); + return fileSize < parameters.getMinFileSizeBytes() || fileSize > parameters.getMaxFileSizeBytes(); + } + + /** + * Check if file has too many delete files + */ + private boolean tooManyDeletes(FileScanTask task) { + if (task.deletes() == null) { + return false; + } + return task.deletes().size() >= parameters.getDeleteFileThreshold(); + } + + /** + * Check if file has too high delete ratio + */ + private boolean tooHighDeleteRatio(FileScanTask task) { + if (task.deletes() == null || task.deletes().isEmpty()) { + return false; + } + + long recordCount = task.file().recordCount(); + if (recordCount == 0) { + return false; + } + + // Calculate known deleted record count (only file-scoped deletes) + long knownDeletedRecordCount = task.deletes().stream() + .filter(ContentFileUtil::isFileScoped) + .mapToLong(ContentFile::recordCount) + .sum(); + + // Calculate delete ratio + double deletedRecords = (double) Math.min(knownDeletedRecordCount, recordCount); + double deleteRatio = deletedRecords / recordCount; + + return deleteRatio >= parameters.getDeleteRatioThreshold(); + } + + /** + * Returns a map from partition to list of file scan tasks in that partition. + */ + private Map> groupTasksByPartition(Iterable allTasks) { + Map> filesByPartition = new HashMap<>(); + for (FileScanTask task : allTasks) { + PartitionSpec spec = task.spec(); + StructLikeWrapper partitionWrapper = StructLikeWrapper.forType(spec.partitionType()); + + // If a task uses an incompatible partition spec, treat it as un-partitioned + // by using an empty partition (all null values) + StructLikeWrapper partition; + if (task.file().specId() == spec.specId()) { + partition = partitionWrapper.copyFor(task.file().partition()); + } else { + // Use empty partition for incompatible spec + // Create an empty GenericRecord with all null values + org.apache.iceberg.StructLike emptyStruct = GenericRecord.create(spec.partitionType()); + partition = partitionWrapper.copyFor(emptyStruct); + } + + filesByPartition.computeIfAbsent(partition, k -> Lists.newArrayList()).add(task); + } + return filesByPartition; + } + + /** + * Pack files in a partition using bin-packing strategy. + *

+ * This method is used to group files in a partition using bin-packing strategy. + * It first filters files if not rewriteAll, then uses bin-packing to group + * files based on their size, and then converts the groups to RewriteDataGroup. + * Finally, it filters groups if not rewriteAll. + *

+ */ + private List packGroupsInPartition(List tasks) { + // Step 1: Filter files if not rewriteAll + Iterable filteredTasks = parameters.isRewriteAll() ? tasks : filterFiles(tasks); + + // Step 2: Use bin-packing to group files + BinPacking.ListPacker packer = new BinPacking.ListPacker<>( + parameters.getMaxFileGroupSizeBytes(), + 1, // lookback: number of bins to look back when packing + false // largestBinFirst: whether to prefer larger bins + ); + + // Pack files using file size as weight + List> groups = packer.pack(filteredTasks, task -> task.file().fileSizeInBytes()); + + // Step 3: Convert to RewriteDataGroup + List rewriteDataGroups = groups.stream() + .map(RewriteDataGroup::new) + .collect(Collectors.toList()); + + // Step 4: Filter groups if not rewriteAll + return parameters.isRewriteAll() ? rewriteDataGroups : filterFileGroups(rewriteDataGroups); + } + + /** + * Filter file groups based on rewrite parameters. + * Only groups that meet the rewrite criteria are kept. + */ + private List filterFileGroups(List groups) { + return groups.stream() + .filter(this::shouldRewriteFileGroup) + .collect(Collectors.toList()); + } + + /** + * Check if a file group should be rewritten based on parameters. + */ + private boolean shouldRewriteFileGroup(RewriteDataGroup group) { + return hasEnoughInputFiles(group) || hasEnoughContent(group) + || hasTooMuchContent(group) || hasDeleteIssues(group); + } + + /** + * Check if group has enough input files + */ + private boolean hasEnoughInputFiles(RewriteDataGroup group) { + return group.getTaskCount() > 1 && group.getTaskCount() >= parameters.getMinInputFiles(); + } + + /** + * Check if group has enough content + */ + private boolean hasEnoughContent(RewriteDataGroup group) { + return group.getTaskCount() > 1 && group.getTotalSize() > parameters.getTargetFileSizeBytes(); + } + + /** + * Check if group has too much content + */ + private boolean hasTooMuchContent(RewriteDataGroup group) { + return group.getTotalSize() > parameters.getMaxFileGroupSizeBytes(); + } + + /** + * Check if any file in the group has too many deletes or high delete ratio + */ + private boolean hasDeleteIssues(RewriteDataGroup group) { + return group.getTasks().stream() + .anyMatch(task -> tooManyDeletes(task) || tooHighDeleteRatio(task)); + } + + /** + * Parameters for Iceberg data file rewrite operation + */ + public static class Parameters { + private final long targetFileSizeBytes; + private final long minFileSizeBytes; + private final long maxFileSizeBytes; + private final int minInputFiles; + private final boolean rewriteAll; + private final long maxFileGroupSizeBytes; + private final int deleteFileThreshold; + private final double deleteRatioThreshold; + + private final Optional whereCondition; + + public Parameters( + long targetFileSizeBytes, + long minFileSizeBytes, + long maxFileSizeBytes, + int minInputFiles, + boolean rewriteAll, + long maxFileGroupSizeBytes, + int deleteFileThreshold, + double deleteRatioThreshold, + long outputSpecId, + Optional whereCondition) { + this.targetFileSizeBytes = targetFileSizeBytes; + this.minFileSizeBytes = minFileSizeBytes; + this.maxFileSizeBytes = maxFileSizeBytes; + this.minInputFiles = minInputFiles; + this.rewriteAll = rewriteAll; + this.maxFileGroupSizeBytes = maxFileGroupSizeBytes; + this.deleteFileThreshold = deleteFileThreshold; + this.deleteRatioThreshold = deleteRatioThreshold; + this.whereCondition = whereCondition; + } + + public long getTargetFileSizeBytes() { + return targetFileSizeBytes; + } + + public long getMinFileSizeBytes() { + return minFileSizeBytes; + } + + public long getMaxFileSizeBytes() { + return maxFileSizeBytes; + } + + public int getMinInputFiles() { + return minInputFiles; + } + + public boolean isRewriteAll() { + return rewriteAll; + } + + public long getMaxFileGroupSizeBytes() { + return maxFileGroupSizeBytes; + } + + public int getDeleteFileThreshold() { + return deleteFileThreshold; + } + + public double getDeleteRatioThreshold() { + return deleteRatioThreshold; + } + + public boolean hasWhereCondition() { + return whereCondition.isPresent(); + } + + public Optional getWhereCondition() { + return whereCondition; + } + + @Override + public String toString() { + return "RewriteDataFilesParameters{" + + ", targetFileSizeBytes=" + targetFileSizeBytes + + ", minFileSizeBytes=" + minFileSizeBytes + + ", maxFileSizeBytes=" + maxFileSizeBytes + + ", minInputFiles=" + minInputFiles + + ", rewriteAll=" + rewriteAll + + ", maxFileGroupSizeBytes=" + maxFileGroupSizeBytes + + ", deleteFileThreshold=" + deleteFileThreshold + + ", deleteRatioThreshold=" + deleteRatioThreshold + + ", hasWhereCondition=" + hasWhereCondition() + + '}'; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataGroup.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataGroup.java new file mode 100644 index 00000000000000..26058ec8f93188 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataGroup.java @@ -0,0 +1,112 @@ +// 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.doris.datasource.iceberg.rewrite; + +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; + +import java.util.ArrayList; +import java.util.List; + +/** + * Group of file scan tasks to be rewritten together + */ +public class RewriteDataGroup { + private final List tasks; + private long totalSize; + private int deleteFileCount; + + public RewriteDataGroup() { + this.tasks = new ArrayList<>(); + this.totalSize = 0; + this.deleteFileCount = 0; + } + + public RewriteDataGroup(List tasks) { + this.tasks = tasks; + this.totalSize = tasks.stream().mapToLong(task -> task.file().fileSizeInBytes()).sum(); + this.deleteFileCount = tasks.stream().mapToInt(task -> task.deletes() != null ? task.deletes().size() : 0) + .sum(); + } + + /** + * Add a task to this group + */ + public void addTask(FileScanTask task) { + tasks.add(task); + totalSize += task.file().fileSizeInBytes(); + if (task.deletes() != null) { + deleteFileCount += task.deletes().size(); + } + } + + /** + * Get all tasks in this group + */ + public List getTasks() { + return tasks; + } + + /** + * Get number of tasks in this group + */ + public int getTaskCount() { + return tasks.size(); + } + + /** + * Get total size of all files in this group + */ + public long getTotalSize() { + return totalSize; + } + + /** + * Get total number of delete files in this group + */ + public int getDeleteFileCount() { + return deleteFileCount; + } + + /** + * Get all data files in this group + */ + public List getDataFiles() { + List dataFiles = new ArrayList<>(); + for (FileScanTask task : tasks) { + dataFiles.add(task.file()); + } + return dataFiles; + } + + /** + * Check if this group is empty + */ + public boolean isEmpty() { + return tasks.isEmpty(); + } + + @Override + public String toString() { + return "RewriteDataGroup{" + + "taskCount=" + tasks.size() + + ", totalSize=" + totalSize + + ", deleteFileCount=" + deleteFileCount + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteGroupTask.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteGroupTask.java new file mode 100644 index 00000000000000..78277dff7781d4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteGroupTask.java @@ -0,0 +1,266 @@ +// 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.doris.datasource.iceberg.rewrite; + +import org.apache.doris.analysis.StatementBase; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Status; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.analyzer.UnboundIcebergTableSink; +import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.glue.LogicalPlanAdapter; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; +import org.apache.doris.nereids.trees.plans.commands.insert.AbstractInsertExecutor; +import org.apache.doris.nereids.trees.plans.commands.insert.IcebergRewriteExecutor; +import org.apache.doris.nereids.trees.plans.commands.insert.RewriteTableCommand; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.OriginStatement; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.qe.VariableMgr; +import org.apache.doris.scheduler.exception.JobException; +import org.apache.doris.scheduler.executor.TransientTaskExecutor; +import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Independent task executor for processing a single rewrite group. + */ +public class RewriteGroupTask implements TransientTaskExecutor { + private static final Logger LOG = LogManager.getLogger(RewriteGroupTask.class); + + private final RewriteDataGroup group; + private final long transactionId; + private final IcebergExternalTable dorisTable; + private final ConnectContext connectContext; + private final long targetFileSizeBytes; + private final RewriteResultCallback resultCallback; + private final Long taskId; + private final AtomicBoolean isCanceled; + private final AtomicBoolean isFinished; + + // for canceling the task + private StmtExecutor stmtExecutor; + + public RewriteGroupTask(RewriteDataGroup group, + long transactionId, + IcebergExternalTable dorisTable, + ConnectContext connectContext, + long targetFileSizeBytes, + RewriteResultCallback resultCallback) { + this.group = group; + this.transactionId = transactionId; + this.dorisTable = dorisTable; + this.connectContext = connectContext; + this.targetFileSizeBytes = targetFileSizeBytes; + this.resultCallback = resultCallback; + this.taskId = UUID.randomUUID().getMostSignificantBits(); + this.isCanceled = new AtomicBoolean(false); + this.isFinished = new AtomicBoolean(false); + } + + @Override + public Long getId() { + return taskId; + } + + @Override + public void execute() throws JobException { + LOG.debug("[Rewrite Task] taskId: {} starting execution for group with {} tasks", + taskId, group.getTaskCount()); + + if (isCanceled.get()) { + LOG.debug("[Rewrite Task] taskId: {} was already canceled before execution", taskId); + throw new JobException("Rewrite task has been canceled, task id: " + taskId); + } + + if (isFinished.get()) { + LOG.debug("[Rewrite Task] taskId: {} was already finished", taskId); + return; + } + + try { + // Step 1: Create and customize a new ConnectContext for this task + ConnectContext taskConnectContext = buildConnectContext(); + // Set target file size for Iceberg write + taskConnectContext.getSessionVariable().setIcebergWriteTargetFileSizeBytes(targetFileSizeBytes); + // Custom file scan tasks for rewrite operations + taskConnectContext.getStatementContext().setIcebergRewriteFileScanTasks(group.getTasks()); + + // Step 2: Build logical plan for this task + RewriteTableCommand taskLogicalPlan = buildRewriteLogicalPlan(); + LogicalPlanAdapter taskParsedStmt = new LogicalPlanAdapter( + taskLogicalPlan, + taskConnectContext.getStatementContext()); + taskParsedStmt.setOrigStmt(new OriginStatement(taskLogicalPlan.toString(), 0)); + + // Step 3: Execute the rewrite operation for this group + executeGroup(taskConnectContext, taskLogicalPlan, taskParsedStmt); + + // Notify result callback + if (resultCallback != null) { + resultCallback.onTaskCompleted(taskId); + } + + LOG.debug("[Rewrite Task] taskId: {} execution completed successfully", taskId); + + } catch (Exception e) { + LOG.warn("Failed to execute rewrite group: {}", e.getMessage(), e); + + // Notify error callback + if (resultCallback != null) { + resultCallback.onTaskFailed(taskId, e); + } + + throw new JobException("Rewrite group execution failed: " + e.getMessage(), e); + } finally { + isFinished.set(true); + } + } + + @Override + public void cancel() throws JobException { + if (isFinished.get()) { + LOG.debug("[Rewrite Task] taskId: {} already finished, cannot cancel", taskId); + return; + } + + isCanceled.set(true); + if (stmtExecutor != null) { + stmtExecutor.cancel(new Status(TStatusCode.CANCELLED, "rewrite task cancelled")); + } + LOG.info("[Rewrite Task] taskId: {} cancelled", taskId); + } + + /** + * Execute rewrite group with task-specific logical plan and parsed statement + */ + private void executeGroup(ConnectContext taskConnectContext, + RewriteTableCommand taskLogicalPlan, + StatementBase taskParsedStmt) throws Exception { + // Step 1: Create stmt executor + stmtExecutor = new StmtExecutor(taskConnectContext, taskParsedStmt); + + // Step 2: Create insert executor + AbstractInsertExecutor insertExecutor = taskLogicalPlan.initPlan(taskConnectContext, stmtExecutor); + Preconditions.checkState(insertExecutor instanceof IcebergRewriteExecutor, + "Expected IcebergRewriteExecutor, got: " + insertExecutor.getClass()); + + // Step 3: Set transaction id for updating CommitData + insertExecutor.getCoordinator().setTxnId(transactionId); + + // Step 4: Execute insert operation + insertExecutor.executeSingleInsert(stmtExecutor); + + LOG.debug("[Rewrite Task] taskId: {} completed execution successfully", taskId); + } + + /** + * Build logical plan for rewrite operation (INSERT INTO ... SELECT ...) + * Each task creates its own independent InsertIntoTableCommand instance + */ + private RewriteTableCommand buildRewriteLogicalPlan() { + // Build table name parts + List tableNameParts = ImmutableList.of( + dorisTable.getCatalog().getName(), + dorisTable.getDbName(), + dorisTable.getName()); + + // Create UnboundRelation for SELECT part (source table) + UnboundRelation sourceRelation = new UnboundRelation( + StatementScopeIdGenerator.newRelationId(), + tableNameParts, + ImmutableList.of(), // partitions + false, // isTemporary + ImmutableList.of(), // tabletIds + ImmutableList.of(), // hints + Optional.empty(), // orderKeys + Optional.empty() // limit + ); + + // Create UnboundIcebergTableSink for INSERT part (target table) + UnboundIcebergTableSink tableSink = new UnboundIcebergTableSink<>( + tableNameParts, + ImmutableList.of(), // colNames (empty means all columns) + ImmutableList.of(), // hints + ImmutableList.of(), // partitions + DMLCommandType.INSERT, + Optional.empty(), // labelName + Optional.empty(), // branchName + sourceRelation); + + // Create RewriteTableCommand for rewrite operation + return new RewriteTableCommand( + tableSink, + Optional.empty(), // labelName + Optional.empty(), // insertCtx + Optional.empty(), // cte + Optional.empty() // branchName + ); + } + + /** + * Build ConnectContext for this task + */ + private ConnectContext buildConnectContext() { + ConnectContext taskContext = new ConnectContext(); + + // Clone session variables from parent + taskContext.setSessionVariable(VariableMgr.cloneSessionVariable(connectContext.getSessionVariable())); + + // Set env and basic identities + taskContext.setEnv(Env.getCurrentEnv()); + taskContext.setDatabase(connectContext.getDatabase()); + taskContext.setCurrentUserIdentity(connectContext.getCurrentUserIdentity()); + taskContext.setRemoteIP(connectContext.getRemoteIP()); + + // Assign unique query id and start time + UUID uuid = UUID.randomUUID(); + TUniqueId queryId = new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); + taskContext.setQueryId(queryId); + taskContext.setThreadLocalInfo(); + taskContext.setStartTime(); + + // Initialize StatementContext for this task + StatementContext statementContext = new StatementContext(); + statementContext.setConnectContext(taskContext); + taskContext.setStatementContext(statementContext); + + return taskContext; + } + + /** + * Callback interface for task completion + */ + public interface RewriteResultCallback { + void onTaskCompleted(Long taskId); + + void onTaskFailed(Long taskId, Exception error); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteResult.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteResult.java new file mode 100644 index 00000000000000..c47ad89fbcc615 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/rewrite/RewriteResult.java @@ -0,0 +1,111 @@ +// 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.doris.datasource.iceberg.rewrite; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * Result of Iceberg data file rewrite operation + */ +public class RewriteResult { + private int rewrittenDataFilesCount; + private int addedDataFilesCount; + private long rewrittenBytesCount; + private int removedDeleteFilesCount; + + public RewriteResult() { + this(0, 0, 0, 0); + } + + public RewriteResult(int rewrittenDataFilesCount, + int addedDataFilesCount, + long rewrittenBytesCount, + int removedDeleteFilesCount) { + this.rewrittenDataFilesCount = rewrittenDataFilesCount; + this.addedDataFilesCount = addedDataFilesCount; + this.rewrittenBytesCount = rewrittenBytesCount; + this.removedDeleteFilesCount = removedDeleteFilesCount; + } + + /** + * Merge another result into this one + */ + public void merge(RewriteResult other) { + this.rewrittenDataFilesCount += other.rewrittenDataFilesCount; + this.addedDataFilesCount += other.addedDataFilesCount; + this.rewrittenBytesCount += other.rewrittenBytesCount; + this.removedDeleteFilesCount += other.removedDeleteFilesCount; + } + + /** + * Convert to string list for display + */ + public List toStringList() { + return Lists.newArrayList( + String.valueOf(rewrittenDataFilesCount), + String.valueOf(addedDataFilesCount), + String.valueOf(rewrittenBytesCount), + String.valueOf(removedDeleteFilesCount)); + } + + // Getters + public int getRewrittenDataFilesCount() { + return rewrittenDataFilesCount; + } + + public int getAddedDataFilesCount() { + return addedDataFilesCount; + } + + public long getRewrittenBytesCount() { + return rewrittenBytesCount; + } + + public int getRemovedDeleteFilesCount() { + return removedDeleteFilesCount; + } + + // Setters + public void setRewrittenDataFilesCount(int rewrittenDataFilesCount) { + this.rewrittenDataFilesCount = rewrittenDataFilesCount; + } + + public void setAddedDataFilesCount(int addedDataFilesCount) { + this.addedDataFilesCount = addedDataFilesCount; + } + + public void setRewrittenBytesCount(long rewrittenBytesCount) { + this.rewrittenBytesCount = rewrittenBytesCount; + } + + public void setRemovedDeleteFilesCount(int removedDeleteFilesCount) { + this.removedDeleteFilesCount = removedDeleteFilesCount; + } + + @Override + public String toString() { + return "RewriteResult{" + + "rewrittenDataFilesCount=" + rewrittenDataFilesCount + + ", addedDataFilesCount=" + addedDataFilesCount + + ", rewrittenBytesCount=" + rewrittenBytesCount + + ", removedDeleteFilesCount=" + removedDeleteFilesCount + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java index 0517117645e904..df6dd92be9dcab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java @@ -41,6 +41,7 @@ import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; import org.apache.doris.spi.Split; import org.apache.doris.statistics.StatisticalType; @@ -250,6 +251,25 @@ public List getSplits(int numBackends) throws UserException { } } + /** + * Get FileScanTasks from StatementContext for rewrite operations. + * This allows setting file scan tasks before the plan is generated. + */ + private List getFileScanTasksFromContext() { + ConnectContext ctx = ConnectContext.get(); + Preconditions.checkNotNull(ctx); + Preconditions.checkNotNull(ctx.getStatementContext()); + + // Get the rewrite file scan tasks from statement context + List tasks = ctx.getStatementContext().getAndClearIcebergRewriteFileScanTasks(); + if (tasks != null && !tasks.isEmpty()) { + LOG.info("Retrieved {} file scan tasks from context for table {}", + tasks.size(), icebergTable.name()); + return new ArrayList<>(tasks); + } + return null; + } + @Override public void startSplit(int numBackends) throws UserException { try { @@ -380,9 +400,21 @@ private Split createIcebergSplit(FileScanTask fileScanTask) { private List doGetSplits(int numBackends) throws UserException { - TableScan scan = createTableScan(); List splits = new ArrayList<>(); + // Use custom file scan tasks if available (for rewrite operations) + List customFileScanTasks = getFileScanTasksFromContext(); + if (customFileScanTasks != null) { + for (FileScanTask task : customFileScanTasks) { + splits.add(createIcebergSplit(task)); + } + selectedPartitionNum = partitionMapInfos.size(); + return splits; + } + + // Normal table scan planning + TableScan scan = createTableScan(); + try (CloseableIterable fileScanTasks = planFileScanTask(scan)) { if (tableLevelPushDownCount) { int needSplitCnt = countFromSnapshot < COUNT_WITH_PARALLEL_SPLITS diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 0a841e14dad582..8fd9aba8b1ebf3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -286,6 +286,10 @@ public enum TableFrom { private Optional>> mvRefreshPredicates = Optional.empty(); + // For Iceberg rewrite operations: store file scan tasks to be used by IcebergScanNode + // TODO: better solution? + private List icebergRewriteFileScanTasks = null; + public StatementContext() { this(ConnectContext.get(), null, 0); } @@ -1019,6 +1023,24 @@ public void setMvRefreshPredicates( this.mvRefreshPredicates = Optional.of(mvRefreshPredicates); } + /** + * Set file scan tasks for Iceberg rewrite operations. + * This allows IcebergScanNode to use specific file scan tasks instead of scanning the full table. + */ + public void setIcebergRewriteFileScanTasks(List tasks) { + this.icebergRewriteFileScanTasks = tasks; + } + + /** + * Get and consume file scan tasks for Iceberg rewrite operations. + * Returns the tasks and clears the field to prevent reuse. + */ + public List getAndClearIcebergRewriteFileScanTasks() { + List tasks = this.icebergRewriteFileScanTasks; + this.icebergRewriteFileScanTasks = null; + return tasks; + } + public boolean isSkipPrunePredicate() { return skipPrunePredicate; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExecuteActionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExecuteActionCommand.java index b0dbff95ba4677..288497300311ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExecuteActionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExecuteActionCommand.java @@ -74,6 +74,7 @@ public ExecuteActionCommand(TableNameInfo tableNameInfo, String actionName, @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + tableNameInfo.analyze(ctx); CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(tableNameInfo.getCtl()); if (catalog == null) { throw new AnalysisException("Catalog " + tableNameInfo.getCtl() + " does not exist"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergRewriteExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergRewriteExecutor.java new file mode 100644 index 00000000000000..fe0efc98b8f8fa --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergRewriteExecutor.java @@ -0,0 +1,60 @@ +// 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.doris.nereids.trees.plans.commands.insert; + +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.transaction.TransactionType; + +import java.util.Optional; + +/** + * Rewrite executor for iceberg table data file rewrite operations. + * + * This executor is specifically designed for rewrite operations and uses + * rewrite-specific transaction logic instead of insert transaction logic. + */ +public class IcebergRewriteExecutor extends BaseExternalTableInsertExecutor { + + /** + * constructor + */ + public IcebergRewriteExecutor(ConnectContext ctx, IcebergExternalTable table, + String labelName, NereidsPlanner planner, + Optional insertCtx, + boolean emptyInsert, long jobId) { + super(ctx, table, labelName, planner, insertCtx, emptyInsert, jobId); + } + + @Override + protected void beforeExec() throws UserException { + // do nothing, the transaction is not managed by IcebergRewriteExecutor + } + + @Override + protected void doBeforeCommit() throws UserException { + // do nothing, the transaction is not managed by IcebergRewriteExecutor + } + + @Override + protected TransactionType transactionType() { + return TransactionType.ICEBERG; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/RewriteTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/RewriteTableCommand.java new file mode 100644 index 00000000000000..3a8edf2d849387 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/RewriteTableCommand.java @@ -0,0 +1,326 @@ +// 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.doris.nereids.trees.plans.commands.insert; + +import org.apache.doris.analysis.RedirectStatus; +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.Util; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.glue.LogicalPlanAdapter; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.plans.Explainable; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.TVFRelation; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; +import org.apache.doris.nereids.trees.plans.commands.ForwardWithSync; +import org.apache.doris.nereids.trees.plans.commands.NeedAuditEncryption; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalIcebergTableSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.RelationUtil; +import org.apache.doris.planner.DataSink; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; + +/** + * RewriteTableCommand: dedicated command for rewrite operations (currently + * Iceberg only). + * This command is used to rewrite the data file of an iceberg table. + */ +public class RewriteTableCommand extends Command implements NeedAuditEncryption, ForwardWithSync, Explainable { + + public static final Logger LOG = LogManager.getLogger(RewriteTableCommand.class); + + private LogicalPlan originLogicalQuery; + private Optional logicalQuery; + private Optional labelName; + private Optional parsedPlan; + private final Optional insertCtx; + private final Optional cte; + private final Optional branchName; + private long jobId; + + /** + * constructor for rewrite operation + */ + public RewriteTableCommand(LogicalPlan logicalQuery, Optional labelName, + Optional insertCtx, Optional cte, Optional branchName) { + super(PlanType.INSERT_INTO_TABLE_COMMAND); + this.originLogicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery should not be null"); + this.labelName = Objects.requireNonNull(labelName, "labelName should not be null"); + this.logicalQuery = Optional.empty(); + this.insertCtx = insertCtx; + this.cte = cte; + this.branchName = branchName; + if (Env.getCurrentEnv().isMaster()) { + this.jobId = Env.getCurrentEnv().getNextId(); + } else { + this.jobId = -1; + } + } + + public LogicalPlan getLogicalQuery() { + return logicalQuery.orElse(originLogicalQuery); + } + + public Optional getParsedPlan() { + return parsedPlan; + } + + protected void setLogicalQuery(LogicalPlan plan) { + this.logicalQuery = Optional.of(plan); + } + + protected TableIf getTargetTableIf(ConnectContext ctx, List qualifiedTargetTableName) { + return RelationUtil.getTable(qualifiedTargetTableName, ctx.getEnv(), Optional.empty()); + } + + /** + * For rewrite, we never begin transaction here. We always finalize sink in init + * stage to keep parity with previous rewrite flow, and let external caller + * inject txnId to coordinator. + */ + public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor stmtExecutor) throws Exception { + List qualifiedTargetTableName = InsertUtils.getTargetTableQualified(originLogicalQuery, ctx); + ctx.getStatementContext().setIsInsert(true); + + TableIf targetTableIf = getTargetTableIf(ctx, qualifiedTargetTableName); + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), targetTableIf.getDatabase().getCatalog().getName(), + targetTableIf.getDatabase().getFullName(), targetTableIf.getName(), + org.apache.doris.mysql.privilege.PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", + ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), + targetTableIf.getDatabase().getFullName() + "." + + Util.getTempTableDisplayName(targetTableIf.getName())); + } + + BuildResult buildResult; + try { + buildResult = initPlanOnce(ctx, stmtExecutor, targetTableIf); + } catch (Throwable e) { + Throwables.throwIfInstanceOf(e, RuntimeException.class); + throw new IllegalStateException(e.getMessage(), e); + } + AbstractInsertExecutor insertExecutor = buildResult.executor; + parsedPlan = Optional.ofNullable(buildResult.planner.getParsedPlan()); + + // For rewrite: do not begin transaction here, but finalize sink + insertExecutor.finalizeSink( + buildResult.planner.getFragments().get(0), buildResult.dataSink, buildResult.physicalSink); + return insertExecutor; + } + + private BuildResult initPlanOnce(ConnectContext ctx, StmtExecutor stmtExecutor, TableIf targetTableIf) + throws Throwable { + targetTableIf.readLock(); + try { + Optional analyzeContext = Optional.of( + CascadesContext.initContext(ctx.getStatementContext(), originLogicalQuery, PhysicalProperties.ANY)); + // rewrite does not need special logic like normalize insert-into-values, just + // use normalize directly + this.logicalQuery = Optional.of((LogicalPlan) InsertUtils.normalizePlan(originLogicalQuery, + targetTableIf, analyzeContext, insertCtx)); + if (cte.isPresent()) { + this.logicalQuery = Optional.of((LogicalPlan) cte.get().withChildren(logicalQuery.get())); + } + } finally { + targetTableIf.readUnlock(); + } + + LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalQuery.get(), ctx.getStatementContext()); + return planInsertExecutor(ctx, stmtExecutor, logicalPlanAdapter, targetTableIf); + } + + private ExecutorFactory selectInsertExecutorFactory(NereidsPlanner planner, ConnectContext ctx, + StmtExecutor stmtExecutor, TableIf targetTableIf) { + try { + stmtExecutor.setPlanner(planner); + stmtExecutor.checkBlockRules(); + if (ctx.getConnectType() == ConnectContext.ConnectType.MYSQL && ctx.getMysqlChannel() != null) { + ctx.getMysqlChannel().reset(); + } + Optional> plan = (planner.getPhysicalPlan() + .>collect(PhysicalSink.class::isInstance)).stream().findAny(); + Preconditions.checkArgument(plan.isPresent(), "rewrite command must contain target table"); + PhysicalSink physicalSink = plan.get(); + DataSink dataSink = planner.getFragments().get(0).getSink(); + String label = this.labelName.orElse(String.format("label_%x_%x", ctx.queryId().hi, ctx.queryId().lo)); + + if (physicalSink instanceof PhysicalIcebergTableSink) { + boolean emptyInsert = childIsEmptyRelation(physicalSink); + IcebergExternalTable icebergExternalTable = (IcebergExternalTable) targetTableIf; + IcebergInsertCommandContext icebergInsertCtx = insertCtx + .map(c -> (IcebergInsertCommandContext) c) + .orElseGet(IcebergInsertCommandContext::new); + branchName.ifPresent(notUsed -> icebergInsertCtx.setBranchName(branchName)); + return ExecutorFactory.from(planner, dataSink, physicalSink, + () -> new IcebergRewriteExecutor(ctx, icebergExternalTable, label, planner, + Optional.of(icebergInsertCtx), emptyInsert, jobId)); + } + throw new AnalysisException("Rewrite only supports iceberg table"); + } catch (Throwable t) { + Throwables.throwIfInstanceOf(t, RuntimeException.class); + throw new IllegalStateException(t.getMessage(), t); + } + } + + private BuildResult planInsertExecutor(ConnectContext ctx, StmtExecutor stmtExecutor, + LogicalPlanAdapter logicalPlanAdapter, TableIf targetTableIf) throws Throwable { + LogicalPlan logicalPlan = logicalPlanAdapter.getLogicalPlan(); + boolean supportFastInsertIntoValues = InsertUtils.supportFastInsertIntoValues(logicalPlan, targetTableIf, ctx); + AtomicReference executorFactoryRef = new AtomicReference<>(); + + FastInsertIntoValuesPlanner planner = new FastInsertIntoValuesPlanner( + ctx.getStatementContext(), supportFastInsertIntoValues) { + @Override + protected void doDistribute(boolean canUseNereidsDistributePlanner, ExplainLevel explainLevel) { + executorFactoryRef.set(selectInsertExecutorFactory(this, ctx, stmtExecutor, targetTableIf)); + super.doDistribute(canUseNereidsDistributePlanner, explainLevel); + } + }; + + planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); + if (LOG.isDebugEnabled()) { + LOG.debug("rewrite plan for query_id: {} is: {}.", DebugUtil.printId(ctx.queryId()), + planner.getPhysicalPlan().treeString()); + } + return executorFactoryRef.get().build(); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + AbstractInsertExecutor insertExecutor = initPlan(ctx, executor); + if (insertExecutor.isEmptyInsert()) { + return; + } + insertExecutor.executeSingleInsert(executor); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCommand(this, context); + } + + @Override + public Plan getExplainPlan(ConnectContext ctx) { + Optional analyzeContext = Optional.of( + CascadesContext.initContext(ctx.getStatementContext(), originLogicalQuery, PhysicalProperties.ANY)); + Plan plan = InsertUtils.getPlanForExplain(ctx, analyzeContext, getLogicalQuery()); + if (cte.isPresent()) { + plan = cte.get().withChildren(plan); + } + return plan; + } + + @Override + public Optional getExplainPlanner(LogicalPlan logicalPlan, StatementContext ctx) { + ConnectContext connectContext = ctx.getConnectContext(); + TableIf targetTableIf = InsertUtils.getTargetTable(originLogicalQuery, connectContext); + boolean supportFastInsertIntoValues = InsertUtils.supportFastInsertIntoValues(logicalPlan, targetTableIf, + connectContext); + return Optional.of(new FastInsertIntoValuesPlanner(ctx, supportFastInsertIntoValues)); + } + + @Override + public StmtType stmtType() { + return StmtType.INSERT; + } + + @Override + public RedirectStatus toRedirectStatus() { + return RedirectStatus.FORWARD_WITH_SYNC; + } + + @Override + public boolean needAuditEncryption() { + return originLogicalQuery + .anyMatch(node -> node instanceof TVFRelation); + } + + private boolean childIsEmptyRelation(PhysicalSink sink) { + if (sink.children() != null && sink.children().size() == 1 + && sink.child(0) instanceof PhysicalEmptyRelation) { + return true; + } + return false; + } + + private static class ExecutorFactory { + public final NereidsPlanner planner; + public final DataSink dataSink; + public final PhysicalSink physicalSink; + public final java.util.function.Supplier executorSupplier; + + private ExecutorFactory(NereidsPlanner planner, DataSink dataSink, PhysicalSink physicalSink, + java.util.function.Supplier executorSupplier) { + this.planner = planner; + this.dataSink = dataSink; + this.physicalSink = physicalSink; + this.executorSupplier = executorSupplier; + } + + public static ExecutorFactory from(NereidsPlanner planner, DataSink dataSink, PhysicalSink physicalSink, + java.util.function.Supplier executorSupplier) { + return new ExecutorFactory(planner, dataSink, physicalSink, executorSupplier); + } + + public BuildResult build() { + AbstractInsertExecutor executor = executorSupplier.get(); + return new BuildResult(planner, executor, dataSink, physicalSink); + } + } + + private static class BuildResult { + private final NereidsPlanner planner; + private final AbstractInsertExecutor executor; + private final DataSink dataSink; + private final PhysicalSink physicalSink; + + public BuildResult(NereidsPlanner planner, AbstractInsertExecutor executor, DataSink dataSink, + PhysicalSink physicalSink) { + this.planner = planner; + this.executor = executor; + this.dataSink = dataSink; + this.physicalSink = physicalSink; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java index f695ee2d024f7e..4084e4a84e8a2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java @@ -378,6 +378,10 @@ public List getFragments() { return coordinatorContext.fragments; } + public CoordinatorContext getCoordinatorContext() { + return coordinatorContext; + } + @Override public ExecutionProfile getExecutionProfile() { return coordinatorContext.executionProfile; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 5d96452898bb5b..61adc668c463f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -509,6 +509,9 @@ public class SessionVariable implements Serializable, Writable { // Split size for ExternalFileScanNode. Default value 0 means use the block size of HDFS/S3. public static final String FILE_SPLIT_SIZE = "file_split_size"; + // Target file size in bytes for Iceberg write operations + public static final String ICEBERG_WRITE_TARGET_FILE_SIZE_BYTES = "iceberg_write_target_file_size_bytes"; + public static final String NUM_PARTITIONS_IN_BATCH_MODE = "num_partitions_in_batch_mode"; public static final String NUM_FILES_IN_BATCH_MODE = "num_files_in_batch_mode"; @@ -2131,6 +2134,11 @@ public boolean isEnableHboNonStrictMatchingMode() { @VariableMgr.VarAttr(name = FILE_SPLIT_SIZE, needForward = true) public long fileSplitSize = 0; + // Target file size for Iceberg write operations + // Default 0 means use config::iceberg_sink_max_file_size + @VariableMgr.VarAttr(name = ICEBERG_WRITE_TARGET_FILE_SIZE_BYTES, needForward = true) + public long icebergWriteTargetFileSizeBytes = 0L; + @VariableMgr.VarAttr( name = NUM_PARTITIONS_IN_BATCH_MODE, fuzzy = true, @@ -4145,6 +4153,14 @@ public void setFileSplitSize(long fileSplitSize) { this.fileSplitSize = fileSplitSize; } + public long getIcebergWriteTargetFileSizeBytes() { + return icebergWriteTargetFileSizeBytes; + } + + public void setIcebergWriteTargetFileSizeBytes(long icebergWriteTargetFileSizeBytes) { + this.icebergWriteTargetFileSizeBytes = icebergWriteTargetFileSizeBytes; + } + public int getNumPartitionsInBatchMode() { return numPartitionsInBatchMode; } @@ -4880,6 +4896,10 @@ public TQueryOptions toThrift() { tResult.setHnswBoundedQueue(hnswBoundedQueue); tResult.setMergeReadSliceSize(mergeReadSliceSizeBytes); tResult.setEnableExtendedRegex(enableExtendedRegex); + + // Set Iceberg write target file size + tResult.setIcebergWriteTargetFileSizeBytes(icebergWriteTargetFileSizeBytes); + return tResult; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergNereidsUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergNereidsUtilsTest.java new file mode 100644 index 00000000000000..3bb8f005828931 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/IcebergNereidsUtilsTest.java @@ -0,0 +1,1004 @@ +// 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.doris.datasource.iceberg; + +import org.apache.doris.common.UserException; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.Between; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.GreaterThan; +import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; +import org.apache.doris.nereids.trees.expressions.InPredicate; +import org.apache.doris.nereids.trees.expressions.LessThan; +import org.apache.doris.nereids.trees.expressions.LessThanEqual; +import org.apache.doris.nereids.trees.expressions.Not; +import org.apache.doris.nereids.trees.expressions.Or; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import org.apache.doris.nereids.trees.expressions.literal.CharLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DateLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DecimalLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DecimalV3Literal; +import org.apache.doris.nereids.trees.expressions.literal.DoubleLiteral; +import org.apache.doris.nereids.trees.expressions.literal.FloatLiteral; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; +import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.BooleanType; +import org.apache.doris.nereids.types.CharType; +import org.apache.doris.nereids.types.DateType; +import org.apache.doris.nereids.types.DecimalV2Type; +import org.apache.doris.nereids.types.DoubleType; +import org.apache.doris.nereids.types.FloatType; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.SmallIntType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.TinyIntType; +import org.apache.doris.nereids.types.VarcharType; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.Mockito; + +import java.math.BigDecimal; +import java.util.Arrays; +import java.util.List; + +/** + * Unit tests for IcebergNereidsUtils + */ +public class IcebergNereidsUtilsTest { + + @Mock + private Schema mockSchema; + + @Mock + private Types.NestedField mockNestedField; + + private Schema testSchema; + + @BeforeEach + public void setUp() { + // Create a real schema for testing + testSchema = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()), + Types.NestedField.required(3, "age", Types.IntegerType.get()), + Types.NestedField.required(4, "salary", Types.DoubleType.get()), + Types.NestedField.required(5, "is_active", Types.BooleanType.get()), + Types.NestedField.required(6, "birth_date", Types.DateType.get()), + Types.NestedField.required(7, "event_time_tz", Types.TimestampType.withZone()), + Types.NestedField.required(8, "event_time_ntz", Types.TimestampType.withoutZone()), + Types.NestedField.required(9, "dec_col", Types.DecimalType.of(10, 2)), + Types.NestedField.required(10, "time_col", Types.TimeType.get())); + } + + @Test + public void testConvertNereidsToIcebergExpression_NullInput() { + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(null, testSchema); + }); + Assertions.assertEquals("Nereids expression is null", exception.getDetailMessage()); + } + + @Test + public void testConvertNereidsToIcebergExpression_EqualTo() throws UserException { + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(100); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("id", 100).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_GreaterThan() throws UserException { + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(18); + GreaterThan greaterThan = new GreaterThan(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(greaterThan, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.greaterThan("age", 18).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_GreaterThanEqual() throws UserException { + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(18); + GreaterThanEqual greaterThanEqual = new GreaterThanEqual(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(greaterThanEqual, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.greaterThanOrEqual("age", 18).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_LessThan() throws UserException { + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(65); + LessThan lessThan = new LessThan(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(lessThan, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.lessThan("age", 65).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_LessThanEqual() throws UserException { + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(65); + LessThanEqual lessThanEqual = new LessThanEqual(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(lessThanEqual, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.lessThanOrEqual("age", 65).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_And() throws UserException { + SlotReference slotRef1 = new SlotReference("age", IntegerType.INSTANCE, false); + SlotReference slotRef2 = new SlotReference("salary", DoubleType.INSTANCE, false); + IntegerLiteral literal1 = new IntegerLiteral(18); + DoubleLiteral literal2 = new DoubleLiteral(50000.0); + + GreaterThan greaterThan = new GreaterThan(slotRef1, literal1); + GreaterThanEqual greaterThanEqual = new GreaterThanEqual(slotRef2, literal2); + And andExpr = new And(greaterThan, greaterThanEqual); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(andExpr, testSchema); + + Assertions.assertNotNull(result); + } + + @Test + public void testConvertNereidsToIcebergExpression_Or() throws UserException { + SlotReference slotRef1 = new SlotReference("age", IntegerType.INSTANCE, false); + SlotReference slotRef2 = new SlotReference("age", IntegerType.INSTANCE, false); + IntegerLiteral literal1 = new IntegerLiteral(18); + IntegerLiteral literal2 = new IntegerLiteral(65); + + LessThan lessThan = new LessThan(slotRef1, literal1); + GreaterThan greaterThan = new GreaterThan(slotRef2, literal2); + Or orExpr = new Or(lessThan, greaterThan); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils.convertNereidsToIcebergExpression(orExpr, + testSchema); + + Assertions.assertNotNull(result); + } + + @Test + public void testConvertNereidsToIcebergExpression_Not() throws UserException { + SlotReference slotRef = new SlotReference("is_active", BooleanType.INSTANCE, false); + BooleanLiteral literal = BooleanLiteral.of(true); + EqualTo equalTo = new EqualTo(slotRef, literal); + Not notExpr = new Not(equalTo); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(notExpr, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertTrue(result.toString().toLowerCase().contains("not")); + } + + @Test + public void testConvertNereidsToIcebergExpression_InPredicate() throws UserException { + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + IntegerLiteral literal1 = new IntegerLiteral(1); + IntegerLiteral literal2 = new IntegerLiteral(2); + IntegerLiteral literal3 = new IntegerLiteral(3); + + InPredicate inPredicate = new InPredicate(slotRef, Arrays.asList(literal1, literal2, literal3)); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(inPredicate, testSchema); + + Assertions.assertNotNull(result); + String s = result.toString(); + Assertions.assertTrue(s.contains("id")); + Assertions.assertTrue(s.contains("1")); + Assertions.assertTrue(s.contains("2")); + Assertions.assertTrue(s.contains("3")); + } + + @Test + public void testConvertNereidsToIcebergExpression_ComplexNested() throws UserException { + // Test complex nested expression: (age > 18 AND salary >= 50000) OR (age < 65 + // AND salary < 100000) + SlotReference ageRef = new SlotReference("age", IntegerType.INSTANCE, false); + SlotReference salaryRef = new SlotReference("salary", DoubleType.INSTANCE, false); + + GreaterThan ageGt = new GreaterThan(ageRef, new IntegerLiteral(18)); + GreaterThanEqual salaryGte = new GreaterThanEqual(salaryRef, new DoubleLiteral(50000.0)); + And leftAnd = new And(ageGt, salaryGte); + + LessThan ageLt = new LessThan(ageRef, new IntegerLiteral(65)); + LessThan salaryLt = new LessThan(salaryRef, new DoubleLiteral(100000.0)); + And rightAnd = new And(ageLt, salaryLt); + + Or orExpr = new Or(leftAnd, rightAnd); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils.convertNereidsToIcebergExpression(orExpr, + testSchema); + + Assertions.assertNotNull(result); + Assertions.assertTrue(result.toString().toLowerCase().contains("or")); + } + + @Test + public void testConvertNereidsToIcebergExpression_WithNullLiteral() throws UserException { + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + NullLiteral nullLiteral = new NullLiteral(); + EqualTo equalTo = new EqualTo(slotRef, nullLiteral); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.isNull("id").toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_ColumnNotFound() { + SlotReference slotRef = new SlotReference("non_existent_column", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(100); + EqualTo equalTo = new EqualTo(slotRef, literal); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(equalTo, testSchema); + }); + Assertions.assertEquals("Column not found in Iceberg schema: non_existent_column", + exception.getDetailMessage()); + } + + @Test + public void testConvertNereidsToIcebergExpression_CaseInsensitiveColumnName() throws UserException { + // Test case insensitive column name matching + SlotReference slotRef = new SlotReference("ID", IntegerType.INSTANCE, false); // uppercase + IntegerLiteral literal = new IntegerLiteral(100); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("id", 100).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_UnsupportedExpression() { + // Test with an unsupported expression type + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(100); + + // Create a mock expression that's not supported + org.apache.doris.nereids.trees.expressions.Expression unsupportedExpr = Mockito.mock( + org.apache.doris.nereids.trees.expressions.Expression.class); + Mockito.when(unsupportedExpr.children()).thenReturn(Arrays.asList(slotRef, literal)); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(unsupportedExpr, testSchema); + }); + Assertions.assertTrue(exception.getDetailMessage().contains("Unsupported expression type")); + } + + @Test + public void testConvertNereidsToIcebergExpression_StringLiteral() throws UserException { + SlotReference slotRef = new SlotReference("name", StringType.INSTANCE, false); + StringLiteral literal = new StringLiteral("John"); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("name", "John").toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_BooleanLiteral() throws UserException { + SlotReference slotRef = new SlotReference("is_active", BooleanType.INSTANCE, false); + BooleanLiteral literal = BooleanLiteral.of(true); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("is_active", true).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_DoubleLiteral() throws UserException { + SlotReference slotRef = new SlotReference("salary", DoubleType.INSTANCE, false); + DoubleLiteral literal = new DoubleLiteral(50000.5); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("salary", 50000.5).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_FloatLiteral() throws UserException { + SlotReference slotRef = new SlotReference("salary", FloatType.INSTANCE, false); + FloatLiteral literal = new FloatLiteral(50000.5f); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("salary", 50000.5f).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_BigIntLiteral() throws UserException { + SlotReference slotRef = new SlotReference("id", BigIntType.INSTANCE, false); + BigIntLiteral literal = new BigIntLiteral(123456789L); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("id", 123456789L).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_DecimalLiteral() throws UserException { + SlotReference slotRef = new SlotReference("salary", DecimalV2Type.SYSTEM_DEFAULT, false); + DecimalLiteral literal = new DecimalLiteral(new BigDecimal("50000.50")); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + } + + @Test + public void testConvertNereidsToIcebergExpression_DateLiteral() throws UserException { + SlotReference slotRef = new SlotReference("birth_date", DateType.INSTANCE, false); + DateLiteral literal = new DateLiteral("2023-01-01"); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("birth_date", "2023-01-01").toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_TimestampWithZoneMicros() throws UserException { + org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal literal = + new org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal( + org.apache.doris.nereids.types.DateTimeV2Type.forTypeFromString("2023-01-02 03:04:05.123456"), + 2023, 1, 2, 3, 4, 5, 123456); + EqualTo equalTo = new EqualTo(new SlotReference("event_time_tz", + org.apache.doris.nereids.types.DateTimeV2Type.forTypeFromString("2023-01-02 03:04:05.123456"), false), + literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + java.time.ZoneId zone = org.apache.doris.nereids.util.DateUtils.getTimeZone(); + java.time.LocalDateTime ldt = java.time.LocalDateTime.of(2023, 1, 2, 3, 4, 5, 123456000); + long expectedMicros = ldt.atZone(zone).toInstant().toEpochMilli() * 1000L + 123456; + Assertions.assertEquals(Expressions.equal("event_time_tz", expectedMicros).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_TimestampWithoutZoneMicros() throws UserException { + org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral literal = + new org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral(2023, 1, 2, 3, 4, 5); + EqualTo equalTo = new EqualTo(new SlotReference("event_time_ntz", + org.apache.doris.nereids.types.DateTimeType.INSTANCE, false), literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + java.time.ZoneId zone = java.time.ZoneId.of("UTC"); + java.time.LocalDateTime ldt = java.time.LocalDateTime.of(2023, 1, 2, 3, 4, 5, 0); + long expectedMicros = ldt.atZone(zone).toInstant().toEpochMilli() * 1000L; + Assertions.assertEquals(Expressions.equal("event_time_ntz", expectedMicros).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_DecimalMapping() throws UserException { + SlotReference slotRef = new SlotReference("dec_col", DecimalV2Type.SYSTEM_DEFAULT, false); + DecimalLiteral literal = new DecimalLiteral(new BigDecimal("12.34")); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertTrue(result.toString().contains("12.34")); + } + + @Test + public void testConvertNereidsToIcebergExpression_DecimalV3Mapping() throws UserException { + SlotReference slotRef = new SlotReference("dec_col", DecimalV2Type.SYSTEM_DEFAULT, false); + DecimalV3Literal literal = + new DecimalV3Literal(new BigDecimal("99.990")); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertTrue(result.toString().contains("99.99")); + } + + @Test + public void testConvertNereidsToIcebergExpression_TimeAsLong() throws UserException { + SlotReference slotRef = new SlotReference("time_col", IntegerType.INSTANCE, false); + // use a numeric literal to represent micros since midnight + BigIntLiteral literal = new BigIntLiteral(12_345_678L); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("time_col", 12_345_678L).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_StringToIntParsing() throws UserException { + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + StringLiteral literal = new StringLiteral("123"); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("id", 123).toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_CharLiteral() throws UserException { + SlotReference slotRef = new SlotReference("name", CharType.createCharType(1), false); + CharLiteral literal = new CharLiteral("A", 1); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("name", "A").toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_TinyIntLiteral() throws UserException { + SlotReference slotRef = new SlotReference("age", TinyIntType.INSTANCE, false); + TinyIntLiteral literal = new TinyIntLiteral((byte) 25); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertTrue(result.toString().contains("age")); + Assertions.assertTrue(result.toString().contains("25")); + } + + @Test + public void testConvertNereidsToIcebergExpression_SmallIntLiteral() throws UserException { + SlotReference slotRef = new SlotReference("age", SmallIntType.INSTANCE, false); + SmallIntLiteral literal = new SmallIntLiteral((short) 25); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertTrue(result.toString().contains("age")); + Assertions.assertTrue(result.toString().contains("25")); + } + + @Test + public void testConvertNereidsToIcebergExpression_VarcharLiteral() throws UserException { + SlotReference slotRef = new SlotReference("name", VarcharType.SYSTEM_DEFAULT, false); + StringLiteral literal = new StringLiteral("John Doe"); + EqualTo equalTo = new EqualTo(slotRef, literal); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + + Assertions.assertNotNull(result); + Assertions.assertEquals(Expressions.equal("name", "John Doe").toString(), result.toString()); + } + + @Test + public void testConvertNereidsToIcebergExpression_MixedLiteralTypesInInPredicate() throws UserException { + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + IntegerLiteral literal1 = new IntegerLiteral(1); + IntegerLiteral literal2 = new IntegerLiteral(2); + IntegerLiteral literal3 = new IntegerLiteral(3); + + InPredicate inPredicate = new InPredicate(slotRef, Arrays.asList(literal1, literal2, literal3)); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(inPredicate, testSchema); + + Assertions.assertNotNull(result); + String s = result.toString(); + Assertions.assertTrue(s.contains("id")); + Assertions.assertTrue(s.contains("1")); + Assertions.assertTrue(s.contains("2")); + Assertions.assertTrue(s.contains("3")); + } + + @Test + public void testConvertNereidsToIcebergExpression_DeeplyNestedExpression() throws UserException { + // Test deeply nested expression: NOT ((age > 18 AND salary >= 50000) OR (age < + // 65 AND salary < 100000)) + SlotReference ageRef = new SlotReference("age", IntegerType.INSTANCE, false); + SlotReference salaryRef = new SlotReference("salary", DoubleType.INSTANCE, false); + + GreaterThan ageGt = new GreaterThan(ageRef, new IntegerLiteral(18)); + GreaterThanEqual salaryGte = new GreaterThanEqual(salaryRef, new DoubleLiteral(50000.0)); + And leftAnd = new And(ageGt, salaryGte); + + LessThan ageLt = new LessThan(ageRef, new IntegerLiteral(65)); + LessThan salaryLt = new LessThan(salaryRef, new DoubleLiteral(100000.0)); + And rightAnd = new And(ageLt, salaryLt); + + Or orExpr = new Or(leftAnd, rightAnd); + Not notExpr = new Not(orExpr); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(notExpr, testSchema); + + Assertions.assertNotNull(result); + String s = result.toString().toLowerCase(); + Assertions.assertTrue(s.contains("not")); + Assertions.assertTrue(s.contains("or")); + Assertions.assertTrue(s.contains("and")); + } + + @Test + public void testConvertNereidsToIcebergExpression_AllComparisonOperators() throws UserException { + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(25); + + // Test all comparison operators + EqualTo equalTo = new EqualTo(slotRef, literal); + GreaterThan greaterThan = new GreaterThan(slotRef, literal); + GreaterThanEqual greaterThanEqual = new GreaterThanEqual(slotRef, literal); + LessThan lessThan = new LessThan(slotRef, literal); + LessThanEqual lessThanEqual = new LessThanEqual(slotRef, literal); + + org.apache.iceberg.expressions.Expression equalResult = IcebergNereidsUtils + .convertNereidsToIcebergExpression(equalTo, testSchema); + org.apache.iceberg.expressions.Expression greaterThanResult = IcebergNereidsUtils + .convertNereidsToIcebergExpression(greaterThan, testSchema); + org.apache.iceberg.expressions.Expression greaterThanEqualResult = IcebergNereidsUtils + .convertNereidsToIcebergExpression(greaterThanEqual, testSchema); + org.apache.iceberg.expressions.Expression lessThanResult = IcebergNereidsUtils + .convertNereidsToIcebergExpression(lessThan, testSchema); + org.apache.iceberg.expressions.Expression lessThanEqualResult = IcebergNereidsUtils + .convertNereidsToIcebergExpression(lessThanEqual, testSchema); + + Assertions.assertNotNull(equalResult); + Assertions.assertNotNull(greaterThanResult); + Assertions.assertNotNull(greaterThanEqualResult); + Assertions.assertNotNull(lessThanResult); + Assertions.assertNotNull(lessThanEqualResult); + + String eq = equalResult.toString().toLowerCase(); + String gt = greaterThanResult.toString().toLowerCase(); + String gte = greaterThanEqualResult.toString().toLowerCase(); + String lt = lessThanResult.toString().toLowerCase(); + String lte = lessThanEqualResult.toString().toLowerCase(); + Assertions.assertTrue(eq.contains("age") && eq.contains("25")); + Assertions.assertTrue(gt.contains("age") && gt.contains(">") && gt.contains("25")); + Assertions.assertTrue(gte.contains("age") && gte.contains(">=") && gte.contains("25")); + Assertions.assertTrue(lt.contains("age") && lt.contains("<") && lt.contains("25")); + Assertions.assertTrue(lte.contains("age") && lte.contains("<=") && lte.contains("25")); + } + + @Test + public void testConvertNereidsToIcebergExpression_ComplexInPredicate() throws UserException { + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + List literals = Arrays.asList( + new IntegerLiteral(1), + new IntegerLiteral(2), + new IntegerLiteral(3), + new IntegerLiteral(4), + new IntegerLiteral(5)); + + InPredicate inPredicate = new InPredicate(slotRef, literals); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(inPredicate, testSchema); + + Assertions.assertNotNull(result); + String s = result.toString(); + Assertions.assertTrue(s.contains("id")); + Assertions.assertTrue(s.contains("1")); + Assertions.assertTrue(s.contains("2")); + Assertions.assertTrue(s.contains("3")); + Assertions.assertTrue(s.contains("4")); + Assertions.assertTrue(s.contains("5")); + } + + @Test + public void testConvertNereidsToIcebergExpression_StringInPredicate() throws UserException { + SlotReference slotRef = new SlotReference("name", StringType.INSTANCE, false); + List literals = Arrays.asList( + new StringLiteral("Alice"), + new StringLiteral("Bob"), + new StringLiteral("Charlie")); + + InPredicate inPredicate = new InPredicate(slotRef, literals); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(inPredicate, testSchema); + + Assertions.assertNotNull(result); + String s = result.toString(); + Assertions.assertTrue(s.contains("name")); + Assertions.assertTrue(s.contains("Alice")); + Assertions.assertTrue(s.contains("Bob")); + Assertions.assertTrue(s.contains("Charlie")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BooleanInPredicate() throws UserException { + SlotReference slotRef = new SlotReference("is_active", BooleanType.INSTANCE, false); + List literals = Arrays.asList( + BooleanLiteral.of(true), + BooleanLiteral.of(false)); + + InPredicate inPredicate = new InPredicate(slotRef, literals); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(inPredicate, testSchema); + + Assertions.assertNotNull(result); + String s = result.toString().toLowerCase(); + Assertions.assertTrue(s.contains("is_active")); + Assertions.assertTrue(s.contains("true")); + Assertions.assertTrue(s.contains("false")); + } + + @Test + public void testConvertNereidsToIcebergExpression_AllLogicalOperators() throws UserException { + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(100); + EqualTo equalTo = new EqualTo(slotRef, literal); + + // Test all logical operators + And andExpr = new And(equalTo, equalTo); + Or orExpr = new Or(equalTo, equalTo); + Not notExpr = new Not(equalTo); + + org.apache.iceberg.expressions.Expression andResult = IcebergNereidsUtils + .convertNereidsToIcebergExpression(andExpr, testSchema); + org.apache.iceberg.expressions.Expression orResult = IcebergNereidsUtils + .convertNereidsToIcebergExpression(orExpr, testSchema); + org.apache.iceberg.expressions.Expression notResult = IcebergNereidsUtils + .convertNereidsToIcebergExpression(notExpr, testSchema); + + Assertions.assertNotNull(andResult); + Assertions.assertNotNull(orResult); + Assertions.assertNotNull(notResult); + + String andStr = andResult.toString().toLowerCase(); + String orStr = orResult.toString().toLowerCase(); + String notStr = notResult.toString().toLowerCase(); + Assertions.assertTrue(andStr.contains("and")); + Assertions.assertTrue(orStr.contains("or")); + Assertions.assertTrue(notStr.contains("not")); + } + + @Test + public void testConvertNereidsToIcebergExpression_EmptySchema() { + // Test with empty schema + Schema emptySchema = new Schema(); + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(100); + EqualTo equalTo = new EqualTo(slotRef, literal); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(equalTo, emptySchema); + }); + Assertions.assertEquals("Column not found in Iceberg schema: id", exception.getDetailMessage()); + } + + @Test + public void testConvertNereidsToIcebergExpression_AllSupportedExpressionTypes() throws UserException { + // Test all supported expression types in one comprehensive test + SlotReference slotRef = new SlotReference("id", IntegerType.INSTANCE, false); + IntegerLiteral literal = new IntegerLiteral(100); + IntegerLiteral lowerBound = new IntegerLiteral(50); + IntegerLiteral upperBound = new IntegerLiteral(150); + + // Test all supported expressions + EqualTo equalTo = new EqualTo(slotRef, literal); + GreaterThan greaterThan = new GreaterThan(slotRef, literal); + GreaterThanEqual greaterThanEqual = new GreaterThanEqual(slotRef, literal); + LessThan lessThan = new LessThan(slotRef, literal); + LessThanEqual lessThanEqual = new LessThanEqual(slotRef, literal); + InPredicate inPredicate = new InPredicate(slotRef, Arrays.asList(literal)); + Between between = new Between(slotRef, lowerBound, upperBound); + And andExpr = new And(equalTo, greaterThan); + Or orExpr = new Or(equalTo, greaterThan); + Not notExpr = new Not(equalTo); + + // All should convert successfully + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(equalTo, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(greaterThan, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(greaterThanEqual, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(lessThan, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(lessThanEqual, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(inPredicate, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(between, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(andExpr, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(orExpr, testSchema)); + Assertions.assertNotNull(IcebergNereidsUtils.convertNereidsToIcebergExpression(notExpr, testSchema)); + } + + @Test + public void testConvertNereidsToIcebergExpression_Between() throws UserException { + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + IntegerLiteral lowerBound = new IntegerLiteral(18); + IntegerLiteral upperBound = new IntegerLiteral(65); + Between between = new Between(slotRef, lowerBound, upperBound); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(between, testSchema); + + Assertions.assertNotNull(result); + String resultStr = result.toString().toLowerCase(); + Assertions.assertTrue(resultStr.contains("age")); + Assertions.assertTrue(resultStr.contains("18")); + Assertions.assertTrue(resultStr.contains("65")); + Assertions.assertTrue(resultStr.contains("and")); + // Verify it's equivalent to: age >= 18 AND age <= 65 + Assertions.assertTrue(resultStr.contains(">=") || resultStr.contains("greaterthanequal")); + Assertions.assertTrue(resultStr.contains("<=") || resultStr.contains("lessthanequal")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenWithUnboundSlot() throws UserException { + UnboundSlot unboundSlot = new UnboundSlot("age"); + IntegerLiteral lowerBound = new IntegerLiteral(18); + IntegerLiteral upperBound = new IntegerLiteral(65); + Between between = new Between(unboundSlot, lowerBound, upperBound); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(between, testSchema); + + Assertions.assertNotNull(result); + String resultStr = result.toString().toLowerCase(); + Assertions.assertTrue(resultStr.contains("age")); + Assertions.assertTrue(resultStr.contains("18")); + Assertions.assertTrue(resultStr.contains("65")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenWithDouble() throws UserException { + SlotReference slotRef = new SlotReference("salary", DoubleType.INSTANCE, false); + DoubleLiteral lowerBound = new DoubleLiteral(10000.0); + DoubleLiteral upperBound = new DoubleLiteral(100000.0); + Between between = new Between(slotRef, lowerBound, upperBound); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(between, testSchema); + + Assertions.assertNotNull(result); + String resultStr = result.toString().toLowerCase(); + Assertions.assertTrue(resultStr.contains("salary")); + Assertions.assertTrue(resultStr.contains("10000")); + Assertions.assertTrue(resultStr.contains("100000")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenWithString() throws UserException { + SlotReference slotRef = new SlotReference("name", StringType.INSTANCE, false); + StringLiteral lowerBound = new StringLiteral("Alice"); + StringLiteral upperBound = new StringLiteral("Charlie"); + Between between = new Between(slotRef, lowerBound, upperBound); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(between, testSchema); + + Assertions.assertNotNull(result); + String resultStr = result.toString(); + Assertions.assertTrue(resultStr.contains("name")); + Assertions.assertTrue(resultStr.contains("Alice")); + Assertions.assertTrue(resultStr.contains("Charlie")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenWithDate() throws UserException { + SlotReference slotRef = new SlotReference("birth_date", DateType.INSTANCE, false); + DateLiteral lowerBound = new DateLiteral("2000-01-01"); + DateLiteral upperBound = new DateLiteral("2010-12-31"); + Between between = new Between(slotRef, lowerBound, upperBound); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(between, testSchema); + + Assertions.assertNotNull(result); + String resultStr = result.toString(); + Assertions.assertTrue(resultStr.contains("birth_date")); + Assertions.assertTrue(resultStr.contains("2000-01-01")); + Assertions.assertTrue(resultStr.contains("2010-12-31")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenInvalidCompareExpr() { + // Test with non-slot compareExpr + IntegerLiteral compareExpr = new IntegerLiteral(100); + IntegerLiteral lowerBound = new IntegerLiteral(18); + IntegerLiteral upperBound = new IntegerLiteral(65); + Between between = new Between(compareExpr, lowerBound, upperBound); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(between, testSchema); + }); + Assertions.assertTrue(exception.getDetailMessage().contains("must be a slot")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenInvalidLowerBound() { + // Test with non-literal lowerBound + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + SlotReference lowerBound = new SlotReference("min_age", IntegerType.INSTANCE, false); + IntegerLiteral upperBound = new IntegerLiteral(65); + Between between = new Between(slotRef, lowerBound, upperBound); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(between, testSchema); + }); + Assertions.assertTrue(exception.getDetailMessage().contains("Lower bound") + && exception.getDetailMessage().contains("must be a literal")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenInvalidUpperBound() { + // Test with non-literal upperBound + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + IntegerLiteral lowerBound = new IntegerLiteral(18); + SlotReference upperBound = new SlotReference("max_age", IntegerType.INSTANCE, false); + Between between = new Between(slotRef, lowerBound, upperBound); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(between, testSchema); + }); + Assertions.assertTrue(exception.getDetailMessage().contains("Upper bound") + && exception.getDetailMessage().contains("must be a literal")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenColumnNotFound() { + SlotReference slotRef = new SlotReference("non_existent_column", IntegerType.INSTANCE, false); + IntegerLiteral lowerBound = new IntegerLiteral(18); + IntegerLiteral upperBound = new IntegerLiteral(65); + Between between = new Between(slotRef, lowerBound, upperBound); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(between, testSchema); + }); + Assertions.assertEquals("Column not found in Iceberg schema: non_existent_column", + exception.getDetailMessage()); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenWithNullBounds() { + SlotReference slotRef = new SlotReference("age", IntegerType.INSTANCE, false); + NullLiteral nullLiteral = new NullLiteral(); + IntegerLiteral upperBound = new IntegerLiteral(65); + Between between = new Between(slotRef, nullLiteral, upperBound); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(between, testSchema); + }); + Assertions.assertTrue(exception.getDetailMessage().contains("cannot be null")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenInComplexExpression() throws UserException { + // Test BETWEEN in AND expression: age BETWEEN 18 AND 65 AND salary > 50000 + SlotReference ageRef = new SlotReference("age", IntegerType.INSTANCE, false); + SlotReference salaryRef = new SlotReference("salary", DoubleType.INSTANCE, false); + + Between between = new Between(ageRef, new IntegerLiteral(18), new IntegerLiteral(65)); + GreaterThan salaryGt = new GreaterThan(salaryRef, new DoubleLiteral(50000.0)); + And andExpr = new And(between, salaryGt); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(andExpr, testSchema); + + Assertions.assertNotNull(result); + String resultStr = result.toString().toLowerCase(); + Assertions.assertTrue(resultStr.contains("age")); + Assertions.assertTrue(resultStr.contains("salary")); + Assertions.assertTrue(resultStr.contains("and")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenWithUnboundSlotInvalidNameParts() { + // Test UnboundSlot with multiple nameParts (should fail) + UnboundSlot unboundSlot = new UnboundSlot("table", "age"); + IntegerLiteral lowerBound = new IntegerLiteral(18); + IntegerLiteral upperBound = new IntegerLiteral(65); + Between between = new Between(unboundSlot, lowerBound, upperBound); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + IcebergNereidsUtils.convertNereidsToIcebergExpression(between, testSchema); + }); + Assertions.assertTrue(exception.getDetailMessage().contains("single name part")); + } + + @Test + public void testConvertNereidsToIcebergExpression_BetweenNestedInOr() throws UserException { + // Test: (age BETWEEN 18 AND 30) OR (age BETWEEN 50 AND 65) + SlotReference ageRef = new SlotReference("age", IntegerType.INSTANCE, false); + + Between between1 = new Between(ageRef, new IntegerLiteral(18), new IntegerLiteral(30)); + Between between2 = new Between(ageRef, new IntegerLiteral(50), new IntegerLiteral(65)); + Or orExpr = new Or(between1, between2); + + org.apache.iceberg.expressions.Expression result = IcebergNereidsUtils + .convertNereidsToIcebergExpression(orExpr, testSchema); + + Assertions.assertNotNull(result); + String resultStr = result.toString().toLowerCase(); + Assertions.assertTrue(resultStr.contains("age")); + Assertions.assertTrue(resultStr.contains("or")); + Assertions.assertTrue(resultStr.contains("18")); + Assertions.assertTrue(resultStr.contains("30")); + Assertions.assertTrue(resultStr.contains("50")); + Assertions.assertTrue(resultStr.contains("65")); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFilePlannerTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFilePlannerTest.java new file mode 100644 index 00000000000000..22fa117c8073a2 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/iceberg/rewrite/RewriteDataFilePlannerTest.java @@ -0,0 +1,1159 @@ +// 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.doris.datasource.iceberg.rewrite; + +import org.apache.doris.common.UserException; + +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +/** + * Unit tests for RewriteDataFilePlanner + */ +public class RewriteDataFilePlannerTest { + + @Mock + private Table mockTable; + + @Mock + private TableScan mockTableScan; + + @Mock + private Schema mockSchema; + + @Mock + private PartitionSpec mockPartitionSpec; + + @Mock + private DataFile mockDataFile; + + @Mock + private DeleteFile mockDeleteFile; + + @Mock + private FileScanTask mockFileScanTask; + + @Mock + private StructLike mockPartition; + + private RewriteDataFilePlanner.Parameters defaultParameters; + private RewriteDataFilePlanner planner; + + @BeforeEach + public void setUp() { + MockitoAnnotations.openMocks(this); + // Create default parameters for testing + defaultParameters = new RewriteDataFilePlanner.Parameters( + 128 * 1024 * 1024L, // targetFileSizeBytes: 128MB + 64 * 1024 * 1024L, // minFileSizeBytes: 64MB + 256 * 1024 * 1024L, // maxFileSizeBytes: 256MB + 2, // minInputFiles + false, // rewriteAll + 512 * 1024 * 1024L, // maxFileGroupSizeBytes: 512MB + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + planner = new RewriteDataFilePlanner(defaultParameters); + } + + @Test + public void testParametersGetters() { + Assertions.assertEquals(128 * 1024 * 1024L, defaultParameters.getTargetFileSizeBytes()); + Assertions.assertEquals(64 * 1024 * 1024L, defaultParameters.getMinFileSizeBytes()); + Assertions.assertEquals(256 * 1024 * 1024L, defaultParameters.getMaxFileSizeBytes()); + Assertions.assertEquals(2, defaultParameters.getMinInputFiles()); + Assertions.assertFalse(defaultParameters.isRewriteAll()); + Assertions.assertEquals(512 * 1024 * 1024L, defaultParameters.getMaxFileGroupSizeBytes()); + Assertions.assertEquals(3, defaultParameters.getDeleteFileThreshold()); + Assertions.assertEquals(0.1, defaultParameters.getDeleteRatioThreshold(), 0.001); + Assertions.assertFalse(defaultParameters.hasWhereCondition()); + } + + @Test + public void testParametersToString() { + String toString = defaultParameters.toString(); + Assertions.assertTrue(toString.contains("targetFileSizeBytes=134217728")); + Assertions.assertTrue(toString.contains("minFileSizeBytes=67108864")); + Assertions.assertTrue(toString.contains("maxFileSizeBytes=268435456")); + Assertions.assertTrue(toString.contains("minInputFiles=2")); + Assertions.assertTrue(toString.contains("rewriteAll=false")); + Assertions.assertTrue(toString.contains("hasWhereCondition=false")); + } + + @Test + public void testPlanAndOrganizeTasksWithRewriteAll() throws UserException { + // Test with rewriteAll = true + RewriteDataFilePlanner.Parameters rewriteAllParams = new RewriteDataFilePlanner.Parameters( + 128 * 1024 * 1024L, 64 * 1024 * 1024L, 256 * 1024 * 1024L, + 2, true, 512 * 1024 * 1024L, 3, 0.1, 1L, + Optional.empty()); + + RewriteDataFilePlanner rewriteAllPlanner = new RewriteDataFilePlanner(rewriteAllParams); + + // Mock table scan + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + + // Mock file scan task + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockFileScanTask.deletes()).thenReturn(null); + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = rewriteAllPlanner.planAndOrganizeTasks(mockTable); + + Assertions.assertNotNull(result); + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals(1, result.get(0).getTaskCount()); + Assertions.assertEquals(100 * 1024 * 1024L, result.get(0).getTotalSize()); + } + + @Test + public void testFileSizeFiltering() throws UserException { + // Test file size filtering logic + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockFileScanTask.deletes()).thenReturn(null); + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + // Test file too small (should be selected for rewrite but filtered by group size - single file) + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(32 * 1024 * 1024L); // 32MB < 64MB min + + List result = planner.planAndOrganizeTasks(mockTable); + // Single file groups are filtered unless they meet tooMuchContent threshold + Assertions.assertTrue(result.isEmpty(), "Single small file should be filtered by group rules"); + + // Test file too large (should be selected for rewrite but filtered by group size - single file) + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(300 * 1024 * 1024L); // 300MB > 256MB max + + result = planner.planAndOrganizeTasks(mockTable); + // Single file groups are filtered unless they meet tooMuchContent threshold + Assertions.assertTrue(result.isEmpty(), "Single large file should be filtered by group rules"); + + // Test file in acceptable range (should be filtered out as it doesn't need rewriting) + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); // 100MB between 64MB-256MB + + result = planner.planAndOrganizeTasks(mockTable); + // File in acceptable range with no deletes doesn't need rewriting + Assertions.assertTrue(result.isEmpty(), "File in acceptable range should not be rewritten"); + } + + @Test + public void testDeleteFileThreshold() throws UserException { + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + // Test with delete files below threshold (should be filtered out) + Mockito.when(mockFileScanTask.deletes()).thenReturn(Arrays.asList(mockDeleteFile, mockDeleteFile)); // 2 < 3 threshold + + List result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertTrue(result.isEmpty(), "File with 2 delete files (< 3 threshold) should not be selected"); + + // Test with delete files at threshold (should be included) + Mockito.when(mockFileScanTask.deletes()).thenReturn(Arrays.asList(mockDeleteFile, mockDeleteFile, mockDeleteFile)); // 3 = 3 threshold + + result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertEquals(1, result.size(), "Should have exactly 1 group"); + Assertions.assertEquals(1, result.get(0).getTaskCount(), "Group should contain 1 task"); + Assertions.assertEquals(100 * 1024 * 1024L, result.get(0).getTotalSize(), "Group size should be 100MB"); + } + + @Test + public void testDeleteRatioThreshold() throws UserException { + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + // Mock delete file with record count + Mockito.when(mockDeleteFile.recordCount()).thenReturn(5L); + Mockito.when(mockDataFile.recordCount()).thenReturn(100L); // 5/100 = 5% < 10% threshold + + Mockito.when(mockFileScanTask.deletes()).thenReturn(Collections.singletonList(mockDeleteFile)); + + List result = planner.planAndOrganizeTasks(mockTable); + // Low delete ratio + single file = filtered out + Assertions.assertTrue(result.isEmpty(), "File with 5% delete ratio (< 10% threshold) should not be selected"); + + // Test with high delete ratio (should be included) + Mockito.when(mockDeleteFile.recordCount()).thenReturn(15L); // 15/100 = 15% > 10% threshold + + result = planner.planAndOrganizeTasks(mockTable); + // Note: delete ratio calculation requires ContentFileUtil.isFileScoped to return true + // which cannot be easily mocked. Single file groups are also filtered out. + Assertions.assertTrue(result.isEmpty(), "Single file group filtered even with high delete ratio (ContentFileUtil limitation)"); + } + + @Test + public void testDeleteRatioWithZeroRecordCount() throws UserException { + // Test that recordCount == 0 doesn't cause division by zero + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + // Mock delete file with record count = 0 (should not cause division by zero) + Mockito.when(mockDeleteFile.recordCount()).thenReturn(10L); + Mockito.when(mockDataFile.recordCount()).thenReturn(0L); // Zero record count + + Mockito.when(mockFileScanTask.deletes()).thenReturn(Collections.singletonList(mockDeleteFile)); + + List result = planner.planAndOrganizeTasks(mockTable); + // File with zero record count should not be selected (should return false without throwing exception) + Assertions.assertTrue(result.isEmpty(), "File with zero record count should not be selected"); + } + + @Test + public void testGroupFilteringByInputFiles() throws UserException { + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockFileScanTask.deletes()).thenReturn(null); + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + // Single file group (1 < 2 minInputFiles) should be filtered out + List result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertTrue(result.isEmpty(), "Single file group with taskCount=1 < minInputFiles=2 should be filtered"); + } + + @Test + public void testGroupFilteringByContentSize() throws UserException { + // Create parameters with lower target file size for testing + RewriteDataFilePlanner.Parameters params = new RewriteDataFilePlanner.Parameters( + 50 * 1024 * 1024L, // targetFileSizeBytes: 50MB + 64 * 1024 * 1024L, // minFileSizeBytes: 64MB + 256 * 1024 * 1024L, // maxFileSizeBytes: 256MB + 1, // minInputFiles + false, // rewriteAll + 512 * 1024 * 1024L, // maxFileGroupSizeBytes: 512MB + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + RewriteDataFilePlanner testPlanner = new RewriteDataFilePlanner(params); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockFileScanTask.deletes()).thenReturn(null); + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); // 100MB > 50MB target + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = testPlanner.planAndOrganizeTasks(mockTable); + // Single file in acceptable range doesn't need rewriting + // enoughContent requires taskCount > 1 + Assertions.assertTrue(result.isEmpty()); + } + + @Test + public void testGroupFilteringByMaxFileGroupSize() throws UserException { + // Create parameters with very small max file group size + RewriteDataFilePlanner.Parameters params = new RewriteDataFilePlanner.Parameters( + 128 * 1024 * 1024L, // targetFileSizeBytes: 128MB + 64 * 1024 * 1024L, // minFileSizeBytes: 64MB + 256 * 1024 * 1024L, // maxFileSizeBytes: 256MB + 1, // minInputFiles + false, // rewriteAll + 50 * 1024 * 1024L, // maxFileGroupSizeBytes: 50MB (very small) + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + RewriteDataFilePlanner testPlanner = new RewriteDataFilePlanner(params); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockFileScanTask.deletes()).thenReturn(null); + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); // 100MB > 50MB max group size + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = testPlanner.planAndOrganizeTasks(mockTable); + // File in acceptable range (64-256MB), so not selected by filterFiles + // Even though 100MB > 50MB maxFileGroupSize, the file is filtered before grouping + Assertions.assertTrue(result.isEmpty()); + } + + @Test + public void testPartitionGrouping() throws UserException { + // Create two file scan tasks with different partitions + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + DataFile dataFile1 = Mockito.mock(DataFile.class); + DataFile dataFile2 = Mockito.mock(DataFile.class); + StructLike partition1 = Mockito.mock(StructLike.class); + StructLike partition2 = Mockito.mock(StructLike.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose(Arrays.asList(task1, task2))); + + // Task 1 + Mockito.when(task1.file()).thenReturn(dataFile1); + Mockito.when(task1.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task1.deletes()).thenReturn(null); + Mockito.when(dataFile1.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile1.partition()).thenReturn(partition1); + + // Task 2 + Mockito.when(task2.file()).thenReturn(dataFile2); + Mockito.when(task2.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task2.deletes()).thenReturn(null); + Mockito.when(dataFile2.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile2.partition()).thenReturn(partition2); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + // Use rewriteAll to avoid filtering + RewriteDataFilePlanner.Parameters rewriteAllParams = new RewriteDataFilePlanner.Parameters( + 128 * 1024 * 1024L, 64 * 1024 * 1024L, 256 * 1024 * 1024L, + 1, true, 512 * 1024 * 1024L, 3, 0.1, 1L, + Optional.empty()); + + RewriteDataFilePlanner rewriteAllPlanner = new RewriteDataFilePlanner(rewriteAllParams); + List result = rewriteAllPlanner.planAndOrganizeTasks(mockTable); + + // With rewriteAll=true, all files are included + // StructLikeWrapper groups by partition, but since we can't mock partition equality, + // we just verify that groups are created + Assertions.assertFalse(result.isEmpty()); + Assertions.assertTrue(result.size() >= 1 && result.size() <= 2); + } + + @Test + public void testExceptionHandling() { + Mockito.when(mockTable.newScan()).thenThrow(new RuntimeException("Table scan failed")); + + UserException exception = Assertions.assertThrows(UserException.class, () -> { + planner.planAndOrganizeTasks(mockTable); + }); + + Assertions.assertTrue(exception.getMessage().contains("Failed to plan file scan tasks")); + Assertions.assertTrue(exception.getCause() instanceof RuntimeException); + Assertions.assertEquals("Table scan failed", exception.getCause().getMessage()); + } + + @Test + public void testEmptyTableScan() throws UserException { + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose(Collections.emptyList())); + + List result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertTrue(result.isEmpty()); + } + + @Test + public void testRewriteDataGroupOperations() { + RewriteDataGroup group = new RewriteDataGroup(); + + Assertions.assertTrue(group.isEmpty()); + Assertions.assertEquals(0, group.getTaskCount()); + Assertions.assertEquals(0, group.getTotalSize()); + Assertions.assertTrue(group.getDataFiles().isEmpty()); + + // Add a task + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + + group.addTask(mockFileScanTask); + + Assertions.assertFalse(group.isEmpty()); + Assertions.assertEquals(1, group.getTaskCount()); + Assertions.assertEquals(100 * 1024 * 1024L, group.getTotalSize()); + Assertions.assertEquals(1, group.getDataFiles().size()); + Assertions.assertTrue(group.getDataFiles().contains(mockDataFile)); + } + + @Test + public void testRewriteDataGroupConstructorWithTasks() { + // Test constructor that takes a list of tasks + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + DataFile dataFile1 = Mockito.mock(DataFile.class); + DataFile dataFile2 = Mockito.mock(DataFile.class); + DeleteFile deleteFile1 = Mockito.mock(DeleteFile.class); + DeleteFile deleteFile2 = Mockito.mock(DeleteFile.class); + + Mockito.when(task1.file()).thenReturn(dataFile1); + Mockito.when(task1.deletes()).thenReturn(Arrays.asList(deleteFile1)); + Mockito.when(dataFile1.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + + Mockito.when(task2.file()).thenReturn(dataFile2); + Mockito.when(task2.deletes()).thenReturn(Arrays.asList(deleteFile2)); + Mockito.when(dataFile2.fileSizeInBytes()).thenReturn(200 * 1024 * 1024L); + + List tasks = Arrays.asList(task1, task2); + RewriteDataGroup group = new RewriteDataGroup(tasks); + + Assertions.assertFalse(group.isEmpty()); + Assertions.assertEquals(2, group.getTaskCount()); + Assertions.assertEquals(300 * 1024 * 1024L, group.getTotalSize()); // 100MB + 200MB + Assertions.assertEquals(2, group.getDeleteFileCount()); // 1 + 1 + Assertions.assertEquals(2, group.getDataFiles().size()); + Assertions.assertTrue(group.getDataFiles().contains(dataFile1)); + Assertions.assertTrue(group.getDataFiles().contains(dataFile2)); + } + + @Test + public void testParametersEdgeCases() { + // Test with zero values + RewriteDataFilePlanner.Parameters zeroParams = new RewriteDataFilePlanner.Parameters( + 0L, 0L, 0L, 0, true, 0L, 0, 0.0, 0L, + Optional.empty()); + + Assertions.assertEquals(0L, zeroParams.getTargetFileSizeBytes()); + Assertions.assertEquals(0L, zeroParams.getMinFileSizeBytes()); + Assertions.assertEquals(0L, zeroParams.getMaxFileSizeBytes()); + Assertions.assertEquals(0, zeroParams.getMinInputFiles()); + Assertions.assertTrue(zeroParams.isRewriteAll()); + Assertions.assertEquals(0L, zeroParams.getMaxFileGroupSizeBytes()); + Assertions.assertEquals(0, zeroParams.getDeleteFileThreshold()); + Assertions.assertEquals(0.0, zeroParams.getDeleteRatioThreshold(), 0.001); + } + + @Test + public void testFileSizeFilteringWithMultipleFiles() throws UserException { + // Create multiple file scan tasks with different sizes + FileScanTask smallFile = Mockito.mock(FileScanTask.class); + FileScanTask mediumFile = Mockito.mock(FileScanTask.class); + FileScanTask largeFile = Mockito.mock(FileScanTask.class); + DataFile smallDataFile = Mockito.mock(DataFile.class); + DataFile mediumDataFile = Mockito.mock(DataFile.class); + DataFile largeDataFile = Mockito.mock(DataFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Arrays.asList(smallFile, mediumFile, largeFile))); + + // Small file (should be filtered out) + Mockito.when(smallFile.file()).thenReturn(smallDataFile); + Mockito.when(smallFile.spec()).thenReturn(mockPartitionSpec); + Mockito.when(smallFile.deletes()).thenReturn(null); + Mockito.when(smallDataFile.fileSizeInBytes()).thenReturn(32 * 1024 * 1024L); // 32MB < 64MB min + Mockito.when(smallDataFile.partition()).thenReturn(mockPartition); + + // Medium file (should be included) + Mockito.when(mediumFile.file()).thenReturn(mediumDataFile); + Mockito.when(mediumFile.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mediumFile.deletes()).thenReturn(null); + Mockito.when(mediumDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); // 100MB between 64MB-256MB + Mockito.when(mediumDataFile.partition()).thenReturn(mockPartition); + + // Large file (should be filtered out) + Mockito.when(largeFile.file()).thenReturn(largeDataFile); + Mockito.when(largeFile.spec()).thenReturn(mockPartitionSpec); + Mockito.when(largeFile.deletes()).thenReturn(null); + Mockito.when(largeDataFile.fileSizeInBytes()).thenReturn(300 * 1024 * 1024L); // 300MB > 256MB max + Mockito.when(largeDataFile.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = planner.planAndOrganizeTasks(mockTable); + + // Small file (32MB) and large file (300MB) are outside range, but single files are filtered by group rules + // Medium file (100MB) is in range, so not selected for rewrite + // With minInputFiles=2, we need at least 2 files. We have 2 files that need rewriting (small+large) + // but they form a group of 2 files which meets the minInputFiles requirement + Assertions.assertTrue(result.size() >= 1); + if (!result.isEmpty()) { + Assertions.assertEquals(2, result.get(0).getTaskCount()); + } + } + + @Test + public void testDeleteFileThresholdWithMultipleFiles() throws UserException { + // Create multiple file scan tasks with different delete file counts + FileScanTask lowDeletes = Mockito.mock(FileScanTask.class); + FileScanTask highDeletes = Mockito.mock(FileScanTask.class); + DataFile dataFile1 = Mockito.mock(DataFile.class); + DataFile dataFile2 = Mockito.mock(DataFile.class); + DeleteFile deleteFile1 = Mockito.mock(DeleteFile.class); + DeleteFile deleteFile2 = Mockito.mock(DeleteFile.class); + DeleteFile deleteFile3 = Mockito.mock(DeleteFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Arrays.asList(lowDeletes, highDeletes))); + + // File with low delete count (should be filtered out) + Mockito.when(lowDeletes.file()).thenReturn(dataFile1); + Mockito.when(lowDeletes.spec()).thenReturn(mockPartitionSpec); + Mockito.when(lowDeletes.deletes()).thenReturn(Arrays.asList(deleteFile1, deleteFile2)); // 2 < 3 threshold + Mockito.when(dataFile1.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile1.partition()).thenReturn(mockPartition); + + // File with high delete count (should be included) + Mockito.when(highDeletes.file()).thenReturn(dataFile2); + Mockito.when(highDeletes.spec()).thenReturn(mockPartitionSpec); + Mockito.when(highDeletes.deletes()).thenReturn(Arrays.asList(deleteFile1, deleteFile2, deleteFile3)); // 3 = 3 threshold + Mockito.when(dataFile2.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile2.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = planner.planAndOrganizeTasks(mockTable); + + // Only high delete count file should be included (low delete file filtered by filterFiles) + Assertions.assertEquals(1, result.size(), "Should have exactly 1 group"); + Assertions.assertEquals(1, result.get(0).getTaskCount(), "Group should contain only 1 file (high delete count)"); + Assertions.assertEquals(100 * 1024 * 1024L, result.get(0).getTotalSize(), "Total size should be 100MB"); + Assertions.assertTrue(result.get(0).getDataFiles().contains(dataFile2), "Should contain the high-delete file"); + Assertions.assertFalse(result.get(0).getDataFiles().contains(dataFile1), "Should NOT contain the low-delete file"); + } + + @Test + public void testDeleteRatioThresholdWithMultipleFiles() throws UserException { + // Create multiple file scan tasks with different delete ratios + FileScanTask lowRatio = Mockito.mock(FileScanTask.class); + FileScanTask highRatio = Mockito.mock(FileScanTask.class); + DataFile dataFile1 = Mockito.mock(DataFile.class); + DataFile dataFile2 = Mockito.mock(DataFile.class); + DeleteFile deleteFile1 = Mockito.mock(DeleteFile.class); + DeleteFile deleteFile2 = Mockito.mock(DeleteFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose(Arrays.asList(lowRatio, highRatio))); + + // File with low delete ratio (should be filtered out) + Mockito.when(lowRatio.file()).thenReturn(dataFile1); + Mockito.when(lowRatio.spec()).thenReturn(mockPartitionSpec); + Mockito.when(lowRatio.deletes()).thenReturn(Collections.singletonList(deleteFile1)); + Mockito.when(dataFile1.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile1.partition()).thenReturn(mockPartition); + Mockito.when(deleteFile1.recordCount()).thenReturn(5L); + Mockito.when(dataFile1.recordCount()).thenReturn(100L); // 5/100 = 5% < 10% threshold + + // File with high delete ratio (should be included) + Mockito.when(highRatio.file()).thenReturn(dataFile2); + Mockito.when(highRatio.spec()).thenReturn(mockPartitionSpec); + Mockito.when(highRatio.deletes()).thenReturn(Collections.singletonList(deleteFile2)); + Mockito.when(dataFile2.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile2.partition()).thenReturn(mockPartition); + Mockito.when(deleteFile2.recordCount()).thenReturn(15L); + Mockito.when(dataFile2.recordCount()).thenReturn(100L); // 15/100 = 15% > 10% threshold + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = planner.planAndOrganizeTasks(mockTable); + + // Delete ratio calculation requires ContentFileUtil.isFileScoped which cannot be easily mocked + // Single file groups are also filtered out by group rules + // So result should be empty or have limited entries + Assertions.assertTrue(result.size() <= 1, "Result should have at most 1 group (limited by ContentFileUtil and group rules)"); + if (!result.isEmpty()) { + Assertions.assertTrue(result.get(0).getTaskCount() <= 2, "If group exists, should have at most 2 tasks"); + } + } + + @Test + public void testGroupFilteringWithMultipleFiles() throws UserException { + // Create parameters that require multiple files for grouping + RewriteDataFilePlanner.Parameters params = new RewriteDataFilePlanner.Parameters( + 128 * 1024 * 1024L, // targetFileSizeBytes: 128MB + 64 * 1024 * 1024L, // minFileSizeBytes: 64MB + 256 * 1024 * 1024L, // maxFileSizeBytes: 256MB + 3, // minInputFiles: 3 + false, // rewriteAll + 512 * 1024 * 1024L, // maxFileGroupSizeBytes: 512MB + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + RewriteDataFilePlanner testPlanner = new RewriteDataFilePlanner(params); + + // Create multiple file scan tasks + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + FileScanTask task3 = Mockito.mock(FileScanTask.class); + DataFile dataFile1 = Mockito.mock(DataFile.class); + DataFile dataFile2 = Mockito.mock(DataFile.class); + DataFile dataFile3 = Mockito.mock(DataFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose(Arrays.asList(task1, task2, task3))); + + // All files have acceptable size and no delete issues + Mockito.when(task1.file()).thenReturn(dataFile1); + Mockito.when(task1.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task1.deletes()).thenReturn(null); + Mockito.when(dataFile1.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile1.partition()).thenReturn(mockPartition); + + Mockito.when(task2.file()).thenReturn(dataFile2); + Mockito.when(task2.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task2.deletes()).thenReturn(null); + Mockito.when(dataFile2.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile2.partition()).thenReturn(mockPartition); + + Mockito.when(task3.file()).thenReturn(dataFile3); + Mockito.when(task3.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task3.deletes()).thenReturn(null); + Mockito.when(dataFile3.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dataFile3.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = testPlanner.planAndOrganizeTasks(mockTable); + + // Files in acceptable range (64-256MB) are not selected for rewrite + // All 3 files are 100MB which is in range, so they are filtered out + Assertions.assertTrue(result.isEmpty(), "All 3 files (100MB each) are in acceptable range [64-256MB], should not be rewritten"); + } + + @Test + public void testBoundaryValueFileSizes() throws UserException { + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockFileScanTask.deletes()).thenReturn(null); + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + // Test file size exactly at minFileSizeBytes (should NOT be selected for rewrite) + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(64 * 1024 * 1024L); // Exactly 64MB + List result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertTrue(result.isEmpty(), "File at exactly minFileSizeBytes (64MB) should NOT be selected"); + + // Test file size just below minFileSizeBytes (should be selected but filtered by group rules) + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(64 * 1024 * 1024L - 1); // 64MB - 1 + result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertTrue(result.isEmpty(), "Single file at 64MB-1 selected but filtered by group rules"); + + // Test file size exactly at maxFileSizeBytes (should NOT be selected for rewrite) + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(256 * 1024 * 1024L); // Exactly 256MB + result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertTrue(result.isEmpty(), "File at exactly maxFileSizeBytes (256MB) should NOT be selected"); + + // Test file size just above maxFileSizeBytes (should be selected but filtered by group rules) + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(256 * 1024 * 1024L + 1); // 256MB + 1 + result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertTrue(result.isEmpty(), "Single file at 256MB+1 selected but filtered by group rules"); + } + + @Test + public void testEnoughContentTrigger() throws UserException { + // Create parameters with specific target size + RewriteDataFilePlanner.Parameters params = new RewriteDataFilePlanner.Parameters( + 100 * 1024 * 1024L, // targetFileSizeBytes: 100MB + 50 * 1024 * 1024L, // minFileSizeBytes: 50MB + 200 * 1024 * 1024L, // maxFileSizeBytes: 200MB + 2, // minInputFiles: 2 + false, // rewriteAll + 500 * 1024 * 1024L, // maxFileGroupSizeBytes: 500MB + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + RewriteDataFilePlanner testPlanner = new RewriteDataFilePlanner(params); + + // Create two small files that together exceed target size + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + DataFile dataFile1 = Mockito.mock(DataFile.class); + DataFile dataFile2 = Mockito.mock(DataFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose(Arrays.asList(task1, task2))); + + // Both files are too small (< 50MB min), so they will be selected for rewrite + Mockito.when(task1.file()).thenReturn(dataFile1); + Mockito.when(task1.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task1.deletes()).thenReturn(null); + Mockito.when(dataFile1.fileSizeInBytes()).thenReturn(40 * 1024 * 1024L); // 40MB < 50MB min + Mockito.when(dataFile1.partition()).thenReturn(mockPartition); + + Mockito.when(task2.file()).thenReturn(dataFile2); + Mockito.when(task2.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task2.deletes()).thenReturn(null); + Mockito.when(dataFile2.fileSizeInBytes()).thenReturn(70 * 1024 * 1024L); // 70MB > 50MB min but < 200MB max, in range! + Mockito.when(dataFile2.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = testPlanner.planAndOrganizeTasks(mockTable); + + // file1 (40MB) is selected by filterFiles (< 50MB min) + // file2 (70MB) is in range, not selected + // Only 1 file in group, doesn't meet minInputFiles requirement + Assertions.assertTrue(result.isEmpty(), "Only 1 file selected (taskCount=1 < minInputFiles=2), should be filtered"); + } + + @Test + public void testEnoughContentTriggerWithBothFilesTooSmall() throws UserException { + // Create parameters with specific target size + RewriteDataFilePlanner.Parameters params = new RewriteDataFilePlanner.Parameters( + 100 * 1024 * 1024L, // targetFileSizeBytes: 100MB + 50 * 1024 * 1024L, // minFileSizeBytes: 50MB + 200 * 1024 * 1024L, // maxFileSizeBytes: 200MB + 2, // minInputFiles: 2 + false, // rewriteAll + 500 * 1024 * 1024L, // maxFileGroupSizeBytes: 500MB + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + RewriteDataFilePlanner testPlanner = new RewriteDataFilePlanner(params); + + // Create two small files that together exceed target size + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + DataFile dataFile1 = Mockito.mock(DataFile.class); + DataFile dataFile2 = Mockito.mock(DataFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose(Arrays.asList(task1, task2))); + + // Both files are too small (< 50MB min), so they will be selected for rewrite + Mockito.when(task1.file()).thenReturn(dataFile1); + Mockito.when(task1.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task1.deletes()).thenReturn(null); + Mockito.when(dataFile1.fileSizeInBytes()).thenReturn(40 * 1024 * 1024L); // 40MB < 50MB min + Mockito.when(dataFile1.partition()).thenReturn(mockPartition); + + Mockito.when(task2.file()).thenReturn(dataFile2); + Mockito.when(task2.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task2.deletes()).thenReturn(null); + Mockito.when(dataFile2.fileSizeInBytes()).thenReturn(45 * 1024 * 1024L); // 45MB < 50MB min + Mockito.when(dataFile2.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = testPlanner.planAndOrganizeTasks(mockTable); + + // Both files are too small and selected + // Total size = 85MB < 100MB target (so enoughContent doesn't trigger) + // But taskCount = 2 >= 2 minInputFiles (so enoughInputFiles triggers) + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals(2, result.get(0).getTaskCount()); + Assertions.assertEquals(85 * 1024 * 1024L, result.get(0).getTotalSize()); + } + + @Test + public void testEnoughContentWithLargerFiles() throws UserException { + // Test specifically for enoughContent condition: taskCount > 1 && totalSize > targetSize + RewriteDataFilePlanner.Parameters params = new RewriteDataFilePlanner.Parameters( + 80 * 1024 * 1024L, // targetFileSizeBytes: 80MB + 50 * 1024 * 1024L, // minFileSizeBytes: 50MB + 200 * 1024 * 1024L, // maxFileSizeBytes: 200MB + 5, // minInputFiles: 5 (high threshold, won't be met) + false, // rewriteAll + 500 * 1024 * 1024L, // maxFileGroupSizeBytes: 500MB + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + RewriteDataFilePlanner testPlanner = new RewriteDataFilePlanner(params); + + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + DataFile dataFile1 = Mockito.mock(DataFile.class); + DataFile dataFile2 = Mockito.mock(DataFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose(Arrays.asList(task1, task2))); + + // Both files too small + Mockito.when(task1.file()).thenReturn(dataFile1); + Mockito.when(task1.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task1.deletes()).thenReturn(null); + Mockito.when(dataFile1.fileSizeInBytes()).thenReturn(45 * 1024 * 1024L); // 45MB < 50MB + Mockito.when(dataFile1.partition()).thenReturn(mockPartition); + + Mockito.when(task2.file()).thenReturn(dataFile2); + Mockito.when(task2.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task2.deletes()).thenReturn(null); + Mockito.when(dataFile2.fileSizeInBytes()).thenReturn(48 * 1024 * 1024L); // 48MB < 50MB + Mockito.when(dataFile2.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = testPlanner.planAndOrganizeTasks(mockTable); + + // taskCount = 2 < 5 minInputFiles (enoughInputFiles = false) + // totalSize = 93MB > 80MB target && taskCount = 2 > 1 (enoughContent = true!) + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals(2, result.get(0).getTaskCount()); + Assertions.assertEquals(93 * 1024 * 1024L, result.get(0).getTotalSize()); + } + + @Test + public void testTooMuchContentTrigger() throws UserException { + // Create parameters with very small maxFileGroupSizeBytes + RewriteDataFilePlanner.Parameters params = new RewriteDataFilePlanner.Parameters( + 128 * 1024 * 1024L, // targetFileSizeBytes: 128MB + 64 * 1024 * 1024L, // minFileSizeBytes: 64MB + 256 * 1024 * 1024L, // maxFileSizeBytes: 256MB + 2, // minInputFiles: 2 + false, // rewriteAll + 50 * 1024 * 1024L, // maxFileGroupSizeBytes: 50MB (very small!) + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + RewriteDataFilePlanner testPlanner = new RewriteDataFilePlanner(params); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockFileScanTask.deletes()).thenReturn(null); + // File is too large (> 256MB max), so it will be selected by filterFiles + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(300 * 1024 * 1024L); // 300MB + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = testPlanner.planAndOrganizeTasks(mockTable); + + // Even though it's a single file (normally filtered), 300MB > 50MB maxFileGroupSize + // This triggers tooMuchContent, so the file should be included + Assertions.assertEquals(1, result.size()); + Assertions.assertEquals(1, result.get(0).getTaskCount()); + Assertions.assertEquals(300 * 1024 * 1024L, result.get(0).getTotalSize()); + } + + @Test + public void testGroupWithAnyFileHavingDeletes() throws UserException { + // Test that if any file in a group has delete issues, the whole group is selected + FileScanTask cleanTask = Mockito.mock(FileScanTask.class); + FileScanTask dirtyTask = Mockito.mock(FileScanTask.class); + DataFile cleanFile = Mockito.mock(DataFile.class); + DataFile dirtyFile = Mockito.mock(DataFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose(Arrays.asList(cleanTask, dirtyTask))); + + // Clean file - no deletes, size in range + Mockito.when(cleanTask.file()).thenReturn(cleanFile); + Mockito.when(cleanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(cleanTask.deletes()).thenReturn(null); + Mockito.when(cleanFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(cleanFile.partition()).thenReturn(mockPartition); + + // Dirty file - has deletes exceeding threshold, size in range + Mockito.when(dirtyTask.file()).thenReturn(dirtyFile); + Mockito.when(dirtyTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(dirtyTask.deletes()).thenReturn(Arrays.asList(mockDeleteFile, mockDeleteFile, mockDeleteFile)); // 3 >= 3 threshold + Mockito.when(dirtyFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(dirtyFile.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = planner.planAndOrganizeTasks(mockTable); + + // The dirty file triggers the group to be selected (via shouldRewriteGroup) + // Only dirty file is selected by filterFiles (cleanFile is in acceptable range) + Assertions.assertEquals(1, result.size(), "Should have exactly 1 group"); + Assertions.assertEquals(1, result.get(0).getTaskCount(), "Group should contain 1 task (dirty file only)"); + Assertions.assertEquals(100 * 1024 * 1024L, result.get(0).getTotalSize(), "Total size should be 100MB (dirty file)"); + Assertions.assertTrue(result.get(0).getDataFiles().contains(dirtyFile), "Should contain dirty file"); + Assertions.assertFalse(result.get(0).getDataFiles().contains(cleanFile), "Should NOT contain clean file (not selected by filterFiles)"); + } + + @Test + public void testMixedScenarioWithMultiplePartitions() throws UserException { + // Create a complex scenario with multiple partitions and mixed file sizes + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + FileScanTask task3 = Mockito.mock(FileScanTask.class); + FileScanTask task4 = Mockito.mock(FileScanTask.class); + DataFile file1 = Mockito.mock(DataFile.class); + DataFile file2 = Mockito.mock(DataFile.class); + DataFile file3 = Mockito.mock(DataFile.class); + DataFile file4 = Mockito.mock(DataFile.class); + StructLike partition1 = Mockito.mock(StructLike.class); + StructLike partition2 = Mockito.mock(StructLike.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose( + Arrays.asList(task1, task2, task3, task4))); + + // Partition 1: Two small files that should be rewritten + Mockito.when(task1.file()).thenReturn(file1); + Mockito.when(task1.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task1.deletes()).thenReturn(null); + Mockito.when(file1.fileSizeInBytes()).thenReturn(30 * 1024 * 1024L); // Too small + Mockito.when(file1.partition()).thenReturn(partition1); + + Mockito.when(task2.file()).thenReturn(file2); + Mockito.when(task2.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task2.deletes()).thenReturn(null); + Mockito.when(file2.fileSizeInBytes()).thenReturn(40 * 1024 * 1024L); // Too small + Mockito.when(file2.partition()).thenReturn(partition1); + + // Partition 2: One large file + one good file + Mockito.when(task3.file()).thenReturn(file3); + Mockito.when(task3.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task3.deletes()).thenReturn(null); + Mockito.when(file3.fileSizeInBytes()).thenReturn(300 * 1024 * 1024L); // Too large + Mockito.when(file3.partition()).thenReturn(partition2); + + Mockito.when(task4.file()).thenReturn(file4); + Mockito.when(task4.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task4.deletes()).thenReturn(null); + Mockito.when(file4.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); // Good size + Mockito.when(file4.partition()).thenReturn(partition2); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = planner.planAndOrganizeTasks(mockTable); + + // Note: StructLikeWrapper may group all mocked StructLike objects together + // Because we can't properly mock partition equality, the exact grouping is unpredictable + // We can only verify that groups are created for files needing rewrite + Assertions.assertTrue(result.size() >= 1, "Should have at least 1 group"); + + // Verify total files selected for rewrite + int totalFiles = result.stream().mapToInt(RewriteDataGroup::getTaskCount).sum(); + long totalSize = result.stream().mapToLong(RewriteDataGroup::getTotalSize).sum(); + // file1 (30MB), file2 (40MB), file3 (300MB) are outside range → selected + // file4 (100MB) is in range → NOT selected + // So we expect 2-3 files (depending on grouping and single-file filtering) + Assertions.assertTrue(totalFiles >= 2, "Should have at least 2 files selected for rewrite"); + Assertions.assertTrue(totalFiles <= 3, "Should have at most 3 files selected"); + // Total size should be at least the 2 small files + Assertions.assertTrue(totalSize >= 70 * 1024 * 1024L, "Total size should be at least 70MB (file1+file2)"); + } + + @Test + public void testMultipleSmallFilesGroupedTogether() throws UserException { + // Test that multiple small files in same partition are grouped and selected + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + FileScanTask task3 = Mockito.mock(FileScanTask.class); + DataFile file1 = Mockito.mock(DataFile.class); + DataFile file2 = Mockito.mock(DataFile.class); + DataFile file3 = Mockito.mock(DataFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose( + Arrays.asList(task1, task2, task3))); + + // All files are too small + Mockito.when(task1.file()).thenReturn(file1); + Mockito.when(task1.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task1.deletes()).thenReturn(null); + Mockito.when(file1.fileSizeInBytes()).thenReturn(40 * 1024 * 1024L); // 40MB < 64MB min + Mockito.when(file1.partition()).thenReturn(mockPartition); + + Mockito.when(task2.file()).thenReturn(file2); + Mockito.when(task2.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task2.deletes()).thenReturn(null); + Mockito.when(file2.fileSizeInBytes()).thenReturn(50 * 1024 * 1024L); // 50MB < 64MB min + Mockito.when(file2.partition()).thenReturn(mockPartition); + + Mockito.when(task3.file()).thenReturn(file3); + Mockito.when(task3.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task3.deletes()).thenReturn(null); + Mockito.when(file3.fileSizeInBytes()).thenReturn(45 * 1024 * 1024L); // 45MB < 64MB min + Mockito.when(file3.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = planner.planAndOrganizeTasks(mockTable); + + // All 3 files are too small, grouped together, total = 135MB > 128MB target + // taskCount = 3 >= 2 minInputFiles, should be selected via enoughInputFiles or enoughContent + Assertions.assertEquals(1, result.size(), "Should have exactly 1 group"); + Assertions.assertEquals(3, result.get(0).getTaskCount(), "Group should contain all 3 small files"); + Assertions.assertEquals(135 * 1024 * 1024L, result.get(0).getTotalSize(), "Total size should be 135MB (40+50+45)"); + // Verify all three files are in the group + Assertions.assertTrue(result.get(0).getDataFiles().contains(file1), "Should contain file1"); + Assertions.assertTrue(result.get(0).getDataFiles().contains(file2), "Should contain file2"); + Assertions.assertTrue(result.get(0).getDataFiles().contains(file3), "Should contain file3"); + } + + @Test + public void testDeleteFileThresholdBoundary() throws UserException { + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()) + .thenReturn(CloseableIterable.withNoopClose(Collections.singletonList(mockFileScanTask))); + Mockito.when(mockFileScanTask.file()).thenReturn(mockDataFile); + Mockito.when(mockFileScanTask.spec()).thenReturn(mockPartitionSpec); + Mockito.when(mockDataFile.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); + Mockito.when(mockDataFile.partition()).thenReturn(mockPartition); + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + // Test with delete count exactly at threshold - 1 (should NOT be selected) + DeleteFile delete1 = Mockito.mock(DeleteFile.class); + DeleteFile delete2 = Mockito.mock(DeleteFile.class); + Mockito.when(mockFileScanTask.deletes()).thenReturn(Arrays.asList(delete1, delete2)); // 2 < 3 threshold + + List result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertTrue(result.isEmpty(), "File with 2 delete files (< 3 threshold) should not be selected"); + + // Test with delete count exactly at threshold (should be selected) + DeleteFile delete3 = Mockito.mock(DeleteFile.class); + Mockito.when(mockFileScanTask.deletes()).thenReturn(Arrays.asList(delete1, delete2, delete3)); // 3 >= 3 threshold + + result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertEquals(1, result.size(), "File with 3 delete files (>= 3 threshold) should be selected"); + Assertions.assertEquals(1, result.get(0).getTaskCount(), "Group should contain 1 task"); + Assertions.assertEquals(100 * 1024 * 1024L, result.get(0).getTotalSize(), "Total size should be 100MB"); + + // Test with delete count above threshold (should be selected) + DeleteFile delete4 = Mockito.mock(DeleteFile.class); + Mockito.when(mockFileScanTask.deletes()).thenReturn(Arrays.asList(delete1, delete2, delete3, delete4)); // 4 >= 3 + + result = planner.planAndOrganizeTasks(mockTable); + Assertions.assertEquals(1, result.size(), "File with 4 delete files (> 3 threshold) should be selected"); + Assertions.assertEquals(1, result.get(0).getTaskCount(), "Group should contain 1 task"); + Assertions.assertEquals(100 * 1024 * 1024L, result.get(0).getTotalSize(), "Total size should be 100MB"); + } + + @Test + public void testBinPackGroupingWithLargePartition() throws UserException { + // Test binPack grouping: when a partition has files exceeding maxFileGroupSizeBytes, + // they should be split into multiple groups + RewriteDataFilePlanner.Parameters params = new RewriteDataFilePlanner.Parameters( + 128 * 1024 * 1024L, // targetFileSizeBytes: 128MB + 64 * 1024 * 1024L, // minFileSizeBytes: 64MB + 256 * 1024 * 1024L, // maxFileSizeBytes: 256MB + 1, // minInputFiles: 1 + true, // rewriteAll: true (to avoid filtering) + 200 * 1024 * 1024L, // maxFileGroupSizeBytes: 200MB (small to trigger splitting) + 3, // deleteFileThreshold + 0.1, // deleteRatioThreshold: 10% + 1L, // outputSpecId + Optional.empty() // whereCondition + ); + + RewriteDataFilePlanner testPlanner = new RewriteDataFilePlanner(params); + + // Create multiple files in the same partition that together exceed maxFileGroupSizeBytes + FileScanTask task1 = Mockito.mock(FileScanTask.class); + FileScanTask task2 = Mockito.mock(FileScanTask.class); + FileScanTask task3 = Mockito.mock(FileScanTask.class); + DataFile file1 = Mockito.mock(DataFile.class); + DataFile file2 = Mockito.mock(DataFile.class); + DataFile file3 = Mockito.mock(DataFile.class); + + Mockito.when(mockTable.newScan()).thenReturn(mockTableScan); + Mockito.when(mockTableScan.planFiles()).thenReturn(CloseableIterable.withNoopClose( + Arrays.asList(task1, task2, task3))); + + // All files in the same partition + Mockito.when(task1.file()).thenReturn(file1); + Mockito.when(task1.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task1.deletes()).thenReturn(null); + Mockito.when(file1.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); // 100MB + Mockito.when(file1.partition()).thenReturn(mockPartition); + + Mockito.when(task2.file()).thenReturn(file2); + Mockito.when(task2.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task2.deletes()).thenReturn(null); + Mockito.when(file2.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); // 100MB + Mockito.when(file2.partition()).thenReturn(mockPartition); + + Mockito.when(task3.file()).thenReturn(file3); + Mockito.when(task3.spec()).thenReturn(mockPartitionSpec); + Mockito.when(task3.deletes()).thenReturn(null); + Mockito.when(file3.fileSizeInBytes()).thenReturn(100 * 1024 * 1024L); // 100MB + Mockito.when(file3.partition()).thenReturn(mockPartition); + + Mockito.when(mockPartitionSpec.partitionType()).thenReturn(Types.StructType.of()); + + List result = testPlanner.planAndOrganizeTasks(mockTable); + + // Total size = 300MB > 200MB maxFileGroupSizeBytes + // binPack should split into multiple groups + // Expected: 2 groups (100MB + 100MB in first group, 100MB in second group) + Assertions.assertTrue(result.size() >= 2, "Should have at least 2 groups due to binPack splitting"); + + // Verify total files are preserved + int totalFiles = result.stream().mapToInt(RewriteDataGroup::getTaskCount).sum(); + long totalSize = result.stream().mapToLong(RewriteDataGroup::getTotalSize).sum(); + Assertions.assertEquals(3, totalFiles, "Should have all 3 files"); + Assertions.assertEquals(300 * 1024 * 1024L, totalSize, "Total size should be 300MB"); + + // Verify each group doesn't exceed maxFileGroupSizeBytes + for (RewriteDataGroup group : result) { + Assertions.assertTrue(group.getTotalSize() <= 200 * 1024 * 1024L, + "Each group should not exceed maxFileGroupSizeBytes (200MB)"); + } + } + +} diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index bfd1c81edcf814..cb65832fdb158c 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -412,6 +412,9 @@ struct TQueryOptions { 176: optional list shuffled_agg_ids; 177: optional bool enable_extended_regex = false; + // Target file size in bytes for Iceberg write operations + // Default 0 means use config::iceberg_sink_max_file_size + 178: optional i64 iceberg_write_target_file_size_bytes = 0; // For cloud, to control if the content would be written into file cache // In write path, to control if the content would be written into file cache. diff --git a/regression-test/data/external_table_p0/iceberg/action/test_iceberg_execute_actions.out b/regression-test/data/external_table_p0/iceberg/action/test_iceberg_execute_actions.out index ecbe09bc0d3e4e..a815666d92aa9b 100644 --- a/regression-test/data/external_table_p0/iceberg/action/test_iceberg_execute_actions.out +++ b/regression-test/data/external_table_p0/iceberg/action/test_iceberg_execute_actions.out @@ -66,6 +66,3 @@ 2 record2 200 3 record3 300 --- !test_rewrite_data_files_results -- -0 1 2 3 - diff --git a/regression-test/data/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files.out b/regression-test/data/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files.out new file mode 100644 index 00000000000000..0c449ed9b4457b --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files.out @@ -0,0 +1,70 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !before_rewrite_data -- +1 item1 electronics 100 2024-01-01 +2 item2 electronics 200 2024-01-02 +3 item3 electronics 300 2024-01-03 +4 item4 electronics 400 2024-01-04 +5 item5 books 150 2024-01-05 +6 item6 books 250 2024-01-06 +7 item7 books 350 2024-01-07 +8 item8 books 450 2024-01-08 +9 item9 clothing 180 2024-01-09 +10 item10 clothing 280 2024-01-10 + +-- !after_rewrite_data -- +1 item1 electronics 100 2024-01-01 +2 item2 electronics 200 2024-01-02 +3 item3 electronics 300 2024-01-03 +4 item4 electronics 400 2024-01-04 +5 item5 books 150 2024-01-05 +6 item6 books 250 2024-01-06 +7 item7 books 350 2024-01-07 +8 item8 books 450 2024-01-08 +9 item9 clothing 180 2024-01-09 +10 item10 clothing 280 2024-01-10 + +-- !before_rewrite_partitioned -- +1 user1 login 1.00 2024-01-01 +2 user2 view 2.00 2024-01-01 +3 user3 click 3.00 2024-01-01 +4 user4 purchase 4.00 2024-01-01 +5 user5 logout 5.00 2024-01-01 +6 user6 login 6.00 2024-01-02 +7 user7 view 7.00 2024-01-02 +8 user8 click 8.00 2024-01-02 +9 user9 purchase 9.00 2024-01-02 +10 user10 logout 10.00 2024-01-02 +11 user11 login 11.00 2024-01-03 +12 user12 view 12.00 2024-01-03 +13 user13 click 13.00 2024-01-03 +14 user14 purchase 14.00 2024-01-03 +15 user15 logout 15.00 2024-01-03 + +-- !after_rewrite_partitioned -- +1 user1 login 1.00 2024-01-01 +2 user2 view 2.00 2024-01-01 +3 user3 click 3.00 2024-01-01 +4 user4 purchase 4.00 2024-01-01 +5 user5 logout 5.00 2024-01-01 +6 user6 login 6.00 2024-01-02 +7 user7 view 7.00 2024-01-02 +8 user8 click 8.00 2024-01-02 +9 user9 purchase 9.00 2024-01-02 +10 user10 logout 10.00 2024-01-02 +11 user11 login 11.00 2024-01-03 +12 user12 view 12.00 2024-01-03 +13 user13 click 13.00 2024-01-03 +14 user14 purchase 14.00 2024-01-03 +15 user15 logout 15.00 2024-01-03 + +-- !after_specific_partition_rewrite -- +1 NORTH 100.00 2024-01-01 +2 NORTH 200.00 2024-01-01 +3 NORTH 300.00 2024-01-01 +4 SOUTH 150.00 2024-01-01 +5 SOUTH 250.00 2024-01-01 +6 SOUTH 350.00 2024-01-01 +7 EAST 180.00 2024-01-02 +8 EAST 280.00 2024-01-02 +9 EAST 380.00 2024-01-02 + diff --git a/regression-test/data/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_where_conditions.out b/regression-test/data/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_where_conditions.out new file mode 100644 index 00000000000000..ff59d39c4960a3 --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_where_conditions.out @@ -0,0 +1,38 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !initial_data -- +1 Alice 25 50000.5 true 2024-01-01 2024-01-01T10:00 85.50 +2 Bob 30 60000 true 2024-01-02 2024-01-02T11:30 92.75 +3 Charlie 35 70000.5 false 2024-01-03 2024-01-03T09:15 78.25 +4 David 28 55000 true 2024-01-04 2024-01-04T14:45 88.00 +5 Eve 32 65000.5 false 2024-01-05 2024-01-05T16:20 95.50 +6 Frank 27 52000 true 2024-01-06 2024-01-06T08:30 82.25 +7 Grace 29 58000.5 true 2024-01-07 2024-01-07T12:00 90.75 +8 Henry 33 72000 false 2024-01-08 2024-01-08T15:30 87.50 +9 Ivy 26 48000.5 true 2024-01-09 2024-01-09T13:45 93.00 +10 Jack 31 68000 true 2024-01-10 2024-01-10T17:15 89.25 +11 Kate 24 45000 false 2024-01-11 2024-01-11T10:30 76.50 +12 Liam 36 75000.5 true 2024-01-12 2024-01-12T11:45 91.25 +13 Mia 23 42000 false 2024-01-13 2024-01-13T14:20 84.75 +14 Noah 34 71000.5 true 2024-01-14 2024-01-14T09:00 86.50 +15 Olivia 25 51000 true 2024-01-15 2024-01-15T16:45 94.25 + +-- !final_data -- +1 Alice 25 50000.5 true 2024-01-01 2024-01-01T10:00 85.50 +2 Bob 30 60000 true 2024-01-02 2024-01-02T11:30 92.75 +3 Charlie 35 70000.5 false 2024-01-03 2024-01-03T09:15 78.25 +4 David 28 55000 true 2024-01-04 2024-01-04T14:45 88.00 +5 Eve 32 65000.5 false 2024-01-05 2024-01-05T16:20 95.50 +6 Frank 27 52000 true 2024-01-06 2024-01-06T08:30 82.25 +7 Grace 29 58000.5 true 2024-01-07 2024-01-07T12:00 90.75 +8 Henry 33 72000 false 2024-01-08 2024-01-08T15:30 87.50 +9 Ivy 26 48000.5 true 2024-01-09 2024-01-09T13:45 93.00 +10 Jack 31 68000 true 2024-01-10 2024-01-10T17:15 89.25 +11 Kate 24 45000 false 2024-01-11 2024-01-11T10:30 76.50 +12 Liam 36 75000.5 true 2024-01-12 2024-01-12T11:45 91.25 +13 Mia 23 42000 false 2024-01-13 2024-01-13T14:20 84.75 +14 Noah 34 71000.5 true 2024-01-14 2024-01-14T09:00 86.50 +15 Olivia 25 51000 true 2024-01-15 2024-01-15T16:45 94.25 +16 NullName \N 45000 \N 2024-01-16 2024-01-16T10:00 \N +17 \N 28 \N true \N 2024-01-17T11:00 75.50 +18 TestUser 30 50000 false 2024-01-18 \N 88.25 + diff --git a/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_execute_actions.groovy b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_execute_actions.groovy index 94810040e8e11e..0dc02c2b26f556 100644 --- a/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_execute_actions.groovy +++ b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_execute_actions.groovy @@ -396,13 +396,6 @@ suite("test_iceberg_optimize_actions_ddl", "p0,external,doris,external_docker,ex exception "Iceberg expire_snapshots procedure is not implemented yet" } - // Test rewrite_data_files action - qt_test_rewrite_data_files_results """ - ALTER TABLE ${catalog_name}.${db_name}.${table_name} EXECUTE rewrite_data_files - ("target-file-size-bytes" = "134217728") - """ - - // Test validation - missing required property test { sql """ diff --git a/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files.groovy b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files.groovy new file mode 100644 index 00000000000000..52261782bfd026 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files.groovy @@ -0,0 +1,495 @@ +// 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. + +import java.time.format.DateTimeFormatter +import java.time.format.DateTimeFormatterBuilder +import java.time.temporal.ChronoField +import java.time.LocalDateTime +import java.time.ZoneId + +suite("test_iceberg_rewrite_data_files", "p0,external,doris,external_docker,external_docker_doris") { + DateTimeFormatter unifiedFormatter = new DateTimeFormatterBuilder() + .appendPattern("yyyy-MM-dd") + .optionalStart() + .appendLiteral('T') + .optionalEnd() + .optionalStart() + .appendLiteral(' ') + .optionalEnd() + .appendPattern("HH:mm:ss") + .optionalStart() + .appendFraction(ChronoField.MILLI_OF_SECOND, 0, 3, true) + .optionalEnd() + .toFormatter() + + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable iceberg test.") + return + } + + String catalog_name = "test_iceberg_rewrite_data_files" + String db_name = "test_db" + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + sql """drop catalog if exists ${catalog_name}""" + sql """ + CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + sql """switch ${catalog_name}""" + sql """CREATE DATABASE IF NOT EXISTS ${db_name} """ + sql """use ${db_name}""" + + // ===================================================================================== + // Test Case 1: Basic rewrite_data_files operation + // Tests the ability to rewrite multiple small data files into larger optimized files + // ===================================================================================== + logger.info("Starting basic rewrite_data_files test case") + + def table_name = "test_rewrite_data_basic" + + // Clean up if table exists + sql """DROP TABLE IF EXISTS ${db_name}.${table_name}""" + + // Create a test table with partitioning + sql """ + CREATE TABLE ${db_name}.${table_name} ( + id BIGINT, + name STRING, + category STRING, + value INT, + created_date DATE + ) ENGINE=iceberg + """ + logger.info("Created test table: ${table_name}") + + // Insert data multiple times to create multiple small files + // This simulates the scenario where files need to be compacted + sql """ + INSERT INTO ${db_name}.${table_name} VALUES + (1, 'item1', 'electronics', 100, '2024-01-01'), + (2, 'item2', 'electronics', 200, '2024-01-02') + """ + + sql """ + INSERT INTO ${db_name}.${table_name} VALUES + (3, 'item3', 'electronics', 300, '2024-01-03'), + (4, 'item4', 'electronics', 400, '2024-01-04') + """ + + sql """ + INSERT INTO ${db_name}.${table_name} VALUES + (5, 'item5', 'books', 150, '2024-01-05'), + (6, 'item6', 'books', 250, '2024-01-06') + """ + + sql """ + INSERT INTO ${db_name}.${table_name} VALUES + (7, 'item7', 'books', 350, '2024-01-07'), + (8, 'item8', 'books', 450, '2024-01-08') + """ + + sql """ + INSERT INTO ${db_name}.${table_name} VALUES + (9, 'item9', 'clothing', 180, '2024-01-09'), + (10, 'item10', 'clothing', 280, '2024-01-10') + """ + + logger.info("Inserted data in 5 separate batches to create multiple small files") + + // Verify table data before rewrite + qt_before_rewrite_data """SELECT * FROM ${table_name} ORDER BY id""" + + // Check files system table before rewrite + List> filesBeforeRewrite = sql """ + SELECT file_path, file_format, record_count, file_size_in_bytes + FROM ${table_name}\$files + ORDER BY file_path + """ + logger.info("Files before rewrite: ${filesBeforeRewrite.size()} files") + logger.info("File details before rewrite: ${filesBeforeRewrite}") + + // Ensure we have multiple files before rewriting + assertTrue(filesBeforeRewrite.size() >= 5, + "Expected at least 5 data files before rewrite, but got ${filesBeforeRewrite.size()}") + + // Check snapshots before rewrite + List> snapshotsBeforeRewrite = sql """ + SELECT snapshot_id, parent_id, operation, summary + FROM ${table_name}\$snapshots + ORDER BY committed_at + """ + logger.info("Snapshots before rewrite: ${snapshotsBeforeRewrite.size()} snapshots") + int snapshotsCountBefore = snapshotsBeforeRewrite.size() + + // Execute rewrite_data_files action with custom parameters + // Using smaller target file size to ensure rewrite happens with our test data + def rewriteResult = sql """ + ALTER TABLE ${catalog_name}.${db_name}.${table_name} + EXECUTE rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "2" + ) + """ + logger.info("Rewrite data files result: ${rewriteResult}") + + // Verify the result contains expected columns + assertTrue(rewriteResult.size() > 0, "Rewrite result should not be empty") + assertTrue(rewriteResult[0].size() == 4, + "Expected 4 columns in result: rewritten_data_files_count, added_data_files_count, rewritten_bytes_count, removed_delete_files_count") + + int rewrittenFilesCount = rewriteResult[0][0] as int + int addedFilesCount = rewriteResult[0][1] as int + long rewrittenBytesCount = rewriteResult[0][2] as long + + logger.info("Rewritten files count: ${rewrittenFilesCount}") + logger.info("Added files count: ${addedFilesCount}") + logger.info("Rewritten bytes count: ${rewrittenBytesCount}") + + // Verify table data after rewrite (data should remain the same) + qt_after_rewrite_data """SELECT * FROM ${table_name} ORDER BY id""" + + // Check files system table after rewrite + List> filesAfterRewrite = sql """ + SELECT file_path, file_format, record_count, file_size_in_bytes + FROM ${table_name}\$files + ORDER BY file_path + """ + logger.info("Files after rewrite: ${filesAfterRewrite.size()} files") + logger.info("File details after rewrite: ${filesAfterRewrite}") + + // Verify that files were actually rewritten (file count should be different) + // After rewrite, we should have fewer files (compaction happened) + assertTrue(filesAfterRewrite.size() <= filesBeforeRewrite.size(), + "File count after rewrite should be less than or equal to before rewrite") + + // Check snapshots after rewrite + List> snapshotsAfterRewrite = sql """ + SELECT snapshot_id, parent_id, operation, summary + FROM ${table_name}\$snapshots + ORDER BY committed_at + """ + logger.info("Snapshots after rewrite: ${snapshotsAfterRewrite.size()} snapshots") + + // Verify that a new snapshot was created by the rewrite operation + int snapshotsCountAfter = snapshotsAfterRewrite.size() + + // If rewrite actually happened (rewrittenFilesCount > 0), should have new snapshot + if (rewrittenFilesCount > 0) { + assertTrue(snapshotsCountAfter > snapshotsCountBefore, + "A new snapshot should be created after rewrite operation") + + // Check the latest snapshot details + def latestSnapshot = snapshotsAfterRewrite[snapshotsCountAfter - 1] + logger.info("Latest snapshot details: ${latestSnapshot}") + + // The operation in the latest snapshot should indicate it's a rewrite + // (Iceberg typically uses "replace" operation for rewrite) + } + + // Verify total record count is preserved + def totalRecords = sql """SELECT COUNT(*) as cnt FROM ${table_name}""" + assertTrue(totalRecords[0][0] == 10, "Total record count should be 10 after rewrite") + + logger.info("Basic rewrite_data_files test completed successfully") + + // ===================================================================================== + // Test Case 2: Rewrite data files for multi-partition table + // Tests the rewrite operation on a partitioned table with multiple partitions + // Verifies that files in each partition are correctly rewritten independently + // ===================================================================================== + logger.info("Starting multi-partition rewrite_data_files test case") + + def table_name_partitioned = "test_rewrite_data_partitioned" + + // Clean up if table exists + sql """DROP TABLE IF EXISTS ${db_name}.${table_name_partitioned}""" + + // Create a partitioned table using DATE partition + sql """ + CREATE TABLE ${db_name}.${table_name_partitioned} ( + id BIGINT, + user_name STRING, + event_type STRING, + event_value DECIMAL(10, 2), + event_date DATE + ) ENGINE=iceberg + PARTITION BY LIST (event_date) () + """ + logger.info("Created partitioned test table: ${table_name_partitioned}") + + // Insert data for partition 2024-01-01 (multiple inserts to create multiple files) + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (1, 'user1', 'login', 1.0, '2024-01-01'), + (2, 'user2', 'view', 2.0, '2024-01-01') + """ + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (3, 'user3', 'click', 3.0, '2024-01-01'), + (4, 'user4', 'purchase', 4.0, '2024-01-01') + """ + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (5, 'user5', 'logout', 5.0, '2024-01-01') + """ + + // Insert data for partition 2024-01-02 (multiple inserts) + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (6, 'user6', 'login', 6.0, '2024-01-02'), + (7, 'user7', 'view', 7.0, '2024-01-02') + """ + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (8, 'user8', 'click', 8.0, '2024-01-02'), + (9, 'user9', 'purchase', 9.0, '2024-01-02') + """ + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (10, 'user10', 'logout', 10.0, '2024-01-02') + """ + + // Insert data for partition 2024-01-03 (multiple inserts) + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (11, 'user11', 'login', 11.0, '2024-01-03'), + (12, 'user12', 'view', 12.0, '2024-01-03') + """ + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (13, 'user13', 'click', 13.0, '2024-01-03'), + (14, 'user14', 'purchase', 14.0, '2024-01-03') + """ + sql """ + INSERT INTO ${db_name}.${table_name_partitioned} VALUES + (15, 'user15', 'logout', 15.0, '2024-01-03') + """ + + logger.info("Inserted data into 3 partitions (2024-01-01, 2024-01-02, 2024-01-03) with multiple batches each") + + // Verify table data before rewrite + qt_before_rewrite_partitioned """SELECT * FROM ${table_name_partitioned} ORDER BY id""" + + // Check files per partition before rewrite + List> filesBeforeRewritePartitioned = sql """ + SELECT `partition`, file_path, record_count, file_size_in_bytes + FROM ${table_name_partitioned}\$files + ORDER BY `partition`, file_path + """ + logger.info("Total files before rewrite: ${filesBeforeRewritePartitioned.size()} files") + + // Count files per partition + def filesPerPartitionBefore = filesBeforeRewritePartitioned.groupBy { it[0] } + filesPerPartitionBefore.each { partition, files -> + logger.info("Partition ${partition}: ${files.size()} files before rewrite") + } + + // Verify we have multiple partitions with multiple files each + assertTrue(filesPerPartitionBefore.size() >= 3, + "Expected at least 3 partitions, but got ${filesPerPartitionBefore.size()}") + + // Check snapshots before rewrite + List> snapshotsBeforeRewritePartitioned = sql """ + SELECT snapshot_id, operation, summary + FROM ${table_name_partitioned}\$snapshots + ORDER BY committed_at + """ + logger.info("Snapshots before rewrite (partitioned): ${snapshotsBeforeRewritePartitioned.size()} snapshots") + int snapshotsCountBeforePartitioned = snapshotsBeforeRewritePartitioned.size() + + // Execute rewrite_data_files on all partitions + def rewriteResultPartitioned = sql """ + ALTER TABLE ${catalog_name}.${db_name}.${table_name_partitioned} + EXECUTE rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "2" + ) + """ + logger.info("Rewrite data files result (all partitions): ${rewriteResultPartitioned}") + + int rewrittenFilesCountPartitioned = rewriteResultPartitioned[0][0] as int + int addedFilesCountPartitioned = rewriteResultPartitioned[0][1] as int + + logger.info("Rewritten files count (partitioned): ${rewrittenFilesCountPartitioned}") + logger.info("Added files count (partitioned): ${addedFilesCountPartitioned}") + + // Verify table data after rewrite (should remain the same) + qt_after_rewrite_partitioned """SELECT * FROM ${table_name_partitioned} ORDER BY id""" + + // Check files per partition after rewrite + List> filesAfterRewritePartitioned = sql """ + SELECT `partition`, file_path, record_count, file_size_in_bytes + FROM ${table_name_partitioned}\$files + ORDER BY `partition`, file_path + """ + logger.info("Total files after rewrite: ${filesAfterRewritePartitioned.size()} files") + + // Count files per partition after rewrite + def filesPerPartitionAfter = filesAfterRewritePartitioned.groupBy { it[0] } + filesPerPartitionAfter.each { partition, files -> + logger.info("Partition ${partition}: ${files.size()} files after rewrite") + } + + // Verify file count decreased or stayed the same (compaction happened) + assertTrue(filesAfterRewritePartitioned.size() <= filesBeforeRewritePartitioned.size(), + "File count after rewrite should be <= before rewrite") + + // Check snapshots after rewrite + List> snapshotsAfterRewritePartitioned = sql """ + SELECT snapshot_id, operation, summary + FROM ${table_name_partitioned}\$snapshots + ORDER BY committed_at + """ + int snapshotsCountAfterPartitioned = snapshotsAfterRewritePartitioned.size() + logger.info("Snapshots after rewrite (partitioned): ${snapshotsCountAfterPartitioned} snapshots") + + // If files were rewritten, a new snapshot should be created + if (rewrittenFilesCountPartitioned > 0) { + assertTrue(snapshotsCountAfterPartitioned > snapshotsCountBeforePartitioned, + "A new snapshot should be created after rewrite operation") + } + + // Verify record count per partition is preserved + def partition1Count = sql """SELECT COUNT(*) FROM ${table_name_partitioned} WHERE event_date = '2024-01-01'""" + def partition2Count = sql """SELECT COUNT(*) FROM ${table_name_partitioned} WHERE event_date = '2024-01-02'""" + def partition3Count = sql """SELECT COUNT(*) FROM ${table_name_partitioned} WHERE event_date = '2024-01-03'""" + + assertTrue(partition1Count[0][0] == 5, "Partition 2024-01-01 should have 5 records") + assertTrue(partition2Count[0][0] == 5, "Partition 2024-01-02 should have 5 records") + assertTrue(partition3Count[0][0] == 5, "Partition 2024-01-03 should have 5 records") + + // Verify total record count + def totalRecordsPartitioned = sql """SELECT COUNT(*) FROM ${table_name_partitioned}""" + assertTrue(totalRecordsPartitioned[0][0] == 15, "Total record count should be 15 after rewrite") + + logger.info("Multi-partition rewrite_data_files test completed successfully") + + // ===================================================================================== + // Test Case 3: Rewrite data files for specific partition using WHERE clause + // Tests the rewrite operation targeting a specific partition + // ===================================================================================== + logger.info("Starting specific partition rewrite test case") + + def table_name_specific = "test_rewrite_specific_partition" + + // Clean up if table exists + sql """DROP TABLE IF EXISTS ${db_name}.${table_name_specific}""" + + // Create another partitioned table + sql """ + CREATE TABLE ${db_name}.${table_name_specific} ( + id BIGINT, + region STRING, + sales_amount DECIMAL(12, 2), + order_date DATE + ) ENGINE=iceberg + PARTITION BY LIST (region, order_date) () + """ + logger.info("Created table for specific partition rewrite: ${table_name_specific}") + + // Insert data for region 'NORTH', date 2024-01-01 + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (1, 'NORTH', 100.00, '2024-01-01')""" + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (2, 'NORTH', 200.00, '2024-01-01')""" + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (3, 'NORTH', 300.00, '2024-01-01')""" + + // Insert data for region 'SOUTH', date 2024-01-01 + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (4, 'SOUTH', 150.00, '2024-01-01')""" + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (5, 'SOUTH', 250.00, '2024-01-01')""" + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (6, 'SOUTH', 350.00, '2024-01-01')""" + + // Insert data for region 'EAST', date 2024-01-02 + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (7, 'EAST', 180.00, '2024-01-02')""" + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (8, 'EAST', 280.00, '2024-01-02')""" + sql """INSERT INTO ${db_name}.${table_name_specific} VALUES (9, 'EAST', 380.00, '2024-01-02')""" + + logger.info("Inserted data into multiple region-date partitions") + + // Check files before specific partition rewrite + List> filesBeforeSpecific = sql """ + SELECT `partition`, file_path, record_count + FROM ${table_name_specific}\$files + ORDER BY `partition` + """ + logger.info("Files before specific partition rewrite: ${filesBeforeSpecific.size()} files") + + def partitionGroupsBefore = filesBeforeSpecific.groupBy { it[0] } + partitionGroupsBefore.each { partition, files -> + logger.info("Partition ${partition}: ${files.size()} files") + } + + // Count files in NORTH partition before rewrite + int northFilesCountBefore = filesBeforeSpecific.findAll { it[0].toString().contains('NORTH') }.size() + logger.info("NORTH partition files before rewrite: ${northFilesCountBefore}") + + // Execute rewrite only for NORTH region using WHERE clause + def rewriteResultSpecific = sql """ + ALTER TABLE ${catalog_name}.${db_name}.${table_name_specific} + EXECUTE rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "2" + ) WHERE region = 'NORTH' + """ + logger.info("Rewrite result for NORTH partition only: ${rewriteResultSpecific}") + + int rewrittenFilesSpecific = rewriteResultSpecific[0][0] as int + logger.info("Files rewritten in NORTH partition: ${rewrittenFilesSpecific}") + + // Check files after specific partition rewrite + List> filesAfterSpecific = sql """ + SELECT `partition`, file_path, record_count + FROM ${table_name_specific}\$files + ORDER BY `partition` + """ + logger.info("Files after specific partition rewrite: ${filesAfterSpecific.size()} files") + + def partitionGroupsAfter = filesAfterSpecific.groupBy { it[0] } + partitionGroupsAfter.each { partition, files -> + logger.info("Partition ${partition}: ${files.size()} files") + } + + // Count files in NORTH partition after rewrite + int northFilesCountAfter = filesAfterSpecific.findAll { it[0].toString().contains('NORTH') }.size() + logger.info("NORTH partition files after rewrite: ${northFilesCountAfter}") + + // Verify NORTH partition was affected + if (rewrittenFilesSpecific > 0) { + assertTrue(northFilesCountAfter <= northFilesCountBefore, + "NORTH partition file count should decrease or stay same after rewrite") + } + + // Verify data integrity - all records should still be present + qt_after_specific_partition_rewrite """SELECT * FROM ${table_name_specific} ORDER BY id""" + + def totalRecordsSpecific = sql """SELECT COUNT(*) FROM ${table_name_specific}""" + assertTrue(totalRecordsSpecific[0][0] == 9, "Total record count should be 9 after specific partition rewrite") + + // Verify NORTH region records + def northRecords = sql """SELECT COUNT(*) FROM ${table_name_specific} WHERE region = 'NORTH'""" + assertTrue(northRecords[0][0] == 3, "NORTH region should still have 3 records") + + logger.info("Specific partition rewrite test completed successfully") +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_expression_conversion.groovy b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_expression_conversion.groovy new file mode 100644 index 00000000000000..4bf587343d48d9 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_expression_conversion.groovy @@ -0,0 +1,430 @@ +// 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. + +suite("test_iceberg_rewrite_data_files_expression_conversion", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable iceberg test.") + return + } + + String catalog_name = "test_iceberg_rewrite_expression_conversion" + String db_name = "test_db" + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + sql """drop catalog if exists ${catalog_name}""" + sql """ + CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + sql """switch ${catalog_name}""" + sql """CREATE DATABASE IF NOT EXISTS ${db_name} """ + sql """use ${db_name}""" + + // ===================================================================================== + // Test: Expression conversion coverage tests + // + // Test strategy: + // These tests verify that various expression types can be successfully converted + // to Iceberg expressions without throwing exceptions. The tables are created using + // Doris (not pre-created in Docker) and we don't care about filter effectiveness, + // only that the expression conversion succeeds. + // + // Tests cover: + // 1. Various column types (BOOLEAN, INT, BIGINT, FLOAT, DOUBLE, STRING, DATE, DATETIME, DECIMAL) + // 2. Various expression types (=, >, >=, <, <=, IN, BETWEEN, AND, OR, NOT, IS NULL) + // ===================================================================================== + logger.info("Starting expression conversion coverage tests") + + def expr_test_table = "test_rewrite_where_expression_conversion" + + // Create a test table with various column types + sql """DROP TABLE IF EXISTS ${db_name}.${expr_test_table}""" + sql """ + CREATE TABLE ${db_name}.${expr_test_table} ( + id BIGINT, + name STRING, + age INT, + salary DOUBLE, + score FLOAT, + is_active BOOLEAN, + birth_date DATE, + created_at DATETIME, + balance DECIMAL(10, 2) + ) ENGINE=iceberg + """ + logger.info("Created expression conversion test table: ${expr_test_table}") + + // Insert some test data + sql """ + INSERT INTO ${db_name}.${expr_test_table} VALUES + (1, 'Alice', 25, 50000.0, 85.5, true, '1998-01-01', '2024-01-01 10:00:00', 1000.50), + (2, 'Bob', 30, 60000.0, 90.0, false, '1993-02-15', '2024-01-02 11:00:00', 2000.75), + (3, 'Charlie', 35, 70000.0, 92.5, true, '1988-03-20', '2024-01-03 12:00:00', 3000.00) + """ + + logger.info("Inserted test data for expression conversion tests") + + // Test expression conversion - should not throw exceptions + // We only check that the command executes without exception, + // not that the filtering works correctly + + // Test 1: Basic comparison operators on different column types + logger.info("Test: Basic comparison operators (=, >, >=, <, <=)") + + // BIGINT: = + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE id = 1 + """ + logger.info("✓ BIGINT = conversion succeeded") + } catch (Exception e) { + logger.error("✗ BIGINT = conversion failed: ${e.getMessage()}") + throw e + } + + // INT: > + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE age > 25 + """ + logger.info("✓ INT > conversion succeeded") + } catch (Exception e) { + logger.error("✗ INT > conversion failed: ${e.getMessage()}") + throw e + } + + // DOUBLE: >= + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE salary >= 60000.0 + """ + logger.info("✓ DOUBLE >= conversion succeeded") + } catch (Exception e) { + logger.error("✗ DOUBLE >= conversion failed: ${e.getMessage()}") + throw e + } + + // FLOAT: < + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE score < 90.0 + """ + logger.info("✓ FLOAT < conversion succeeded") + } catch (Exception e) { + logger.error("✗ FLOAT < conversion failed: ${e.getMessage()}") + throw e + } + + // STRING: <= + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE name <= 'Charlie' + """ + logger.info("✓ STRING <= conversion succeeded") + } catch (Exception e) { + logger.error("✗ STRING <= conversion failed: ${e.getMessage()}") + throw e + } + + // BOOLEAN: = + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE is_active = true + """ + logger.info("✓ BOOLEAN = conversion succeeded") + } catch (Exception e) { + logger.error("✗ BOOLEAN = conversion failed: ${e.getMessage()}") + throw e + } + + // DATE: = + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE birth_date = '1998-01-01' + """ + logger.info("✓ DATE = conversion succeeded") + } catch (Exception e) { + logger.error("✗ DATE = conversion failed: ${e.getMessage()}") + throw e + } + + // DATETIME: > + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE created_at > '2024-01-01 10:00:00' + """ + logger.info("✓ DATETIME > conversion succeeded") + } catch (Exception e) { + logger.error("✗ DATETIME > conversion failed: ${e.getMessage()}") + throw e + } + + // DECIMAL: >= + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE balance >= 2000.0 + """ + logger.info("✓ DECIMAL >= conversion succeeded") + } catch (Exception e) { + logger.error("✗ DECIMAL >= conversion failed: ${e.getMessage()}") + throw e + } + + // Test 2: IN predicate on different column types + logger.info("Test: IN predicate") + + // INT: IN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE age IN (25, 30, 35) + """ + logger.info("✓ INT IN conversion succeeded") + } catch (Exception e) { + logger.error("✗ INT IN conversion failed: ${e.getMessage()}") + throw e + } + + // STRING: IN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE name IN ('Alice', 'Bob', 'Charlie') + """ + logger.info("✓ STRING IN conversion succeeded") + } catch (Exception e) { + logger.error("✗ STRING IN conversion failed: ${e.getMessage()}") + throw e + } + + // DOUBLE: IN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE salary IN (50000.0, 60000.0, 70000.0) + """ + logger.info("✓ DOUBLE IN conversion succeeded") + } catch (Exception e) { + logger.error("✗ DOUBLE IN conversion failed: ${e.getMessage()}") + throw e + } + + // Test 3: BETWEEN predicate on different column types + logger.info("Test: BETWEEN predicate") + + // BIGINT: BETWEEN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE id BETWEEN 1 AND 3 + """ + logger.info("✓ BIGINT BETWEEN conversion succeeded") + } catch (Exception e) { + logger.error("✗ BIGINT BETWEEN conversion failed: ${e.getMessage()}") + throw e + } + + // INT: BETWEEN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE age BETWEEN 25 AND 35 + """ + logger.info("✓ INT BETWEEN conversion succeeded") + } catch (Exception e) { + logger.error("✗ INT BETWEEN conversion failed: ${e.getMessage()}") + throw e + } + + // DOUBLE: BETWEEN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE salary BETWEEN 50000.0 AND 70000.0 + """ + logger.info("✓ DOUBLE BETWEEN conversion succeeded") + } catch (Exception e) { + logger.error("✗ DOUBLE BETWEEN conversion failed: ${e.getMessage()}") + throw e + } + + // STRING: BETWEEN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE name BETWEEN 'Alice' AND 'Charlie' + """ + logger.info("✓ STRING BETWEEN conversion succeeded") + } catch (Exception e) { + logger.error("✗ STRING BETWEEN conversion failed: ${e.getMessage()}") + throw e + } + + // DATE: BETWEEN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE birth_date BETWEEN '1993-01-01' AND '1998-12-31' + """ + logger.info("✓ DATE BETWEEN conversion succeeded") + } catch (Exception e) { + logger.error("✗ DATE BETWEEN conversion failed: ${e.getMessage()}") + throw e + } + + // DATETIME: BETWEEN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE created_at BETWEEN '2024-01-01 10:00:00' AND '2024-01-03 12:00:00' + """ + logger.info("✓ DATETIME BETWEEN conversion succeeded") + } catch (Exception e) { + logger.error("✗ DATETIME BETWEEN conversion failed: ${e.getMessage()}") + throw e + } + + // DECIMAL: BETWEEN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE balance BETWEEN 1000.0 AND 3000.0 + """ + logger.info("✓ DECIMAL BETWEEN conversion succeeded") + } catch (Exception e) { + logger.error("✗ DECIMAL BETWEEN conversion failed: ${e.getMessage()}") + throw e + } + + // Test 4: Logical operators (AND, OR, NOT) + logger.info("Test: Logical operators (AND, OR, NOT)") + + // AND + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE id > 1 AND age < 35 + """ + logger.info("✓ AND conversion succeeded") + } catch (Exception e) { + logger.error("✗ AND conversion failed: ${e.getMessage()}") + throw e + } + + // OR + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE id = 1 OR id = 2 + """ + logger.info("✓ OR conversion succeeded") + } catch (Exception e) { + logger.error("✗ OR conversion failed: ${e.getMessage()}") + throw e + } + + // NOT + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE NOT id = 1 + """ + logger.info("✓ NOT conversion succeeded") + } catch (Exception e) { + logger.error("✗ NOT conversion failed: ${e.getMessage()}") + throw e + } + + // Complex AND with OR + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE (id > 1 AND age < 35) OR salary > 65000.0 + """ + logger.info("✓ Complex AND/OR conversion succeeded") + } catch (Exception e) { + logger.error("✗ Complex AND/OR conversion failed: ${e.getMessage()}") + throw e + } + + // Test 5: IS NULL + logger.info("Test: IS NULL") + + // Note: We don't have NULL values in our test data, but the conversion should still work + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE name IS NULL + """ + logger.info("✓ IS NULL conversion succeeded") + } catch (Exception e) { + logger.error("✗ IS NULL conversion failed: ${e.getMessage()}") + throw e + } + + // Test 6: Combined expressions with different operators + logger.info("Test: Combined expressions") + + // BETWEEN AND IN + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE id BETWEEN 1 AND 3 AND name IN ('Alice', 'Bob') + """ + logger.info("✓ BETWEEN AND IN combination succeeded") + } catch (Exception e) { + logger.error("✗ BETWEEN AND IN combination failed: ${e.getMessage()}") + throw e + } + + // Multiple AND conditions + try { + sql """ + ALTER TABLE ${catalog_name}.${db_name}.${expr_test_table} + EXECUTE rewrite_data_files("target-file-size-bytes" = "10485760") WHERE id >= 1 AND age <= 35 AND salary > 50000.0 AND is_active = true + """ + logger.info("✓ Multiple AND conditions succeeded") + } catch (Exception e) { + logger.error("✗ Multiple AND conditions failed: ${e.getMessage()}") + throw e + } + + logger.info("Expression conversion coverage tests completed successfully") + + // Clean up + sql """DROP TABLE IF EXISTS ${db_name}.${expr_test_table}""" + logger.info("Expression conversion test cleanup completed") +} + diff --git a/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_where_conditions.groovy b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_where_conditions.groovy new file mode 100644 index 00000000000000..03236e075029d2 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/action/test_iceberg_rewrite_data_files_where_conditions.groovy @@ -0,0 +1,152 @@ +// 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. + +suite("test_iceberg_rewrite_data_files_where_conditions", "p0,external,doris,external_docker,external_docker_doris") { + String enabled = context.config.otherConfigs.get("enableIcebergTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("disable iceberg test.") + return + } + + String catalog_name = "test_iceberg_rewrite_where_conditions" + String db_name = "test_db" + String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") + String minio_port = context.config.otherConfigs.get("iceberg_minio_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + sql """drop catalog if exists ${catalog_name}""" + sql """ + CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='iceberg', + 'iceberg.catalog.type'='rest', + 'uri' = 'http://${externalEnvIp}:${rest_port}', + "s3.access_key" = "admin", + "s3.secret_key" = "password", + "s3.endpoint" = "http://${externalEnvIp}:${minio_port}", + "s3.region" = "us-east-1" + );""" + + sql """switch ${catalog_name}""" + sql """CREATE DATABASE IF NOT EXISTS ${db_name} """ + sql """use ${db_name}""" + + // ===================================================================================== + // Test: Verify WHERE conditions affect rewrite data files operation + // + // Test strategy: + // Tables are pre-created in Docker initialization (run21.sql) using Spark SQL + // to ensure min/max metadata is properly generated. This is required for WHERE + // condition filtering to work correctly, as Doris-written Iceberg tables may + // not have min/max information. + // + // 1. Test without WHERE condition - should rewrite files (baseline) + // 2. Test with WHERE condition matching subset of data - should rewrite fewer files + // 3. Test with WHERE condition matching no data - should rewrite no files + // + // Result format: [files_rewritten, files_added, bytes_written, bytes_deleted] + // ===================================================================================== + logger.info("Starting WHERE conditions test for rewrite_data_files") + logger.info("Using pre-created tables from Docker initialization (run21.sql)") + + // Tables are pre-created in demo.test_db database (Spark SQL), + // but accessed as test_db in Doris Iceberg catalog + // Table names for each test case (pre-created in run21.sql) + def table_baseline = "test_rewrite_where_conditions_baseline" + def table_with_where = "test_rewrite_where_conditions_with_where" + def table_no_match = "test_rewrite_where_conditions_no_match" + + // Test 1: Rewrite without WHERE condition (baseline - should rewrite files) + logger.info("Test 1: Rewrite without WHERE condition (baseline)") + + def rewriteResultNoWhere = sql """ + ALTER TABLE ${catalog_name}.${db_name}.${table_baseline} + EXECUTE rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "2" + ) + """ + logger.info("Rewrite result without WHERE: ${rewriteResultNoWhere}") + // Assert: Should rewrite some files when no WHERE condition + assertTrue(rewriteResultNoWhere[0][0] > 0, "Files should be rewritten without WHERE condition") + assertTrue(rewriteResultNoWhere[0][1] > 0, "Files should be added without WHERE condition") + assertTrue(rewriteResultNoWhere[0][2] > 0, "Bytes should be written without WHERE condition") + assertTrue(rewriteResultNoWhere[0][3] == 0, "No bytes should be deleted") + + int filesRewrittenBaseline = rewriteResultNoWhere[0][0] as int + logger.info("Baseline: ${filesRewrittenBaseline} files rewritten") + + // Verify data integrity + def totalRecords1 = sql """SELECT COUNT(*) FROM ${table_baseline}""" + assertTrue(totalRecords1[0][0] == 30, "Total record count should be 30 after baseline test") + + // Test 2: Rewrite with WHERE condition matching subset of data + // WHERE condition: id >= 11 AND id <= 20 (only matches 10 records) + logger.info("Test 2: Rewrite with WHERE condition matching subset (id >= 11 AND id <= 20)") + + def rewriteResultWithWhere = sql """ + ALTER TABLE ${catalog_name}.${db_name}.${table_with_where} + EXECUTE rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "2" + ) WHERE id >= 11 AND id <= 20 + """ + logger.info("Rewrite result with WHERE (id >= 11 AND id <= 20): ${rewriteResultWithWhere}") + // Assert: Should rewrite fewer or equal files than baseline when WHERE condition filters data + int filesRewrittenWithWhere = rewriteResultWithWhere[0][0] as int + assertTrue(filesRewrittenWithWhere > 0, "Files rewritten count should be positive") + assertTrue(filesRewrittenWithWhere < filesRewrittenBaseline, + "Files rewritten with WHERE condition (${filesRewrittenWithWhere}) should be <= baseline (${filesRewrittenBaseline})") + assertTrue(rewriteResultWithWhere[0][1] > 0, "Files added count should be positive") + assertTrue(rewriteResultWithWhere[0][2] > 0, "Bytes written count should be positive") + assertTrue(rewriteResultWithWhere[0][3] == 0, "No bytes should be deleted") + logger.info("With WHERE: ${filesRewrittenWithWhere} files rewritten (expected <= ${filesRewrittenBaseline})") + + // Verify data integrity + def totalRecords2 = sql """SELECT COUNT(*) FROM ${table_with_where}""" + assertTrue(totalRecords2[0][0] == 30, "Total record count should be 30 after WHERE condition test") + + // Test 3: Rewrite with WHERE condition matching no data + // WHERE condition: id = 99999 (matches no records) + logger.info("Test 3: Rewrite with WHERE condition matching no data (id = 99999)") + + def rewriteResultNoMatch = sql """ + ALTER TABLE ${catalog_name}.${db_name}.${table_no_match} + EXECUTE rewrite_data_files( + "target-file-size-bytes" = "10485760", + "min-input-files" = "2" + ) WHERE id = 99999 + """ + logger.info("Rewrite result with WHERE (id = 99999, no matches): ${rewriteResultNoMatch}") + // Assert: Should rewrite no files when WHERE condition matches no data + assertTrue(rewriteResultNoMatch[0][0] == 0, + "No files should be rewritten when WHERE condition matches no data (expected 0, got ${rewriteResultNoMatch[0][0]})") + assertTrue(rewriteResultNoMatch[0][1] == 0, + "No files should be added when WHERE condition matches no data (expected 0, got ${rewriteResultNoMatch[0][1]})") + assertTrue(rewriteResultNoMatch[0][2] == 0, + "No bytes should be written when WHERE condition matches no data (expected 0, got ${rewriteResultNoMatch[0][2]})") + assertTrue(rewriteResultNoMatch[0][3] == 0, "No bytes should be deleted") + logger.info("With WHERE (no match): 0 files rewritten as expected") + + // Verify data integrity + def totalRecords3 = sql """SELECT COUNT(*) FROM ${table_no_match}""" + assertTrue(totalRecords3[0][0] == 30, "Total record count should be 30 after no-match WHERE condition test") + + logger.info("WHERE conditions test completed successfully") + + // Note: Tables are not cleaned up as they are pre-created in Docker initialization + // and may be reused or manually cleaned up if needed + logger.info("Test completed - pre-created tables remain for potential reuse") +}