diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java index 905de471d1..a7fb7e6ac0 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java @@ -52,6 +52,7 @@ import org.apache.fluss.exception.TooManyPartitionsException; import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.FsPathAndFileName; +import org.apache.fluss.metadata.AggFunctions; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metadata.DatabaseInfo; import org.apache.fluss.metadata.DeleteBehavior; @@ -512,6 +513,47 @@ void testCreateTableWithDeleteBehavior() { TableInfo tableInfo2 = admin.getTableInfo(tablePath2).join(); assertThat(tableInfo2.getTableConfig().getDeleteBehavior()).hasValue(DeleteBehavior.IGNORE); + // Test 2.5: AGGREGATION merge engine - should set delete behavior to IGNORE + TablePath tablePathAggregate = TablePath.of("fluss", "test_ignore_delete_for_aggregate"); + Schema aggregateSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + Map propertiesAggregate = new HashMap<>(); + propertiesAggregate.put(ConfigOptions.TABLE_MERGE_ENGINE.key(), "aggregation"); + TableDescriptor tableDescriptorAggregate = + TableDescriptor.builder() + .schema(aggregateSchema) + .comment("aggregate merge engine table") + .properties(propertiesAggregate) + .build(); + admin.createTable(tablePathAggregate, tableDescriptorAggregate, false).join(); + // Get the table and verify delete behavior is changed to IGNORE + TableInfo tableInfoAggregate = admin.getTableInfo(tablePathAggregate).join(); + assertThat(tableInfoAggregate.getTableConfig().getDeleteBehavior()) + .hasValue(DeleteBehavior.IGNORE); + + // Test 2.6: AGGREGATION merge engine with delete behavior explicitly set to ALLOW - should + // be allowed + TablePath tablePathAggregateAllow = + TablePath.of("fluss", "test_allow_delete_for_aggregate"); + Map propertiesAggregateAllow = new HashMap<>(); + propertiesAggregateAllow.put(ConfigOptions.TABLE_MERGE_ENGINE.key(), "aggregation"); + propertiesAggregateAllow.put(ConfigOptions.TABLE_DELETE_BEHAVIOR.key(), "ALLOW"); + TableDescriptor tableDescriptorAggregateAllow = + TableDescriptor.builder() + .schema(aggregateSchema) + .comment("aggregate merge engine table with allow delete") + .properties(propertiesAggregateAllow) + .build(); + admin.createTable(tablePathAggregateAllow, tableDescriptorAggregateAllow, false).join(); + // Get the table and verify delete behavior is set to ALLOW + TableInfo tableInfoAggregateAllow = admin.getTableInfo(tablePathAggregateAllow).join(); + assertThat(tableInfoAggregateAllow.getTableConfig().getDeleteBehavior()) + .hasValue(DeleteBehavior.ALLOW); + // Test 3: FIRST_ROW merge engine with delete behavior explicitly set to ALLOW TablePath tablePath3 = TablePath.of("fluss", "test_allow_delete_for_first_row"); Map properties3 = new HashMap<>(); diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index c1059ac062..9f4b603e98 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -1420,9 +1420,10 @@ public class ConfigOptions { .noDefaultValue() .withDescription( "Defines the merge engine for the primary key table. By default, primary key table doesn't have merge engine. " - + "The supported merge engines are `first_row` and `versioned`. " + + "The supported merge engines are `first_row`, `versioned`, and `aggregation`. " + "The `first_row` merge engine will keep the first row of the same primary key. " - + "The `versioned` merge engine will keep the row with the largest version of the same primary key."); + + "The `versioned` merge engine will keep the row with the largest version of the same primary key. " + + "The `aggregation` merge engine will aggregate rows with the same primary key using field-level aggregate functions."); public static final ConfigOption TABLE_MERGE_ENGINE_VERSION_COLUMN = // we may need to introduce "del-column" in the future to support delete operation @@ -1440,10 +1441,11 @@ public class ConfigOptions { .withDescription( "Defines the delete behavior for the primary key table. " + "The supported delete behaviors are `allow`, `ignore`, and `disable`. " - + "The `allow` behavior allows normal delete operations (default). " + + "The `allow` behavior allows normal delete operations (default for default merge engine). " + "The `ignore` behavior silently skips delete requests without error. " + "The `disable` behavior rejects delete requests with a clear error message. " - + "For tables with FIRST_ROW or VERSIONED merge engines, this option defaults to `ignore`."); + + "For tables with FIRST_ROW, VERSIONED, or AGGREGATION merge engines, this option defaults to `ignore`. " + + "Note: For AGGREGATION merge engine, when set to `allow`, delete operations will remove the entire record."); public static final ConfigOption TABLE_AUTO_INCREMENT_FIELDS = key("table.auto-increment.fields") diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunction.java b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunction.java new file mode 100644 index 0000000000..996b3d3e3c --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunction.java @@ -0,0 +1,148 @@ +/* + * 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.fluss.metadata; + +import org.apache.fluss.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** + * Aggregation function with optional parameters for aggregate merge engine. + * + *

This class represents a parameterized aggregation function that can be applied to non-primary + * key columns in aggregation merge engine tables. It encapsulates both the function type and + * function-specific parameters (e.g., delimiter for LISTAGG). + * + *

Use {@link AggFunctions} utility class to create instances: + * + *

{@code
+ * AggFunction sumFunc = AggFunctions.SUM();
+ * AggFunction listaggFunc = AggFunctions.LISTAGG(";");
+ * }
+ * + * @since 0.9 + */ +@PublicEvolving +public final class AggFunction implements Serializable { + + private static final long serialVersionUID = 1L; + + private final AggFunctionType type; + private final Map parameters; + + /** + * Creates an aggregation function with the specified type and parameters. + * + * @param type the aggregation function type + * @param parameters the function parameters (nullable) + */ + AggFunction(AggFunctionType type, @Nullable Map parameters) { + this.type = Objects.requireNonNull(type, "Aggregation function type must not be null"); + this.parameters = + parameters == null || parameters.isEmpty() + ? Collections.emptyMap() + : Collections.unmodifiableMap(new HashMap<>(parameters)); + } + + /** + * Returns the aggregation function type. + * + * @return the function type + */ + public AggFunctionType getType() { + return type; + } + + /** + * Returns the function parameters. + * + * @return an immutable map of parameters + */ + public Map getParameters() { + return parameters; + } + + /** + * Gets a specific parameter value. + * + * @param key the parameter key + * @return the parameter value, or null if not found + */ + @Nullable + public String getParameter(String key) { + return parameters.get(key); + } + + /** + * Checks if this function has any parameters. + * + * @return true if parameters are present, false otherwise + */ + public boolean hasParameters() { + return !parameters.isEmpty(); + } + + /** + * Validates all parameters of this aggregation function. + * + *

This method checks that: + * + *

    + *
  • All parameter names are supported by the function type + *
  • All parameter values are valid + *
+ * + * @throws IllegalArgumentException if any parameter is invalid + */ + public void validate() { + for (Map.Entry entry : parameters.entrySet()) { + type.validateParameter(entry.getKey(), entry.getValue()); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AggFunction that = (AggFunction) o; + return type == that.type && parameters.equals(that.parameters); + } + + @Override + public int hashCode() { + return Objects.hash(type, parameters); + } + + @Override + public String toString() { + if (parameters.isEmpty()) { + return type.toString(); + } + return type.toString() + parameters; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctionType.java b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctionType.java new file mode 100644 index 0000000000..9cf148ad2e --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctionType.java @@ -0,0 +1,157 @@ +/* + * 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.fluss.metadata; + +import org.apache.fluss.annotation.PublicEvolving; + +import java.util.Collections; +import java.util.Locale; +import java.util.Set; + +/** + * Aggregation function type for aggregate merge engine. + * + *

This enum represents all supported aggregation function types that can be applied to + * non-primary key columns in aggregation merge engine tables. + */ +@PublicEvolving +public enum AggFunctionType { + // Numeric aggregation + SUM, + PRODUCT, + MAX, + MIN, + + // Value selection + LAST_VALUE, + LAST_VALUE_IGNORE_NULLS, + FIRST_VALUE, + FIRST_VALUE_IGNORE_NULLS, + + // String aggregation + LISTAGG, + STRING_AGG, // Alias for LISTAGG - maps to same factory + + // Boolean aggregation + BOOL_AND, + BOOL_OR; + + /** Parameter name for delimiter used in LISTAGG and STRING_AGG functions. */ + public static final String PARAM_DELIMITER = "delimiter"; + + /** + * Returns the set of supported parameter names for this aggregation function. + * + * @return an immutable set of parameter names + */ + public Set getSupportedParameters() { + switch (this) { + case LISTAGG: + case STRING_AGG: + // LISTAGG and STRING_AGG support optional "delimiter" parameter + return Collections.singleton(PARAM_DELIMITER); + default: + // All other functions do not accept parameters + return Collections.emptySet(); + } + } + + /** + * Validates a parameter value for this aggregation function. + * + * @param parameterName the parameter name + * @param parameterValue the parameter value + * @throws IllegalArgumentException if the parameter value is invalid + */ + public void validateParameter(String parameterName, String parameterValue) { + // Check if parameter is supported + if (!getSupportedParameters().contains(parameterName)) { + throw new IllegalArgumentException( + String.format( + "Parameter '%s' is not supported for aggregation function '%s'. " + + "Supported parameters: %s", + parameterName, + this, + getSupportedParameters().isEmpty() + ? "none" + : getSupportedParameters())); + } + + // Validate parameter value based on function type and parameter name + switch (this) { + case LISTAGG: + case STRING_AGG: + if (PARAM_DELIMITER.equals(parameterName)) { + if (parameterValue == null || parameterValue.isEmpty()) { + throw new IllegalArgumentException( + String.format( + "Parameter '%s' for aggregation function '%s' must be a non-empty string", + parameterName, this)); + } + } + break; + default: + // No validation needed for other functions (they don't have parameters) + break; + } + } + + /** + * Converts a string to an AggFunctionType enum value. + * + *

This method supports multiple naming formats: + * + *

    + *
  • Underscore format: "last_value_ignore_nulls" + *
  • Hyphen format: "last-value-ignore-nulls" + *
  • Case insensitive matching + *
+ * + *

Note: For string_agg, this will return STRING_AGG enum, but the server-side factory will + * map it to the same implementation as listagg. + * + * @param name the aggregation function type name + * @return the AggFunctionType enum value, or null if not found + */ + public static AggFunctionType fromString(String name) { + if (name == null || name.trim().isEmpty()) { + return null; + } + + // Normalize the input: convert hyphens to underscores and uppercase + String normalized = name.replace('-', '_').toUpperCase(Locale.ROOT).trim(); + + try { + return AggFunctionType.valueOf(normalized); + } catch (IllegalArgumentException e) { + return null; + } + } + + /** + * Converts this AggFunctionType to its string identifier. + * + *

The identifier is the lowercase name with underscores, e.g., "sum", "last_value". + * + * @return the identifier string + */ + @Override + public String toString() { + return name().toLowerCase(Locale.ROOT); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctions.java b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctions.java new file mode 100644 index 0000000000..7cc7024d0f --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/AggFunctions.java @@ -0,0 +1,300 @@ +/* + * 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.fluss.metadata; + +import org.apache.fluss.annotation.PublicEvolving; + +import java.util.HashMap; +import java.util.Map; + +/** + * Utility class for creating parameterized aggregation functions. + * + *

This class provides factory methods for all supported aggregation functions with their + * respective parameters. Use these methods when defining aggregate columns in a schema. + * + *

Example usage: + * + *

{@code
+ * Schema schema = Schema.newBuilder()
+ *     .column("id", DataTypes.BIGINT())
+ *     .column("count", DataTypes.BIGINT(), AggFunctions.SUM())
+ *     .column("tags", DataTypes.STRING(), AggFunctions.LISTAGG(";"))
+ *     .primaryKey("id")
+ *     .build();
+ * }
+ * + * @since 0.9 + */ +@PublicEvolving +// CHECKSTYLE.OFF: MethodName - Factory methods use uppercase naming convention like DataTypes +public final class AggFunctions { + + private AggFunctions() { + // Utility class, no instantiation + } + + // =================================================================================== + // Numeric Aggregation Functions + // =================================================================================== + + /** + * Creates a SUM aggregation function that computes the sum of numeric values. + * + *

Supported data types: TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, DECIMAL + * + * @return a SUM aggregation function + */ + public static AggFunction SUM() { + return new AggFunction(AggFunctionType.SUM, null); + } + + /** + * Creates a PRODUCT aggregation function that computes the product of numeric values. + * + *

Supported data types: TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, DECIMAL + * + * @return a PRODUCT aggregation function + */ + public static AggFunction PRODUCT() { + return new AggFunction(AggFunctionType.PRODUCT, null); + } + + /** + * Creates a MAX aggregation function that selects the maximum value. + * + *

Supported data types: CHAR, STRING, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, + * DECIMAL, DATE, TIME, TIMESTAMP, TIMESTAMP_LTZ + * + * @return a MAX aggregation function + */ + public static AggFunction MAX() { + return new AggFunction(AggFunctionType.MAX, null); + } + + /** + * Creates a MIN aggregation function that selects the minimum value. + * + *

Supported data types: CHAR, STRING, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, + * DECIMAL, DATE, TIME, TIMESTAMP, TIMESTAMP_LTZ + * + * @return a MIN aggregation function + */ + public static AggFunction MIN() { + return new AggFunction(AggFunctionType.MIN, null); + } + + // =================================================================================== + // Value Selection Functions + // =================================================================================== + + /** + * Creates a LAST_VALUE aggregation function that replaces the previous value with the most + * recently received value. + * + *

Supported data types: All data types + * + *

Null handling: Null values will overwrite previous values + * + * @return a LAST_VALUE aggregation function + */ + public static AggFunction LAST_VALUE() { + return new AggFunction(AggFunctionType.LAST_VALUE, null); + } + + /** + * Creates a LAST_VALUE_IGNORE_NULLS aggregation function that replaces the previous value with + * the latest non-null value. + * + *

This is the default aggregate function when no function is specified. + * + *

Supported data types: All data types + * + *

Null handling: Null values are ignored, previous value is retained + * + * @return a LAST_VALUE_IGNORE_NULLS aggregation function + */ + public static AggFunction LAST_VALUE_IGNORE_NULLS() { + return new AggFunction(AggFunctionType.LAST_VALUE_IGNORE_NULLS, null); + } + + /** + * Creates a FIRST_VALUE aggregation function that retains the first value seen for a field. + * + *

Supported data types: All data types + * + *

Null handling: Null values are retained if received first + * + * @return a FIRST_VALUE aggregation function + */ + public static AggFunction FIRST_VALUE() { + return new AggFunction(AggFunctionType.FIRST_VALUE, null); + } + + /** + * Creates a FIRST_VALUE_IGNORE_NULLS aggregation function that selects the first non-null value + * in a data set. + * + *

Supported data types: All data types + * + *

Null handling: Null values are ignored until a non-null value is received + * + * @return a FIRST_VALUE_IGNORE_NULLS aggregation function + */ + public static AggFunction FIRST_VALUE_IGNORE_NULLS() { + return new AggFunction(AggFunctionType.FIRST_VALUE_IGNORE_NULLS, null); + } + + // =================================================================================== + // String Aggregation Functions + // =================================================================================== + + /** + * Default delimiter for LISTAGG and STRING_AGG aggregation functions. + * + * @since 0.9 + */ + public static final String DEFAULT_LISTAGG_DELIMITER = ","; + + /** + * Creates a LISTAGG aggregation function with default comma delimiter. + * + *

Concatenates multiple string values into a single string with a delimiter. + * + *

Supported data types: STRING, CHAR + * + *

Null handling: Null values are skipped + * + * @return a LISTAGG aggregation function with default delimiter + */ + public static AggFunction LISTAGG() { + return new AggFunction(AggFunctionType.LISTAGG, null); + } + + /** + * Creates a LISTAGG aggregation function with the specified delimiter. + * + *

Concatenates multiple string values into a single string with the specified delimiter. + * + *

Supported data types: STRING, CHAR + * + *

Null handling: Null values are skipped + * + * @param delimiter the delimiter to use for concatenation + * @return a LISTAGG aggregation function with the specified delimiter + */ + public static AggFunction LISTAGG(String delimiter) { + Map params = new HashMap<>(); + params.put("delimiter", delimiter); + return new AggFunction(AggFunctionType.LISTAGG, params); + } + + /** + * Creates a STRING_AGG aggregation function with default comma delimiter. + * + *

Alias for {@link #LISTAGG()}. Concatenates multiple string values into a single string + * with a delimiter. + * + *

Supported data types: STRING, CHAR + * + *

Null handling: Null values are skipped + * + * @return a STRING_AGG aggregation function with default delimiter + */ + public static AggFunction STRING_AGG() { + return new AggFunction(AggFunctionType.STRING_AGG, null); + } + + /** + * Creates a STRING_AGG aggregation function with the specified delimiter. + * + *

Alias for {@link #LISTAGG(String)}. Concatenates multiple string values into a single + * string with the specified delimiter. + * + *

Supported data types: STRING, CHAR + * + *

Null handling: Null values are skipped + * + * @param delimiter the delimiter to use for concatenation + * @return a STRING_AGG aggregation function with the specified delimiter + */ + public static AggFunction STRING_AGG(String delimiter) { + Map params = new HashMap<>(); + params.put("delimiter", delimiter); + return new AggFunction(AggFunctionType.STRING_AGG, params); + } + + // =================================================================================== + // Boolean Aggregation Functions + // =================================================================================== + + /** + * Creates a BOOL_AND aggregation function that evaluates whether all boolean values in a set + * are true (logical AND). + * + *

Supported data types: BOOLEAN + * + *

Null handling: Null values are ignored + * + * @return a BOOL_AND aggregation function + */ + public static AggFunction BOOL_AND() { + return new AggFunction(AggFunctionType.BOOL_AND, null); + } + + /** + * Creates a BOOL_OR aggregation function that checks if at least one boolean value in a set is + * true (logical OR). + * + *

Supported data types: BOOLEAN + * + *

Null handling: Null values are ignored + * + * @return a BOOL_OR aggregation function + */ + public static AggFunction BOOL_OR() { + return new AggFunction(AggFunctionType.BOOL_OR, null); + } + + // =================================================================================== + // Internal Factory Methods + // =================================================================================== + + /** + * Creates an aggregation function from a type and parameters map. Used internally for + * deserialization. + * + * @param type the aggregation function type + * @param parameters the function parameters + * @return an aggregation function + */ + public static AggFunction of(AggFunctionType type, Map parameters) { + return new AggFunction(type, parameters); + } + + /** + * Creates an aggregation function from a type. Used internally for deserialization. + * + * @param type the aggregation function type + * @return an aggregation function + */ + public static AggFunction of(AggFunctionType type) { + return new AggFunction(type, null); + } +} +// CHECKSTYLE.ON: MethodName diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/DeleteBehavior.java b/fluss-common/src/main/java/org/apache/fluss/metadata/DeleteBehavior.java index 23ebf641d9..d4816a276e 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/DeleteBehavior.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/DeleteBehavior.java @@ -36,7 +36,7 @@ public enum DeleteBehavior { /** * Silently ignore delete requests without error. Delete operations will be dropped at the * server side, and no deletion will be performed. This is the default behavior for tables with - * FIRST_ROW or VERSIONED merge engines. + * FIRST_ROW, VERSIONED, or AGGREGATION merge engines. */ IGNORE, diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/MergeEngineType.java b/fluss-common/src/main/java/org/apache/fluss/metadata/MergeEngineType.java index 60573935ac..f4f94d5a74 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/MergeEngineType.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/MergeEngineType.java @@ -52,9 +52,16 @@ public enum MergeEngineType { *

  • Null version value is treated as the smallest version (i.e., Long.MIN_VALUE) * */ - VERSIONED; + VERSIONED, - // introduce AGGREGATE merge engine in the future + /** + * A merge engine that aggregates rows with the same primary key using field-level aggregate + * functions. Each non-primary-key field can have its own aggregate function (e.g., sum, max, + * min, last_value, etc.). This allows for flexible aggregation semantics at the field level. + * + * @since 0.9 + */ + AGGREGATION; /** Creates a {@link MergeEngineType} from the given string. */ public static MergeEngineType fromString(String type) { @@ -63,6 +70,8 @@ public static MergeEngineType fromString(String type) { return FIRST_ROW; case "VERSIONED": return VERSIONED; + case "AGGREGATION": + return AGGREGATION; default: throw new IllegalArgumentException("Unsupported merge engine type: " + type); } diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/Schema.java b/fluss-common/src/main/java/org/apache/fluss/metadata/Schema.java index 715fabf7c7..63066849b5 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/Schema.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/Schema.java @@ -106,6 +106,19 @@ public RowType getRowType() { return rowType; } + /** + * Gets the aggregation function for a specific column. + * + * @param columnName the column name + * @return the aggregation function, or empty if not configured + */ + public Optional getAggFunction(String columnName) { + return columns.stream() + .filter(col -> col.getName().equals(columnName)) + .findFirst() + .flatMap(Column::getAggFunction); + } + /** Returns the primary key indexes, if any, otherwise returns an empty array. */ public int[] getPrimaryKeyIndexes() { final List columns = getColumnNames(); @@ -282,10 +295,16 @@ public Builder fromColumns(List inputColumns) { } else { // if all columnId is not set, this maybe from old version schema. Just use its // position as columnId. - inputColumns.forEach( - column -> - this.column(column.columnName, column.dataType) - .withComment(column.comment)); + for (Column column : inputColumns) { + int newColumnId = highestFieldId.incrementAndGet(); + columns.add( + new Column( + column.columnName, + column.dataType, + column.comment, + newColumnId, + column.aggFunction)); + } } return this; @@ -303,7 +322,37 @@ public Builder fromColumns(List inputColumns) { public Builder column(String columnName, DataType dataType) { checkNotNull(columnName, "Column name must not be null."); checkNotNull(dataType, "Data type must not be null."); - columns.add(new Column(columnName, dataType, null, highestFieldId.incrementAndGet())); + columns.add( + new Column(columnName, dataType, null, highestFieldId.incrementAndGet(), null)); + return this; + } + + /** + * Declares a column with aggregation function that is appended to this schema. + * + *

    This method associates an aggregation function with a non-primary key column. It is + * only applicable when the table uses aggregation merge engine. + * + *

    If aggregation function is not specified for a non-primary key column, it defaults to + * {@link AggFunctions#LAST_VALUE_IGNORE_NULLS}. + * + * @param columnName the name of the column + * @param dataType the data type of the column + * @param aggFunction the aggregation function to apply + * @return the builder instance + */ + public Builder column(String columnName, DataType dataType, AggFunction aggFunction) { + checkNotNull(columnName, "Column name must not be null."); + checkNotNull(dataType, "Data type must not be null."); + checkNotNull(aggFunction, "Aggregation function must not be null."); + + columns.add( + new Column( + columnName, + dataType, + null, + highestFieldId.incrementAndGet(), + aggFunction)); return this; } @@ -402,6 +451,7 @@ public Schema build() { checkState( columns.stream().map(Column::getColumnId).distinct().count() == columns.size(), "Column ids must be unique."); + return new Schema(columns, primaryKey, highestFieldId.get(), autoIncrementColumnNames); } } @@ -423,21 +473,32 @@ public static final class Column implements Serializable { private final String columnName; private final DataType dataType; private final @Nullable String comment; + private final @Nullable AggFunction aggFunction; public Column(String columnName, DataType dataType) { - this(columnName, dataType, null, UNKNOWN_COLUMN_ID); + this(columnName, dataType, null, UNKNOWN_COLUMN_ID, null); } public Column(String columnName, DataType dataType, @Nullable String comment) { - this(columnName, dataType, comment, UNKNOWN_COLUMN_ID); + this(columnName, dataType, comment, UNKNOWN_COLUMN_ID, null); } public Column( String columnName, DataType dataType, @Nullable String comment, int columnId) { + this(columnName, dataType, comment, columnId, null); + } + + public Column( + String columnName, + DataType dataType, + @Nullable String comment, + int columnId, + @Nullable AggFunction aggFunction) { this.columnName = columnName; this.dataType = dataType; this.comment = comment; this.columnId = columnId; + this.aggFunction = aggFunction; } public String getName() { @@ -456,8 +517,21 @@ public DataType getDataType() { return dataType; } + /** + * Gets the aggregation function for this column. + * + * @return the aggregation function, or empty if not configured + */ + public Optional getAggFunction() { + return Optional.ofNullable(aggFunction); + } + public Column withComment(String comment) { - return new Column(columnName, dataType, comment, columnId); + return new Column(columnName, dataType, comment, columnId, aggFunction); + } + + public Column withAggFunction(@Nullable AggFunction aggFunction) { + return new Column(columnName, dataType, comment, columnId, aggFunction); } @Override @@ -486,12 +560,13 @@ public boolean equals(Object o) { return Objects.equals(columnName, that.columnName) && Objects.equals(dataType, that.dataType) && Objects.equals(comment, that.comment) - && Objects.equals(columnId, that.columnId); + && Objects.equals(columnId, that.columnId) + && Objects.equals(aggFunction, that.aggFunction); } @Override public int hashCode() { - return Objects.hash(columnName, dataType, comment, columnId); + return Objects.hash(columnName, dataType, comment, columnId, aggFunction); } } @@ -598,6 +673,19 @@ private static List normalizeColumns( !pkSet.contains(autoIncrementColumn), "Auto increment column can not be used as the primary key."); } + + // Validate that aggregation functions are only set for non-primary key columns + for (Column column : columns) { + // check presentation first for better performance + if (column.getAggFunction().isPresent() && pkSet.contains(column.getName())) { + throw new IllegalArgumentException( + String.format( + "Cannot set aggregation function for primary key column '%s'. " + + "Primary key columns automatically use 'primary-key' aggregation.", + column.getName())); + } + } + List newColumns = new ArrayList<>(); for (Column column : columns) { if (autoIncrementColumnNames.contains(column.getName())) { @@ -616,7 +704,8 @@ private static List normalizeColumns( column.getName(), column.getDataType().copy(false), column.getComment().isPresent() ? column.getComment().get() : null, - column.getColumnId())); + column.getColumnId(), + column.getAggFunction().orElse(null))); } else { newColumns.add(column); } diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java index 6275182c4c..903198fd6e 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/BinaryStringUtils.java @@ -17,11 +17,13 @@ package org.apache.fluss.utils; +import org.apache.fluss.row.BinarySegmentUtils; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; import java.time.DateTimeException; +import java.util.Arrays; import java.util.List; import java.util.TimeZone; import java.util.stream.Collectors; @@ -81,4 +83,125 @@ public static TimestampLtz toTimestampLtz( BinaryString input, int precision, TimeZone localTimeZone) throws DateTimeException { return DateTimeUtils.parseTimestampData(input.toString(), precision, localTimeZone); } + + /** + * Concatenates input strings together into a single string. Returns NULL if any argument is + * NULL. + * + *

    This method is optimized to avoid unnecessary string conversions and memory allocations. + * It directly operates on the underlying UTF-8 byte arrays of BinaryString objects. + * + * @param inputs the strings to concatenate + * @return the concatenated string, or NULL if any input is NULL + */ + public static BinaryString concat(BinaryString... inputs) { + return concat(Arrays.asList(inputs)); + } + + /** + * Concatenates input strings together into a single string. Returns NULL if any argument is + * NULL. + * + *

    This method is optimized to avoid unnecessary string conversions and memory allocations. + * It directly operates on the underlying UTF-8 byte arrays of BinaryString objects. + * + * @param inputs the strings to concatenate + * @return the concatenated string, or NULL if any input is NULL + */ + public static BinaryString concat(Iterable inputs) { + // Compute the total length of the result. + int totalLength = 0; + for (BinaryString input : inputs) { + if (input == null) { + return null; + } + + totalLength += input.getSizeInBytes(); + } + + // Allocate a new byte array, and copy the inputs one by one into it. + final byte[] result = new byte[totalLength]; + int offset = 0; + for (BinaryString input : inputs) { + if (input != null) { + int len = input.getSizeInBytes(); + BinarySegmentUtils.copyToBytes( + input.getSegments(), input.getOffset(), result, offset, len); + offset += len; + } + } + return BinaryString.fromBytes(result); + } + + /** + * Concatenates input strings together into a single string using the separator. Returns NULL if + * the separator is NULL. + * + *

    Note: CONCAT_WS() does not skip any empty strings, however it does skip any NULL values + * after the separator. For example, concat_ws(",", "a", null, "c") would yield "a,c". + * + * @param separator the separator to use between strings + * @param inputs the strings to concatenate + * @return the concatenated string with separator, or NULL if separator is NULL + */ + public static BinaryString concatWs(BinaryString separator, BinaryString... inputs) { + return concatWs(separator, Arrays.asList(inputs)); + } + + /** + * Concatenates input strings together into a single string using the separator. Returns NULL if + * the separator is NULL. + * + *

    Note: CONCAT_WS() does not skip any empty strings, however it does skip any NULL values + * after the separator. For example, concat_ws(",", "a", null, "c") would yield "a,c". + * + * @param separator the separator to use between strings + * @param inputs the strings to concatenate + * @return the concatenated string with separator, or NULL if separator is NULL + */ + public static BinaryString concatWs(BinaryString separator, Iterable inputs) { + if (null == separator) { + return null; + } + + int numInputBytes = 0; // total number of bytes from the inputs + int numInputs = 0; // number of non-null inputs + for (BinaryString input : inputs) { + if (input != null) { + numInputBytes += input.getSizeInBytes(); + numInputs++; + } + } + + if (numInputs == 0) { + // Return an empty string if there is no input, or all the inputs are null. + return BinaryString.EMPTY_UTF8; + } + + // Allocate a new byte array, and copy the inputs one by one into it. + // The size of the new array is the size of all inputs, plus the separators. + final byte[] result = + new byte[numInputBytes + (numInputs - 1) * separator.getSizeInBytes()]; + int offset = 0; + + int i = 0; + for (BinaryString input : inputs) { + if (input != null) { + int len = input.getSizeInBytes(); + BinarySegmentUtils.copyToBytes( + input.getSegments(), input.getOffset(), result, offset, len); + offset += len; + + i++; + // Add separator if this is not the last input + if (i < numInputs) { + int sepLen = separator.getSizeInBytes(); + BinarySegmentUtils.copyToBytes( + separator.getSegments(), separator.getOffset(), result, offset, sepLen); + offset += sepLen; + } + } + } + return BinaryString.fromBytes(result); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/InternalRowUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/InternalRowUtils.java new file mode 100644 index 0000000000..438e17e9f6 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/utils/InternalRowUtils.java @@ -0,0 +1,105 @@ +/* + * 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.fluss.utils; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.DataTypeRoot; + +/** Utility class for {@link org.apache.fluss.row.InternalRow} related operations. */ +public class InternalRowUtils { + + /** + * Compares two objects based on their data type. + * + * @param x the first object + * @param y the second object + * @param type the data type root + * @return a negative integer, zero, or a positive integer as x is less than, equal to, or + * greater than y + */ + public static int compare(Object x, Object y, DataTypeRoot type) { + int ret; + switch (type) { + case DECIMAL: + Decimal xDD = (Decimal) x; + Decimal yDD = (Decimal) y; + ret = xDD.compareTo(yDD); + break; + case TINYINT: + ret = Byte.compare((byte) x, (byte) y); + break; + case SMALLINT: + ret = Short.compare((short) x, (short) y); + break; + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + ret = Integer.compare((int) x, (int) y); + break; + case BIGINT: + ret = Long.compare((long) x, (long) y); + break; + case FLOAT: + ret = Float.compare((float) x, (float) y); + break; + case DOUBLE: + ret = Double.compare((double) x, (double) y); + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampNtz xNtz = (TimestampNtz) x; + TimestampNtz yNtz = (TimestampNtz) y; + ret = xNtz.compareTo(yNtz); + break; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + TimestampLtz xLtz = (TimestampLtz) x; + TimestampLtz yLtz = (TimestampLtz) y; + ret = xLtz.compareTo(yLtz); + break; + case BINARY: + case BYTES: + ret = byteArrayCompare((byte[]) x, (byte[]) y); + break; + case STRING: + case CHAR: + ret = ((BinaryString) x).compareTo((BinaryString) y); + break; + default: + throw new IllegalArgumentException("Incomparable type: " + type); + } + return ret; + } + + private static int byteArrayCompare(byte[] array1, byte[] array2) { + for (int i = 0, j = 0; i < array1.length && j < array2.length; i++, j++) { + int a = (array1[i] & 0xff); + int b = (array2[j] & 0xff); + if (a != b) { + return a - b; + } + } + return array1.length - array2.length; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/json/ColumnJsonSerde.java b/fluss-common/src/main/java/org/apache/fluss/utils/json/ColumnJsonSerde.java index 5db5b65ed6..cbddfacead 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/json/ColumnJsonSerde.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/json/ColumnJsonSerde.java @@ -18,12 +18,18 @@ package org.apache.fluss.utils.json; import org.apache.fluss.annotation.Internal; +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.metadata.AggFunctions; import org.apache.fluss.metadata.Schema; import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.fluss.types.DataType; import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; import static org.apache.fluss.metadata.Schema.Column.UNKNOWN_COLUMN_ID; @@ -37,6 +43,9 @@ public class ColumnJsonSerde static final String ID = "id"; static final String DATA_TYPE = "data_type"; static final String COMMENT = "comment"; + static final String AGG_FUNCTION = "agg_function"; + static final String AGG_FUNCTION_TYPE = "type"; + static final String AGG_FUNCTION_PARAMS = "parameters"; @Override public void serialize(Schema.Column column, JsonGenerator generator) throws IOException { @@ -49,6 +58,19 @@ public void serialize(Schema.Column column, JsonGenerator generator) throws IOEx if (column.getComment().isPresent()) { generator.writeStringField(COMMENT, column.getComment().get()); } + if (column.getAggFunction().isPresent()) { + AggFunction aggFunc = column.getAggFunction().get(); + generator.writeObjectFieldStart(AGG_FUNCTION); + generator.writeStringField(AGG_FUNCTION_TYPE, aggFunc.getType().toString()); + if (aggFunc.hasParameters()) { + generator.writeObjectFieldStart(AGG_FUNCTION_PARAMS); + for (Map.Entry entry : aggFunc.getParameters().entrySet()) { + generator.writeStringField(entry.getKey(), entry.getValue()); + } + generator.writeEndObject(); + } + generator.writeEndObject(); + } generator.writeNumberField(ID, column.getColumnId()); generator.writeEndObject(); @@ -60,10 +82,34 @@ public Schema.Column deserialize(JsonNode node) { DataType dataType = DataTypeJsonSerde.INSTANCE.deserialize(node.get(DATA_TYPE)); + AggFunction aggFunction = null; + if (node.hasNonNull(AGG_FUNCTION)) { + JsonNode aggFuncNode = node.get(AGG_FUNCTION); + + // Parse new format: object with type and parameters + String typeStr = aggFuncNode.get(AGG_FUNCTION_TYPE).asText(); + AggFunctionType type = AggFunctionType.fromString(typeStr); + + if (type != null) { + Map parameters = null; + if (aggFuncNode.hasNonNull(AGG_FUNCTION_PARAMS)) { + parameters = new HashMap<>(); + JsonNode paramsNode = aggFuncNode.get(AGG_FUNCTION_PARAMS); + Iterator> fields = paramsNode.fields(); + while (fields.hasNext()) { + Map.Entry entry = fields.next(); + parameters.put(entry.getKey(), entry.getValue().asText()); + } + } + aggFunction = AggFunctions.of(type, parameters); + } + } + return new Schema.Column( columnName, dataType, node.hasNonNull(COMMENT) ? node.get(COMMENT).asText() : null, - node.has(ID) ? node.get(ID).asInt() : UNKNOWN_COLUMN_ID); + node.has(ID) ? node.get(ID).asInt() : UNKNOWN_COLUMN_ID, + aggFunction); } } diff --git a/fluss-common/src/test/java/org/apache/fluss/config/TableConfigTest.java b/fluss-common/src/test/java/org/apache/fluss/config/TableConfigTest.java new file mode 100644 index 0000000000..5d18fcd1c9 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/config/TableConfigTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.config; + +import org.apache.fluss.metadata.DeleteBehavior; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link TableConfig}. */ +class TableConfigTest { + + @Test + void testDeleteBehavior() { + Configuration conf = new Configuration(); + TableConfig tableConfig = new TableConfig(conf); + + // Test default value (empty optional since not set) + assertThat(tableConfig.getDeleteBehavior()).isEmpty(); + + // Test configured value + conf.set(ConfigOptions.TABLE_DELETE_BEHAVIOR, DeleteBehavior.ALLOW); + TableConfig tableConfig2 = new TableConfig(conf); + assertThat(tableConfig2.getDeleteBehavior()).hasValue(DeleteBehavior.ALLOW); + + // Test IGNORE + conf.set(ConfigOptions.TABLE_DELETE_BEHAVIOR, DeleteBehavior.IGNORE); + TableConfig tableConfig3 = new TableConfig(conf); + assertThat(tableConfig3.getDeleteBehavior()).hasValue(DeleteBehavior.IGNORE); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/metadata/TableDescriptorTest.java b/fluss-common/src/test/java/org/apache/fluss/metadata/TableDescriptorTest.java index 2f8fffce98..430d47d904 100644 --- a/fluss-common/src/test/java/org/apache/fluss/metadata/TableDescriptorTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/metadata/TableDescriptorTest.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.HashMap; +import java.util.Map; import java.util.Optional; import static org.assertj.core.api.Assertions.assertThat; @@ -316,4 +317,36 @@ void testPartitionedTable() { .hasMessage( "Bucket key [f0, f3] shouldn't include any column in partition keys [f0]."); } + + @Test + void testInvalidListaggParameterEmptyDelimiter() { + // LISTAGG with empty delimiter - should fail + assertThatThrownBy(() -> AggFunctions.LISTAGG("").validate()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("must be a non-empty string"); + } + + @Test + void testInvalidListaggParameterUnknownParameter() { + // LISTAGG with unknown parameter - should fail + Map params = new HashMap<>(); + params.put("unknown_param", "value"); + + assertThatThrownBy(() -> AggFunctions.of(AggFunctionType.LISTAGG, params).validate()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("unknown_param") + .hasMessageContaining("not supported"); + } + + @Test + void testInvalidSumFunctionWithParameters() { + // SUM function does not accept parameters - should fail + Map params = new HashMap<>(); + params.put("some_param", "value"); + + assertThatThrownBy(() -> AggFunctions.of(AggFunctionType.SUM, params).validate()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("some_param") + .hasMessageContaining("not supported"); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/metadata/TableSchemaTest.java b/fluss-common/src/test/java/org/apache/fluss/metadata/TableSchemaTest.java index 9f35dd6021..32ba62b069 100644 --- a/fluss-common/src/test/java/org/apache/fluss/metadata/TableSchemaTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/metadata/TableSchemaTest.java @@ -121,4 +121,158 @@ void testAutoIncrementColumnSchema() { .isInstanceOf(IllegalStateException.class) .hasMessage("Auto increment column can only be used in primary-key table."); } + + @Test + void testSchemaBuilderColumnWithAggFunction() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("sum_val", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("max_val", DataTypes.INT(), AggFunctions.MAX()) + .column("min_val", DataTypes.INT(), AggFunctions.MIN()) + .column("last_val", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .column( + "last_non_null", + DataTypes.STRING(), + AggFunctions.LAST_VALUE_IGNORE_NULLS()) + .column("first_val", DataTypes.STRING(), AggFunctions.FIRST_VALUE()) + .column( + "first_non_null", + DataTypes.STRING(), + AggFunctions.FIRST_VALUE_IGNORE_NULLS()) + .column("listagg_val", DataTypes.STRING(), AggFunctions.LISTAGG()) + .column("string_agg_val", DataTypes.STRING(), AggFunctions.STRING_AGG()) + .column("bool_and_val", DataTypes.BOOLEAN(), AggFunctions.BOOL_AND()) + .column("bool_or_val", DataTypes.BOOLEAN(), AggFunctions.BOOL_OR()) + .primaryKey("id") + .build(); + + assertThat(schema.getAggFunction("sum_val").get()).isEqualTo(AggFunctions.SUM()); + assertThat(schema.getAggFunction("max_val").get()).isEqualTo(AggFunctions.MAX()); + assertThat(schema.getAggFunction("min_val").get()).isEqualTo(AggFunctions.MIN()); + assertThat(schema.getAggFunction("last_val").get()).isEqualTo(AggFunctions.LAST_VALUE()); + assertThat(schema.getAggFunction("last_non_null").get()) + .isEqualTo(AggFunctions.LAST_VALUE_IGNORE_NULLS()); + assertThat(schema.getAggFunction("first_val").get()).isEqualTo(AggFunctions.FIRST_VALUE()); + assertThat(schema.getAggFunction("first_non_null").get()) + .isEqualTo(AggFunctions.FIRST_VALUE_IGNORE_NULLS()); + assertThat(schema.getAggFunction("listagg_val").get()).isEqualTo(AggFunctions.LISTAGG()); + assertThat(schema.getAggFunction("string_agg_val").get()) + .isEqualTo(AggFunctions.STRING_AGG()); + assertThat(schema.getAggFunction("bool_and_val").get()).isEqualTo(AggFunctions.BOOL_AND()); + assertThat(schema.getAggFunction("bool_or_val").get()).isEqualTo(AggFunctions.BOOL_OR()); + } + + @Test + void testSchemaBuilderColumnWithAggFunctionThrowsExceptionForPrimaryKey() { + assertThatThrownBy( + () -> + Schema.newBuilder() + .column("id", DataTypes.INT(), AggFunctions.SUM()) + .column("value", DataTypes.BIGINT()) + .primaryKey("id") + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot set aggregation function for primary key column"); + } + + @Test + void testSchemaEqualityWithAggFunction() { + Schema schema1 = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + Schema schema2 = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + Schema schema3 = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BIGINT(), AggFunctions.MAX()) + .primaryKey("id") + .build(); + + assertThat(schema1).isEqualTo(schema2); + assertThat(schema1).isNotEqualTo(schema3); + } + + @Test + void testSchemaFromSchemaPreservesAggFunction() { + Schema original = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("max_val", DataTypes.INT(), AggFunctions.MAX()) + .primaryKey("id") + .build(); + + Schema copied = Schema.newBuilder().fromSchema(original).build(); + + assertThat(copied.getAggFunction("value")).isPresent(); + assertThat(copied.getAggFunction("value").get()).isEqualTo(AggFunctions.SUM()); + assertThat(copied.getAggFunction("max_val")).isPresent(); + assertThat(copied.getAggFunction("max_val").get()).isEqualTo(AggFunctions.MAX()); + } + + @Test + void testListaggWithCustomDelimiter() { + // Test LISTAGG with default delimiter (comma) + Schema schemaDefault = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("tags", DataTypes.STRING(), AggFunctions.LISTAGG()) + .primaryKey("id") + .build(); + + assertThat(schemaDefault.getAggFunction("tags")).isPresent(); + assertThat(schemaDefault.getAggFunction("tags").get()).isEqualTo(AggFunctions.LISTAGG()); + assertThat(schemaDefault.getAggFunction("tags").get().hasParameters()).isFalse(); + + // Test LISTAGG with custom delimiter + Schema schemaCustom = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("tags", DataTypes.STRING(), AggFunctions.LISTAGG(";")) + .column("values", DataTypes.STRING(), AggFunctions.LISTAGG("|")) + .column("paths", DataTypes.STRING(), AggFunctions.LISTAGG("/")) + .primaryKey("id") + .build(); + + assertThat(schemaCustom.getAggFunction("tags")).isPresent(); + assertThat(schemaCustom.getAggFunction("tags").get()).isEqualTo(AggFunctions.LISTAGG(";")); + assertThat(schemaCustom.getAggFunction("tags").get().getParameter("delimiter")) + .isEqualTo(";"); + + assertThat(schemaCustom.getAggFunction("values")).isPresent(); + assertThat(schemaCustom.getAggFunction("values").get()) + .isEqualTo(AggFunctions.LISTAGG("|")); + assertThat(schemaCustom.getAggFunction("values").get().getParameter("delimiter")) + .isEqualTo("|"); + + assertThat(schemaCustom.getAggFunction("paths")).isPresent(); + assertThat(schemaCustom.getAggFunction("paths").get()).isEqualTo(AggFunctions.LISTAGG("/")); + assertThat(schemaCustom.getAggFunction("paths").get().getParameter("delimiter")) + .isEqualTo("/"); + + // Test STRING_AGG with custom delimiter + Schema schemaStringAgg = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("items", DataTypes.STRING(), AggFunctions.STRING_AGG(", ")) + .primaryKey("id") + .build(); + + assertThat(schemaStringAgg.getAggFunction("items")).isPresent(); + assertThat(schemaStringAgg.getAggFunction("items").get()) + .isEqualTo(AggFunctions.STRING_AGG(", ")); + assertThat(schemaStringAgg.getAggFunction("items").get().getParameter("delimiter")) + .isEqualTo(", "); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/BinaryStringUtilsTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/BinaryStringUtilsTest.java new file mode 100644 index 0000000000..c615f5f640 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/utils/BinaryStringUtilsTest.java @@ -0,0 +1,162 @@ +/* + * 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.fluss.utils; + +import org.apache.fluss.row.BinaryString; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link BinaryStringUtils}. */ +public class BinaryStringUtilsTest { + + @Test + public void testConcat() { + // Test basic concatenation + BinaryString s1 = BinaryString.fromString("hello"); + BinaryString s2 = BinaryString.fromString(" "); + BinaryString s3 = BinaryString.fromString("world"); + + BinaryString result = BinaryStringUtils.concat(s1, s2, s3); + assertThat(result.toString()).isEqualTo("hello world"); + + // Test concatenation with empty string + BinaryString empty = BinaryString.fromString(""); + result = BinaryStringUtils.concat(s1, empty, s3); + assertThat(result.toString()).isEqualTo("helloworld"); + + // Test concatenation with single string + result = BinaryStringUtils.concat(s1); + assertThat(result.toString()).isEqualTo("hello"); + + // Test concatenation with null + result = BinaryStringUtils.concat(s1, null, s3); + assertThat(result).isNull(); + + // Test concatenation with Chinese characters + BinaryString chinese1 = BinaryString.fromString("你好"); + BinaryString chinese2 = BinaryString.fromString("世界"); + result = BinaryStringUtils.concat(chinese1, chinese2); + assertThat(result.toString()).isEqualTo("你好世界"); + + // Test concatenation with mixed ASCII and multi-byte characters + BinaryString mixed1 = BinaryString.fromString("Hello"); + BinaryString mixed2 = BinaryString.fromString("世界"); + BinaryString mixed3 = BinaryString.fromString("!"); + result = BinaryStringUtils.concat(mixed1, mixed2, mixed3); + assertThat(result.toString()).isEqualTo("Hello世界!"); + } + + @Test + public void testConcatWs() { + BinaryString sep = BinaryString.fromString(","); + BinaryString s1 = BinaryString.fromString("a"); + BinaryString s2 = BinaryString.fromString("b"); + BinaryString s3 = BinaryString.fromString("c"); + + // Test basic concatenation with separator + BinaryString result = BinaryStringUtils.concatWs(sep, s1, s2, s3); + assertThat(result.toString()).isEqualTo("a,b,c"); + + // Test with null separator + result = BinaryStringUtils.concatWs(null, s1, s2, s3); + assertThat(result).isNull(); + + // Test with null values in inputs (should skip nulls) + result = BinaryStringUtils.concatWs(sep, s1, null, s3); + assertThat(result.toString()).isEqualTo("a,c"); + + // Test with all null inputs + result = BinaryStringUtils.concatWs(sep, null, null, null); + assertThat(result).isEqualTo(BinaryString.EMPTY_UTF8); + + // Test with single input + result = BinaryStringUtils.concatWs(sep, s1); + assertThat(result.toString()).isEqualTo("a"); + + // Test with empty strings (should not skip empty strings) + BinaryString empty = BinaryString.fromString(""); + result = BinaryStringUtils.concatWs(sep, s1, empty, s3); + assertThat(result.toString()).isEqualTo("a,,c"); + + // Test with different separator + BinaryString dashSep = BinaryString.fromString("-"); + result = BinaryStringUtils.concatWs(dashSep, s1, s2, s3); + assertThat(result.toString()).isEqualTo("a-b-c"); + + // Test with multi-character separator + BinaryString multiSep = BinaryString.fromString(" | "); + result = BinaryStringUtils.concatWs(multiSep, s1, s2, s3); + assertThat(result.toString()).isEqualTo("a | b | c"); + + // Test with Chinese characters + BinaryString chineseSep = BinaryString.fromString(","); + BinaryString chinese1 = BinaryString.fromString("你好"); + BinaryString chinese2 = BinaryString.fromString("世界"); + result = BinaryStringUtils.concatWs(chineseSep, chinese1, chinese2); + assertThat(result.toString()).isEqualTo("你好,世界"); + } + + @Test + public void testConcatIterable() { + BinaryString s1 = BinaryString.fromString("a"); + BinaryString s2 = BinaryString.fromString("b"); + BinaryString s3 = BinaryString.fromString("c"); + + // Test with iterable + BinaryString result = BinaryStringUtils.concat(java.util.Arrays.asList(s1, s2, s3)); + assertThat(result.toString()).isEqualTo("abc"); + + // Test with null in iterable + result = BinaryStringUtils.concat(java.util.Arrays.asList(s1, null, s3)); + assertThat(result).isNull(); + } + + @Test + public void testConcatWsIterable() { + BinaryString sep = BinaryString.fromString(","); + BinaryString s1 = BinaryString.fromString("a"); + BinaryString s2 = BinaryString.fromString("b"); + BinaryString s3 = BinaryString.fromString("c"); + + // Test with iterable + BinaryString result = BinaryStringUtils.concatWs(sep, java.util.Arrays.asList(s1, s2, s3)); + assertThat(result.toString()).isEqualTo("a,b,c"); + + // Test with null values in iterable (should skip nulls) + result = BinaryStringUtils.concatWs(sep, java.util.Arrays.asList(s1, null, s3)); + assertThat(result.toString()).isEqualTo("a,c"); + } + + @Test + public void testConcatPerformance() { + // Test to ensure concat works correctly for repeated concatenations + // simulating listagg aggregation + BinaryString delimiter = BinaryString.fromString(","); + BinaryString accumulator = BinaryString.fromString("item1"); + + for (int i = 2; i <= 10; i++) { + BinaryString newItem = BinaryString.fromString("item" + i); + accumulator = BinaryStringUtils.concat(accumulator, delimiter, newItem); + } + + assertThat(accumulator.toString()) + .isEqualTo("item1,item2,item3,item4,item5,item6,item7,item8,item9,item10"); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/InternalRowUtilsTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/InternalRowUtilsTest.java new file mode 100644 index 0000000000..b0ffab698b --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/utils/InternalRowUtilsTest.java @@ -0,0 +1,118 @@ +/* + * 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.fluss.utils; + +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.DataTypeRoot; + +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; +import java.time.LocalDateTime; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link InternalRowUtils}. */ +public class InternalRowUtilsTest { + + @Test + public void testCompare() { + // test DECIMAL data type + Decimal xDecimalData = Decimal.fromBigDecimal(new BigDecimal("12.34"), 4, 2); + Decimal yDecimalData = Decimal.fromBigDecimal(new BigDecimal("13.14"), 4, 2); + assertThat(InternalRowUtils.compare(xDecimalData, yDecimalData, DataTypeRoot.DECIMAL)) + .isLessThan(0); + + // test DOUBLE data type + double xDouble = 13.14; + double yDouble = 12.13; + assertThat(InternalRowUtils.compare(xDouble, yDouble, DataTypeRoot.DOUBLE)) + .isGreaterThan(0); + + // test TIMESTAMP_WITHOUT_TIME_ZONE data type + TimestampNtz xTimestamp = TimestampNtz.fromLocalDateTime(LocalDateTime.now()); + TimestampNtz yTimestamp = TimestampNtz.fromLocalDateTime(xTimestamp.toLocalDateTime()); + assertThat( + InternalRowUtils.compare( + xTimestamp, yTimestamp, DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE)) + .isEqualTo(0); + + // test TIME_WITHOUT_TIME_ZONE data type + assertThat(InternalRowUtils.compare(165, 168, DataTypeRoot.TIME_WITHOUT_TIME_ZONE)) + .isLessThan(0); + + // test STRING type (fluss uses STRING instead of VARCHAR) + assertThat( + InternalRowUtils.compare( + BinaryString.fromString("a"), + BinaryString.fromString("b"), + DataTypeRoot.STRING)) + .isLessThan(0); + + // test CHAR type + assertThat( + InternalRowUtils.compare( + BinaryString.fromString("a"), + BinaryString.fromString("b"), + DataTypeRoot.CHAR)) + .isLessThan(0); + + // test TIMESTAMP_WITH_LOCAL_TIME_ZONE data type + long currentMillis = System.currentTimeMillis(); + TimestampLtz xLtz = TimestampLtz.fromEpochMillis(currentMillis); + TimestampLtz yLtz = TimestampLtz.fromEpochMillis(xLtz.getEpochMillisecond()); + assertThat( + InternalRowUtils.compare( + xLtz, yLtz, DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE)) + .isEqualTo(0); + + // test TINYINT + assertThat(InternalRowUtils.compare((byte) 5, (byte) 10, DataTypeRoot.TINYINT)) + .isLessThan(0); + + // test SMALLINT + assertThat(InternalRowUtils.compare((short) 5, (short) 10, DataTypeRoot.SMALLINT)) + .isLessThan(0); + + // test INTEGER + assertThat(InternalRowUtils.compare(5, 10, DataTypeRoot.INTEGER)).isLessThan(0); + + // test BIGINT + assertThat(InternalRowUtils.compare(5L, 10L, DataTypeRoot.BIGINT)).isLessThan(0); + + // test FLOAT + assertThat(InternalRowUtils.compare(5.0f, 10.0f, DataTypeRoot.FLOAT)).isLessThan(0); + + // test DATE + assertThat(InternalRowUtils.compare(100, 200, DataTypeRoot.DATE)).isLessThan(0); + + // test BINARY + byte[] xBinary = new byte[] {1, 2, 3}; + byte[] yBinary = new byte[] {1, 2, 4}; + assertThat(InternalRowUtils.compare(xBinary, yBinary, DataTypeRoot.BINARY)).isLessThan(0); + + // test BYTES + byte[] xBytes = new byte[] {1, 2}; + byte[] yBytes = new byte[] {1, 2, 3}; + assertThat(InternalRowUtils.compare(xBytes, yBytes, DataTypeRoot.BYTES)).isLessThan(0); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/json/SchemaJsonSerdeTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/json/SchemaJsonSerdeTest.java index 33acb30838..018214b3fb 100644 --- a/fluss-common/src/test/java/org/apache/fluss/utils/json/SchemaJsonSerdeTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/utils/json/SchemaJsonSerdeTest.java @@ -17,6 +17,7 @@ package org.apache.fluss.utils.json; +import org.apache.fluss.metadata.AggFunctions; import org.apache.fluss.metadata.Schema; import org.apache.fluss.types.DataTypes; @@ -89,19 +90,42 @@ public class SchemaJsonSerdeTest extends JsonSerdeTestBase { static final String SCHEMA_JSON_4 = "{\"version\":1,\"columns\":[{\"name\":\"a\",\"data_type\":{\"type\":\"INTEGER\",\"nullable\":false},\"comment\":\"a is first column\",\"id\":0},{\"name\":\"b\",\"data_type\":{\"type\":\"INTEGER\",\"nullable\":false},\"comment\":\"b is second column\",\"id\":1},{\"name\":\"c\",\"data_type\":{\"type\":\"CHAR\",\"nullable\":false,\"length\":10},\"comment\":\"c is third column\",\"id\":2}],\"primary_key\":[\"a\",\"c\"],\"auto_increment_column\":[\"b\"],\"highest_field_id\":2}"; + static final Schema SCHEMA_WITH_AGG = + Schema.newBuilder() + .column("product_id", DataTypes.BIGINT().copy(false)) + .column("total_sales", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("max_price", DataTypes.DECIMAL(10, 2), AggFunctions.MAX()) + .column( + "last_update_time", + DataTypes.TIMESTAMP(), + AggFunctions.LAST_VALUE_IGNORE_NULLS()) + .column("tags", DataTypes.STRING(), AggFunctions.LISTAGG(";")) + .column("categories", DataTypes.STRING(), AggFunctions.STRING_AGG("|")) + .column("labels", DataTypes.STRING(), AggFunctions.LISTAGG()) + .primaryKey("product_id") + .build(); + + static final String SCHEMA_JSON_WITH_AGG = + "{\"version\":1,\"columns\":[{\"name\":\"product_id\",\"data_type\":{\"type\":\"BIGINT\",\"nullable\":false},\"id\":0},{\"name\":\"total_sales\",\"data_type\":{\"type\":\"BIGINT\"},\"agg_function\":{\"type\":\"sum\"},\"id\":1},{\"name\":\"max_price\",\"data_type\":{\"type\":\"DECIMAL\",\"precision\":10,\"scale\":2},\"agg_function\":{\"type\":\"max\"},\"id\":2},{\"name\":\"last_update_time\",\"data_type\":{\"type\":\"TIMESTAMP_WITHOUT_TIME_ZONE\",\"precision\":6},\"agg_function\":{\"type\":\"last_value_ignore_nulls\"},\"id\":3},{\"name\":\"tags\",\"data_type\":{\"type\":\"STRING\"},\"agg_function\":{\"type\":\"listagg\",\"parameters\":{\"delimiter\":\";\"}},\"id\":4},{\"name\":\"categories\",\"data_type\":{\"type\":\"STRING\"},\"agg_function\":{\"type\":\"string_agg\",\"parameters\":{\"delimiter\":\"|\"}},\"id\":5},{\"name\":\"labels\",\"data_type\":{\"type\":\"STRING\"},\"agg_function\":{\"type\":\"listagg\"},\"id\":6}],\"primary_key\":[\"product_id\"],\"highest_field_id\":6}"; + SchemaJsonSerdeTest() { super(SchemaJsonSerde.INSTANCE); } @Override protected Schema[] createObjects() { - return new Schema[] {SCHEMA_0, SCHEMA_1, SCHEMA_2, SCHEMA_3, SCHEMA_4}; + return new Schema[] {SCHEMA_0, SCHEMA_1, SCHEMA_2, SCHEMA_3, SCHEMA_4, SCHEMA_WITH_AGG}; } @Override protected String[] expectedJsons() { return new String[] { - SCHEMA_JSON_0, SCHEMA_JSON_1, SCHEMA_JSON_1, SCHEMA_JSON_3, SCHEMA_JSON_4 + SCHEMA_JSON_0, + SCHEMA_JSON_1, + SCHEMA_JSON_1, + SCHEMA_JSON_3, + SCHEMA_JSON_4, + SCHEMA_JSON_WITH_AGG }; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java index 29478bb425..7fc53b96f8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java @@ -461,12 +461,14 @@ private TableDescriptor applySystemDefaults( ConfigOptions.TABLE_DATALAKE_ENABLED.key())); } - // For tables with first_row or versioned merge engines, automatically set to IGNORE if - // delete behavior is not set + // For tables with first_row, versioned or aggregation merge engines, automatically set to + // IGNORE if delete behavior is not set Configuration tableConf = Configuration.fromMap(tableDescriptor.getProperties()); MergeEngineType mergeEngine = tableConf.getOptional(ConfigOptions.TABLE_MERGE_ENGINE).orElse(null); - if (mergeEngine == MergeEngineType.FIRST_ROW || mergeEngine == MergeEngineType.VERSIONED) { + if (mergeEngine == MergeEngineType.FIRST_ROW + || mergeEngine == MergeEngineType.VERSIONED + || mergeEngine == MergeEngineType.AGGREGATION) { if (tableDescriptor.hasPrimaryKey() && !tableConf.getOptional(ConfigOptions.TABLE_DELETE_BEHAVIOR).isPresent()) { Map newProperties = new HashMap<>(newDescriptor.getProperties()); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java index 4353530903..0986c67f95 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java @@ -247,7 +247,7 @@ public KvTablet getOrCreateKv( } File tabletDir = getOrCreateTabletDir(tablePath, tableBucket); - RowMerger merger = RowMerger.create(tableConfig, kvFormat); + RowMerger merger = RowMerger.create(tableConfig, kvFormat, schemaGetter); KvTablet tablet = KvTablet.create( tablePath, @@ -356,7 +356,8 @@ public KvTablet loadKv(File tabletDir, SchemaGetter schemaGetter) throws Excepti TableInfo tableInfo = getTableInfo(zkClient, tablePath); TableConfig tableConfig = tableInfo.getTableConfig(); - RowMerger rowMerger = RowMerger.create(tableConfig, tableConfig.getKvFormat()); + RowMerger rowMerger = + RowMerger.create(tableConfig, tableConfig.getKvFormat(), schemaGetter); KvTablet kvTablet = KvTablet.create( physicalTablePath, diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMerger.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMerger.java new file mode 100644 index 0000000000..54583a90b3 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMerger.java @@ -0,0 +1,392 @@ +/* + * 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.fluss.server.kv.rowmerger; + +import org.apache.fluss.config.TableConfig; +import org.apache.fluss.metadata.DeleteBehavior; +import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaGetter; +import org.apache.fluss.record.BinaryValue; +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.row.encode.RowEncoder; +import org.apache.fluss.server.kv.rowmerger.aggregate.AggregateFieldsProcessor; +import org.apache.fluss.server.kv.rowmerger.aggregate.AggregationContext; +import org.apache.fluss.server.kv.rowmerger.aggregate.AggregationContextCache; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Arrays; +import java.util.BitSet; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * A row merger that aggregates rows with the same primary key using field-level aggregate + * functions. + * + *

    Each field can have its own aggregate function (e.g., sum, max, min, etc.). This allows for + * flexible aggregation semantics at the field level. + * + *

    This merger supports schema evolution by dynamically retrieving schemas based on schema IDs + * when merging rows with different schema versions. + * + *

    This class is thread-safe as it is guaranteed to be accessed by a single thread at a time + * (protected by KvTablet's write lock). + */ +public class AggregateRowMerger implements RowMerger { + + // Cache configuration constants + private static final int PARTIAL_MERGER_CACHE_MAX_SIZE = 4; + private static final Duration PARTIAL_MERGER_CACHE_EXPIRE_DURATION = Duration.ofMinutes(5); + + private final SchemaGetter schemaGetter; + private final DeleteBehavior deleteBehavior; + private final AggregationContextCache contextCache; + + // Cache for PartialAggregateRowMerger instances to avoid repeated creation + private final Cache partialMergerCache; + + // the current target schema id which is updated before merge() operation + private short targetSchemaId = -1; + + public AggregateRowMerger( + TableConfig tableConfig, KvFormat kvFormat, SchemaGetter schemaGetter) { + this.schemaGetter = schemaGetter; + // Extract configuration from TableConfig to ensure single source of truth + this.deleteBehavior = tableConfig.getDeleteBehavior().orElse(DeleteBehavior.IGNORE); + this.contextCache = new AggregationContextCache(schemaGetter, kvFormat); + // Initialize cache with same settings as PartialUpdaterCache and AggregationContextCache + this.partialMergerCache = + Caffeine.newBuilder() + .maximumSize(PARTIAL_MERGER_CACHE_MAX_SIZE) + .expireAfterAccess(PARTIAL_MERGER_CACHE_EXPIRE_DURATION) + .build(); + } + + @Override + public BinaryValue merge(BinaryValue oldValue, BinaryValue newValue) { + // First write: no existing row + if (oldValue == null || oldValue.row == null) { + return newValue; + } + + // Get contexts for schema evolution support + AggregationContext oldContext = contextCache.getContext(oldValue.schemaId); + AggregationContext newContext = contextCache.getContext(newValue.schemaId); + AggregationContext targetContext = contextCache.getContext(targetSchemaId); + + // Use target schema encoder to ensure merged row uses latest schema + RowEncoder encoder = targetContext.getRowEncoder(); + encoder.startNewRow(); + + // Aggregate using target schema context to ensure output uses server's latest schema + AggregateFieldsProcessor.aggregateAllFieldsWithTargetSchema( + oldValue.row, newValue.row, oldContext, newContext, targetContext, encoder); + BinaryRow mergedRow = encoder.finishRow(); + + return new BinaryValue(targetSchemaId, mergedRow); + } + + @Override + public BinaryValue delete(BinaryValue oldValue) { + // Remove the entire row (returns null to indicate deletion) + return null; + } + + @Override + public DeleteBehavior deleteBehavior() { + return deleteBehavior; + } + + @Override + public RowMerger configureTargetColumns( + @Nullable int[] targetColumns, short latestSchemaId, Schema latestSchema) { + if (targetColumns == null) { + this.targetSchemaId = latestSchemaId; + return this; + } + + // Use cache to get or create PartialAggregateRowMerger + // This avoids repeated object creation and BitSet construction + CacheKey cacheKey = new CacheKey(latestSchemaId, targetColumns); + return partialMergerCache.get( + cacheKey, + k -> { + // TODO: Currently, this conversion is broken when DROP COLUMN is supported, + // because `targetColumns` still references column indexes from an outdated + // schema, which no longer align with the current (latest) schema. + // In #2239, we plan to refactor `targetColumns` to use column IDs instead of + // indexes. Once that change is in place, this conversion logic can be safely + // removed. + List columns = latestSchema.getColumns(); + Set targetColumnIds = new HashSet<>(); + for (int colIdx : targetColumns) { + targetColumnIds.add(columns.get(colIdx).getColumnId()); + } + + // Build BitSet for fast target column lookup + BitSet targetColumnIdBitSet = new BitSet(); + for (Integer columnId : targetColumnIds) { + targetColumnIdBitSet.set(columnId); + } + + // Create the PartialAggregateRowMerger instance + return new PartialAggregateRowMerger( + targetColumnIdBitSet, + deleteBehavior, + schemaGetter, + contextCache, + latestSchema, + latestSchemaId); + }); + } + + /** + * Cache key for PartialAggregateRowMerger instances. + * + *

    Efficiently encodes schema ID and target column indices for cache lookup. Uses array + * content-based equality and hashCode for correct cache behavior. + */ + private static class CacheKey { + private final short schemaId; + private final int[] targetColumns; + private final int hashCode; + + CacheKey(short schemaId, int[] targetColumns) { + this.schemaId = schemaId; + this.targetColumns = targetColumns; + // Pre-compute hash code for efficiency + this.hashCode = computeHashCode(); + } + + private int computeHashCode() { + int result = schemaId; + result = 31 * result + Arrays.hashCode(targetColumns); + return result; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CacheKey cacheKey = (CacheKey) o; + return schemaId == cacheKey.schemaId + && Arrays.equals(targetColumns, cacheKey.targetColumns); + } + + @Override + public int hashCode() { + return hashCode; + } + } + + /** + * A merger that partially aggregates specified columns with the new row while keeping other + * columns unchanged. + * + *

    This merger is used when only a subset of columns need to be aggregated during merge + * operations, allowing for partial updates while preserving other column values. + * + *

    This merger supports schema evolution by using column IDs to identify target columns, + * rather than relying on field positions. + * + *

    Performance optimization: Uses BitSet for O(1) target column lookup and fast path for same + * schema scenarios. + * + *

    This class is thread-safe as it is guaranteed to be accessed by a single thread at a time + * (protected by KvTablet's write lock). + */ + private static class PartialAggregateRowMerger implements RowMerger { + // Cache size and expiration configuration + private static final int TARGET_POS_BITSET_CACHE_MAX_SIZE = 4; + private static final Duration TARGET_POS_BITSET_CACHE_EXPIRE_DURATION = + Duration.ofMinutes(5); + + // BitSet for fast O(1) lookup of target columns by columnId + private final BitSet targetColumnIdBitSet; + + private final DeleteBehavior deleteBehavior; + + // Schema evolution support + private final SchemaGetter schemaGetter; + private final AggregationContextCache contextCache; + // The schema ID this PartialAggregateRowMerger was created for (used for delete operations) + private final short targetSchemaId; + + // Cache for target position BitSets by schema ID to support schema evolution in delete + // operations + private final Cache targetPosBitSetCache; + + PartialAggregateRowMerger( + BitSet targetColumnIdBitSet, + DeleteBehavior deleteBehavior, + SchemaGetter schemaGetter, + AggregationContextCache contextCache, + Schema schema, + short schemaId) { + this.targetColumnIdBitSet = targetColumnIdBitSet; + this.deleteBehavior = deleteBehavior; + this.schemaGetter = schemaGetter; + this.contextCache = contextCache; + this.targetSchemaId = schemaId; + + // Perform sanity check using the provided schema + AggregationContext context = contextCache.getOrCreateContext(schemaId, schema); + context.sanityCheckTargetColumns(targetColumnIdBitSet); + + // Initialize cache for target position BitSets + this.targetPosBitSetCache = + Caffeine.newBuilder() + .maximumSize(TARGET_POS_BITSET_CACHE_MAX_SIZE) + .expireAfterAccess(TARGET_POS_BITSET_CACHE_EXPIRE_DURATION) + .build(); + } + + @Override + public BinaryValue merge(BinaryValue oldValue, BinaryValue newValue) { + // First write: no existing row + if (oldValue == null || oldValue.row == null) { + return newValue; + } + + // Get contexts for schema evolution support + AggregationContext oldContext = contextCache.getContext(oldValue.schemaId); + AggregationContext newContext = contextCache.getContext(newValue.schemaId); + AggregationContext targetContext = contextCache.getContext(targetSchemaId); + + // Use target schema encoder to ensure merged row uses latest schema + RowEncoder encoder = targetContext.getRowEncoder(); + encoder.startNewRow(); + + // Aggregate using target schema to ensure output uses server's latest schema + AggregateFieldsProcessor.aggregateTargetFieldsWithTargetSchema( + oldValue.row, + newValue.row, + oldContext, + newContext, + targetContext, + targetColumnIdBitSet, + encoder); + BinaryRow mergedRow = encoder.finishRow(); + + return new BinaryValue(targetSchemaId, mergedRow); + } + + @Override + public BinaryValue delete(BinaryValue oldValue) { + // Fast path: if oldValue uses the same schema as target, use simple logic + if (oldValue.schemaId == targetSchemaId) { + BitSet targetPosBitSet = getOrComputeTargetPosBitSet(targetSchemaId); + + // Check if all non-target columns are null + if (!AggregateFieldsProcessor.hasNonTargetNonNullField( + oldValue.row, targetPosBitSet)) { + return null; + } + + // Partial delete: set target columns (except primary key) to null + AggregationContext context = contextCache.getContext(targetSchemaId); + BitSet pkPosBitSet = context.getPrimaryKeyColsBitSet(); + + RowEncoder encoder = context.getRowEncoder(); + encoder.startNewRow(); + AggregateFieldsProcessor.encodePartialDeleteWithSameSchema( + oldValue.row, context, targetPosBitSet, pkPosBitSet, encoder); + BinaryRow deletedRow = encoder.finishRow(); + return new BinaryValue(targetSchemaId, deletedRow); + } + + // Schema evolution path: oldValue uses different schema + // Check non-target columns using old schema + BitSet oldTargetPosBitSet = getOrComputeTargetPosBitSet(oldValue.schemaId); + if (!AggregateFieldsProcessor.hasNonTargetNonNullField( + oldValue.row, oldTargetPosBitSet)) { + return null; + } + + // Get contexts for both schemas + AggregationContext oldContext = contextCache.getContext(oldValue.schemaId); + AggregationContext targetContext = contextCache.getContext(targetSchemaId); + BitSet targetPkPosBitSet = targetContext.getPrimaryKeyColsBitSet(); + + RowEncoder encoder = targetContext.getRowEncoder(); + encoder.startNewRow(); + AggregateFieldsProcessor.encodePartialDeleteWithDifferentSchema( + oldValue.row, + oldContext, + targetContext, + targetColumnIdBitSet, + targetPkPosBitSet, + encoder); + BinaryRow deletedRow = encoder.finishRow(); + return new BinaryValue(targetSchemaId, deletedRow); + } + + @Override + public DeleteBehavior deleteBehavior() { + return deleteBehavior; + } + + @Override + public RowMerger configureTargetColumns( + @Nullable int[] targetColumns, short latestSchemaId, Schema latestSchema) { + throw new IllegalStateException( + "PartialAggregateRowMerger does not support reconfigure target merge columns."); + } + + /** + * Get or compute target position BitSet for a given schema ID. + * + *

    This method uses a cache to avoid repeated computation of target position BitSets for + * different schemas during delete operations. + * + * @param schemaId the schema ID + * @return BitSet marking target column positions + */ + private BitSet getOrComputeTargetPosBitSet(short schemaId) { + return targetPosBitSetCache.get( + schemaId, + sid -> { + Schema schema = schemaGetter.getSchema(sid); + if (schema == null) { + throw new IllegalStateException( + String.format("Schema with ID %d not found", sid)); + } + // Compute target position BitSet by mapping column IDs to positions + BitSet targetPosBitSet = new BitSet(); + for (int pos = 0; pos < schema.getColumns().size(); pos++) { + int columnId = schema.getColumns().get(pos).getColumnId(); + if (targetColumnIdBitSet.get(columnId)) { + targetPosBitSet.set(pos); + } + } + return targetPosBitSet; + }); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/DefaultRowMerger.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/DefaultRowMerger.java index 2db73c819b..3717850fa8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/DefaultRowMerger.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/DefaultRowMerger.java @@ -65,14 +65,14 @@ public DeleteBehavior deleteBehavior() { @Override public RowMerger configureTargetColumns( - @Nullable int[] targetColumns, short schemaId, Schema schema) { + @Nullable int[] targetColumns, short latestShemaId, Schema latestSchema) { if (targetColumns == null) { return this; } else { // this also sanity checks the validity of the partial update PartialUpdater partialUpdater = partialUpdaterCache.getOrCreatePartialUpdater( - kvFormat, schemaId, schema, targetColumns); + kvFormat, latestShemaId, latestSchema, targetColumns); return new PartialUpdateRowMerger(partialUpdater, deleteBehavior); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/FirstRowRowMerger.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/FirstRowRowMerger.java index 33bd6ee1fd..7a279dd29d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/FirstRowRowMerger.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/FirstRowRowMerger.java @@ -63,7 +63,7 @@ public DeleteBehavior deleteBehavior() { @Override public RowMerger configureTargetColumns( - @Nullable int[] targetColumns, short schemaId, Schema schema) { + @Nullable int[] targetColumns, short latestShemaId, Schema latestSchema) { if (targetColumns == null) { return this; } else { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/RowMerger.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/RowMerger.java index 89bff8c0a2..ed8b37a658 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/RowMerger.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/RowMerger.java @@ -23,6 +23,7 @@ import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.MergeEngineType; import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaGetter; import org.apache.fluss.record.BinaryValue; import javax.annotation.Nullable; @@ -65,13 +66,22 @@ public interface RowMerger { * Dynamically configure the target columns to merge and return the effective merger. * * @param targetColumns the partial update target column positions, null means full update - * @param schemaId the schema id used to generate new rows - * @param schema the schema used to generate new rows + * @param latestShemaId the schema id used to generate new rows + * @param latestSchema the schema used to generate new rows */ - RowMerger configureTargetColumns(@Nullable int[] targetColumns, short schemaId, Schema schema); + RowMerger configureTargetColumns( + @Nullable int[] targetColumns, short latestShemaId, Schema latestSchema); - /** Create a row merger based on the given configuration. */ - static RowMerger create(TableConfig tableConf, KvFormat kvFormat) { + /** + * Create a row merger based on the given configuration. + * + * @param tableConf the table configuration + * @param kvFormat the kv format + * @param schemaGetter the schema getter for retrieving schemas by schema id (required for + * schema evolution support) + * @return the created row merger + */ + static RowMerger create(TableConfig tableConf, KvFormat kvFormat, SchemaGetter schemaGetter) { Optional mergeEngineType = tableConf.getMergeEngineType(); @Nullable DeleteBehavior deleteBehavior = tableConf.getDeleteBehavior().orElse(null); @@ -88,6 +98,8 @@ static RowMerger create(TableConfig tableConf, KvFormat kvFormat) { ConfigOptions.TABLE_MERGE_ENGINE_VERSION_COLUMN.key())); } return new VersionedRowMerger(versionColumn.get(), deleteBehavior); + case AGGREGATION: + return new AggregateRowMerger(tableConf, kvFormat, schemaGetter); default: throw new IllegalArgumentException( "Unsupported merge engine type: " + mergeEngineType.get()); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/VersionedRowMerger.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/VersionedRowMerger.java index 4b555602e7..0bc64f6c9b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/VersionedRowMerger.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/VersionedRowMerger.java @@ -84,12 +84,12 @@ public DeleteBehavior deleteBehavior() { @Override public RowMerger configureTargetColumns( - @Nullable int[] targetColumns, short schemaId, Schema schema) { + @Nullable int[] targetColumns, short latestShemaId, Schema latestSchema) { if (targetColumns == null) { - if (schemaId != this.schemaId) { - this.schemaId = schemaId; + if (latestShemaId != this.schemaId) { + this.schemaId = latestShemaId; this.versionComparator = - createVersionComparator(schema.getRowType(), versionColumnName); + createVersionComparator(latestSchema.getRowType(), versionColumnName); } return this; } else { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregateFieldsProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregateFieldsProcessor.java new file mode 100644 index 0000000000..0429a2015b --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregateFieldsProcessor.java @@ -0,0 +1,385 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate; + +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.encode.RowEncoder; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldAggregator; + +import java.util.BitSet; +import java.util.List; + +/** + * Utility class for processing aggregated fields from old and new rows. + * + *

    This class provides reusable methods for aggregating fields with different strategies (full + * aggregation or partial aggregation with target columns). It handles schema evolution by matching + * fields using column IDs. + * + *

    Note: This class processes aggregation logic and delegates encoding to {@link RowEncoder}. The + * class is designed to be stateless and thread-safe, with all methods being static. + */ +public final class AggregateFieldsProcessor { + + // A FieldGetter that always returns null, used for non-existent columns in schema evolution + private static final InternalRow.FieldGetter NULL_FIELD_GETTER = row -> null; + + // Private constructor to prevent instantiation + private AggregateFieldsProcessor() {} + + /** + * Aggregate all fields from old and new rows with explicit target schema (full aggregation). + * + *

    This variant allows specifying a different target schema for the output, which is useful + * when the client's newValue uses an outdated schema but we want to output using the latest + * server schema. + * + *

    This method handles schema evolution by matching fields using column IDs across three + * potentially different schemas: old row schema, new row schema, and target output schema. + * + * @param oldRow the old row + * @param newRow the new row + * @param oldContext context for the old row schema + * @param newInputContext context for the new row schema (for reading newRow) + * @param targetContext context for the target output schema + * @param encoder the row encoder to encode results (should match targetContext) + */ + public static void aggregateAllFieldsWithTargetSchema( + BinaryRow oldRow, + BinaryRow newRow, + AggregationContext oldContext, + AggregationContext newInputContext, + AggregationContext targetContext, + RowEncoder encoder) { + // Fast path: all three schemas are the same + if (targetContext == oldContext && targetContext == newInputContext) { + aggregateAllFieldsWithSameSchema(oldRow, newRow, targetContext, encoder); + return; + } + + // General path: iterate over target schema columns and aggregate using column ID matching + InternalRow.FieldGetter[] oldFieldGetters = oldContext.getFieldGetters(); + InternalRow.FieldGetter[] newFieldGetters = newInputContext.getFieldGetters(); + FieldAggregator[] targetAggregators = targetContext.getAggregators(); + List targetColumns = targetContext.getSchema().getColumns(); + + for (int targetIdx = 0; targetIdx < targetColumns.size(); targetIdx++) { + Schema.Column targetColumn = targetColumns.get(targetIdx); + int columnId = targetColumn.getColumnId(); + + // Find corresponding fields in old and new schemas using column ID + Integer oldIdx = oldContext.getFieldIndex(columnId); + Integer newIdx = newInputContext.getFieldIndex(columnId); + + // Get field getters (use NULL_FIELD_GETTER if column doesn't exist in that schema) + InternalRow.FieldGetter oldFieldGetter = + (oldIdx != null) ? oldFieldGetters[oldIdx] : NULL_FIELD_GETTER; + InternalRow.FieldGetter newFieldGetter = + (newIdx != null) ? newFieldGetters[newIdx] : NULL_FIELD_GETTER; + + // Aggregate and encode using target schema's aggregator + aggregateAndEncode( + oldFieldGetter, + newFieldGetter, + oldRow, + newRow, + targetAggregators[targetIdx], + targetIdx, + encoder); + } + } + + /** + * Aggregate and encode a single field. + * + * @param oldFieldGetter getter for the old field + * @param newFieldGetter getter for the new field + * @param oldRow the old row + * @param newRow the new row + * @param aggregator the aggregator for this field + * @param targetIdx the target index to encode + * @param encoder the row encoder + */ + private static void aggregateAndEncode( + InternalRow.FieldGetter oldFieldGetter, + InternalRow.FieldGetter newFieldGetter, + BinaryRow oldRow, + BinaryRow newRow, + FieldAggregator aggregator, + int targetIdx, + RowEncoder encoder) { + Object accumulator = oldFieldGetter.getFieldOrNull(oldRow); + Object inputField = newFieldGetter.getFieldOrNull(newRow); + Object mergedField = aggregator.agg(accumulator, inputField); + encoder.encodeField(targetIdx, mergedField); + } + + /** + * Copy and encode a field value from old row. + * + * @param fieldGetter getter for the field + * @param oldRow the old row + * @param targetIdx the target index to encode + * @param encoder the row encoder + */ + private static void copyOldValueAndEncode( + InternalRow.FieldGetter fieldGetter, + BinaryRow oldRow, + int targetIdx, + RowEncoder encoder) { + encoder.encodeField(targetIdx, fieldGetter.getFieldOrNull(oldRow)); + } + + /** + * Aggregate target fields from old and new rows with explicit target schema (partial + * aggregation). + * + *

    This variant allows specifying a different target schema for the output, which is useful + * when the client's newValue uses an outdated schema but we want to output using the latest + * server schema. + * + *

    For target columns, aggregate with the aggregation function. For non-target columns, keep + * the old value unchanged. For columns that don't exist in old schema, copy from newRow. For + * columns that exist only in target schema, set to null. + * + * @param oldRow the old row + * @param newRow the new row + * @param oldContext context for the old row schema + * @param newInputContext context for the new row schema (for reading newRow) + * @param targetContext context for the target output schema + * @param targetColumnIdBitSet BitSet marking target columns by column ID + * @param encoder the row encoder to encode results (should match targetContext) + */ + public static void aggregateTargetFieldsWithTargetSchema( + BinaryRow oldRow, + BinaryRow newRow, + AggregationContext oldContext, + AggregationContext newInputContext, + AggregationContext targetContext, + BitSet targetColumnIdBitSet, + RowEncoder encoder) { + // Fast path: all three schemas are the same + if (targetContext == oldContext && targetContext == newInputContext) { + aggregateTargetFieldsWithSameSchema( + oldRow, newRow, targetContext, targetColumnIdBitSet, encoder); + return; + } + + // General path: iterate over target schema columns + InternalRow.FieldGetter[] oldFieldGetters = oldContext.getFieldGetters(); + InternalRow.FieldGetter[] newFieldGetters = newInputContext.getFieldGetters(); + FieldAggregator[] targetAggregators = targetContext.getAggregators(); + List targetColumns = targetContext.getSchema().getColumns(); + + for (int targetIdx = 0; targetIdx < targetColumns.size(); targetIdx++) { + Schema.Column targetColumn = targetColumns.get(targetIdx); + int columnId = targetColumn.getColumnId(); + + // Find corresponding fields in old and new schemas using column ID + Integer oldIdx = oldContext.getFieldIndex(columnId); + Integer newIdx = newInputContext.getFieldIndex(columnId); + + if (targetColumnIdBitSet.get(columnId)) { + // Target column: aggregate and encode + InternalRow.FieldGetter oldFieldGetter = + (oldIdx != null) ? oldFieldGetters[oldIdx] : NULL_FIELD_GETTER; + InternalRow.FieldGetter newFieldGetter = + (newIdx != null) ? newFieldGetters[newIdx] : NULL_FIELD_GETTER; + aggregateAndEncode( + oldFieldGetter, + newFieldGetter, + oldRow, + newRow, + targetAggregators[targetIdx], + targetIdx, + encoder); + } else if (oldIdx != null) { + // Non-target column that exists in old schema: copy old value + copyOldValueAndEncode(oldFieldGetters[oldIdx], oldRow, targetIdx, encoder); + } else { + // Non-target column that doesn't exist in old schema: set to null + // NOTE: In partial aggregation, non-target columns should not use values from + // newRow, even if they exist in newRow's schema, as only target columns are + // aggregated + encoder.encodeField(targetIdx, null); + } + } + } + + /** + * Aggregate all fields when old and new schemas are identical. + * + *

    Fast path: field positions match directly, no column ID lookup needed. + */ + private static void aggregateAllFieldsWithSameSchema( + BinaryRow oldRow, BinaryRow newRow, AggregationContext context, RowEncoder encoder) { + InternalRow.FieldGetter[] fieldGetters = context.getFieldGetters(); + FieldAggregator[] aggregators = context.getAggregators(); + int fieldCount = context.getFieldCount(); + + for (int idx = 0; idx < fieldCount; idx++) { + aggregateAndEncode( + fieldGetters[idx], + fieldGetters[idx], + oldRow, + newRow, + aggregators[idx], + idx, + encoder); + } + } + + /** + * Aggregate target fields when old and new schemas are identical. + * + *

    Fast path: field positions match directly, no column ID lookup needed. + */ + private static void aggregateTargetFieldsWithSameSchema( + BinaryRow oldRow, + BinaryRow newRow, + AggregationContext context, + BitSet targetColumnIdBitSet, + RowEncoder encoder) { + InternalRow.FieldGetter[] fieldGetters = context.getFieldGetters(); + FieldAggregator[] aggregators = context.getAggregators(); + List columns = context.getSchema().getColumns(); + int fieldCount = context.getFieldCount(); + + for (int idx = 0; idx < fieldCount; idx++) { + int columnId = columns.get(idx).getColumnId(); + + if (targetColumnIdBitSet.get(columnId)) { + // Target column: aggregate and encode + aggregateAndEncode( + fieldGetters[idx], + fieldGetters[idx], + oldRow, + newRow, + aggregators[idx], + idx, + encoder); + } else { + // Non-target column: encode old value + copyOldValueAndEncode(fieldGetters[idx], oldRow, idx, encoder); + } + } + } + + /** + * Encode a partial delete when old and new schemas are identical. + * + *

    Set target columns (except primary key) to null, keep other columns unchanged. + * + *

    Fast path: field positions match directly, no column ID lookup needed. + * + * @param oldRow the old row to partially delete + * @param context the aggregation context for encoding + * @param targetPosBitSet BitSet marking target column positions + * @param pkPosBitSet BitSet marking primary key positions + * @param encoder the row encoder to encode results + */ + public static void encodePartialDeleteWithSameSchema( + BinaryRow oldRow, + AggregationContext context, + BitSet targetPosBitSet, + BitSet pkPosBitSet, + RowEncoder encoder) { + InternalRow.FieldGetter[] fieldGetters = context.getFieldGetters(); + + for (int i = 0; i < oldRow.getFieldCount(); i++) { + if (targetPosBitSet.get(i) && !pkPosBitSet.get(i)) { + // Target column (not primary key): set to null + encoder.encodeField(i, null); + } else { + // Non-target column or primary key: keep old value + copyOldValueAndEncode(fieldGetters[i], oldRow, i, encoder); + } + } + } + + /** + * Encode a partial delete when old and new schemas differ (schema evolution). + * + *

    Set target columns (except primary key) to null, keep other columns unchanged. For new + * columns that don't exist in old schema, set to null. + * + *

    Slow path: requires column ID matching to find corresponding fields between schemas. + * + * @param oldRow the old row to partially delete + * @param oldContext context for the old schema + * @param targetContext context for the target schema + * @param targetColumnIdBitSet BitSet marking target columns by column ID + * @param pkPosBitSet BitSet marking primary key positions in target schema + * @param encoder the row encoder to encode results + */ + public static void encodePartialDeleteWithDifferentSchema( + BinaryRow oldRow, + AggregationContext oldContext, + AggregationContext targetContext, + BitSet targetColumnIdBitSet, + BitSet pkPosBitSet, + RowEncoder encoder) { + InternalRow.FieldGetter[] oldFieldGetters = oldContext.getFieldGetters(); + List targetColumns = targetContext.getSchema().getColumns(); + + for (int targetIdx = 0; targetIdx < targetColumns.size(); targetIdx++) { + Schema.Column targetColumn = targetColumns.get(targetIdx); + int columnId = targetColumn.getColumnId(); + + // Find corresponding field in old schema using column ID + Integer oldIdx = oldContext.getFieldIndex(columnId); + + // Check if this is a target column using columnId (not position) + boolean isTargetColumn = targetColumnIdBitSet.get(columnId); + boolean isPrimaryKey = pkPosBitSet.get(targetIdx); + + if (isTargetColumn && !isPrimaryKey) { + // Target column (not primary key): set to null + encoder.encodeField(targetIdx, null); + } else if (oldIdx != null) { + // Column exists in old schema: copy value from old row + copyOldValueAndEncode(oldFieldGetters[oldIdx], oldRow, targetIdx, encoder); + } else { + // New column that doesn't exist in old schema: set to null + encoder.encodeField(targetIdx, null); + } + } + } + + /** + * Check if there are any non-null fields in non-target columns. + * + * @param row the row to check + * @param targetPosBitSet BitSet marking target column positions + * @return true if at least one non-target column has a non-null value + */ + public static boolean hasNonTargetNonNullField(BinaryRow row, BitSet targetPosBitSet) { + int fieldCount = row.getFieldCount(); + // Use nextClearBit to iterate over non-target fields (bits not set in targetPosBitSet) + for (int pos = targetPosBitSet.nextClearBit(0); + pos < fieldCount; + pos = targetPosBitSet.nextClearBit(pos + 1)) { + if (!row.isNullAt(pos)) { + return true; + } + } + return false; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContext.java new file mode 100644 index 0000000000..7cec63a4e8 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContext.java @@ -0,0 +1,296 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate; + +import org.apache.fluss.exception.InvalidTargetColumnException; +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.metadata.AggFunctions; +import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.encode.RowEncoder; +import org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldAggregatorFactory; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldAggregator; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.RowType; + +import java.util.BitSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Context for aggregation operations, containing field getters, aggregators, and encoder for a + * specific schema. + * + *

    This class supports schema evolution by using column IDs to match fields across different + * schema versions, rather than relying on field positions. + * + *

    This class is thread-safe as it is guaranteed to be accessed by a single thread at a time + * (protected by KvTablet's write lock). + */ +public class AggregationContext { + final Schema schema; + final RowType rowType; + final InternalRow.FieldGetter[] fieldGetters; + final FieldAggregator[] aggregators; + final RowEncoder rowEncoder; + final int fieldCount; + + /** + * Mapping from column ID to field index in this schema. This is used for schema evolution to + * correctly match fields between old and new schemas. + */ + private final Map columnIdToIndex; + + /** + * BitSet marking primary key column positions for fast O(1) lookup. This is useful for partial + * update and delete operations. + */ + private final BitSet primaryKeyColsBitSet; + + private AggregationContext( + Schema schema, + RowType rowType, + InternalRow.FieldGetter[] fieldGetters, + FieldAggregator[] aggregators, + RowEncoder rowEncoder) { + this.schema = schema; + this.rowType = rowType; + this.fieldGetters = fieldGetters; + this.aggregators = aggregators; + this.rowEncoder = rowEncoder; + this.fieldCount = rowType.getFieldCount(); + + // Build columnId to index mapping for schema evolution support + this.columnIdToIndex = new HashMap<>(); + List columns = schema.getColumns(); + for (int i = 0; i < columns.size(); i++) { + columnIdToIndex.put(columns.get(i).getColumnId(), i); + } + + // Initialize primary key columns BitSet for fast lookup + this.primaryKeyColsBitSet = new BitSet(); + for (int pkIndex : schema.getPrimaryKeyIndexes()) { + primaryKeyColsBitSet.set(pkIndex); + } + } + + public Schema getSchema() { + return schema; + } + + public RowType getRowType() { + return rowType; + } + + public InternalRow.FieldGetter[] getFieldGetters() { + return fieldGetters; + } + + public FieldAggregator[] getAggregators() { + return aggregators; + } + + public int getFieldCount() { + return fieldCount; + } + + /** + * Get the field index for a given column ID. + * + * @param columnId the column ID + * @return the field index, or null if the column doesn't exist in this schema + */ + public Integer getFieldIndex(int columnId) { + return columnIdToIndex.get(columnId); + } + + /** + * Get the BitSet marking primary key column positions. + * + * @return BitSet with primary key positions set to true + */ + public BitSet getPrimaryKeyColsBitSet() { + return primaryKeyColsBitSet; + } + + /** + * Sanity check for target columns used in partial aggregate operations. + * + *

    Validates that: 1. Target columns must contain all primary key columns 2. All non-primary + * key columns must be nullable (for partial update semantics) + * + * @param targetColumnIdBitSet BitSet of target column IDs + * @throws InvalidTargetColumnException if validation fails + */ + public void sanityCheckTargetColumns(BitSet targetColumnIdBitSet) { + // Build target column position set from targetColumnIds + BitSet targetColumnPosBitSet = new BitSet(); + DataType[] fieldDataTypes = schema.getRowType().getChildren().toArray(new DataType[0]); + + // Map column IDs to positions + for (int pos = 0; pos < schema.getColumns().size(); pos++) { + int columnId = schema.getColumns().get(pos).getColumnId(); + if (targetColumnIdBitSet.get(columnId)) { + targetColumnPosBitSet.set(pos); + } + } + + // Check 1: target columns must contain all primary key columns + for (int pkIndex : schema.getPrimaryKeyIndexes()) { + if (!targetColumnPosBitSet.get(pkIndex)) { + throw new InvalidTargetColumnException( + String.format( + "The target write columns must contain the primary key columns %s.", + schema.getColumnNames(schema.getPrimaryKeyIndexes()))); + } + } + + // Check 2: all non-primary key columns must be nullable + for (int i = 0; i < fieldDataTypes.length; i++) { + if (!primaryKeyColsBitSet.get(i)) { + if (!fieldDataTypes[i].isNullable()) { + throw new InvalidTargetColumnException( + String.format( + "Partial aggregate requires all columns except primary key to be nullable, " + + "but column %s is NOT NULL.", + schema.getRowType().getFieldNames().get(i))); + } + } + } + } + + /** + * Get the row encoder for encoding rows. + * + *

    This method returns the RowEncoder instance for encoding rows. Callers should use the + * standard RowEncoder pattern: + * + *

    {@code
    +     * RowEncoder encoder = context.getRowEncoder();
    +     * encoder.startNewRow();
    +     * for (int i = 0; i < fieldCount; i++) {
    +     *     encoder.encodeField(i, value);
    +     * }
    +     * BinaryRow result = encoder.finishRow();
    +     * }
    + * + *

    This is safe because KvTablet ensures single-threaded access via write lock. + * + * @return the row encoder for this schema + */ + public RowEncoder getRowEncoder() { + return rowEncoder; + } + + /** + * Create an aggregation context for a given schema. + * + * @param schema the schema + * @param kvFormat the KV format + * @return the aggregation context + */ + public static AggregationContext create(Schema schema, KvFormat kvFormat) { + RowType rowType = schema.getRowType(); + int fieldCount = rowType.getFieldCount(); + + // Create field getters + InternalRow.FieldGetter[] fieldGetters = new InternalRow.FieldGetter[fieldCount]; + for (int i = 0; i < fieldCount; i++) { + fieldGetters[i] = + InternalRow.createFieldGetter(rowType.getFields().get(i).getType(), i); + } + + // Create aggregators + FieldAggregator[] aggregators = createAggregators(schema); + + // Create row encoder + RowEncoder rowEncoder = RowEncoder.create(kvFormat, rowType); + + return new AggregationContext(schema, rowType, fieldGetters, aggregators, rowEncoder); + } + + /** + * Creates an array of field aggregators for all fields in the schema. + * + *

    This method reads aggregation functions from the Schema object and creates the appropriate + * aggregators with their parameters. + * + * @param schema the Schema object containing column definitions and aggregation functions + * @return an array of field aggregators, one for each field + */ + private static FieldAggregator[] createAggregators(Schema schema) { + RowType rowType = schema.getRowType(); + List primaryKeys = schema.getPrimaryKeyColumnNames(); + List fieldNames = rowType.getFieldNames(); + int fieldCount = rowType.getFieldCount(); + + FieldAggregator[] aggregators = new FieldAggregator[fieldCount]; + + for (int i = 0; i < fieldCount; i++) { + String fieldName = fieldNames.get(i); + DataType fieldType = rowType.getTypeAt(i); + + // Get the aggregate function for this field + AggFunction aggFunc = getAggFunction(fieldName, primaryKeys, schema); + + // Get the factory for this aggregation function type and create the aggregator + AggFunctionType type = aggFunc.getType(); + FieldAggregatorFactory factory = FieldAggregatorFactory.getFactory(type); + if (factory == null) { + throw new IllegalArgumentException( + String.format( + "Unsupported aggregation function: %s or spell aggregate function incorrectly!", + type)); + } + aggregators[i] = factory.create(fieldType, aggFunc); + } + + return aggregators; + } + + /** + * Determines the aggregate function for a field. + * + *

    The priority is: + * + *

      + *
    1. Primary key fields use "last_value" (no aggregation) + *
    2. Schema.getAggFunction() - aggregation function defined in Schema (from Column) + *
    3. Final fallback: "last_value_ignore_nulls" + *
    + * + * @param fieldName the field name + * @param primaryKeys the list of primary key field names + * @param schema the Schema object + * @return the aggregate function to use + */ + private static AggFunction getAggFunction( + String fieldName, List primaryKeys, Schema schema) { + + // 1. Primary key fields don't aggregate + if (primaryKeys.contains(fieldName)) { + return AggFunctions.of(AggFunctionType.LAST_VALUE); + } + + // 2. Check Schema for aggregation function, or use default fallback + return schema.getAggFunction(fieldName).orElseGet(AggFunctions::LAST_VALUE_IGNORE_NULLS); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContextCache.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContextCache.java new file mode 100644 index 0000000000..2549e54007 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContextCache.java @@ -0,0 +1,86 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate; + +import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaGetter; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; + +import javax.annotation.concurrent.ThreadSafe; + +import java.time.Duration; + +/** The cache for {@link AggregationContext}. */ +@ThreadSafe +public class AggregationContextCache { + + private final Cache contexts; + private final SchemaGetter schemaGetter; + private final KvFormat kvFormat; + + public AggregationContextCache(SchemaGetter schemaGetter, KvFormat kvFormat) { + this.schemaGetter = schemaGetter; + this.kvFormat = kvFormat; + // Limit cache size to prevent memory leak, and expire entries after 5 minutes of inactivity + this.contexts = + Caffeine.newBuilder() + .maximumSize(5) + .expireAfterAccess(Duration.ofMinutes(5)) + .build(); + } + + /** + * Get aggregation context for a given schema ID with null safety check. + * + *

    This method retrieves the schema from SchemaGetter and creates or retrieves the + * AggregationContext from cache. Note: Schema is not returned separately as it's already + * accessible via {@link AggregationContext#getSchema()}. + * + * @param schemaId the schema ID + * @return AggregationContext for the schema + * @throws IllegalStateException if schema is not found + */ + public AggregationContext getContext(short schemaId) { + if (schemaId < 0) { + throw new IllegalArgumentException( + "Schema ID must be non-negative, but got: " + schemaId); + } + Schema schema = schemaGetter.getSchema(schemaId); + if (schema == null) { + throw new IllegalStateException(String.format("Schema with ID %d not found", schemaId)); + } + return getOrCreateContext(schemaId, schema); + } + + /** + * Get or create aggregation context for a given schema ID when schema is already available. + * + *

    This method is useful when the schema is already retrieved (e.g., in constructors) to + * avoid redundant schema lookups. + * + * @param schemaId the schema ID + * @param schema the schema (must not be null) + * @return the aggregation context + */ + public AggregationContext getOrCreateContext(int schemaId, Schema schema) { + return contexts.get(schemaId, k -> AggregationContext.create(schema, kvFormat)); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldAggregatorFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldAggregatorFactory.java new file mode 100644 index 0000000000..7e0901d396 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldAggregatorFactory.java @@ -0,0 +1,90 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldAggregator; +import org.apache.fluss.types.DataType; + +import java.util.EnumMap; +import java.util.ServiceLoader; + +/** Factory interface for creating {@link FieldAggregator} instances. */ +public interface FieldAggregatorFactory { + + /** + * Creates a field aggregator with parameters from AggFunction. + * + * @param fieldType the data type of the field + * @param aggFunction the aggregation function with parameters + * @return the field aggregator + */ + FieldAggregator create(DataType fieldType, AggFunction aggFunction); + + /** + * Returns the unique identifier for this factory. + * + * @return the identifier string + */ + String identifier(); + + /** + * Gets a factory by its aggregation function type. + * + * @param type the aggregation function type + * @return the factory, or null if not found + */ + static FieldAggregatorFactory getFactory(AggFunctionType type) { + return FactoryRegistry.INSTANCE.getFactory(type); + } + + /** Registry for field aggregator factories using Java SPI. */ + class FactoryRegistry { + private static final FactoryRegistry INSTANCE = new FactoryRegistry(); + private final EnumMap factories; + + private FactoryRegistry() { + this.factories = new EnumMap<>(AggFunctionType.class); + loadFactories(); + } + + private void loadFactories() { + ServiceLoader loader = + ServiceLoader.load( + FieldAggregatorFactory.class, + FieldAggregatorFactory.class.getClassLoader()); + for (FieldAggregatorFactory factory : loader) { + // Map factory identifier to AggFunctionType + AggFunctionType type = AggFunctionType.fromString(factory.identifier()); + if (type != null) { + factories.put(type, factory); + } + } + } + + FieldAggregatorFactory getFactory(AggFunctionType type) { + return factories.get(type); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldBoolAndAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldBoolAndAggFactory.java new file mode 100644 index 0000000000..174454bfa5 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldBoolAndAggFactory.java @@ -0,0 +1,49 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldBoolAndAgg; +import org.apache.fluss.types.BooleanType; +import org.apache.fluss.types.DataType; + +import static org.apache.fluss.utils.Preconditions.checkArgument; + +/** Factory for {@link FieldBoolAndAgg}. */ +public class FieldBoolAndAggFactory implements FieldAggregatorFactory { + + @Override + public FieldBoolAndAgg create(DataType fieldType, AggFunction aggFunction) { + checkArgument( + fieldType instanceof BooleanType, + "Data type for bool_and column must be 'BooleanType' but was '%s'.", + fieldType); + return new FieldBoolAndAgg((BooleanType) fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.BOOL_AND.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldBoolOrAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldBoolOrAggFactory.java new file mode 100644 index 0000000000..661070cc7a --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldBoolOrAggFactory.java @@ -0,0 +1,49 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldBoolOrAgg; +import org.apache.fluss.types.BooleanType; +import org.apache.fluss.types.DataType; + +import static org.apache.fluss.utils.Preconditions.checkArgument; + +/** Factory for {@link FieldBoolOrAgg}. */ +public class FieldBoolOrAggFactory implements FieldAggregatorFactory { + + @Override + public FieldBoolOrAgg create(DataType fieldType, AggFunction aggFunction) { + checkArgument( + fieldType instanceof BooleanType, + "Data type for bool_or column must be 'BooleanType' but was '%s'.", + fieldType); + return new FieldBoolOrAgg((BooleanType) fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.BOOL_OR.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldFirstNonNullValueAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldFirstNonNullValueAggFactory.java new file mode 100644 index 0000000000..d1041fe417 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldFirstNonNullValueAggFactory.java @@ -0,0 +1,42 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldFirstNonNullValueAgg; +import org.apache.fluss.types.DataType; + +/** Factory for {@link FieldFirstNonNullValueAgg}. */ +public class FieldFirstNonNullValueAggFactory implements FieldAggregatorFactory { + + @Override + public FieldFirstNonNullValueAgg create(DataType fieldType, AggFunction aggFunction) { + return new FieldFirstNonNullValueAgg(fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.FIRST_VALUE_IGNORE_NULLS.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldFirstValueAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldFirstValueAggFactory.java new file mode 100644 index 0000000000..30f8acbf71 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldFirstValueAggFactory.java @@ -0,0 +1,42 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldFirstValueAgg; +import org.apache.fluss.types.DataType; + +/** Factory for {@link FieldFirstValueAgg}. */ +public class FieldFirstValueAggFactory implements FieldAggregatorFactory { + + @Override + public FieldFirstValueAgg create(DataType fieldType, AggFunction aggFunction) { + return new FieldFirstValueAgg(fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.FIRST_VALUE.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldLastNonNullValueAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldLastNonNullValueAggFactory.java new file mode 100644 index 0000000000..dccc294b7c --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldLastNonNullValueAggFactory.java @@ -0,0 +1,42 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldLastNonNullValueAgg; +import org.apache.fluss.types.DataType; + +/** Factory for {@link FieldLastNonNullValueAgg}. */ +public class FieldLastNonNullValueAggFactory implements FieldAggregatorFactory { + + @Override + public FieldLastNonNullValueAgg create(DataType fieldType, AggFunction aggFunction) { + return new FieldLastNonNullValueAgg(fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.LAST_VALUE_IGNORE_NULLS.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldLastValueAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldLastValueAggFactory.java new file mode 100644 index 0000000000..071ce8f4f1 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldLastValueAggFactory.java @@ -0,0 +1,42 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldLastValueAgg; +import org.apache.fluss.types.DataType; + +/** Factory for {@link FieldLastValueAgg}. */ +public class FieldLastValueAggFactory implements FieldAggregatorFactory { + + @Override + public FieldLastValueAgg create(DataType fieldType, AggFunction aggFunction) { + return new FieldLastValueAgg(fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.LAST_VALUE.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldListaggAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldListaggAggFactory.java new file mode 100644 index 0000000000..aeb4b01d6f --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldListaggAggFactory.java @@ -0,0 +1,57 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.metadata.AggFunctions; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldListaggAgg; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.StringType; + +import static org.apache.fluss.utils.Preconditions.checkArgument; + +/** Factory for {@link FieldListaggAgg}. */ +public class FieldListaggAggFactory implements FieldAggregatorFactory { + + @Override + public FieldListaggAgg create(DataType fieldType, AggFunction aggFunction) { + checkArgument( + fieldType instanceof StringType, + "Data type for listagg column must be 'StringType' but was '%s'.", + fieldType); + + // Get delimiter from function parameters, default to comma + String delimiter = aggFunction.getParameter(AggFunctionType.PARAM_DELIMITER); + if (delimiter == null) { + delimiter = AggFunctions.DEFAULT_LISTAGG_DELIMITER; + } + + return new FieldListaggAgg((StringType) fieldType, delimiter); + } + + @Override + public String identifier() { + return AggFunctionType.LISTAGG.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldMaxAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldMaxAggFactory.java new file mode 100644 index 0000000000..57d948105e --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldMaxAggFactory.java @@ -0,0 +1,42 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldMaxAgg; +import org.apache.fluss.types.DataType; + +/** Factory for {@link FieldMaxAgg}. */ +public class FieldMaxAggFactory implements FieldAggregatorFactory { + + @Override + public FieldMaxAgg create(DataType fieldType, AggFunction aggFunction) { + return new FieldMaxAgg(fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.MAX.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldMinAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldMinAggFactory.java new file mode 100644 index 0000000000..489fbfd336 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldMinAggFactory.java @@ -0,0 +1,42 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldMinAgg; +import org.apache.fluss.types.DataType; + +/** Factory for {@link FieldMinAgg}. */ +public class FieldMinAggFactory implements FieldAggregatorFactory { + + @Override + public FieldMinAgg create(DataType fieldType, AggFunction aggFunction) { + return new FieldMinAgg(fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.MIN.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldProductAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldProductAggFactory.java new file mode 100644 index 0000000000..ec7594c682 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldProductAggFactory.java @@ -0,0 +1,49 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldProductAgg; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeFamily; + +import static org.apache.fluss.utils.Preconditions.checkArgument; + +/** Factory for {@link FieldProductAgg}. */ +public class FieldProductAggFactory implements FieldAggregatorFactory { + + @Override + public FieldProductAgg create(DataType fieldType, AggFunction aggFunction) { + checkArgument( + fieldType.getTypeRoot().getFamilies().contains(DataTypeFamily.NUMERIC), + "Data type for product column must be 'NumericType' but was '%s'.", + fieldType); + return new FieldProductAgg(fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.PRODUCT.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldStringAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldStringAggFactory.java new file mode 100644 index 0000000000..784776f363 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldStringAggFactory.java @@ -0,0 +1,63 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.metadata.AggFunctions; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldListaggAgg; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.StringType; + +import static org.apache.fluss.utils.Preconditions.checkArgument; + +/** + * Factory for {@link FieldListaggAgg} with identifier "string_agg". + * + *

    This is an alias factory for listagg aggregation function. It creates the same aggregator as + * {@link FieldListaggAggFactory} but uses "string_agg" as the identifier, providing an alternative + * name for the same functionality. + */ +public class FieldStringAggFactory implements FieldAggregatorFactory { + + @Override + public FieldListaggAgg create(DataType fieldType, AggFunction aggFunction) { + checkArgument( + fieldType instanceof StringType, + "Data type for string_agg column must be 'StringType' but was '%s'.", + fieldType); + + // Get delimiter from function parameters, default to comma + String delimiter = aggFunction.getParameter(AggFunctionType.PARAM_DELIMITER); + if (delimiter == null) { + delimiter = AggFunctions.DEFAULT_LISTAGG_DELIMITER; + } + + return new FieldListaggAgg((StringType) fieldType, delimiter); + } + + @Override + public String identifier() { + return AggFunctionType.STRING_AGG.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldSumAggFactory.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldSumAggFactory.java new file mode 100644 index 0000000000..673ad66e12 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/factory/FieldSumAggFactory.java @@ -0,0 +1,49 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.factory; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.metadata.AggFunction; +import org.apache.fluss.metadata.AggFunctionType; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldSumAgg; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeFamily; + +import static org.apache.fluss.utils.Preconditions.checkArgument; + +/** Factory for {@link FieldSumAgg}. */ +public class FieldSumAggFactory implements FieldAggregatorFactory { + + @Override + public FieldSumAgg create(DataType fieldType, AggFunction aggFunction) { + checkArgument( + fieldType.getTypeRoot().getFamilies().contains(DataTypeFamily.NUMERIC), + "Data type for sum column must be 'NumericType' but was '%s'.", + fieldType); + return new FieldSumAgg(fieldType); + } + + @Override + public String identifier() { + return AggFunctionType.SUM.toString(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldAggregator.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldAggregator.java new file mode 100644 index 0000000000..f6ec77a494 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldAggregator.java @@ -0,0 +1,66 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeRoot; + +import java.io.Serializable; + +/** Abstract class for aggregating a field of a row. */ +public abstract class FieldAggregator implements Serializable { + + private static final long serialVersionUID = 1L; + + protected final DataType fieldType; + protected final DataTypeRoot typeRoot; + + public FieldAggregator(DataType dataType) { + this.fieldType = dataType; + this.typeRoot = dataType.getTypeRoot(); + } + + /** + * Aggregates the accumulator with the input field. + * + * @param accumulator the current accumulator value + * @param inputField the input field value to aggregate + * @return the aggregated result + */ + public abstract Object agg(Object accumulator, Object inputField); + + /** + * Aggregates the accumulator with the input field in reversed order. By default, it calls + * {@link #agg(Object, Object)} with swapped parameters. + * + * @param accumulator the current accumulator value + * @param inputField the input field value to aggregate + * @return the aggregated result + */ + public Object aggReversed(Object accumulator, Object inputField) { + return agg(inputField, accumulator); + } + + /** Resets the aggregator to a clean start state. */ + public void reset() {} +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldBoolAndAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldBoolAndAgg.java new file mode 100644 index 0000000000..c7504c7094 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldBoolAndAgg.java @@ -0,0 +1,43 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.BooleanType; + +/** Boolean AND aggregator - computes logical AND of boolean values. */ +public class FieldBoolAndAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldBoolAndAgg(BooleanType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + if (accumulator == null || inputField == null) { + return accumulator == null ? inputField : accumulator; + } + return (boolean) accumulator && (boolean) inputField; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldBoolOrAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldBoolOrAgg.java new file mode 100644 index 0000000000..cf259c9518 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldBoolOrAgg.java @@ -0,0 +1,43 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.BooleanType; + +/** Boolean OR aggregator - computes logical OR of boolean values. */ +public class FieldBoolOrAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldBoolOrAgg(BooleanType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + if (accumulator == null || inputField == null) { + return accumulator == null ? inputField : accumulator; + } + return (boolean) accumulator || (boolean) inputField; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldFirstNonNullValueAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldFirstNonNullValueAgg.java new file mode 100644 index 0000000000..5cafd87e8e --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldFirstNonNullValueAgg.java @@ -0,0 +1,46 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.DataType; + +/** First non-null value aggregator - keeps the first seen non-null value. */ +public class FieldFirstNonNullValueAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldFirstNonNullValueAgg(DataType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + // Keep the first non-null value seen + // If accumulator is non-null, it's the first non-null value, keep it + if (accumulator != null) { + return accumulator; + } + // Otherwise, return inputField (which becomes the new accumulator if it's non-null) + return inputField; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldFirstValueAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldFirstValueAgg.java new file mode 100644 index 0000000000..7f281f8176 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldFirstValueAgg.java @@ -0,0 +1,45 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.DataType; + +/** First value aggregator - keeps the first seen value. */ +public class FieldFirstValueAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldFirstValueAgg(DataType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + // If accumulator is not null, it's the first value, keep it + if (accumulator != null) { + return accumulator; + } + // Otherwise, return inputField (which becomes the first value) + return inputField; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastNonNullValueAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastNonNullValueAgg.java new file mode 100644 index 0000000000..b35f9ce477 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastNonNullValueAgg.java @@ -0,0 +1,43 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.DataType; + +/** + * Last non-null value aggregator - returns the input field if not null, otherwise keeps the + * accumulator. + */ +public class FieldLastNonNullValueAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldLastNonNullValueAgg(DataType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + return (inputField == null) ? accumulator : inputField; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastValueAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastValueAgg.java new file mode 100644 index 0000000000..69593c6713 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldLastValueAgg.java @@ -0,0 +1,40 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.DataType; + +/** Last value aggregator - always returns the input field value. */ +public class FieldLastValueAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldLastValueAgg(DataType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + return inputField; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldListaggAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldListaggAgg.java new file mode 100644 index 0000000000..138f284de8 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldListaggAgg.java @@ -0,0 +1,54 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.types.StringType; +import org.apache.fluss.utils.BinaryStringUtils; + +/** List aggregation aggregator - concatenates string values with a delimiter. */ +public class FieldListaggAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + private final BinaryString delimiter; + + public FieldListaggAgg(StringType dataType, String delimiter) { + super(dataType); + // Cache delimiter as BinaryString to avoid repeated conversions + this.delimiter = BinaryString.fromString(delimiter); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + if (accumulator == null || inputField == null) { + return accumulator == null ? inputField : accumulator; + } + + BinaryString mergeFieldSD = (BinaryString) accumulator; + BinaryString inFieldSD = (BinaryString) inputField; + + // Use optimized concat method to avoid string conversion and reduce memory allocation + return BinaryStringUtils.concat(mergeFieldSD, delimiter, inFieldSD); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldMaxAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldMaxAgg.java new file mode 100644 index 0000000000..812dc84306 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldMaxAgg.java @@ -0,0 +1,46 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.DataType; +import org.apache.fluss.utils.InternalRowUtils; + +/** Max aggregator - keeps the maximum value. */ +public class FieldMaxAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldMaxAgg(DataType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + if (accumulator == null || inputField == null) { + return accumulator == null ? inputField : accumulator; + } + return InternalRowUtils.compare(accumulator, inputField, typeRoot) < 0 + ? inputField + : accumulator; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldMinAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldMinAgg.java new file mode 100644 index 0000000000..eba6f49f5b --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldMinAgg.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.types.DataType; +import org.apache.fluss.utils.InternalRowUtils; + +/** Min aggregator - keeps the minimum value. */ +public class FieldMinAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldMinAgg(DataType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + if (accumulator == null || inputField == null) { + return accumulator == null ? inputField : accumulator; + } + + return InternalRowUtils.compare(accumulator, inputField, typeRoot) < 0 + ? accumulator + : inputField; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldProductAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldProductAgg.java new file mode 100644 index 0000000000..da640abcb7 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldProductAgg.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.row.Decimal; +import org.apache.fluss.types.DataType; + +import java.math.BigDecimal; + +import static org.apache.fluss.row.Decimal.fromBigDecimal; + +/** Product aggregator - computes the product of numeric values. */ +public class FieldProductAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldProductAgg(DataType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + if (accumulator == null || inputField == null) { + return accumulator == null ? inputField : accumulator; + } + + Object product; + + // ordered by type root definition + switch (typeRoot) { + case DECIMAL: + Decimal mergeFieldDD = (Decimal) accumulator; + Decimal inFieldDD = (Decimal) inputField; + assert mergeFieldDD.scale() == inFieldDD.scale() + : "Inconsistent scale of aggregate Decimal!"; + assert mergeFieldDD.precision() == inFieldDD.precision() + : "Inconsistent precision of aggregate Decimal!"; + BigDecimal bigDecimal = mergeFieldDD.toBigDecimal(); + BigDecimal bigDecimal1 = inFieldDD.toBigDecimal(); + BigDecimal mul = bigDecimal.multiply(bigDecimal1); + product = fromBigDecimal(mul, mergeFieldDD.precision(), mergeFieldDD.scale()); + break; + case TINYINT: + product = (byte) ((byte) accumulator * (byte) inputField); + break; + case SMALLINT: + product = (short) ((short) accumulator * (short) inputField); + break; + case INTEGER: + product = (int) accumulator * (int) inputField; + break; + case BIGINT: + product = (long) accumulator * (long) inputField; + break; + case FLOAT: + product = (float) accumulator * (float) inputField; + break; + case DOUBLE: + product = (double) accumulator * (double) inputField; + break; + default: + String msg = + String.format( + "type %s not support in %s", typeRoot, this.getClass().getName()); + throw new IllegalArgumentException(msg); + } + return product; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldSumAgg.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldSumAgg.java new file mode 100644 index 0000000000..3c2e101df3 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rowmerger/aggregate/functions/FieldSumAgg.java @@ -0,0 +1,87 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate.functions; + +/* This file is based on source code of Apache Paimon Project (https://paimon.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +import org.apache.fluss.row.Decimal; +import org.apache.fluss.types.DataType; +import org.apache.fluss.utils.DecimalUtils; + +/** Sum aggregator - computes the sum of numeric values. */ +public class FieldSumAgg extends FieldAggregator { + + private static final long serialVersionUID = 1L; + + public FieldSumAgg(DataType dataType) { + super(dataType); + } + + @Override + public Object agg(Object accumulator, Object inputField) { + if (accumulator == null || inputField == null) { + return accumulator == null ? inputField : accumulator; + } + Object sum; + + // ordered by type root definition + switch (typeRoot) { + case DECIMAL: + Decimal mergeFieldDD = (Decimal) accumulator; + Decimal inFieldDD = (Decimal) inputField; + assert mergeFieldDD.scale() == inFieldDD.scale() + : "Inconsistent scale of aggregate Decimal!"; + assert mergeFieldDD.precision() == inFieldDD.precision() + : "Inconsistent precision of aggregate Decimal!"; + sum = + DecimalUtils.add( + mergeFieldDD, + inFieldDD, + mergeFieldDD.precision(), + mergeFieldDD.scale()); + break; + case TINYINT: + sum = (byte) ((byte) accumulator + (byte) inputField); + break; + case SMALLINT: + sum = (short) ((short) accumulator + (short) inputField); + break; + case INTEGER: + sum = (int) accumulator + (int) inputField; + break; + case BIGINT: + sum = (long) accumulator + (long) inputField; + break; + case FLOAT: + sum = (float) accumulator + (float) inputField; + break; + case DOUBLE: + sum = (double) accumulator + (double) inputField; + break; + default: + String msg = + String.format( + "type %s not support in %s", typeRoot, this.getClass().getName()); + throw new IllegalArgumentException(msg); + } + return sum; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/TableDescriptorValidation.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/TableDescriptorValidation.java index 0941e32c16..45065ca415 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/TableDescriptorValidation.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/TableDescriptorValidation.java @@ -26,6 +26,7 @@ import org.apache.fluss.exception.InvalidConfigException; import org.apache.fluss.exception.InvalidTableException; import org.apache.fluss.exception.TooManyBucketsException; +import org.apache.fluss.metadata.AggFunction; import org.apache.fluss.metadata.DeleteBehavior; import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.LogFormat; @@ -72,13 +73,14 @@ public class TableDescriptorValidation { /** Validate table descriptor to create is valid and contain all necessary information. */ public static void validateTableDescriptor(TableDescriptor tableDescriptor, int maxBucketNum) { - boolean hasPrimaryKey = tableDescriptor.getSchema().getPrimaryKey().isPresent(); - RowType schema = tableDescriptor.getSchema().getRowType(); + Schema schema = tableDescriptor.getSchema(); + boolean hasPrimaryKey = schema.getPrimaryKey().isPresent(); Configuration tableConf = Configuration.fromMap(tableDescriptor.getProperties()); // check properties should only contain table.* options, // and this cluster know it, and value is valid for (String key : tableConf.keySet()) { + if (!TABLE_OPTIONS.containsKey(key)) { if (isTableStorageConfig(key)) { throw new InvalidConfigException( @@ -107,8 +109,8 @@ public static void validateTableDescriptor(TableDescriptor tableDescriptor, int checkMergeEngine(tableConf, hasPrimaryKey, schema); checkDeleteBehavior(tableConf, hasPrimaryKey); checkTieredLog(tableConf); - checkPartition(tableConf, tableDescriptor.getPartitionKeys(), schema); - checkSystemColumns(schema); + checkPartition(tableConf, tableDescriptor.getPartitionKeys(), schema.getRowType()); + checkSystemColumns(schema.getRowType()); } public static void validateAlterTableProperties( @@ -245,7 +247,7 @@ private static void checkArrowCompression(Configuration tableConf) { } private static void checkMergeEngine( - Configuration tableConf, boolean hasPrimaryKey, RowType schema) { + Configuration tableConf, boolean hasPrimaryKey, Schema schema) { MergeEngineType mergeEngine = tableConf.get(ConfigOptions.TABLE_MERGE_ENGINE); if (mergeEngine != null) { if (!hasPrimaryKey) { @@ -261,7 +263,8 @@ private static void checkMergeEngine( "'%s' must be set for versioned merge engine.", ConfigOptions.TABLE_MERGE_ENGINE_VERSION_COLUMN.key())); } - int columnIndex = schema.getFieldIndex(versionColumn.get()); + RowType rowType = schema.getRowType(); + int columnIndex = rowType.getFieldIndex(versionColumn.get()); if (columnIndex < 0) { throw new InvalidConfigException( String.format( @@ -274,7 +277,7 @@ private static void checkMergeEngine( DataTypeRoot.BIGINT, DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); - DataType columnType = schema.getTypeAt(columnIndex); + DataType columnType = rowType.getTypeAt(columnIndex); if (!supportedTypes.contains(columnType.getTypeRoot())) { throw new InvalidConfigException( String.format( @@ -283,6 +286,45 @@ private static void checkMergeEngine( + ", but got %s.", versionColumn.get(), columnType)); } + } else if (mergeEngine == MergeEngineType.AGGREGATION) { + // Validate aggregation function parameters for aggregation merge engine + validateAggregationFunctionParameters(schema); + } + } + } + + /** + * Validates aggregation function parameters in the schema. + * + *

    This method delegates to {@link AggFunction#validate()} to ensure all parameters are valid + * according to the function's requirements. + * + * @param schema the schema to validate + * @throws InvalidConfigException if any aggregation function has invalid parameters + */ + private static void validateAggregationFunctionParameters(Schema schema) { + // Get primary key columns for early exit + List primaryKeys = schema.getPrimaryKeyColumnNames(); + + for (Schema.Column column : schema.getColumns()) { + // Skip primary key columns (they don't use user-defined aggregation functions) + if (primaryKeys.contains(column.getName())) { + continue; + } + + Optional aggFunctionOpt = column.getAggFunction(); + if (!aggFunctionOpt.isPresent()) { + continue; + } + + // Validate aggregation function parameters + try { + aggFunctionOpt.get().validate(); + } catch (IllegalArgumentException e) { + throw new InvalidConfigException( + String.format( + "Invalid aggregation function for column '%s': %s", + column.getName(), e.getMessage())); } } } @@ -371,16 +413,24 @@ private static void checkDeleteBehavior(Configuration tableConf, boolean hasPrim // For tables with merge engines, automatically set appropriate delete behavior MergeEngineType mergeEngine = tableConf.get(ConfigOptions.TABLE_MERGE_ENGINE); - if (mergeEngine == MergeEngineType.FIRST_ROW || mergeEngine == MergeEngineType.VERSIONED) { - // For FIRST_ROW and VERSIONED merge engines, delete operations are not supported - // If user explicitly sets delete behavior to ALLOW, throw an exception + if (mergeEngine == MergeEngineType.FIRST_ROW + || mergeEngine == MergeEngineType.VERSIONED + || mergeEngine == MergeEngineType.AGGREGATION) { + // For FIRST_ROW, VERSIONED and AGGREGATION merge engines, delete operations are not + // supported by default + // If user explicitly sets delete behavior to ALLOW, validate it if (deleteBehaviorOptional.isPresent() && deleteBehaviorOptional.get() == DeleteBehavior.ALLOW) { - throw new InvalidConfigException( - String.format( - "Table with '%s' merge engine does not support delete operations. " - + "The 'table.delete.behavior' config must be set to 'ignore' or 'disable', but got 'allow'.", - mergeEngine)); + // For FIRST_ROW and VERSIONED, ALLOW is not permitted + if (mergeEngine == MergeEngineType.FIRST_ROW + || mergeEngine == MergeEngineType.VERSIONED) { + throw new InvalidConfigException( + String.format( + "Table with '%s' merge engine does not support delete operations. " + + "The 'table.delete.behavior' config must be set to 'ignore' or 'disable', but got 'allow'.", + mergeEngine)); + } + // For AGGREGATION, ALLOW is permitted (removes entire record) } } } diff --git a/fluss-server/src/main/resources/META-INF/services/org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldAggregatorFactory b/fluss-server/src/main/resources/META-INF/services/org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldAggregatorFactory new file mode 100644 index 0000000000..f3a27da5f7 --- /dev/null +++ b/fluss-server/src/main/resources/META-INF/services/org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldAggregatorFactory @@ -0,0 +1,30 @@ +# +# 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. +# + +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldSumAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldMaxAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldMinAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldLastValueAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldLastNonNullValueAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldFirstValueAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldFirstNonNullValueAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldProductAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldBoolAndAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldBoolOrAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldListaggAggFactory +org.apache.fluss.server.kv.rowmerger.aggregate.factory.FieldStringAggFactory diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java index 06db30b340..ec2fab37c7 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java @@ -22,6 +22,7 @@ import org.apache.fluss.exception.InvalidTargetColumnException; import org.apache.fluss.exception.OutOfOrderSequenceException; import org.apache.fluss.memory.TestingMemorySegmentPool; +import org.apache.fluss.metadata.AggFunctions; import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.PhysicalTablePath; @@ -177,7 +178,7 @@ private KvTablet createKvTablet( Map tableConfig) throws Exception { TableConfig tableConf = new TableConfig(Configuration.fromMap(tableConfig)); - RowMerger rowMerger = RowMerger.create(tableConf, KvFormat.COMPACTED); + RowMerger rowMerger = RowMerger.create(tableConf, KvFormat.COMPACTED, schemaGetter); return KvTablet.create( tablePath, tableBucket, @@ -1015,6 +1016,107 @@ void testVersionRowMergeEngine(boolean doProjection) throws Exception { .isEqualTo(expectedLogs); } + @Test + void testAggregateMergeEngine() throws Exception { + // Create schema with aggregate-able fields + Schema schema = + Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("max_val", DataTypes.INT(), AggFunctions.MAX()) + .column("name", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .primaryKey("a") + .build(); + + Map config = new HashMap<>(); + config.put("table.merge-engine", "aggregation"); + + TablePath tablePath = TablePath.of("testDb", "test_aggregate_merge_engine"); + initLogTabletAndKvTablet(tablePath, schema, config); + RowType rowType = schema.getRowType(); + KvRecordTestUtils.KvRecordFactory kvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(rowType); + + // First batch: insert two records + List kvData1 = + Arrays.asList( + kvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 10L, 100, "Alice"}), + kvRecordFactory.ofRecord( + "k2".getBytes(), new Object[] {2, 20L, 200, "Bob"})); + KvRecordBatch kvRecordBatch1 = kvRecordBatchFactory.ofRecords(kvData1); + kvTablet.putAsLeader(kvRecordBatch1, null); + + long endOffset = logTablet.localLogEndOffset(); + LogRecords actualLogRecords = readLogRecords(logTablet, 0L, null); + MemoryLogRecords expectedLogs = + logRecords( + rowType, + 0, + Arrays.asList(ChangeType.INSERT, ChangeType.INSERT), + Arrays.asList( + new Object[] {1, 10L, 100, "Alice"}, + new Object[] {2, 20L, 200, "Bob"})); + assertThatLogRecords(actualLogRecords) + .withSchema(rowType) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + + // Second batch: update same keys with aggregation + // k1: count 10+15=25, max_val max(100,150)=150, name="Alice2" + // k2: count 20+25=45, max_val max(200,180)=200, name="Bob2" + List kvData2 = + Arrays.asList( + kvRecordFactory.ofRecord( + "k1".getBytes(), new Object[] {1, 15L, 150, "Alice2"}), + kvRecordFactory.ofRecord( + "k2".getBytes(), new Object[] {2, 25L, 180, "Bob2"})); + KvRecordBatch kvRecordBatch2 = kvRecordBatchFactory.ofRecords(kvData2); + kvTablet.putAsLeader(kvRecordBatch2, null); + + // Aggregate merge engine produces UPDATE_BEFORE and UPDATE_AFTER + expectedLogs = + logRecords( + rowType, + endOffset, + Arrays.asList( + ChangeType.UPDATE_BEFORE, + ChangeType.UPDATE_AFTER, + ChangeType.UPDATE_BEFORE, + ChangeType.UPDATE_AFTER), + Arrays.asList( + new Object[] {1, 10L, 100, "Alice"}, + new Object[] {1, 25L, 150, "Alice2"}, + new Object[] {2, 20L, 200, "Bob"}, + new Object[] {2, 45L, 200, "Bob2"})); + actualLogRecords = readLogRecords(logTablet, endOffset, null); + assertThatLogRecords(actualLogRecords) + .withSchema(rowType) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + + // Third batch: insert new k3 only + endOffset = logTablet.localLogEndOffset(); + List kvData3 = + Arrays.asList( + kvRecordFactory.ofRecord( + "k3".getBytes(), new Object[] {3, 30L, 300, "Charlie"})); + KvRecordBatch kvRecordBatch3 = kvRecordBatchFactory.ofRecords(kvData3); + kvTablet.putAsLeader(kvRecordBatch3, null); + + expectedLogs = + logRecords( + rowType, + endOffset, + Collections.singletonList(ChangeType.INSERT), + Collections.singletonList(new Object[] {3, 30L, 300, "Charlie"})); + actualLogRecords = readLogRecords(logTablet, endOffset, null); + assertThatLogRecords(actualLogRecords) + .withSchema(rowType) + .assertCheckSum(true) + .isEqualTo(expectedLogs); + } + @Test void testAppendDuplicatedKvBatch() throws Exception { initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMergerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMergerTest.java new file mode 100644 index 0000000000..387fb95bb0 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/AggregateRowMergerTest.java @@ -0,0 +1,1008 @@ +/* + * 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.fluss.server.kv.rowmerger; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.TableConfig; +import org.apache.fluss.metadata.AggFunctions; +import org.apache.fluss.metadata.DeleteBehavior; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaInfo; +import org.apache.fluss.record.BinaryValue; +import org.apache.fluss.record.TestingSchemaGetter; +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; + +import org.junit.jupiter.api.Test; + +import static org.apache.fluss.testutils.DataTestUtils.compactedRow; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link AggregateRowMerger}. */ +class AggregateRowMergerTest { + + private static final short SCHEMA_ID = (short) 1; + + private BinaryValue toBinaryValue(BinaryRow row) { + return new BinaryValue(SCHEMA_ID, row); + } + + private static final Schema SCHEMA_SUM = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("total", DataTypes.DOUBLE(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + private static final RowType ROW_TYPE_SUM = SCHEMA_SUM.getRowType(); + + @Test + void testBasicAggregation() { + // Create a single schema with all aggregation types + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("sum_count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("sum_total", DataTypes.DOUBLE(), AggFunctions.SUM()) + .column("max_val", DataTypes.INT(), AggFunctions.MAX()) + .column("min_val", DataTypes.INT(), AggFunctions.MIN()) + .column("name", DataTypes.STRING()) // defaults to last_value_ignore_nulls + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, schema); + RowType rowType = schema.getRowType(); + + // First row: id=1, sum_count=5, sum_total=10.5, max_val=10, min_val=3, name="Alice" + BinaryRow row1 = compactedRow(rowType, new Object[] {1, 5L, 10.5, 10, 3, "Alice"}); + + // Second row: id=1, sum_count=3, sum_total=7.5, max_val=15, min_val=1, name="Bob" + BinaryRow row2 = compactedRow(rowType, new Object[] {1, 3L, 7.5, 15, 1, "Bob"}); + + // Merge rows + BinaryValue value1 = toBinaryValue(row1); + BinaryValue result = merger.merge(null, value1); + assertThat(result).isSameAs(value1); + + BinaryValue value2 = toBinaryValue(row2); + BinaryValue merged = merger.merge(value1, value2); + + // Enhanced assertions: verify all fields including schemaId and field count + assertThat(merged.schemaId).isEqualTo(SCHEMA_ID); + assertThat(merged.row.getFieldCount()).isEqualTo(6); + assertThat(merged.row.getInt(0)).isEqualTo(1); // id stays the same + assertThat(merged.row.getLong(1)).isEqualTo(8L); // sum_count = 5 + 3 + assertThat(merged.row.getDouble(2)).isEqualTo(18.0); // sum_total = 10.5 + 7.5 + assertThat(merged.row.getInt(3)).isEqualTo(15); // max_val = max(10, 15) + assertThat(merged.row.getInt(4)).isEqualTo(1); // min_val = min(3, 1) + assertThat(merged.row.getString(5).toString()).isEqualTo("Bob"); // name = last non-null + + // Third row: id=1, name=null (other fields don't matter for last_value test) + BinaryRow row3 = compactedRow(rowType, new Object[] {1, 0L, 0.0, 0, 0, null}); + + // Merge with null name should keep "Bob" + BinaryValue merged2 = merger.merge(merged, toBinaryValue(row3)); + // Enhanced assertion: verify null handling for last_value_ignore_nulls + assertThat(merged2.schemaId).isEqualTo(SCHEMA_ID); + assertThat(merged2.row.getString(5).toString()) + .isEqualTo("Bob"); // name should still be "Bob" (null ignored) + assertThat(merged2.row.isNullAt(5)).isFalse(); // Should not be null + } + + @Test + void testDeleteBehaviorRemoveRecord() { + Configuration conf = new Configuration(); + conf.set(ConfigOptions.TABLE_DELETE_BEHAVIOR, DeleteBehavior.ALLOW); + TableConfig tableConfig = new TableConfig(conf); + + AggregateRowMerger merger = createMerger(SCHEMA_SUM, tableConfig); + + BinaryRow row = compactedRow(ROW_TYPE_SUM, new Object[] {1, 5L, 10.5}); + + // Delete should remove the record + BinaryValue deleted = merger.delete(toBinaryValue(row)); + // Enhanced assertion: verify complete removal + assertThat(deleted).isNull(); + // Verify deleteBehavior is correctly configured + assertThat(merger.deleteBehavior()).isEqualTo(DeleteBehavior.ALLOW); + } + + @Test + void testDeleteBehaviorNotAllowed() { + // deleteBehavior defaults to IGNORE + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(SCHEMA_SUM, tableConfig); + + // Enhanced assertion: verify deleteBehavior + assertThat(merger.deleteBehavior()).isEqualTo(DeleteBehavior.IGNORE); + // Delete method should not be called when deleteBehavior is IGNORE, + // as KvTablet.processDeletion() will skip it. + // However, if called directly, it will remove the record. + BinaryRow row = compactedRow(ROW_TYPE_SUM, new Object[] {1, 5L, 10.5}); + BinaryValue deleted = merger.delete(toBinaryValue(row)); + assertThat(deleted).isNull(); + } + + @Test + void testNullValueHandling() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + merger.configureTargetColumns(null, SCHEMA_ID, schema); + + // First row: id=1, value=null + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, null}); + + // Second row: id=1, value=10 + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, 10L}); + + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + + // Enhanced assertions: verify schemaId and null handling + assertThat(merged.schemaId).isEqualTo(SCHEMA_ID); + assertThat(merged.row.getFieldCount()).isEqualTo(2); + assertThat(merged.row.getInt(0)).isEqualTo(1); // id unchanged + assertThat(merged.row.getLong(1)).isEqualTo(10L); // Result should be 10 (null + 10 = 10) + } + + @Test + void testPartialUpdate() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("total", DataTypes.DOUBLE(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // Configure partial update for id and count (excluding total) + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1}, SCHEMA_ID, schema); + + // First row: id=1, count=10, total=100.0 + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, 10L, 100.0}); + + // Second row (partial): id=1, count=5, total=50.0 + // Only count should be aggregated (10 + 5 = 15) + // total should remain unchanged (100.0) + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, 5L, 50.0}); + + BinaryValue merged = partialMerger.merge(toBinaryValue(row1), toBinaryValue(row2)); + + // Enhanced assertions: verify all fields including schemaId + assertThat(merged.schemaId).isEqualTo(SCHEMA_ID); + assertThat(merged.row.getFieldCount()).isEqualTo(3); + assertThat(merged.row.getInt(0)).isEqualTo(1); // id unchanged + assertThat(merged.row.getLong(1)).isEqualTo(15L); // count: 10 + 5 (aggregated) + assertThat(merged.row.getDouble(2)) + .isEqualTo(100.0); // total unchanged (not in targetColumns) + // Verify non-target column is truly unchanged (not aggregated) + assertThat(merged.row.getDouble(2)).isEqualTo(row1.getDouble(2)); + } + + @Test + void testConfigureTargetColumnsNull() { + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(SCHEMA_SUM, tableConfig); + + // Null target columns should return the same merger + RowMerger result = merger.configureTargetColumns(null, SCHEMA_ID, SCHEMA_SUM); + assertThat(result).isSameAs(merger); + } + + @Test + void testPartialUpdateWithSchemaEvolutionAddColumn() { + // Create old schema: id(columnId=0), count(columnId=1), total(columnId=2) + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("total", DataTypes.DOUBLE(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + // Create new schema: add a new column "new_field" (columnId=3) at the end + // Ensure columnId inheritance: use fromColumns to preserve columnIds from old schema + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + // id: preserve columnId=0 + new Schema.Column("id", DataTypes.INT(), null, 0), + // count: preserve columnId=1 + new Schema.Column( + "count", + DataTypes.BIGINT(), + null, + 1, + AggFunctions.SUM()), + // total: preserve columnId=2 + new Schema.Column( + "total", + DataTypes.DOUBLE(), + null, + 2, + AggFunctions.SUM()), + // new_field: new column with columnId=3 + new Schema.Column( + "new_field", + DataTypes.BIGINT(), + null, + 3, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + // Create schema getter with both schemas + // Initialize with newSchema as latest schema for AggregateRowMerger constructor + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + // Add old schema to cache + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + // Ensure newSchema is latest + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + + // Configure partial update for id and count (excluding total and new_field) + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1}, newSchemaId, newSchema); + + // Create old row with old schema: id=1, count=10, total=100.0 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, 10L, 100.0}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // Create new row with new schema: id=1, count=5, total=50.0, new_field=20 + BinaryRow newRow = compactedRow(newSchema.getRowType(), new Object[] {1, 5L, 50.0, 20L}); + BinaryValue newValue = new BinaryValue(newSchemaId, newRow); + + // Merge old and new rows + BinaryValue merged = partialMerger.merge(oldValue, newValue); + + // Enhanced assertions: verify schema evolution and all fields + assertThat(merged.schemaId).isEqualTo(newSchemaId); + assertThat(merged.row.getFieldCount()).isEqualTo(4); + + // Verify old fields + assertThat(merged.row.getInt(0)).isEqualTo(1); // id stays the same + assertThat(merged.row.getLong(1)) + .isEqualTo(15L); // count = 10 + 5 (aggregated, target column) + assertThat(merged.row.getDouble(2)) + .isEqualTo(100.0); // total unchanged (not in targetColumns) + // Verify non-target column is truly unchanged + assertThat(merged.row.getDouble(2)).isEqualTo(oldRow.getDouble(2)); + + // Verify new field: not in targetColumns and not in old schema, so should be null + // In partial aggregation, non-target columns should not use values from newRow + assertThat(merged.row.isNullAt(3)).isTrue(); // new_field should be null + } + + @Test + void testPartialUpdateWithSchemaEvolutionAddColumnTargetNewField() { + // Create old schema: id(columnId=0), count(columnId=1) + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + // Create new schema: add a new column "new_field" (columnId=2) at the end + // Ensure columnId inheritance: use fromColumns to preserve columnIds from old schema + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + // id: preserve columnId=0 + new Schema.Column("id", DataTypes.INT(), null, 0), + // count: preserve columnId=1 + new Schema.Column( + "count", + DataTypes.BIGINT(), + null, + 1, + AggFunctions.SUM()), + // new_field: new column with columnId=2 + new Schema.Column( + "new_field", + DataTypes.BIGINT(), + null, + 2, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + // Create schema getter with both schemas + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + + // Configure partial update for id and new_field (excluding count) + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 2}, newSchemaId, newSchema); + + // Create old row with old schema: id=1, count=10 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, 10L}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // Create new row with new schema: id=1, count=5, new_field=20 + BinaryRow newRow = compactedRow(newSchema.getRowType(), new Object[] {1, 5L, 20L}); + BinaryValue newValue = new BinaryValue(newSchemaId, newRow); + + // Merge old and new rows + BinaryValue merged = partialMerger.merge(oldValue, newValue); + + // Enhanced assertions: verify schema evolution and field handling + assertThat(merged.schemaId).isEqualTo(newSchemaId); + assertThat(merged.row.getFieldCount()).isEqualTo(3); + + // Verify fields + assertThat(merged.row.getInt(0)).isEqualTo(1); // id stays the same + assertThat(merged.row.getLong(1)).isEqualTo(10L); // count unchanged (not in targetColumns) + // Verify non-target column is truly unchanged + assertThat(merged.row.getLong(1)).isEqualTo(oldRow.getLong(1)); + assertThat(merged.row.getLong(2)) + .isEqualTo(20L); // new_field = null + 20 (aggregated, target column) + } + + @Test + void testSchemaEvolutionAddColumnWithMultipleAggTypes() { + // Create old schema: id(columnId=0), sum_count(columnId=1), max_val(columnId=2) + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("sum_count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("max_val", DataTypes.INT(), AggFunctions.MAX()) + .primaryKey("id") + .build(); + + // Create new schema: add new columns "min_val" (columnId=3) and "new_sum" (columnId=4) + // Ensure columnId inheritance: use fromColumns to preserve columnIds from old schema + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + // id: preserve columnId=0 + new Schema.Column("id", DataTypes.INT(), null, 0), + // sum_count: preserve columnId=1 + new Schema.Column( + "sum_count", + DataTypes.BIGINT(), + null, + 1, + AggFunctions.SUM()), + // max_val: preserve columnId=2 + new Schema.Column( + "max_val", + DataTypes.INT(), + null, + 2, + AggFunctions.MAX()), + // min_val: new column with columnId=3 + new Schema.Column( + "min_val", + DataTypes.INT(), + null, + 3, + AggFunctions.MIN()), + // new_sum: new column with columnId=4 + new Schema.Column( + "new_sum", + DataTypes.BIGINT(), + null, + 4, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + // Create schema getter with both schemas + // Initialize with newSchema as latest schema for AggregateRowMerger constructor + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + // Add old schema to cache + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + // Ensure newSchema is latest + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + merger.configureTargetColumns(null, newSchemaId, newSchema); + + // Create old row with old schema: id=1, sum_count=100, max_val=50 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, 100L, 50}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // Create new row with new schema: id=1, sum_count=20, max_val=80, min_val=10, new_sum=30 + BinaryRow newRow = compactedRow(newSchema.getRowType(), new Object[] {1, 20L, 80, 10, 30L}); + BinaryValue newValue = new BinaryValue(newSchemaId, newRow); + + // Merge old and new rows + BinaryValue merged = merger.merge(oldValue, newValue); + + // Enhanced assertions: verify schema evolution and all aggregation functions + assertThat(merged.schemaId).isEqualTo(newSchemaId); + assertThat(merged.row.getFieldCount()).isEqualTo(5); + + // Verify old fields are aggregated correctly + assertThat(merged.row.getInt(0)).isEqualTo(1); // id stays the same + assertThat(merged.row.getLong(1)).isEqualTo(120L); // sum_count = 100 + 20 + assertThat(merged.row.getInt(2)).isEqualTo(80); // max_val = max(50, 80) + + // Verify new fields are aggregated correctly (null handling) + assertThat(merged.row.getInt(3)).isEqualTo(10); // min_val = min(null, 10) = 10 + assertThat(merged.row.isNullAt(3)).isFalse(); // Should not be null + assertThat(merged.row.getLong(4)).isEqualTo(30L); // new_sum = null + 30 = 30 + assertThat(merged.row.isNullAt(4)).isFalse(); // Should not be null + } + + // ========== Schema Evolution Tests ========== + + @Test + void testSchemaEvolutionColumnReorder() { + // Old schema: [id(0), name(1), age(2)] + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) // columnId = 0 + .column("name", DataTypes.STRING()) // columnId = 1 + .column("age", DataTypes.INT()) // columnId = 2 + .primaryKey("id") + .build(); + + // New schema: [id(0), age(2), name(1)] - columns reordered! + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + new Schema.Column("id", DataTypes.INT(), null, 0), + new Schema.Column( + "age", + DataTypes.INT(), + null, + 2, + AggFunctions.SUM()), + new Schema.Column( + "name", + DataTypes.STRING(), + null, + 1, + AggFunctions.LAST_VALUE()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + merger.configureTargetColumns(null, newSchemaId, newSchema); + + // Old row: id=1, name="Alice", age=20 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, "Alice", 20}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // New row: id=1, age=30, name="Bob" (reordered!) + BinaryRow newRow = compactedRow(newSchema.getRowType(), new Object[] {1, 30, "Bob"}); + BinaryValue newValue = new BinaryValue(newSchemaId, newRow); + + // Merge + BinaryValue merged = merger.merge(oldValue, newValue); + + // Enhanced assertions: verify column ID matching and schema evolution + assertThat(merged.schemaId).isEqualTo(newSchemaId); + assertThat(merged.row.getFieldCount()).isEqualTo(3); + // Verify: should correctly match by column ID, not position + assertThat(merged.row.getInt(0)).isEqualTo(1); // id + assertThat(merged.row.getInt(1)).isEqualTo(50); // age: 20 + 30 = 50 (SUM) + assertThat(merged.row.getString(2).toString()).isEqualTo("Bob"); // name: last_value + // Verify column reordering: old schema [id, name, age], new schema [id, age, name] + // Should match by columnId, not position + } + + @Test + void testSchemaEvolutionDropMiddleColumn() { + // Old schema: [id(0), dropped(1), count(2)] + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("dropped", DataTypes.STRING()) + .column("count", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + // New schema: [id(0), count(2)] - dropped column removed + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + new Schema.Column("id", DataTypes.INT(), null, 0), + new Schema.Column( + "count", + DataTypes.BIGINT(), + null, + 2, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + merger.configureTargetColumns(null, newSchemaId, newSchema); + + // Old row: id=1, dropped="ignore", count=100 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, "ignore", 100L}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // New row: id=1, count=50 + BinaryRow newRow = compactedRow(newSchema.getRowType(), new Object[] {1, 50L}); + BinaryValue newValue = new BinaryValue(newSchemaId, newRow); + + // Merge + BinaryValue merged = merger.merge(oldValue, newValue); + + // Enhanced assertions: verify column dropping and aggregation + assertThat(merged.schemaId).isEqualTo(newSchemaId); + assertThat(merged.row.getFieldCount()).isEqualTo(2); + // Verify: should correctly aggregate count despite position change + assertThat(merged.row.getInt(0)).isEqualTo(1); // id + assertThat(merged.row.getLong(1)).isEqualTo(150L); // count: 100 + 50 + // Verify dropped column is not present in result + } + + @Test + void testSchemaEvolutionInsertMiddleColumn() { + // Old schema: [id(0), count(2)] + Schema oldSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + new Schema.Column("id", DataTypes.INT(), null, 0), + new Schema.Column( + "count", + DataTypes.BIGINT(), + null, + 2, + AggFunctions.SUM()))) + .primaryKey("id") + .highestFieldId((short) 2) + .build(); + + // New schema: [id(0), inserted(1), count(2)] - new column inserted in middle + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + new Schema.Column("id", DataTypes.INT(), null, 0), + new Schema.Column( + "inserted", + DataTypes.STRING(), + null, + 1, + AggFunctions.LAST_VALUE()), + new Schema.Column( + "count", + DataTypes.BIGINT(), + null, + 2, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + merger.configureTargetColumns(null, newSchemaId, newSchema); + + // Old row: id=1, count=100 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, 100L}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // New row: id=1, inserted="new", count=50 + BinaryRow newRow = compactedRow(newSchema.getRowType(), new Object[] {1, "new", 50L}); + BinaryValue newValue = new BinaryValue(newSchemaId, newRow); + + // Merge + BinaryValue merged = merger.merge(oldValue, newValue); + + // Enhanced assertions: verify column insertion and aggregation + assertThat(merged.schemaId).isEqualTo(newSchemaId); + assertThat(merged.row.getFieldCount()).isEqualTo(3); + assertThat(merged.row.getInt(0)).isEqualTo(1); // id + assertThat(merged.row.getString(1).toString()).isEqualTo("new"); // inserted (new column) + assertThat(merged.row.isNullAt(1)).isFalse(); // Should not be null + assertThat(merged.row.getLong(2)).isEqualTo(150L); // count: 100 + 50 + // Verify new column is correctly inserted at middle position + } + + @Test + void testSchemaEvolutionPartialUpdateWithReorder() { + // Old schema: [id(0), a(1), b(2), c(3)] + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("a", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("b", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("c", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + // New schema: [id(0), c(3), b(2), a(1)] - all columns reordered! + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + new Schema.Column("id", DataTypes.INT(), null, 0), + new Schema.Column( + "c", + DataTypes.BIGINT(), + null, + 3, + AggFunctions.SUM()), + new Schema.Column( + "b", + DataTypes.BIGINT(), + null, + 2, + AggFunctions.SUM()), + new Schema.Column( + "a", + DataTypes.BIGINT(), + null, + 1, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + + // Configure partial update: only update "a" and "c" (at positions 3 and 1 in new schema) + // But we specify by original indices in new schema + RowMerger partialMerger = + merger.configureTargetColumns( + new int[] {0, 1, 3}, // id, c, a in new schema positions + newSchemaId, + newSchema); + + // Old row: id=1, a=10, b=20, c=30 + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, 10L, 20L, 30L}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + // New row: id=1, c=5, b=999, a=3 (reordered, b should be ignored) + BinaryRow newRow = compactedRow(newSchema.getRowType(), new Object[] {1, 5L, 999L, 3L}); + BinaryValue newValue = new BinaryValue(newSchemaId, newRow); + + // Merge + BinaryValue merged = partialMerger.merge(oldValue, newValue); + + // Enhanced assertions: verify column ID matching and partial update behavior + assertThat(merged.schemaId).isEqualTo(newSchemaId); + assertThat(merged.row.getFieldCount()).isEqualTo(4); + // Verify: should match by column ID, not position + assertThat(merged.row.getInt(0)).isEqualTo(1); // id + assertThat(merged.row.getLong(1)).isEqualTo(35L); // c: 30 + 5 = 35 (target, aggregated) + assertThat(merged.row.getLong(2)).isEqualTo(20L); // b: kept old value (not target) + // Verify non-target column is truly unchanged + assertThat(merged.row.getLong(2)).isEqualTo(oldRow.getLong(2)); + assertThat(merged.row.getLong(3)).isEqualTo(13L); // a: 10 + 3 = 13 (target, aggregated) + } + + @Test + void testPartialAggregateRowMergerDeleteAllScenarios() { + // Create schema with multiple columns for comprehensive delete testing + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("target_sum", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("target_max", DataTypes.INT(), AggFunctions.MAX()) + .column("non_target", DataTypes.STRING()) + .column("non_target_count", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + short schemaId = SCHEMA_ID; + + // ========== Scenario 1: deleteBehavior = ALLOW, partial delete with non-target values + // ========== + { + Configuration conf = new Configuration(); + conf.set(ConfigOptions.TABLE_DELETE_BEHAVIOR, DeleteBehavior.ALLOW); + TableConfig tableConfig = new TableConfig(conf); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1, 2}, schemaId, schema); + + BinaryRow row = + compactedRow(schema.getRowType(), new Object[] {1, 100L, 50, "keep", 200L}); + BinaryValue oldValue = new BinaryValue(schemaId, row); + + BinaryValue deleted = partialMerger.delete(oldValue); + + // Should perform partial delete: target columns (except PK) set to null, non-target + // columns kept + assertThat(deleted).isNotNull(); + assertThat(deleted.schemaId).isEqualTo(schemaId); + assertThat(deleted.row.getFieldCount()).isEqualTo(5); + + // Primary key should be kept + assertThat(deleted.row.getInt(0)).isEqualTo(1); + + // Target columns (non-PK) should be set to null + assertThat(deleted.row.isNullAt(1)).isTrue(); // target_sum + assertThat(deleted.row.isNullAt(2)).isTrue(); // target_max + + // Non-target columns should be kept unchanged + assertThat(deleted.row.getString(3).toString()).isEqualTo("keep"); + assertThat(deleted.row.getLong(4)).isEqualTo(200L); + } + + // ========== Scenario 2: deleteBehavior = IGNORE (default), partial delete with non-target + // values ========== + { + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + // Target columns: id(0), target_sum(1), target_max(2) + // Non-target columns: non_target(3), non_target_count(4) + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1, 2}, schemaId, schema); + + BinaryRow row = + compactedRow(schema.getRowType(), new Object[] {1, 100L, 50, "keep_me", 200L}); + BinaryValue oldValue = new BinaryValue(schemaId, row); + + BinaryValue deleted = partialMerger.delete(oldValue); + + // Should perform partial delete: target columns (except PK) set to null, non-target + // columns kept + assertThat(deleted).isNotNull(); + assertThat(deleted.schemaId).isEqualTo(schemaId); + assertThat(deleted.row.getFieldCount()).isEqualTo(5); + + // Primary key should be kept + assertThat(deleted.row.getInt(0)).isEqualTo(1); + + // Target columns (non-PK) should be set to null + assertThat(deleted.row.isNullAt(1)).isTrue(); // target_sum + assertThat(deleted.row.isNullAt(2)).isTrue(); // target_max + + // Non-target columns should be kept unchanged + assertThat(deleted.row.getString(3).toString()).isEqualTo("keep_me"); + assertThat(deleted.row.getLong(4)).isEqualTo(200L); + } + + // ========== Scenario 3: deleteBehavior = IGNORE (default), all non-target columns are null + // ========== + { + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(schema, tableConfig); + // Target columns: id(0), target_sum(1) + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1}, schemaId, schema); + + // All non-target columns are null + BinaryRow row = + compactedRow(schema.getRowType(), new Object[] {1, 100L, null, null, null}); + BinaryValue oldValue = new BinaryValue(schemaId, row); + + BinaryValue deleted = partialMerger.delete(oldValue); + + // Should return null (all non-target columns are null) + assertThat(deleted).isNull(); + } + + // ========== Scenario 4: deleteBehavior = IGNORE (default), Schema Evolution + Partial + // Delete ========== + { + // Old schema: id(columnId=0), target_sum(columnId=1), non_target(columnId=2) + Schema oldSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("target_sum", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("non_target", DataTypes.STRING()) + .primaryKey("id") + .build(); + + // New schema: id(columnId=0), target_sum(columnId=1), target_max(columnId=3, new), + // non_target(columnId=2), non_target_count(columnId=4, new) + // Ensure columnId inheritance: use fromColumns to preserve columnIds from old schema + Schema newSchema = + Schema.newBuilder() + .fromColumns( + java.util.Arrays.asList( + // id: preserve columnId=0 + new Schema.Column("id", DataTypes.INT(), null, 0), + // target_sum: preserve columnId=1 + new Schema.Column( + "target_sum", + DataTypes.BIGINT(), + null, + 1, + AggFunctions.SUM()), + // target_max: new column with columnId=3 + new Schema.Column( + "target_max", + DataTypes.INT(), + null, + 3, + AggFunctions.MAX()), + // non_target: preserve columnId=2 + new Schema.Column( + "non_target", DataTypes.STRING(), null, 2), + // non_target_count: new column with columnId=4 + new Schema.Column( + "non_target_count", + DataTypes.BIGINT(), + null, + 4, + AggFunctions.SUM()))) + .primaryKey("id") + .build(); + + short oldSchemaId = (short) 1; + short newSchemaId = (short) 2; + + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(newSchema, newSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(oldSchema, oldSchemaId)); + schemaGetter.updateLatestSchemaInfo(new SchemaInfo(newSchema, newSchemaId)); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + + // Configure partial update for id and target_sum (columns 0, 1 in new schema) + RowMerger partialMerger = + merger.configureTargetColumns(new int[] {0, 1}, newSchemaId, newSchema); + + // Old row with old schema: id=1, target_sum=100, non_target="keep" + BinaryRow oldRow = compactedRow(oldSchema.getRowType(), new Object[] {1, 100L, "keep"}); + BinaryValue oldValue = new BinaryValue(oldSchemaId, oldRow); + + BinaryValue deleted = partialMerger.delete(oldValue); + + // Should perform partial delete using new schema + assertThat(deleted).isNotNull(); + assertThat(deleted.schemaId).isEqualTo(newSchemaId); + assertThat(deleted.row.getFieldCount()).isEqualTo(5); + + // Primary key kept + assertThat(deleted.row.getInt(0)).isEqualTo(1); + + // Target column (target_sum) set to null + assertThat(deleted.row.isNullAt(1)).isTrue(); + + // New column (target_max) should be null (didn't exist in old schema) + assertThat(deleted.row.isNullAt(2)).isTrue(); + + // Non-target column (non_target) kept + assertThat(deleted.row.getString(3).toString()).isEqualTo("keep"); + + // New non-target column (non_target_count) should be null + assertThat(deleted.row.isNullAt(4)).isTrue(); + } + + // ========== Scenario 5: deleteBehavior = IGNORE (default), composite primary key + // ========== + { + Schema compositePkSchema = + Schema.newBuilder() + .column("id1", DataTypes.INT()) + .column("id2", DataTypes.STRING()) + .column("target_sum", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("non_target", DataTypes.STRING()) + .primaryKey("id1", "id2") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregateRowMerger merger = createMerger(compositePkSchema, tableConfig); + // Target columns: id1(0), id2(1), target_sum(2) + RowMerger partialMerger = + merger.configureTargetColumns( + new int[] {0, 1, 2}, SCHEMA_ID, compositePkSchema); + + BinaryRow row = + compactedRow( + compositePkSchema.getRowType(), + new Object[] {1, "pk2", 100L, "keep_me"}); + BinaryValue oldValue = new BinaryValue(SCHEMA_ID, row); + + BinaryValue deleted = partialMerger.delete(oldValue); + + // Both primary key columns should be kept + assertThat(deleted).isNotNull(); + assertThat(deleted.row.getInt(0)).isEqualTo(1); // id1 + assertThat(deleted.row.getString(1).toString()).isEqualTo("pk2"); // id2 + // Target column (non-PK) should be null + assertThat(deleted.row.isNullAt(2)).isTrue(); // target_sum + // Non-target column should be kept + assertThat(deleted.row.getString(3).toString()).isEqualTo("keep_me"); + } + } + + private AggregateRowMerger createMerger(Schema schema, TableConfig tableConfig) { + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(schema, SCHEMA_ID)); + return new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/RowMergerCreateTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/RowMergerCreateTest.java new file mode 100644 index 0000000000..e6d7f48499 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/RowMergerCreateTest.java @@ -0,0 +1,224 @@ +/* + * 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.fluss.server.kv.rowmerger; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.TableConfig; +import org.apache.fluss.metadata.AggFunctions; +import org.apache.fluss.metadata.DeleteBehavior; +import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.metadata.MergeEngineType; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaInfo; +import org.apache.fluss.record.BinaryValue; +import org.apache.fluss.record.TestingSchemaGetter; +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.types.DataTypes; + +import org.junit.jupiter.api.Test; + +import static org.apache.fluss.testutils.DataTestUtils.compactedRow; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Integration tests for {@link RowMerger#create} method. */ +class RowMergerCreateTest { + + private static final short SCHEMA_ID = (short) 1; + + private BinaryValue toBinaryValue(BinaryRow row) { + return new BinaryValue(SCHEMA_ID, row); + } + + private static final Schema TEST_SCHEMA = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BIGINT(), AggFunctions.SUM()) + .primaryKey("id") + .build(); + + private static TestingSchemaGetter createSchemaGetter(Schema schema) { + return new TestingSchemaGetter(new SchemaInfo(schema, SCHEMA_ID)); + } + + @Test + void testCreateAggregateRowMergerWithMaxFunction() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BIGINT(), AggFunctions.MAX()) + .primaryKey("id") + .build(); + + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.AGGREGATION.name()); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(schema)); + merger.configureTargetColumns(null, SCHEMA_ID, schema); + + assertThat(merger).isInstanceOf(AggregateRowMerger.class); + + // Verify max aggregation + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, 10L}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, 20L}); + + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + + assertThat(merged.row.getInt(0)).isEqualTo(1); + assertThat(merged.row.getLong(1)).isEqualTo(20L); // max(10, 20) + } + + @Test + void testCreateAggregateRowMergerWithDeleteBehaviorAllow() { + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.AGGREGATION.name()); + conf.set(ConfigOptions.TABLE_DELETE_BEHAVIOR, DeleteBehavior.ALLOW); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(TEST_SCHEMA)); + + assertThat(merger).isInstanceOf(AggregateRowMerger.class); + assertThat(merger.deleteBehavior()).isEqualTo(DeleteBehavior.ALLOW); + + // Verify delete behavior - should remove the record + BinaryRow row = compactedRow(TEST_SCHEMA.getRowType(), new Object[] {1, 10L}); + BinaryValue deleted = merger.delete(toBinaryValue(row)); + + assertThat(deleted).isNull(); + } + + @Test + void testCreateAggregateRowMergerWithDeleteBehavior() { + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.AGGREGATION.name()); + conf.setString(ConfigOptions.TABLE_DELETE_BEHAVIOR.key(), DeleteBehavior.IGNORE.name()); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(TEST_SCHEMA)); + + assertThat(merger).isInstanceOf(AggregateRowMerger.class); + assertThat(merger.deleteBehavior()).isEqualTo(DeleteBehavior.IGNORE); + } + + @Test + void testCreateFirstRowRowMerger() { + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.FIRST_ROW.name()); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(TEST_SCHEMA)); + + assertThat(merger).isInstanceOf(FirstRowRowMerger.class); + } + + @Test + void testCreateVersionedRowMerger() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("version", DataTypes.BIGINT()) + .column("value", DataTypes.STRING()) + .primaryKey("id") + .build(); + + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.VERSIONED.name()); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE_VERSION_COLUMN.key(), "version"); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(schema)); + + assertThat(merger).isInstanceOf(VersionedRowMerger.class); + } + + @Test + void testCreateVersionedRowMergerWithoutVersionColumnThrowsException() { + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.VERSIONED.name()); + // Missing version column configuration + TableConfig tableConfig = new TableConfig(conf); + + assertThatThrownBy( + () -> + RowMerger.create( + tableConfig, + KvFormat.COMPACTED, + createSchemaGetter(TEST_SCHEMA))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("must be set for versioned merge engine"); + } + + @Test + void testCreateDefaultRowMerger() { + // No merge engine specified + Configuration conf = new Configuration(); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(TEST_SCHEMA)); + + assertThat(merger).isInstanceOf(DefaultRowMerger.class); + } + + @Test + void testCreateAggregateRowMergerWithCompositePrimaryKeyAndMultipleAggTypes() { + // Create schema with composite primary key and various aggregation function types + Schema schema = + Schema.newBuilder() + .column("id1", DataTypes.INT()) + .column("id2", DataTypes.STRING()) + .column("sum_count", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("max_val", DataTypes.INT(), AggFunctions.MAX()) + .column("min_val", DataTypes.INT(), AggFunctions.MIN()) + .column("last_name", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .primaryKey("id1", "id2") + .build(); + + Configuration conf = new Configuration(); + conf.setString(ConfigOptions.TABLE_MERGE_ENGINE.key(), MergeEngineType.AGGREGATION.name()); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(schema)); + + // Verify the merger is AggregateRowMerger + assertThat(merger).isInstanceOf(AggregateRowMerger.class); + } + + @Test + void testCreateAggregateRowMergerCaseInsensitive() { + Configuration conf = new Configuration(); + // Test case-insensitive merge engine type + conf.setString( + ConfigOptions.TABLE_MERGE_ENGINE.key(), + MergeEngineType.AGGREGATION.name().toLowerCase()); + TableConfig tableConfig = new TableConfig(conf); + + RowMerger merger = + RowMerger.create(tableConfig, KvFormat.COMPACTED, createSchemaGetter(TEST_SCHEMA)); + + assertThat(merger).isInstanceOf(AggregateRowMerger.class); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContextTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContextTest.java new file mode 100644 index 0000000000..ab05d18ca5 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/AggregationContextTest.java @@ -0,0 +1,90 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate; + +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.TableConfig; +import org.apache.fluss.metadata.AggFunctions; +import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldBoolAndAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldBoolOrAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldFirstNonNullValueAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldFirstValueAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldLastNonNullValueAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldLastValueAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldListaggAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldMaxAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldMinAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldProductAgg; +import org.apache.fluss.server.kv.rowmerger.aggregate.functions.FieldSumAgg; +import org.apache.fluss.types.DataTypes; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for aggregator creation in {@link AggregationContext}. */ +class AggregationContextTest { + + @Test + void testAllAggregatorTypesFromSchema() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("sum_col", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("product_col", DataTypes.DOUBLE(), AggFunctions.PRODUCT()) + .column("max_col", DataTypes.INT(), AggFunctions.MAX()) + .column("min_col", DataTypes.INT(), AggFunctions.MIN()) + .column("last_val_col", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .column( + "last_nonnull_col", + DataTypes.STRING(), + AggFunctions.LAST_VALUE_IGNORE_NULLS()) + .column("first_val_col", DataTypes.STRING(), AggFunctions.FIRST_VALUE()) + .column( + "first_nonnull_col", + DataTypes.STRING(), + AggFunctions.FIRST_VALUE_IGNORE_NULLS()) + .column("bool_and_col", DataTypes.BOOLEAN(), AggFunctions.BOOL_AND()) + .column("bool_or_col", DataTypes.BOOLEAN(), AggFunctions.BOOL_OR()) + .column("listagg_col", DataTypes.STRING(), AggFunctions.LISTAGG()) + .column("string_agg_col", DataTypes.STRING(), AggFunctions.STRING_AGG()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + AggregationContext context = AggregationContext.create(schema, KvFormat.COMPACTED); + + // Primary key field should use FieldLastValueAgg (not aggregated) + assertThat(context.getAggregators()[0]).isInstanceOf(FieldLastValueAgg.class); + assertThat(context.getAggregators()[1]).isInstanceOf(FieldSumAgg.class); + assertThat(context.getAggregators()[2]).isInstanceOf(FieldProductAgg.class); + assertThat(context.getAggregators()[3]).isInstanceOf(FieldMaxAgg.class); + assertThat(context.getAggregators()[4]).isInstanceOf(FieldMinAgg.class); + assertThat(context.getAggregators()[5]).isInstanceOf(FieldLastValueAgg.class); + assertThat(context.getAggregators()[6]).isInstanceOf(FieldLastNonNullValueAgg.class); + assertThat(context.getAggregators()[7]).isInstanceOf(FieldFirstValueAgg.class); + assertThat(context.getAggregators()[8]).isInstanceOf(FieldFirstNonNullValueAgg.class); + assertThat(context.getAggregators()[9]).isInstanceOf(FieldBoolAndAgg.class); + assertThat(context.getAggregators()[10]).isInstanceOf(FieldBoolOrAgg.class); + assertThat(context.getAggregators()[11]).isInstanceOf(FieldListaggAgg.class); + assertThat(context.getAggregators()[12]) + .isInstanceOf(FieldListaggAgg.class); // string_agg is alias + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/FieldAggregatorParameterizedTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/FieldAggregatorParameterizedTest.java new file mode 100644 index 0000000000..37a4a3d46a --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/rowmerger/aggregate/FieldAggregatorParameterizedTest.java @@ -0,0 +1,807 @@ +/* + * 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.fluss.server.kv.rowmerger.aggregate; + +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.TableConfig; +import org.apache.fluss.metadata.AggFunctions; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaInfo; +import org.apache.fluss.record.BinaryValue; +import org.apache.fluss.record.TestingSchemaGetter; +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.server.kv.rowmerger.AggregateRowMerger; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeChecks; +import org.apache.fluss.types.DataTypes; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.util.stream.Stream; + +import static org.apache.fluss.testutils.DataTestUtils.compactedRow; +import static org.assertj.core.api.Assertions.assertThat; + +/** Parameterized tests for all aggregation functions with different data types. */ +class FieldAggregatorParameterizedTest { + + private static final short SCHEMA_ID = (short) 1; + + private BinaryValue toBinaryValue(BinaryRow row) { + return new BinaryValue(SCHEMA_ID, row); + } + + // =================================================================================== + // Sum Aggregation Tests + // =================================================================================== + + @ParameterizedTest(name = "sum aggregation with {0}") + @MethodSource("sumAggregationTestData") + void testSumAggregation( + String typeName, DataType dataType, Object val1, Object val2, Object expected) { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.SUM()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, val1}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, val2}); + + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + + // Verify result based on data type + assertThat(merged.row.getInt(0)).isEqualTo(1); // id stays the same + assertAggregatedValue(merged.row, 1, dataType, expected); + } + + static Stream sumAggregationTestData() { + return Stream.of( + Arguments.of("TINYINT", DataTypes.TINYINT(), (byte) 5, (byte) 3, (byte) 8), + Arguments.of( + "SMALLINT", DataTypes.SMALLINT(), (short) 100, (short) 200, (short) 300), + Arguments.of("INT", DataTypes.INT(), 1000, 2000, 3000), + Arguments.of("BIGINT", DataTypes.BIGINT(), 10000L, 20000L, 30000L), + Arguments.of("FLOAT", DataTypes.FLOAT(), 1.5f, 2.5f, 4.0f), + Arguments.of("DOUBLE", DataTypes.DOUBLE(), 10.5, 20.5, 31.0), + Arguments.of( + "DECIMAL(10,2)", + DataTypes.DECIMAL(10, 2), + Decimal.fromBigDecimal(new BigDecimal("100.50"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("200.75"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("301.25"), 10, 2))); + } + + @ParameterizedTest(name = "sum aggregation with null values - {0}") + @MethodSource("sumAggregationNullTestData") + void testSumAggregationWithNull( + String typeName, DataType dataType, Object val, Object expected) { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.SUM()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // Test: null + value = value + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, null}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, val}); + + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertAggregatedValue(merged.row, 1, dataType, expected); + + // Test: value + null = value + BinaryRow row3 = compactedRow(schema.getRowType(), new Object[] {1, val}); + BinaryRow row4 = compactedRow(schema.getRowType(), new Object[] {1, null}); + + BinaryValue merged2 = merger.merge(toBinaryValue(row3), toBinaryValue(row4)); + assertAggregatedValue(merged2.row, 1, dataType, expected); + } + + static Stream sumAggregationNullTestData() { + return Stream.of( + Arguments.of("TINYINT", DataTypes.TINYINT(), (byte) 10, (byte) 10), + Arguments.of("SMALLINT", DataTypes.SMALLINT(), (short) 100, (short) 100), + Arguments.of("INT", DataTypes.INT(), 1000, 1000), + Arguments.of("BIGINT", DataTypes.BIGINT(), 10000L, 10000L), + Arguments.of("FLOAT", DataTypes.FLOAT(), 5.5f, 5.5f), + Arguments.of("DOUBLE", DataTypes.DOUBLE(), 10.5, 10.5)); + } + + // =================================================================================== + // Product Aggregation Tests + // =================================================================================== + + @ParameterizedTest(name = "product aggregation with {0}") + @MethodSource("productAggregationTestData") + void testProductAggregation( + String typeName, DataType dataType, Object val1, Object val2, Object expected) { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.PRODUCT()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, val1}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, val2}); + + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + + assertThat(merged.row.getInt(0)).isEqualTo(1); + assertAggregatedValue(merged.row, 1, dataType, expected); + } + + static Stream productAggregationTestData() { + return Stream.of( + Arguments.of("TINYINT", DataTypes.TINYINT(), (byte) 5, (byte) 3, (byte) 15), + Arguments.of("SMALLINT", DataTypes.SMALLINT(), (short) 10, (short) 20, (short) 200), + Arguments.of("INT", DataTypes.INT(), 100, 200, 20000), + Arguments.of("BIGINT", DataTypes.BIGINT(), 100L, 200L, 20000L), + Arguments.of("FLOAT", DataTypes.FLOAT(), 2.5f, 4.0f, 10.0f), + Arguments.of("DOUBLE", DataTypes.DOUBLE(), 2.5, 4.0, 10.0), + Arguments.of( + "DECIMAL(10,2)", + DataTypes.DECIMAL(10, 2), + Decimal.fromBigDecimal(new BigDecimal("2.50"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("4.00"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("10.00"), 10, 2))); + } + + // =================================================================================== + // Max Aggregation Tests + // =================================================================================== + + @ParameterizedTest(name = "max aggregation with {0}") + @MethodSource("maxAggregationTestData") + void testMaxAggregation( + String typeName, + DataType dataType, + Object val1, + Object val2, + Object expectedMax, + Object expectedMin) { + // Test max + Schema schemaMax = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.MAX()) + .primaryKey("id") + .build(); + + TableConfig tableConfigMax = new TableConfig(new Configuration()); + + AggregateRowMerger mergerMax = createMerger(schemaMax, tableConfigMax); + + BinaryRow row1 = compactedRow(schemaMax.getRowType(), new Object[] {1, val1}); + BinaryRow row2 = compactedRow(schemaMax.getRowType(), new Object[] {1, val2}); + + BinaryValue mergedMax = mergerMax.merge(toBinaryValue(row1), toBinaryValue(row2)); + + assertThat(mergedMax.row.getInt(0)).isEqualTo(1); + assertAggregatedValue(mergedMax.row, 1, dataType, expectedMax); + + // Test min + Schema schemaMin = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.MIN()) + .primaryKey("id") + .build(); + + TableConfig tableConfigMin = new TableConfig(new Configuration()); + + AggregateRowMerger mergerMin = createMerger(schemaMin, tableConfigMin); + + BinaryRow row3 = compactedRow(schemaMin.getRowType(), new Object[] {1, val1}); + BinaryRow row4 = compactedRow(schemaMin.getRowType(), new Object[] {1, val2}); + + BinaryValue mergedMin = mergerMin.merge(toBinaryValue(row3), toBinaryValue(row4)); + + assertThat(mergedMin.row.getInt(0)).isEqualTo(1); + assertAggregatedValue(mergedMin.row, 1, dataType, expectedMin); + } + + static Stream maxAggregationTestData() { + return Stream.of( + // Numeric types + Arguments.of( + "TINYINT", DataTypes.TINYINT(), (byte) 5, (byte) 10, (byte) 10, (byte) 5), + Arguments.of( + "TINYINT_NEGATIVE", + DataTypes.TINYINT(), + (byte) -5, + (byte) -10, + (byte) -5, + (byte) -10), + Arguments.of( + "SMALLINT", + DataTypes.SMALLINT(), + (short) 100, + (short) 200, + (short) 200, + (short) 100), + Arguments.of("INT", DataTypes.INT(), 1000, 2000, 2000, 1000), + Arguments.of("BIGINT", DataTypes.BIGINT(), 10000L, 20000L, 20000L, 10000L), + Arguments.of("FLOAT", DataTypes.FLOAT(), 1.5f, 2.5f, 2.5f, 1.5f), + Arguments.of("DOUBLE", DataTypes.DOUBLE(), 10.5, 20.5, 20.5, 10.5), + Arguments.of( + "DECIMAL(10,2)", + DataTypes.DECIMAL(10, 2), + Decimal.fromBigDecimal(new BigDecimal("100.50"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("200.75"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("200.75"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("100.50"), 10, 2)), + // String type + Arguments.of("STRING", DataTypes.STRING(), "apple", "banana", "banana", "apple"), + Arguments.of("STRING_EMPTY", DataTypes.STRING(), "", "test", "test", ""), + // Date and time types + Arguments.of( + "DATE", + DataTypes.DATE(), + (int) LocalDate.of(2025, 1, 1).toEpochDay(), + (int) LocalDate.of(2025, 12, 31).toEpochDay(), + (int) LocalDate.of(2025, 12, 31).toEpochDay(), + (int) LocalDate.of(2025, 1, 1).toEpochDay()), + Arguments.of( + "TIME", + DataTypes.TIME(), + (int) (LocalTime.of(10, 0, 0).toNanoOfDay() / 1_000_000), + (int) (LocalTime.of(14, 30, 0).toNanoOfDay() / 1_000_000), + (int) (LocalTime.of(14, 30, 0).toNanoOfDay() / 1_000_000), + (int) (LocalTime.of(10, 0, 0).toNanoOfDay() / 1_000_000)), + Arguments.of( + "TIMESTAMP", + DataTypes.TIMESTAMP(), + timestampNtz("2025-01-10T12:00:00"), + timestampNtz("2025-12-31T23:59:59"), + timestampNtz("2025-12-31T23:59:59"), + timestampNtz("2025-01-10T12:00:00")), + Arguments.of( + "TIMESTAMP_LTZ", + DataTypes.TIMESTAMP_LTZ(), + timestampLtz("2025-01-10T12:00:00"), + timestampLtz("2025-12-31T23:59:59"), + timestampLtz("2025-12-31T23:59:59"), + timestampLtz("2025-01-10T12:00:00"))); + } + + // =================================================================================== + // Last Value Aggregation Tests + // =================================================================================== + + @ParameterizedTest(name = "last_value with {0}") + @MethodSource("lastValueTestData") + void testLastValueAggregation( + String typeName, DataType dataType, Object val1, Object val2, Object val3) { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.LAST_VALUE()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, val1}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, val2}); + BinaryRow row3 = compactedRow(schema.getRowType(), new Object[] {1, val3}); + + // First merge: last value should be val2 + BinaryValue merged1 = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertAggregatedValue(merged1.row, 1, dataType, val2); + + // Second merge: last value should be val3 + BinaryValue merged2 = merger.merge(merged1, toBinaryValue(row3)); + assertAggregatedValue(merged2.row, 1, dataType, val3); + } + + static Stream lastValueTestData() { + return Stream.of( + // Numeric types + Arguments.of("TINYINT", DataTypes.TINYINT(), (byte) 1, (byte) 2, (byte) 3), + Arguments.of("SMALLINT", DataTypes.SMALLINT(), (short) 10, (short) 20, (short) 30), + Arguments.of("INT", DataTypes.INT(), 100, 200, 300), + Arguments.of("BIGINT", DataTypes.BIGINT(), 1000L, 2000L, 3000L), + Arguments.of("FLOAT", DataTypes.FLOAT(), 1.5f, 2.5f, 3.5f), + Arguments.of("DOUBLE", DataTypes.DOUBLE(), 10.5, 20.5, 30.5), + Arguments.of( + "DECIMAL(10,2)", + DataTypes.DECIMAL(10, 2), + Decimal.fromBigDecimal(new BigDecimal("10.50"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("20.50"), 10, 2), + Decimal.fromBigDecimal(new BigDecimal("30.50"), 10, 2)), + // String types + Arguments.of("STRING", DataTypes.STRING(), "first", "second", "third"), + Arguments.of("CHAR(10)", DataTypes.CHAR(10), "aaa", "bbb", "ccc"), + // Boolean type + Arguments.of("BOOLEAN", DataTypes.BOOLEAN(), true, false, true), + // Date and time types + Arguments.of( + "DATE", + DataTypes.DATE(), + (int) LocalDate.of(2025, 1, 1).toEpochDay(), + (int) LocalDate.of(2025, 6, 15).toEpochDay(), + (int) LocalDate.of(2025, 12, 31).toEpochDay()), + Arguments.of( + "TIME", + DataTypes.TIME(), + (int) (LocalTime.of(10, 0, 0).toNanoOfDay() / 1_000_000), + (int) (LocalTime.of(12, 30, 0).toNanoOfDay() / 1_000_000), + (int) (LocalTime.of(15, 0, 0).toNanoOfDay() / 1_000_000)), + Arguments.of( + "TIMESTAMP", + DataTypes.TIMESTAMP(), + timestampNtz("2025-01-10T12:00:00"), + timestampNtz("2025-06-15T13:30:00"), + timestampNtz("2025-12-31T23:59:59")), + Arguments.of( + "TIMESTAMP_LTZ", + DataTypes.TIMESTAMP_LTZ(), + timestampLtz("2025-01-10T12:00:00"), + timestampLtz("2025-06-15T13:30:00"), + timestampLtz("2025-12-31T23:59:59")), + // Binary types + Arguments.of( + "BYTES", + DataTypes.BYTES(), + new byte[] {1, 2, 3}, + new byte[] {4, 5, 6}, + new byte[] {7, 8, 9}), + Arguments.of( + "BINARY(4)", + DataTypes.BINARY(4), + new byte[] {1, 2, 3, 4}, + new byte[] {5, 6, 7, 8}, + new byte[] {9, 10, 11, 12})); + } + + // =================================================================================== + // Last Non-Null Value Aggregation Tests + // =================================================================================== + + @ParameterizedTest(name = "last_value_ignore_nulls with {0}") + @MethodSource("lastNonNullValueTestData") + void testLastNonNullValueAggregation(String typeName, DataType dataType, Object val) { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.LAST_VALUE_IGNORE_NULLS()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // Test: value + null should keep value + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, val}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, null}); + + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertAggregatedValue(merged.row, 1, dataType, val); + } + + static Stream lastNonNullValueTestData() { + return Stream.of( + Arguments.of("TINYINT", DataTypes.TINYINT(), (byte) 10), + Arguments.of("SMALLINT", DataTypes.SMALLINT(), (short) 100), + Arguments.of("INT", DataTypes.INT(), 1000), + Arguments.of("BIGINT", DataTypes.BIGINT(), 10000L), + Arguments.of("FLOAT", DataTypes.FLOAT(), 5.5f), + Arguments.of("DOUBLE", DataTypes.DOUBLE(), 10.5), + Arguments.of("STRING", DataTypes.STRING(), "test"), + Arguments.of("BOOLEAN", DataTypes.BOOLEAN(), true), + Arguments.of( + "TIMESTAMP", DataTypes.TIMESTAMP(), timestampNtz("2025-01-10T12:00:00"))); + } + + // =================================================================================== + // First Value Aggregation Tests + // =================================================================================== + + @ParameterizedTest(name = "first_value with {0}") + @MethodSource("firstValueTestData") + void testFirstValueAggregation(String typeName, DataType dataType, Object val1, Object val2) { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.FIRST_VALUE()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, val1}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, val2}); + + // First value should always be val1 + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertAggregatedValue(merged.row, 1, dataType, val1); + } + + static Stream firstValueTestData() { + return Stream.of( + Arguments.of("TINYINT", DataTypes.TINYINT(), (byte) 10, (byte) 20), + Arguments.of("SMALLINT", DataTypes.SMALLINT(), (short) 100, (short) 200), + Arguments.of("INT", DataTypes.INT(), 1000, 2000), + Arguments.of("BIGINT", DataTypes.BIGINT(), 10000L, 20000L), + Arguments.of("STRING", DataTypes.STRING(), "first", "second"), + Arguments.of("BOOLEAN", DataTypes.BOOLEAN(), true, false)); + } + + // =================================================================================== + // First Non-Null Value Aggregation Tests + // =================================================================================== + + @ParameterizedTest(name = "first_value_ignore_nulls with {0}") + @MethodSource("firstNonNullValueTestData") + void testFirstNonNullValueAggregation(String typeName, DataType dataType, Object val) { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", dataType, AggFunctions.FIRST_VALUE_IGNORE_NULLS()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // Test: null + value should keep value + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, null}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, val}); + + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertAggregatedValue(merged.row, 1, dataType, val); + + // Test: value + non-null should keep first value + BinaryRow row3 = compactedRow(schema.getRowType(), new Object[] {1, val}); + BinaryRow row4 = compactedRow(schema.getRowType(), new Object[] {1, val}); + + BinaryValue merged2 = merger.merge(toBinaryValue(row3), toBinaryValue(row4)); + assertAggregatedValue(merged2.row, 1, dataType, val); + } + + static Stream firstNonNullValueTestData() { + return Stream.of( + Arguments.of("TINYINT", DataTypes.TINYINT(), (byte) 10), + Arguments.of("SMALLINT", DataTypes.SMALLINT(), (short) 100), + Arguments.of("INT", DataTypes.INT(), 1000), + Arguments.of("BIGINT", DataTypes.BIGINT(), 10000L), + Arguments.of("STRING", DataTypes.STRING(), "test")); + } + + // =================================================================================== + // Boolean Aggregation Tests + // =================================================================================== + + @Test + void testBoolAndAggregation() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BOOLEAN(), AggFunctions.BOOL_AND()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // true AND true = true + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, true}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, true}); + BinaryValue merged1 = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertThat(merged1.row.getBoolean(1)).isTrue(); + + // true AND false = false + BinaryRow row3 = compactedRow(schema.getRowType(), new Object[] {1, true}); + BinaryRow row4 = compactedRow(schema.getRowType(), new Object[] {1, false}); + BinaryValue merged2 = merger.merge(toBinaryValue(row3), toBinaryValue(row4)); + assertThat(merged2.row.getBoolean(1)).isFalse(); + + // false AND false = false + BinaryRow row5 = compactedRow(schema.getRowType(), new Object[] {1, false}); + BinaryRow row6 = compactedRow(schema.getRowType(), new Object[] {1, false}); + BinaryValue merged3 = merger.merge(toBinaryValue(row5), toBinaryValue(row6)); + assertThat(merged3.row.getBoolean(1)).isFalse(); + } + + @Test + void testBoolOrAggregation() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BOOLEAN(), AggFunctions.BOOL_OR()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // false OR false = false + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, false}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, false}); + BinaryValue merged1 = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertThat(merged1.row.getBoolean(1)).isFalse(); + + // true OR false = true + BinaryRow row3 = compactedRow(schema.getRowType(), new Object[] {1, true}); + BinaryRow row4 = compactedRow(schema.getRowType(), new Object[] {1, false}); + BinaryValue merged2 = merger.merge(toBinaryValue(row3), toBinaryValue(row4)); + assertThat(merged2.row.getBoolean(1)).isTrue(); + + // true OR true = true + BinaryRow row5 = compactedRow(schema.getRowType(), new Object[] {1, true}); + BinaryRow row6 = compactedRow(schema.getRowType(), new Object[] {1, true}); + BinaryValue merged3 = merger.merge(toBinaryValue(row5), toBinaryValue(row6)); + assertThat(merged3.row.getBoolean(1)).isTrue(); + } + + @Test + void testBoolAndWithNull() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BOOLEAN(), AggFunctions.BOOL_AND()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // null AND true = true + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, null}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, true}); + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertThat(merged.row.getBoolean(1)).isTrue(); + } + + @Test + void testBoolOrWithNull() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.BOOLEAN(), AggFunctions.BOOL_OR()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // null OR false = false + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, null}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, false}); + BinaryValue merged = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertThat(merged.row.getBoolean(1)).isFalse(); + } + + // =================================================================================== + // Listagg Aggregation Tests + // =================================================================================== + + @Test + void testListaggAggregation() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.STRING(), AggFunctions.LISTAGG()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // Test basic concatenation with default delimiter (comma) + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, "apple"}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, "banana"}); + BinaryRow row3 = compactedRow(schema.getRowType(), new Object[] {1, "cherry"}); + + BinaryValue merged1 = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertThat(merged1.row.getString(1).toString()).isEqualTo("apple,banana"); + + BinaryValue merged2 = merger.merge(merged1, toBinaryValue(row3)); + assertThat(merged2.row.getString(1).toString()).isEqualTo("apple,banana,cherry"); + } + + @Test + void testListaggWithCustomDelimiter() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.STRING(), AggFunctions.LISTAGG("|")) + .primaryKey("id") + .build(); + + Configuration conf = new Configuration(); + TableConfig tableConfig = new TableConfig(conf); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, "a"}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, "b"}); + BinaryRow row3 = compactedRow(schema.getRowType(), new Object[] {1, "c"}); + + BinaryValue merged1 = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertThat(merged1.row.getString(1).toString()).isEqualTo("a|b"); + + BinaryValue merged2 = merger.merge(merged1, toBinaryValue(row3)); + assertThat(merged2.row.getString(1).toString()).isEqualTo("a|b|c"); + } + + @Test + void testListaggWithNull() { + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("value", DataTypes.STRING(), AggFunctions.LISTAGG()) + .primaryKey("id") + .build(); + + TableConfig tableConfig = new TableConfig(new Configuration()); + + AggregateRowMerger merger = createMerger(schema, tableConfig); + + // Test null handling: null values should be skipped + BinaryRow row1 = compactedRow(schema.getRowType(), new Object[] {1, "a"}); + BinaryRow row2 = compactedRow(schema.getRowType(), new Object[] {1, null}); + BinaryRow row3 = compactedRow(schema.getRowType(), new Object[] {1, "b"}); + + BinaryValue merged1 = merger.merge(toBinaryValue(row1), toBinaryValue(row2)); + assertThat(merged1.row.getString(1).toString()).isEqualTo("a"); + + BinaryValue merged2 = merger.merge(merged1, toBinaryValue(row3)); + assertThat(merged2.row.getString(1).toString()).isEqualTo("a,b"); + } + + // =================================================================================== + // Helper Methods + // =================================================================================== + + private AggregateRowMerger createMerger(Schema schema, TableConfig tableConfig) { + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(schema, (short) 1)); + AggregateRowMerger merger = + new AggregateRowMerger(tableConfig, tableConfig.getKvFormat(), schemaGetter); + merger.configureTargetColumns(null, (short) 1, schema); + return merger; + } + + private void assertAggregatedValue(BinaryRow row, int pos, DataType dataType, Object expected) { + if (expected == null) { + assertThat(row.isNullAt(pos)).isTrue(); + return; + } + + String typeName = dataType.getTypeRoot().name(); + switch (typeName) { + case "TINYINT": + assertThat(row.getByte(pos)).isEqualTo((Byte) expected); + break; + case "SMALLINT": + assertThat(row.getShort(pos)).isEqualTo((Short) expected); + break; + case "INTEGER": + assertThat(row.getInt(pos)).isEqualTo((Integer) expected); + break; + case "BIGINT": + assertThat(row.getLong(pos)).isEqualTo((Long) expected); + break; + case "FLOAT": + assertThat(row.getFloat(pos)).isEqualTo((Float) expected); + break; + case "DOUBLE": + assertThat(row.getDouble(pos)).isEqualTo((Double) expected); + break; + case "DECIMAL": + int precision = DataTypeChecks.getPrecision(dataType); + int scale = DataTypeChecks.getScale(dataType); + Decimal actualDecimal = row.getDecimal(pos, precision, scale); + Decimal expectedDecimal = (Decimal) expected; + assertThat(actualDecimal.toBigDecimal()) + .isEqualByComparingTo(expectedDecimal.toBigDecimal()); + break; + case "BOOLEAN": + assertThat(row.getBoolean(pos)).isEqualTo((Boolean) expected); + break; + case "STRING": + case "VARCHAR": + assertThat(row.getString(pos).toString()).isEqualTo((String) expected); + break; + case "CHAR": + // CHAR type is padded with spaces, need to trim + assertThat(row.getString(pos).toString().trim()).isEqualTo((String) expected); + break; + case "DATE": + assertThat(row.getInt(pos)).isEqualTo((Integer) expected); + break; + case "TIME_WITHOUT_TIME_ZONE": + assertThat(row.getInt(pos)).isEqualTo((Integer) expected); + break; + case "TIMESTAMP_WITHOUT_TIME_ZONE": + int tsPrecision = DataTypeChecks.getPrecision(dataType); + TimestampNtz actualTs = row.getTimestampNtz(pos, tsPrecision); + TimestampNtz expectedTs = (TimestampNtz) expected; + assertThat(actualTs.getMillisecond()).isEqualTo(expectedTs.getMillisecond()); + assertThat(actualTs.getNanoOfMillisecond()) + .isEqualTo(expectedTs.getNanoOfMillisecond()); + break; + case "TIMESTAMP_WITH_LOCAL_TIME_ZONE": + int tsLtzPrecision = DataTypeChecks.getPrecision(dataType); + TimestampLtz actualTsLtz = row.getTimestampLtz(pos, tsLtzPrecision); + TimestampLtz expectedTsLtz = (TimestampLtz) expected; + assertThat(actualTsLtz.getEpochMillisecond()) + .isEqualTo(expectedTsLtz.getEpochMillisecond()); + assertThat(actualTsLtz.getNanoOfMillisecond()) + .isEqualTo(expectedTsLtz.getNanoOfMillisecond()); + break; + case "BYTES": + case "VARBINARY": + case "BINARY": + byte[] expectedBytes = (byte[]) expected; + assertThat(row.getBinary(pos, expectedBytes.length)).isEqualTo(expectedBytes); + break; + default: + throw new UnsupportedOperationException( + "Unsupported data type for assertion: " + typeName); + } + } + + private static TimestampNtz timestampNtz(String timestamp) { + return TimestampNtz.fromLocalDateTime(LocalDateTime.parse(timestamp)); + } + + private static TimestampLtz timestampLtz(String timestamp) { + Instant instant = LocalDateTime.parse(timestamp).toInstant(ZoneOffset.UTC); + return TimestampLtz.fromInstant(instant); + } +} diff --git a/website/docs/engine-flink/options.md b/website/docs/engine-flink/options.md index 2fecb25dd6..a7114af492 100644 --- a/website/docs/engine-flink/options.md +++ b/website/docs/engine-flink/options.md @@ -83,9 +83,9 @@ See more details about [ALTER TABLE ... SET](engine-flink/ddl.md#set-properties) | table.datalake.freshness | Duration | 3min | It defines the maximum amount of time that the datalake table's content should lag behind updates to the Fluss table. Based on this target freshness, the Fluss service automatically moves data from the Fluss table and updates to the datalake table, so that the data in the datalake table is kept up to date within this target. If the data does not need to be as fresh, you can specify a longer target freshness time to reduce costs. | | table.datalake.auto-compaction | Boolean | false | If true, compaction will be triggered automatically when tiering service writes to the datalake. It is disabled by default. | | table.datalake.auto-expire-snapshot | Boolean | false | If true, snapshot expiration will be triggered automatically when tiering service commits to the datalake. It is disabled by default. | -| table.merge-engine | Enum | (None) | Defines the merge engine for the primary key table. By default, primary key table uses the [default merge engine(last_row)](table-design/table-types/pk-table/merge-engines/default.md). It also supports two merge engines are `first_row` and `versioned`. The [first_row merge engine](table-design/table-types/pk-table/merge-engines/first-row.md) will keep the first row of the same primary key. The [versioned merge engine](table-design/table-types/pk-table/merge-engines/versioned.md) will keep the row with the largest version of the same primary key. | +| table.merge-engine | Enum | (None) | Defines the merge engine for the primary key table. By default, primary key table uses the [default merge engine(last_row)](table-design/table-types/pk-table/merge-engines/default.md). It also supports two merge engines are `first_row`, `versioned` and `aggregation`. The [first_row merge engine](table-design/table-types/pk-table/merge-engines/first-row.md) will keep the first row of the same primary key. The [versioned merge engine](table-design/table-types/pk-table/merge-engines/versioned.md) will keep the row with the largest version of the same primary key. The `aggregation` merge engine will aggregate rows with the same primary key using field-level aggregate functions. | | table.merge-engine.versioned.ver-column | String | (None) | The column name of the version column for the `versioned` merge engine. If the merge engine is set to `versioned`, the version column must be set. | -| table.delete.behavior | Enum | ALLOW | Controls the behavior of delete operations on primary key tables. Three modes are supported: `ALLOW` (default) - allows normal delete operations; `IGNORE` - silently ignores delete requests without errors; `DISABLE` - rejects delete requests and throws explicit errors. This configuration provides system-level guarantees for some downstream pipelines (e.g., Flink Delta Join) that must not receive any delete events in the changelog of the table. For tables with `first_row` or `versioned` merge engines, this option is automatically set to `IGNORE` and cannot be overridden. Only applicable to primary key tables. | +| table.delete.behavior | Enum | ALLOW | Controls the behavior of delete operations on primary key tables. Three modes are supported: `ALLOW` (default for default merge engine) - allows normal delete operations; `IGNORE` - silently ignores delete requests without errors; `DISABLE` - rejects delete requests and throws explicit errors. This configuration provides system-level guarantees for some downstream pipelines (e.g., Flink Delta Join) that must not receive any delete events in the changelog of the table. For tables with `first_row` or `versioned` or `aggregation` merge engines, this option is automatically set to `IGNORE` and cannot be overridden. Note: For `aggregation` merge engine, when set to `allow`, delete operations will remove the entire record. This configuration only applicable to primary key tables. | | table.changelog.image | Enum | FULL | Defines the changelog image mode for primary key tables. This configuration is inspired by similar settings in database systems like MySQL's `binlog_row_image` and PostgreSQL's `replica identity`. Two modes are supported: `FULL` (default) - produces both UPDATE_BEFORE and UPDATE_AFTER records for update operations, capturing complete information about updates and allowing tracking of previous values; `WAL` - does not produce UPDATE_BEFORE records. Only INSERT, UPDATE_AFTER (and DELETE if allowed) records are emitted. When WAL mode is enabled with default merge engine (no merge engine configured) and full row updates (not partial update), an optimization is applied to skip looking up old values, and in this case INSERT operations are converted to UPDATE_AFTER events. This mode reduces storage and transmission costs but loses the ability to track previous values. Only applicable to primary key tables. | diff --git a/website/docs/table-design/table-types/pk-table/index.md b/website/docs/table-design/table-types/pk-table/index.md index 986b140d3e..261424b5b9 100644 --- a/website/docs/table-design/table-types/pk-table/index.md +++ b/website/docs/table-design/table-types/pk-table/index.md @@ -85,6 +85,7 @@ The following merge engines are supported: 1. [Default Merge Engine (LastRow)](merge-engines/default.md) 2. [FirstRow Merge Engine](merge-engines/first-row.md) 3. [Versioned Merge Engine](merge-engines/versioned.md) +4. [Aggregation Merge Engine](merge-engines/aggregation.md) ## Changelog Generation diff --git a/website/docs/table-design/table-types/pk-table/merge-engines/aggregation.md b/website/docs/table-design/table-types/pk-table/merge-engines/aggregation.md new file mode 100644 index 0000000000..ed44e2e256 --- /dev/null +++ b/website/docs/table-design/table-types/pk-table/merge-engines/aggregation.md @@ -0,0 +1,1019 @@ +--- +sidebar_label: Aggregation +title: Aggregation Merge Engine +sidebar_position: 5 +--- + +# Aggregation Merge Engine + +## Overview + +The **Aggregation Merge Engine** is designed for scenarios where users only care about aggregated results rather than individual records. It aggregates each value field with the latest data one by one under the same primary key according to the specified aggregate function. + +Each field not part of the primary keys can be assigned an aggregate function. The recommended way depends on the client you are working with: +- For **Flink SQL** or **Spark SQL**, use DDL and connector options (`'fields..agg'`) +- For **Java clients**, use the Schema API + +If no function is specified for a field, it will use `last_value_ignore_nulls` aggregation as the default behavior. + +This merge engine is useful for real-time aggregation scenarios such as: +- Computing running totals and statistics +- Maintaining counters and metrics +- Tracking maximum/minimum values over time +- Building real-time dashboards and analytics + +## Configuration + +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + + + +To enable the aggregation merge engine, set the following table property: + +```sql +CREATE TABLE product_stats ( + product_id BIGINT, + price DOUBLE, + sales BIGINT, + last_update_time TIMESTAMP(3), + PRIMARY KEY (product_id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.price.agg' = 'max', + 'fields.sales.agg' = 'sum' + -- last_update_time defaults to 'last_value_ignore_nulls' +); +``` + +Specify the aggregate function for each non-primary key field using connector options: + +```sql +'fields..agg' = '' +``` + + + + +To enable the aggregation merge engine, set the following table property: + +```java +TableDescriptor tableDescriptor = TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); +``` + +Specify the aggregate function for each non-primary key field using the Schema API: + +```java +Schema schema = Schema.newBuilder() + .column("product_id", DataTypes.BIGINT()) + .column("price", DataTypes.DOUBLE(), AggFunctions.MAX()) + .column("sales", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("last_update_time", DataTypes.TIMESTAMP(3)) // Defaults to LAST_VALUE_IGNORE_NULLS + .primaryKey("product_id") + .build(); +``` + + + + +## Usage Examples + + + + +### Creating a Table with Aggregation + +```sql +CREATE TABLE product_stats ( + product_id BIGINT, + price DOUBLE, + sales BIGINT, + last_update_time TIMESTAMP(3), + PRIMARY KEY (product_id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.price.agg' = 'max', + 'fields.sales.agg' = 'sum' + -- last_update_time defaults to 'last_value_ignore_nulls' +); +``` + +### Writing Data + +```sql +-- Insert data - these will be aggregated +INSERT INTO product_stats VALUES + (1, 23.0, 15, TIMESTAMP '2024-01-01 10:00:00'), + (1, 30.2, 20, TIMESTAMP '2024-01-01 11:00:00'); -- Same primary key - triggers aggregation +``` + +### Querying Results + +```sql +SELECT * FROM product_stats; +``` + +**Result after aggregation:** +``` ++------------+-------+-------+---------------------+ +| product_id | price | sales | last_update_time | ++------------+-------+-------+---------------------+ +| 1 | 30.2 | 35 | 2024-01-01 11:00:00 | ++------------+-------+-------+---------------------+ +``` + +- `product_id`: 1 +- `price`: 30.2 (max of 23.0 and 30.2) +- `sales`: 35 (sum of 15 and 20) +- `last_update_time`: 2024-01-01 11:00:00 (last non-null value) + + + + +### Creating a Table with Aggregation + +```java +import org.apache.fluss.client.Connection; +import org.apache.fluss.client.admin.Admin; +import org.apache.fluss.client.table.Table; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.metadata.AggFunction; + +// Create connection +Connection conn = Connection.create(config); +Admin admin = conn.getAdmin(); + +// Define schema with aggregation functions +Schema schema = Schema.newBuilder() + .column("product_id", DataTypes.BIGINT()) + .column("price", DataTypes.DOUBLE(), AggFunctions.MAX()) + .column("sales", DataTypes.BIGINT(), AggFunctions.SUM()) + .column("last_update_time", DataTypes.TIMESTAMP(3)) // Defaults to LAST_VALUE_IGNORE_NULLS + .primaryKey("product_id") + .build(); + +// Create table with aggregation merge engine +TableDescriptor tableDescriptor = TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +TablePath tablePath = TablePath.of("my_database", "product_stats"); +admin.createTable(tablePath, tableDescriptor, false).get(); +``` + +### Writing Data + +```java +// Get table +Table table = conn.getTable(tablePath); + +// Create upsert writer +UpsertWriter writer = table.newUpsert().createWriter(); + +// Write data - these will be aggregated +writer.upsert(row(1L, 23.0, 15L, timestamp1)); +writer.upsert(row(1L, 30.2, 20L, timestamp2)); // Same primary key - triggers aggregation + +writer.flush(); +``` + +**Result after aggregation:** +- `product_id`: 1 +- `price`: 30.2 (max of 23.0 and 30.2) +- `sales`: 35 (sum of 15 and 20) +- `last_update_time`: timestamp2 (last non-null value) + + + + +## Supported Aggregate Functions + +Fluss currently supports the following aggregate functions: + +### sum + +Aggregates values by computing the sum across multiple rows. + +- **Supported Data Types**: `TINYINT`, `SMALLINT`, `INT`, `BIGINT`, `FLOAT`, `DOUBLE`, `DECIMAL` +- **Behavior**: Adds incoming values to the accumulator +- **Null Handling**: Null values are ignored + +**Example:** + + + + +```sql +CREATE TABLE test_sum ( + id BIGINT, + amount DECIMAL(10, 2), + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.amount.agg' = 'sum' +); + +INSERT INTO test_sum VALUES + (1, 100.50), + (1, 200.75); + +SELECT * FROM test_sum; ++------------+---------+ +| id | amount | ++------------+---------+ +| 1 | 301.25 | ++------------+---------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("amount", DataTypes.DECIMAL(10, 2), AggFunctions.SUM()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, 100.50), (1, 200.75) +// Result: (1, 301.25) +``` + + + +### product + +Computes the product of values across multiple rows. + +- **Supported Data Types**: TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, DECIMAL +- **Behavior**: Multiplies incoming values with the accumulator +- **Null Handling**: Null values are ignored + +**Example:** + + + +```sql +CREATE TABLE test_product ( + id BIGINT, + discount_factor DOUBLE, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.discount_factor.agg' = 'product' +); + +INSERT INTO test_product VALUES + (1, 0.9), + (1, 0.8); + +SELECT * FROM test_product; ++------------+---------+ +| id | amount | ++------------+---------+ +| 1 | 0.72 | ++------------+---------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("discount_factor", DataTypes.DOUBLE(), AggFunctions.PRODUCT()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, 0.9), (1, 0.8) +// Result: (1, 0.72) -- 90% * 80% = 72% +``` + + + + +### max + +Identifies and retains the maximum value. + +- **Supported Data Types**: CHAR, STRING, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, DECIMAL, DATE, TIME, TIMESTAMP, TIMESTAMP_LTZ +- **Behavior**: Keeps the larger value between accumulator and incoming value +- **Null Handling**: Null values are ignored + +**Example:** + + + +```sql +CREATE TABLE test_max ( + id BIGINT, + temperature DOUBLE, + reading_time TIMESTAMP(3), + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.temperature.agg' = 'max', + 'fields.reading_time.agg' = 'max' +); + +INSERT INTO test_max VALUES + (1, 25.5, TIMESTAMP '2024-01-01 10:00:00'), + (1, 28.3, TIMESTAMP '2024-01-01 11:00:00'); + +SELECT * FROM test_max; ++------------+----------------+---------------------+ +| id | temperature | reading_time | ++------------+----------------+---------------------+ +| 1 | 28.3 | 2024-01-01 11:00:00 | ++------------+----------------+---------------------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("max_temperature", DataTypes.DOUBLE(), AggFunctions.MAX()) + .column("max_reading_time", DataTypes.TIMESTAMP(3), AggFunctions.MAX()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, 25.5, '2024-01-01 10:00:00'), (1, 28.3, '2024-01-01 11:00:00') +// Result: (1, 28.3, '2024-01-01 11:00:00') +``` + + + +### min + +Identifies and retains the minimum value. + +- **Supported Data Types**: CHAR, STRING, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, DECIMAL, DATE, TIME, TIMESTAMP, TIMESTAMP_LTZ +- **Behavior**: Keeps the smaller value between accumulator and incoming value +- **Null Handling**: Null values are ignored + +**Example:** + + + +```sql +CREATE TABLE test_min ( + id BIGINT, + lowest_price DECIMAL(10, 2), + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.lowest_price.agg' = 'min' +); + +INSERT INTO test_min VALUES + (1, 99.99), + (1, 79.99), + (1, 89.99); + +SELECT * FROM test_min; ++------------+--------------+ +| id | lowest_price | ++------------+--------------+ +| 1 | 79.99 | ++------------+--------------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("lowest_price", DataTypes.DECIMAL(10, 2), AggFunctions.MIN()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, 99.99), (1, 79.99), (1, 89.99) +// Result: (1, 79.99) +``` + + + + +### last_value + +Replaces the previous value with the most recently received value. + +- **Supported Data Types**: All data types +- **Behavior**: Always uses the latest incoming value +- **Null Handling**: Null values will overwrite previous values + +**Example:** + + + +```sql +CREATE TABLE test_last_value ( + id BIGINT, + status STRING, + last_login TIMESTAMP(3), + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.status.agg' = 'last_value', + 'fields.last_login.agg' = 'last_value' +); + + +INSERT INTO test_last_value VALUES + (1, 'online', TIMESTAMP '2024-01-01 10:00:00'), + (1, 'offline', TIMESTAMP '2024-01-01 11:00:00'), + (1, null, TIMESTAMP '2024-01-01 12:00:00'); -- Null overwrites previous 'offline' value + +SELECT * FROM test_last_value; ++------------+---------+---------------------+ +| id | status | last_login | ++------------+---------+---------------------+ +| 1 | NULL | 2024-01-01 12:00:00 | ++------------+---------+---------------------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("status", DataTypes.STRING(), AggFunctions.LAST_VALUE()) + .column("last_login", DataTypes.TIMESTAMP(3), AggFunctions.LAST_VALUE()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Step 1: Insert initial values +// Input: (1, 'online', '2024-01-01 10:00:00') +// Result: (1, 'online', '2024-01-01 10:00:00') + +// Step 2: Upsert with new values +// Input: (1, 'offline', '2024-01-01 11:00:00') +// Result: (1, 'offline', '2024-01-01 11:00:00') + +// Step 3: Upsert with null status - null overwrites the previous 'offline' value +// Input: (1, null, '2024-01-01 12:00:00') +// Result: (1, null, '2024-01-01 12:00:00') +// Note: status becomes null (null overwrites previous value), last_login updated +``` + + + +**Key behavior:** Null values overwrite existing values, treating null as a valid value to be stored. + +### last_value_ignore_nulls + +Replaces the previous value with the latest non-null value. This is the **default aggregate function** when no function is specified. + +- **Supported Data Types**: All data types +- **Behavior**: Uses the latest incoming value only if it's not null +- **Null Handling**: Null values are ignored, previous value is retained + +**Example:** + + + +```sql +CREATE TABLE test_last_value_ignore_nulls ( + id BIGINT, + email STRING, + phone STRING, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.email.agg' = 'last_value_ignore_nulls', + 'fields.phone.agg' = 'last_value_ignore_nulls' +); + + +INSERT INTO test_last_value_ignore_nulls VALUES + (1, 'user@example.com', '123-456'), + (1, null, '789-012'), -- Null is ignored, email retains previous value + (1, 'new@example.com', null); + +SELECT * FROM test_last_value_ignore_nulls; ++------------+-------------------+---------+ +| id | email | phone | ++------------+-------------------+---------+ +| 1 | new@example.com | 789-012 | ++------------+-------------------+---------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("email", DataTypes.STRING(), AggFunctions.LAST_VALUE_IGNORE_NULLS()) + .column("phone", DataTypes.STRING(), AggFunctions.LAST_VALUE_IGNORE_NULLS()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Step 1: Insert initial values +// Input: (1, 'user@example.com', '123-456') +// Result: (1, 'user@example.com', '123-456') + +// Step 2: Upsert with null email - null is ignored, email retains previous value +// Input: (1, null, '789-012') +// Result: (1, 'user@example.com', '789-012') +// Note: email remains 'user@example.com' (null was ignored), phone updated to '789-012' + +// Step 3: Upsert with null phone - null is ignored, phone retains previous value +// Input: (1, 'new@example.com', null) +// Result: (1, 'new@example.com', '789-012') +// Note: email updated to 'new@example.com', phone remains '789-012' (null was ignored) +``` + + + + +**Key behavior:** Null values do not overwrite existing non-null values, making this function ideal for maintaining the most recent valid data. + +### first_value + +Retrieves and retains the first value seen for a field. + +- **Supported Data Types**: All data types +- **Behavior**: Keeps the first received value, ignores all subsequent values +- **Null Handling**: Null values are retained if received first + +**Example:** + + + +```sql +CREATE TABLE test_first_value ( + id BIGINT, + first_purchase_date DATE, + first_product STRING, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.first_purchase_date.agg' = 'first_value', + 'fields.first_product.agg' = 'first_value' +); + +INSERT INTO test_first_value VALUES + (1, '2024-01-01', 'ProductA'), + (1, '2024-02-01', 'ProductB'); -- Ignored, first value retained + +SELECT * FROM test_first_value; ++------------+---------------------+---------------+ +| id | first_purchase_date | first_product | ++------------+---------------------+---------------+ +| 1 | 2024-01-01 | ProductA | ++------------+---------------------+---------------+ +``` + + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("first_purchase_date", DataTypes.DATE(), AggFunctions.FIRST_VALUE()) + .column("first_product", DataTypes.STRING(), AggFunctions.FIRST_VALUE()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, '2024-01-01', 'ProductA'), (1, '2024-02-01', 'ProductB') +// Result: (1, '2024-01-01', 'ProductA') +``` + + + + +### first_value_ignore_nulls + +Selects the first non-null value in a data set. + +- **Supported Data Types**: All data types +- **Behavior**: Keeps the first received non-null value, ignores all subsequent values +- **Null Handling**: Null values are ignored until a non-null value is received + +**Example:** + + + +```sql +CREATE TABLE test_first_value_ignore_nulls ( + id BIGINT, + email STRING, + verified_at TIMESTAMP(3), + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.email.agg' = 'first_value_ignore_nulls', + 'fields.verified_at.agg' = 'first_value_ignore_nulls' +); + +INSERT INTO test_first_value_ignore_nulls VALUES + (1, null, null), + (1, 'user@example.com', '2024-01-01 10:00:00'), + (1, 'other@example.com', '2024-01-02 10:00:00'); -- Only the first non-null value is retained + +SELECT * FROM test_first_value_ignore_nulls; ++------------+-------------------+---------------------+ +| id | email | verified_at | ++------------+-------------------+---------------------+ +| 1 | user@example.com | 2024-01-01 10:00:00 | ++------------+-------------------+---------------------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("email", DataTypes.STRING(), AggFunctions.FIRST_VALUE_IGNORE_NULLS()) + .column("verified_at", DataTypes.TIMESTAMP(3), AggFunctions.FIRST_VALUE_IGNORE_NULLS()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, null, null), (1, 'user@example.com', '2024-01-01 10:00:00'), (1, 'other@example.com', '2024-01-02 10:00:00') +// Result: (1, 'user@example.com', '2024-01-01 10:00:00') +``` + + + + +### listagg + +Concatenates multiple string values into a single string with a delimiter. + +- **Supported Data Types**: STRING, CHAR +- **Behavior**: Concatenates values using the specified delimiter +- **Null Handling**: Null values are skipped +- **Delimiter**: Specify delimiter directly in the aggregation function (default is comma `,`) + +**Example:** + + + +```sql +CREATE TABLE test_listagg ( + id BIGINT, + tags1 STRING, + tags2 STRING, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.tags1.agg' = 'listagg', + 'fields.tags2.agg' = 'listagg', + 'fields.tags2.delimiter' = ';' -- Specify delimiter inline +); + +INSERT INTO test_listagg VALUES + (1, 'developer', 'developer'), + (1, 'java', 'java'), + (1, 'flink', 'flink'); + +SELECT * FROM test_listagg; ++------------+-----------------------+-----------------------+ +| id | tags1 | tags2 | ++------------+-----------------------+-----------------------+ +| 1 | developer,java,flink | developer;java;flink | ++------------+-----------------------+-----------------------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("tags1", DataTypes.STRING(), AggFunctions.LISTAGG()) + .column("tags2", DataTypes.STRING(), AggFunctions.LISTAGG(";")) // Specify delimiter inline + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, 'developer', 'developer'), (1, 'java', 'java'), (1, 'flink', 'flink') +// Result: (1, 'developer,java,flink', 'developer;java;flink') +``` + + + + +### string_agg + +Alias for `listagg`. Concatenates multiple string values into a single string with a delimiter. + +- **Supported Data Types**: STRING, CHAR +- **Behavior**: Same as `listagg` - concatenates values using the specified delimiter +- **Null Handling**: Null values are skipped +- **Delimiter**: Specify delimiter directly in the aggregation function (default is comma `,`) + +**Example:** + + + +```sql +CREATE TABLE test_string_agg ( + id BIGINT, + tags1 STRING, + tags2 STRING, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.tags1.agg' = 'string_agg', + 'fields.tags2.agg' = 'string_agg', + 'fields.tags2.delimiter' = ';' -- Specify delimiter inline +); + +INSERT INTO test_string_agg VALUES + (1, 'developer', 'developer'), + (1, 'java', 'java'), + (1, 'flink', 'flink'); + +SELECT * FROM test_string_agg; ++------------+-----------------------+-----------------------+ +| id | tags1 | tags2 | ++------------+-----------------------+-----------------------+ +| 1 | developer,java,flink | developer;java;flink | ++------------+-----------------------+-----------------------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("tags", DataTypes.STRING(), AggFunctions.STRING_AGG(";")) // Specify delimiter inline + .primaryKey("id") + .build(); +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("tags1", DataTypes.STRING(), AggFunctions.STRING_AGG()) + .column("tags2", DataTypes.STRING(), AggFunctions.STRING_AGG(";")) // Specify delimiter inline + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, 'developer', 'developer'), (1, 'java', 'java'), (1, 'flink', 'flink') +// Result: (1, 'developer,java,flink', 'developer;java;flink') +``` + + + + +### bool_and + +Evaluates whether all boolean values in a set are true (logical AND). + +- **Supported Data Types**: BOOLEAN +- **Behavior**: Returns true only if all values are true +- **Null Handling**: Null values are ignored + +**Example:** + + + +```sql +CREATE TABLE test_bool_and ( + id BIGINT, + has_all_permissions BOOLEAN, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.has_all_permissions.agg' = 'bool_and' +); + +INSERT INTO test_bool_and VALUES + (1, true), + (1, true), + (1, false); + +SELECT * FROM test_bool_and; ++------------+----------------------+ +| id | has_all_permissions | ++------------+----------------------+ +| 1 | false | ++------------+----------------------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("has_all_permissions", DataTypes.BOOLEAN(), AggFunctions.BOOL_AND()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, true), (1, true), (1, false) +// Result: (1, false) -- Not all values are true +``` + + + + +### bool_or + +Checks if at least one boolean value in a set is true (logical OR). + +- **Supported Data Types**: BOOLEAN +- **Behavior**: Returns true if any value is true +- **Null Handling**: Null values are ignored + +**Example:** + + + +```sql +CREATE TABLE test_bool_or ( + id BIGINT, + has_any_alert BOOLEAN, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'table.merge-engine' = 'aggregation', + 'fields.has_any_alert.agg' = 'bool_or' +); + +INSERT INTO test_bool_or VALUES + (1, false), + (1, false), + (1, true); + +SELECT * FROM test_bool_or; ++------------+------------------+ +| id | has_any_alert | ++------------+------------------+ +| 1 | true | ++------------+------------------+ +``` + + + + +```java +Schema schema = Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("has_any_alert", DataTypes.BOOLEAN(), AggFunctions.BOOL_OR()) + .primaryKey("id") + .build(); + +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .build(); + +// Input: (1, false), (1, false), (1, true) +// Result: (1, true) -- At least one value is true +``` + + + + +## Delete Behavior + +The aggregation merge engine provides limited support for delete operations. You can configure the behavior using the `'table.delete.behavior'` option: + +```java +TableDescriptor.builder() + .schema(schema) + .property("table.merge-engine", "aggregation") + .property("table.delete.behavior", "allow") // Enable delete operations + .build(); +``` + +**Configuration options**: +- **`'table.delete.behavior' = 'ignore'`** (default): Delete operations will be silently ignored without error +- **`'table.delete.behavior' = 'disable'`**: Delete operations will be rejected with a clear error message +- **`'table.delete.behavior' = 'allow'`**: Delete operations will remove records based on the update mode (see details below) + +### Delete Behavior with Different Update Modes + +When `'table.delete.behavior' = 'allow'`, the actual delete behavior depends on whether you are using **full update** or **partial update**: + +**Full Update (Default Write Mode)**: +- Delete operations remove the **entire record** from the table +- All aggregated values for that primary key are permanently lost + +**Example**: +```java +// Full update mode (default) +UpsertWriter writer = table.newUpsert().createWriter(); +writer.delete(primaryKeyRow); // Removes the entire record +``` + +**Partial Update Mode**: +- Delete operations perform a **partial delete** on target columns only +- **Target columns** (except primary key): Set to null +- **Non-target columns**: Remain unchanged +- **Special case**: If all non-target columns are null after the delete, the entire record is removed + +**Example**: +```java +// Partial update mode - only targeting specific columns +UpsertWriter partialWriter = table.newUpsert() + .partialUpdate("id", "count1", "sum1") // Target columns + .createWriter(); + +// Delete will: +// - Set count1 and sum1 to null +// - Keep count2 and sum2 unchanged (non-target columns) +// - Remove entire record only if count2 and sum2 are both null +partialWriter.delete(primaryKeyRow); +``` + +:::note +**Current Limitation**: The aggregation merge engine does not support retraction semantics (e.g., subtracting from a sum, reverting a max). + +- **Full update mode**: Delete operations can only remove the entire record +- **Partial update mode**: Delete operations can only null out target columns, not retract aggregated values + +Future versions may support fine-grained retraction by enhancing the protocol to carry row data with delete operations. +::: + +## Limitations + +:::warning Critical Limitations +When using the `aggregation` merge engine, be aware of the following critical limitations: + +### Exactly-Once Semantics + +When writing to an aggregate merge engine table using the Flink engine, Fluss does provide exactly-once guarantees. Thanks to Flink's checkpointing mechanism, in the event of a failure and recovery, the Flink connector automatically performs an undo operation to roll back the table state to what it was at the last successful checkpoint. This ensures no over-counting or under-counting: data remains consistent and accurate. + +However, when using the Fluss client API directly (outside of Flink), exactly-once is not provided out of the box. In such cases, users must implement their own recovery logic (similar to what the Flink connector does) by explicitly resetting the table state to a previous version by performing undo operations. + +For detailed information about Exactly-Once implementation, please refer to: [FIP-21: Aggregation Merge Engine](https://cwiki.apache.org/confluence/display/FLUSS/FIP-21%3A+Aggregation+Merge+Engine) + +::: + +## See Also + +- [Default Merge Engine](./default.md) +- [FirstRow Merge Engine](./first-row.md) +- [Versioned Merge Engine](./versioned.md) +- [Primary Key Tables](../index.md) +- [Fluss Client API](../../../../apis/java-client.md) diff --git a/website/docs/table-design/table-types/pk-table/merge-engines/index.md b/website/docs/table-design/table-types/pk-table/merge-engines/index.md index bff300ae74..dfb6798853 100644 --- a/website/docs/table-design/table-types/pk-table/merge-engines/index.md +++ b/website/docs/table-design/table-types/pk-table/merge-engines/index.md @@ -14,3 +14,4 @@ The following merge engines are supported: 1. [Default Merge Engine (LastRow)](table-design/table-types/pk-table/merge-engines/default.md) 2. [FirstRow Merge Engine](table-design/table-types/pk-table/merge-engines/first-row.md) 3. [Versioned Merge Engine](table-design/table-types/pk-table/merge-engines/versioned.md) +4. [Aggregation Merge Engine](table-design/table-types/pk-table/merge-engines/aggregation.md)