diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java b/fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java index 578ff2a25a..bee407cf07 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/ArrowLogWriteBatch.java @@ -23,12 +23,14 @@ import org.apache.fluss.memory.MemorySegment; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.record.ChangeType; +import org.apache.fluss.record.LogRecordBatchStatisticsCollector; import org.apache.fluss.record.MemoryLogRecordsArrowBuilder; import org.apache.fluss.record.bytesview.BytesView; import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.arrow.ArrowWriter; import org.apache.fluss.rpc.messages.ProduceLogRequest; +import javax.annotation.Nullable; import javax.annotation.concurrent.NotThreadSafe; import java.io.IOException; @@ -55,11 +57,13 @@ public ArrowLogWriteBatch( int schemaId, ArrowWriter arrowWriter, AbstractPagedOutputView outputView, - long createdMs) { + long createdMs, + @Nullable LogRecordBatchStatisticsCollector statisticsCollector) { super(bucketId, physicalTablePath, createdMs); this.outputView = outputView; this.recordsBuilder = - MemoryLogRecordsArrowBuilder.builder(schemaId, arrowWriter, outputView, true); + MemoryLogRecordsArrowBuilder.builder( + schemaId, arrowWriter, outputView, true, statisticsCollector); } @Override diff --git a/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java b/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java index a1f0544a54..991ea7c8ea 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/write/RecordAccumulator.java @@ -32,6 +32,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metrics.MetricNames; +import org.apache.fluss.record.LogRecordBatchStatisticsCollector; import org.apache.fluss.row.arrow.ArrowWriter; import org.apache.fluss.row.arrow.ArrowWriterPool; import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; @@ -628,13 +629,20 @@ private WriteBatch createWriteBatch( outputView.getPreAllocatedSize(), tableInfo.getRowType(), tableInfo.getTableConfig().getArrowCompressionInfo()); + LogRecordBatchStatisticsCollector statisticsCollector = null; + if (tableInfo.isStatisticsEnabled()) { + statisticsCollector = + new LogRecordBatchStatisticsCollector( + tableInfo.getRowType(), tableInfo.getStatsIndexMapping()); + } return new ArrowLogWriteBatch( bucketId, physicalTablePath, tableInfo.getSchemaId(), arrowWriter, outputView, - clock.milliseconds()); + clock.milliseconds(), + statisticsCollector); case COMPACTED_LOG: return new CompactedLogWriteBatch( diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/ArrowLogWriteBatchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/ArrowLogWriteBatchTest.java index afdef56165..07c4aa3383 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/ArrowLogWriteBatchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/ArrowLogWriteBatchTest.java @@ -136,7 +136,8 @@ void testAppendWithPreAllocatedMemorySegments() throws Exception { DATA1_ROW_TYPE, DEFAULT_COMPRESSION), new PreAllocatedPagedOutputView(memorySegmentList), - System.currentTimeMillis()); + System.currentTimeMillis(), + null); assertThat(arrowLogWriteBatch.pooledMemorySegments()).isEqualTo(memorySegmentList); int count = 0; @@ -210,7 +211,8 @@ void testArrowCompressionRatioEstimated() throws Exception { DATA1_TABLE_INFO.getSchemaId(), arrowWriter, new PreAllocatedPagedOutputView(memorySegmentList), - System.currentTimeMillis()); + System.currentTimeMillis(), + null); int recordCount = 0; while (arrowLogWriteBatch.tryAppend( @@ -310,7 +312,8 @@ private ArrowLogWriteBatch createArrowLogWriteBatch(TableBucket tb, int maxSizeI DATA1_ROW_TYPE, DEFAULT_COMPRESSION), new UnmanagedPagedOutputView(128), - System.currentTimeMillis()); + System.currentTimeMillis(), + null); } private WriteCallback newWriteCallback() { 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 bcd00b1300..8da2246b0a 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 @@ -1579,6 +1579,22 @@ public class ConfigOptions { + "This mode reduces storage and transmission costs but loses the ability to track previous values. " + "This option only affects primary key tables."); + public static final ConfigOption TABLE_STATISTICS_COLUMNS = + key("table.statistics.columns") + .stringType() + .noDefaultValue() + .withDescription( + "Configures column-level statistics collection for the table. " + + "By default this option is not set and no column statistics are collected. " + + "The value '*' means collect statistics for all supported columns. " + + "A comma-separated list of column names means collect statistics only for the specified columns. " + + "Supported types include: BOOLEAN, TINYINT, SMALLINT, INTEGER, BIGINT, FLOAT, DOUBLE, " + + "STRING, CHAR, DECIMAL, DATE, TIME, TIMESTAMP, and TIMESTAMP_LTZ. " + + "Example: 'id,name,timestamp' to collect statistics only for specified columns. " + + "Note: enabling column statistics requires the V1 batch format. " + + "Downstream consumers must be upgraded to Fluss v1.0+ before enabling this option, " + + "as older versions cannot parse the extended batch format."); + // ------------------------------------------------------------------------ // ConfigOptions for Kv // ------------------------------------------------------------------------ diff --git a/fluss-common/src/main/java/org/apache/fluss/config/StatisticsColumnsConfig.java b/fluss-common/src/main/java/org/apache/fluss/config/StatisticsColumnsConfig.java new file mode 100644 index 0000000000..75fb323f75 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/config/StatisticsColumnsConfig.java @@ -0,0 +1,127 @@ +/* + * 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.annotation.PublicEvolving; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Configuration for statistics columns collection with three states: + * + * + * + * @since 1.0 + */ +@PublicEvolving +public class StatisticsColumnsConfig { + + /** The mode of statistics columns collection. */ + public enum Mode { + /** Statistics collection is disabled. */ + DISABLED, + /** Collect statistics for all supported columns. */ + ALL, + /** Collect statistics for specified columns only. */ + SPECIFIED + } + + private static final StatisticsColumnsConfig DISABLED_INSTANCE = + new StatisticsColumnsConfig(Mode.DISABLED, Collections.emptyList()); + + private static final StatisticsColumnsConfig ALL_INSTANCE = + new StatisticsColumnsConfig(Mode.ALL, Collections.emptyList()); + + private final Mode mode; + private final List columns; + + private StatisticsColumnsConfig(Mode mode, List columns) { + this.mode = mode; + this.columns = columns; + } + + /** Creates a disabled statistics columns configuration. */ + public static StatisticsColumnsConfig disabled() { + return DISABLED_INSTANCE; + } + + /** Creates a configuration that collects statistics for all supported columns. */ + public static StatisticsColumnsConfig all() { + return ALL_INSTANCE; + } + + /** Creates a configuration that collects statistics for the specified columns. */ + public static StatisticsColumnsConfig of(List columns) { + checkNotNull(columns, "columns must not be null"); + return new StatisticsColumnsConfig(Mode.SPECIFIED, Collections.unmodifiableList(columns)); + } + + /** Returns the mode of statistics columns collection. */ + public Mode getMode() { + return mode; + } + + /** Returns the specified columns. Only meaningful when mode is {@link Mode#SPECIFIED}. */ + public List getColumns() { + return columns; + } + + /** Returns whether statistics collection is enabled (mode is not DISABLED). */ + public boolean isEnabled() { + return mode != Mode.DISABLED; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StatisticsColumnsConfig that = (StatisticsColumnsConfig) o; + return mode == that.mode && Objects.equals(columns, that.columns); + } + + @Override + public int hashCode() { + return Objects.hash(mode, columns); + } + + @Override + public String toString() { + switch (mode) { + case DISABLED: + return "StatisticsColumnsConfig{DISABLED}"; + case ALL: + return "StatisticsColumnsConfig{ALL}"; + case SPECIFIED: + return "StatisticsColumnsConfig{SPECIFIED: " + columns + "}"; + default: + return "StatisticsColumnsConfig{UNKNOWN}"; + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/config/StatisticsConfigUtils.java b/fluss-common/src/main/java/org/apache/fluss/config/StatisticsConfigUtils.java new file mode 100644 index 0000000000..58ebf0e8cf --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/config/StatisticsConfigUtils.java @@ -0,0 +1,125 @@ +/* + * 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.annotation.Internal; +import org.apache.fluss.exception.InvalidConfigException; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.types.DataField; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeChecks; +import org.apache.fluss.types.RowType; + +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Utility class for validating table statistics configuration. + * + *

This provides simple validation methods that can be called during CREATE TABLE operations to + * ensure statistics configuration is valid and compatible with the table schema. + */ +@Internal +public class StatisticsConfigUtils { + + private StatisticsConfigUtils() {} + + /** + * Validates statistics configuration for a table descriptor. + * + * @param tableDescriptor the table descriptor to validate + * @throws InvalidConfigException if the statistics configuration is invalid + */ + public static void validateStatisticsConfig(TableDescriptor tableDescriptor) { + Map properties = tableDescriptor.getProperties(); + String statisticsColumns = properties.get(ConfigOptions.TABLE_STATISTICS_COLUMNS.key()); + + // Not set means statistics disabled - no validation needed + if (statisticsColumns == null) { + return; + } + + RowType rowType = tableDescriptor.getSchema().getRowType(); + + // Wildcard means all supported columns - no validation needed + if ("*".equals(statisticsColumns.trim())) { + return; + } + + // Parse using TableConfig's logic via StatisticsColumnsConfig + Configuration config = new Configuration(); + config.setString(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), statisticsColumns); + StatisticsColumnsConfig columnsConfig = new TableConfig(config).getStatisticsColumns(); + + if (columnsConfig.getMode() == StatisticsColumnsConfig.Mode.SPECIFIED + && columnsConfig.getColumns().isEmpty()) { + throw new InvalidConfigException( + "Statistics columns configuration cannot be empty. " + + "Use '*' to collect statistics for all supported columns, " + + "or remove the property to disable statistics collection."); + } + + if (columnsConfig.getMode() == StatisticsColumnsConfig.Mode.SPECIFIED) { + validateColumns(rowType, columnsConfig); + } + } + + /** + * Validates that the specified columns exist in the schema and are of supported types. + * + * @param rowType the table schema + * @param columnsConfig the statistics columns configuration + * @throws InvalidConfigException if validation fails + */ + private static void validateColumns(RowType rowType, StatisticsColumnsConfig columnsConfig) { + Map columnTypeMap = buildColumnTypeMap(rowType); + + for (String columnName : columnsConfig.getColumns()) { + // Check if column exists + if (!columnTypeMap.containsKey(columnName)) { + throw new InvalidConfigException( + String.format( + "Column '%s' specified in statistics collection does not exist in table schema", + columnName)); + } + + // Check if column type is supported (whitelist approach) + DataType dataType = columnTypeMap.get(columnName); + if (!DataTypeChecks.isSupportedStatisticsType(dataType)) { + throw new InvalidConfigException( + String.format( + "Column '%s' of type '%s' is not supported for statistics collection. " + + "Supported types are: BOOLEAN, TINYINT, SMALLINT, INTEGER, BIGINT, " + + "FLOAT, DOUBLE, STRING, CHAR, DECIMAL, DATE, TIME, TIMESTAMP, " + + "and TIMESTAMP_LTZ.", + columnName, dataType.asSummaryString())); + } + } + } + + /** + * Builds a map from column name to data type for quick lookup. + * + * @param rowType the table schema + * @return map of column name to data type + */ + private static Map buildColumnTypeMap(RowType rowType) { + return rowType.getFields().stream() + .collect(Collectors.toMap(DataField::getName, DataField::getType)); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java b/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java index 547d0f671b..13cb49e9ed 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java @@ -28,7 +28,10 @@ import org.apache.fluss.utils.AutoPartitionStrategy; import java.time.Duration; +import java.util.Arrays; +import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; /** * Helper class to get table configs (prefixed with "table.*" properties). @@ -154,4 +157,31 @@ public AutoPartitionStrategy getAutoPartitionStrategy() { public long getAutoIncrementCacheSize() { return config.get(ConfigOptions.TABLE_AUTO_INCREMENT_CACHE_SIZE); } + + /** Gets whether statistics collection is enabled for the table. */ + public boolean isStatisticsEnabled() { + return getStatisticsColumns().isEnabled(); + } + + /** + * Gets the statistics columns configuration of the table. + * + * @return a {@link StatisticsColumnsConfig} representing the statistics collection mode: + * DISABLED if not configured, ALL if "*", or SPECIFIED with the list of column names. + */ + public StatisticsColumnsConfig getStatisticsColumns() { + String columnsStr = config.get(ConfigOptions.TABLE_STATISTICS_COLUMNS); + if (columnsStr == null) { + return StatisticsColumnsConfig.disabled(); + } + if ("*".equals(columnsStr)) { + return StatisticsColumnsConfig.all(); + } + List columns = + Arrays.stream(columnsStr.split(",")) + .map(String::trim) + .filter(s -> !s.isEmpty()) + .collect(Collectors.toList()); + return StatisticsColumnsConfig.of(columns); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/memory/MemorySegmentInputView.java b/fluss-common/src/main/java/org/apache/fluss/memory/MemorySegmentInputView.java index 9257ec4797..2086445fdc 100644 --- a/fluss-common/src/main/java/org/apache/fluss/memory/MemorySegmentInputView.java +++ b/fluss-common/src/main/java/org/apache/fluss/memory/MemorySegmentInputView.java @@ -103,6 +103,24 @@ public double readDouble() throws IOException { return Double.longBitsToDouble(readLong()); } + /** + * Skips over the given number of bytes by advancing the position pointer directly, without + * reading the data byte-by-byte. + * + * @param n the number of bytes to skip + * @throws EOFException if there are not enough bytes remaining + */ + public void skipBytes(int n) throws EOFException { + if (n < 0) { + throw new IllegalArgumentException("Number of bytes to skip may not be negative."); + } + if (this.position <= this.end - n) { + this.position += n; + } else { + throw new EOFException(); + } + } + @Override public void readFully(byte[] b) throws IOException { readFully(b, 0, b.length); diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java b/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java index be894ee790..4fa5acfb62 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java @@ -19,11 +19,15 @@ import org.apache.fluss.annotation.PublicEvolving; import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.StatisticsColumnsConfig; import org.apache.fluss.config.TableConfig; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypeChecks; import org.apache.fluss.types.RowType; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -64,6 +68,8 @@ public final class TableInfo { private final long createdTime; private final long modifiedTime; + private int[] cachedStatsIndexMapping = null; + public TableInfo( TablePath tablePath, long tableId, @@ -220,6 +226,94 @@ public boolean isAutoPartitioned() { return isPartitioned() && tableConfig.getAutoPartitionStrategy().isAutoPartitionEnabled(); } + /** + * Check if statistics collection is enabled for this table. + * + *

Statistics collection helps optimize query performance by providing data distribution + * information about the table's columns. When enabled, the system collects and maintains + * statistical information such as min/max values, null counts, and cardinality estimates. + * + * @return true if statistics collection is enabled, false otherwise + */ + public boolean isStatisticsEnabled() { + return tableConfig.isStatisticsEnabled(); + } + + /** + * Returns the column index mapping for statistics collection. + * + *

This method provides a mapping array where each element at index {@code i} represents the + * original column index in the table schema for the {@code i}-th statistics column. This + * mapping is used to efficiently project rows when collecting statistics data. + * + *

The behavior varies based on the statistics configuration: + * + *

    + *
  • Disabled: Returns an empty array when statistics collection is disabled + *
  • Specific columns: When specific columns are configured, returns mapping for + * those columns only + *
  • All columns ("*"): Returns mapping for all supported columns in the table schema + * (unsupported types are excluded as they are not suitable for statistical analysis) + *
+ * + *

The result is cached after the first computation to avoid repeated processing. + * + * @return an array where {@code result[i]} is the original column index for the {@code i}-th + * statistics column. Returns an empty array if statistics collection is disabled. + * @throws IllegalArgumentException if a configured statistics column is not found in the table + * schema + */ + public int[] getStatsIndexMapping() { + if (cachedStatsIndexMapping != null) { + return cachedStatsIndexMapping; + } + + StatisticsColumnsConfig statsConfig = tableConfig.getStatisticsColumns(); + List statsColumns; + + switch (statsConfig.getMode()) { + case DISABLED: + cachedStatsIndexMapping = new int[0]; + return cachedStatsIndexMapping; + case SPECIFIED: + statsColumns = statsConfig.getColumns(); + break; + case ALL: + // Collect all supported columns + statsColumns = new ArrayList<>(); + for (int rowIndex = 0; rowIndex < rowType.getFieldCount(); rowIndex++) { + DataType columnType = rowType.getTypeAt(rowIndex); + if (DataTypeChecks.isSupportedStatisticsType(columnType)) { + String columnName = rowType.getFields().get(rowIndex).getName(); + statsColumns.add(columnName); + } + } + break; + default: + throw new IllegalStateException( + "Unknown statistics columns mode: " + statsConfig.getMode()); + } + + // Build mapping from stats column index to original row column index + int[] mapping = new int[statsColumns.size()]; + for (int statsIndex = 0; statsIndex < statsColumns.size(); statsIndex++) { + String statsColumnName = statsColumns.get(statsIndex); + // Find the original column index for this stats column + int originalIndex = rowType.getFieldIndex(statsColumnName); + if (originalIndex == -1) { + throw new IllegalArgumentException( + String.format( + "Statistics column '%s' not found in table schema", + statsColumnName)); + } + mapping[statsIndex] = originalIndex; + } + + // Cache the result + cachedStatsIndexMapping = mapping; + return mapping; + } + /** * Get the partition keys of the table. This will be an empty set if the table is not * partitioned. diff --git a/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatch.java b/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatch.java index 188a62bbd0..4d9c7fd040 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatch.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatch.java @@ -18,6 +18,7 @@ package org.apache.fluss.record; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.exception.CorruptMessageException; import org.apache.fluss.memory.MemorySegment; import org.apache.fluss.metadata.LogFormat; @@ -33,19 +34,23 @@ import org.apache.fluss.utils.MurmurHashUtils; import org.apache.fluss.utils.crc.Crc32C; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.NoSuchElementException; +import java.util.Optional; import static org.apache.fluss.record.LogRecordBatchFormat.BASE_OFFSET_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.COMMIT_TIMESTAMP_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.LENGTH_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V1; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_OVERHEAD; import static org.apache.fluss.record.LogRecordBatchFormat.MAGIC_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.NO_LEADER_EPOCH; -import static org.apache.fluss.record.LogRecordBatchFormat.arrowChangeTypeOffset; import static org.apache.fluss.record.LogRecordBatchFormat.attributeOffset; import static org.apache.fluss.record.LogRecordBatchFormat.batchSequenceOffset; import static org.apache.fluss.record.LogRecordBatchFormat.crcOffset; @@ -54,6 +59,8 @@ import static org.apache.fluss.record.LogRecordBatchFormat.recordBatchHeaderSize; import static org.apache.fluss.record.LogRecordBatchFormat.recordsCountOffset; import static org.apache.fluss.record.LogRecordBatchFormat.schemaIdOffset; +import static org.apache.fluss.record.LogRecordBatchFormat.statisticsDataOffset; +import static org.apache.fluss.record.LogRecordBatchFormat.statisticsLengthOffset; import static org.apache.fluss.record.LogRecordBatchFormat.writeClientIdOffset; /* This file is based on source code of Apache Kafka Project (https://kafka.apache.org/), licensed by the Apache @@ -69,6 +76,7 @@ *

    *
  • V0 => {@link LogRecordBatchFormat#LOG_MAGIC_VALUE_V0} *
  • V1 => {@link LogRecordBatchFormat#LOG_MAGIC_VALUE_V1} + *
  • V2 => {@link LogRecordBatchFormat#LOG_MAGIC_VALUE_V2} *
* * @since 0.1 @@ -76,16 +84,23 @@ // TODO rename to MemoryLogRecordBatch @PublicEvolving public class DefaultLogRecordBatch implements LogRecordBatch { + private static final Logger LOG = LoggerFactory.getLogger(DefaultLogRecordBatch.class); + public static final byte APPEND_ONLY_FLAG_MASK = 0x01; private MemorySegment segment; private int position; private byte magic; + // Cache for statistics to avoid repeated parsing + private Optional cachedStatistics = null; + public void pointTo(MemorySegment segment, int position) { this.segment = segment; this.position = position; this.magic = segment.get(position + MAGIC_OFFSET); + // Reset cache when pointing to new memory segment + this.cachedStatistics = null; } public void setBaseLogOffset(long baseLogOffset) { @@ -107,7 +122,7 @@ public void setCommitTimestamp(long timestamp) { } public void setLeaderEpoch(int leaderEpoch) { - if (magic >= LOG_MAGIC_VALUE_V1) { + if (magic >= LOG_MAGIC_VALUE_V2) { segment.putInt(position + leaderEpochOffset(magic), leaderEpoch); } else { throw new UnsupportedOperationException( @@ -127,7 +142,7 @@ public int batchSequence() { @Override public int leaderEpoch() { - if (magic >= LOG_MAGIC_VALUE_V1) { + if (magic >= LOG_MAGIC_VALUE_V2) { return segment.getInt(position + leaderEpochOffset(magic)); } else { return NO_LEADER_EPOCH; @@ -211,6 +226,13 @@ public int getRecordCount() { return segment.getInt(position + recordsCountOffset(magic)); } + public int getStatisticsLength() { + if (magic < LOG_MAGIC_VALUE_V1) { + return 0; + } + return segment.getInt(position + statisticsLengthOffset(magic)); + } + @Override public CloseableIterator records(ReadContext context) { if (getRecordCount() == 0) { @@ -265,7 +287,7 @@ private CloseableIterator rowRecordIterator( RowType rowType, @Nullable ProjectedRow outputProjection, long timestamp) { DataType[] fieldTypes = rowType.getChildren().toArray(new DataType[0]); return new LogRecordIterator() { - int position = DefaultLogRecordBatch.this.position + recordBatchHeaderSize(magic); + int position = DefaultLogRecordBatch.this.position + recordsDataOffset(); int rowId = 0; @Override @@ -301,7 +323,7 @@ private CloseableIterator compactedRowRecordIterator( RowType rowType, long timestamp) { DataType[] fieldTypes = rowType.getChildren().toArray(new DataType[0]); return new LogRecordIterator() { - int position = DefaultLogRecordBatch.this.position + recordBatchHeaderSize(magic); + int position = DefaultLogRecordBatch.this.position + recordsDataOffset(); int rowId = 0; @Override @@ -331,12 +353,12 @@ private CloseableIterator columnRecordIterator( BufferAllocator allocator, long timestamp) { boolean isAppendOnly = (attributes() & APPEND_ONLY_FLAG_MASK) > 0; + int recordsDataOffset = recordsDataOffset(); if (isAppendOnly) { // append only batch, no change type vector, // the start of the arrow data is the beginning of the batch records - int recordBatchHeaderSize = recordBatchHeaderSize(magic); - int arrowOffset = position + recordBatchHeaderSize; - int arrowLength = sizeInBytes() - recordBatchHeaderSize; + int arrowOffset = position + recordsDataOffset; + int arrowLength = sizeInBytes() - recordsDataOffset; ArrowReader reader = ArrowUtils.createArrowReader( segment, arrowOffset, arrowLength, root, allocator, rowType); @@ -349,12 +371,11 @@ protected ChangeType getChangeType(int rowId) { } else { // with change type, decode the change type vector first, // the arrow data starts after the change type vector - int changeTypeOffset = position + arrowChangeTypeOffset(magic); + int changeTypeOffset = position + recordsDataOffset; ChangeTypeVector changeTypeVector = new ChangeTypeVector(segment, changeTypeOffset, getRecordCount()); int arrowOffset = changeTypeOffset + changeTypeVector.sizeInBytes(); - int arrowLength = - sizeInBytes() - arrowChangeTypeOffset(magic) - changeTypeVector.sizeInBytes(); + int arrowLength = sizeInBytes() - recordsDataOffset - changeTypeVector.sizeInBytes(); ArrowReader reader = ArrowUtils.createArrowReader( segment, arrowOffset, arrowLength, root, allocator, rowType); @@ -468,4 +489,73 @@ public void remove() { throw new UnsupportedOperationException(); } } + + @Override + public Optional getStatistics(ReadContext context) { + if (context == null) { + return Optional.empty(); + } + + if (cachedStatistics != null) { + return cachedStatistics; + } + + cachedStatistics = parseStatistics(context); + return cachedStatistics; + } + + private Optional parseStatistics(ReadContext context) { + try { + if (magic < LOG_MAGIC_VALUE_V1) { + return Optional.empty(); + } + + int statisticsLength = segment.getInt(position + statisticsLengthOffset(magic)); + if (statisticsLength == 0) { + return Optional.empty(); + } + + RowType rowType = context.getRowType(schemaId()); + if (rowType == null) { + LOG.debug("Skipping statistics parsing: schema {} not found", schemaId()); + return Optional.empty(); + } + + int statsDataOffset = statisticsDataOffset(magic); + LogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics( + segment, position + statsDataOffset, rowType, schemaId()); + return Optional.ofNullable(statistics); + } catch (Exception e) { + LOG.warn("Failed to parse statistics", e); + return Optional.empty(); + } + } + + /** + * Get the offset where records data starts, relative to the batch start. For V1+, records start + * after the fixed header + statistics data. For V0, records start right after the header. + */ + private int recordsDataOffset() { + int headerSize = recordBatchHeaderSize(magic); + if (magic >= LOG_MAGIC_VALUE_V1) { + int statsLength = segment.getInt(position + statisticsLengthOffset(magic)); + return headerSize + statsLength; + } + return headerSize; + } + + // ----------------------------------------------------------------------- + // Methods for testing only + // ----------------------------------------------------------------------- + + @VisibleForTesting + MemorySegment segment() { + return segment; + } + + @VisibleForTesting + int position() { + return position; + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatchStatistics.java b/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatchStatistics.java new file mode 100644 index 0000000000..a883e58974 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/record/DefaultLogRecordBatchStatistics.java @@ -0,0 +1,466 @@ +/* + * 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.record; + +import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalMap; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.row.aligned.AlignedRow; +import org.apache.fluss.types.RowType; + +import java.util.Arrays; + +import static org.apache.fluss.utils.Preconditions.checkArgument; + +/** + * Byte view implementation of LogRecordBatchStatistics that provides zero-copy access to statistics + * data without creating heap objects or copying data. Uses AlignedRow for better performance. + * Supports schema-aware format with partial column statistics. + * + *

Statistics Layout Format

+ * + *

The binary format of statistics data stored in memory segment: + * + *

+ * +------------------+---------+-------------------------------------------------------+
+ * | Field            | Size    | Description                                           |
+ * +------------------+---------+-------------------------------------------------------+
+ * | Version          | 1 byte  | Statistics format version                             |
+ * | Column Count     | 2 bytes | Number of columns with statistics                     |
+ * | Column Indexes   | 2*N     | Field indexes in original schema (2 bytes each)      |
+ * | Null Counts      | 4*N     | Null counts for each stats column (4 bytes each)     |
+ * | Min Values Size  | 4 bytes | Size of min values AlignedRow data                   |
+ * | Min Values Data  | Variable| AlignedRow containing minimum values                  |
+ * | Max Values Size  | 4 bytes | Size of max values AlignedRow data                   |
+ * | Max Values Data  | Variable| AlignedRow containing maximum values                  |
+ * +------------------+---------+-------------------------------------------------------+
+ * 
+ * + *

The statistics support partial column statistics through statsIndexMapping, which maps + * statistics column positions to the original table schema field indexes. This allows efficient + * storage when only a subset of columns have statistics collected. + */ +public class DefaultLogRecordBatchStatistics implements LogRecordBatchStatistics { + + private final MemorySegment segment; + private final int position; + private final int size; + private final RowType rowType; + private final int schemaId; + + private final int[] statsIndexMapping; + + private final Long[] statsNullCounts; + + // Offsets for min/max values in the byte array + private final int minValuesOffset; + private final int maxValuesOffset; + private final int minValuesSize; + private final int maxValuesSize; + + private InternalRow cachedMinRow; + private InternalRow cachedMaxRow; + private Long[] cachedNullCounts; + + private final int[] reversedStatsIndexMapping; + + /** Constructor for schema-aware statistics. */ + public DefaultLogRecordBatchStatistics( + MemorySegment segment, + int position, + int size, + RowType rowType, + int schemaId, + Long[] nullCounts, + int minValuesOffset, + int maxValuesOffset, + int minValuesSize, + int maxValuesSize, + int[] statsIndexMapping) { + this.segment = segment; + this.position = position; + this.size = size; + this.rowType = rowType; + this.schemaId = schemaId; + this.statsNullCounts = nullCounts; + this.minValuesOffset = minValuesOffset; + this.maxValuesOffset = maxValuesOffset; + checkArgument(minValuesSize > 0, "minValuesSize must be positive"); + checkArgument(maxValuesSize > 0, "maxValuesSize must be positive"); + this.minValuesSize = minValuesSize; + this.maxValuesSize = maxValuesSize; + this.statsIndexMapping = statsIndexMapping; + this.reversedStatsIndexMapping = new int[rowType.getFieldCount()]; + Arrays.fill(this.reversedStatsIndexMapping, -1); + for (int statsIndex = 0; statsIndex < statsIndexMapping.length; statsIndex++) { + this.reversedStatsIndexMapping[statsIndexMapping[statsIndex]] = statsIndex; + } + } + + @Override + public InternalRow getMinValues() { + // Return cached row if already created + if (cachedMinRow != null) { + return cachedMinRow; + } + + AlignedRow minRow = new AlignedRow(statsIndexMapping.length); + minRow.pointTo(segment, position + minValuesOffset, minValuesSize); + + this.cachedMinRow = new FullRowWrapper(minRow, reversedStatsIndexMapping); + return this.cachedMinRow; + } + + @Override + public InternalRow getMaxValues() { + // Return cached row if already created + if (cachedMaxRow != null) { + return cachedMaxRow; + } + + AlignedRow maxRow = new AlignedRow(statsIndexMapping.length); + maxRow.pointTo(segment, position + maxValuesOffset, maxValuesSize); + + this.cachedMaxRow = new FullRowWrapper(maxRow, reversedStatsIndexMapping); + return this.cachedMaxRow; + } + + @Override + public Long[] getNullCounts() { + if (cachedNullCounts != null) { + return cachedNullCounts; + } + cachedNullCounts = new Long[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); i++) { + if (this.reversedStatsIndexMapping[i] >= 0) { + cachedNullCounts[i] = statsNullCounts[reversedStatsIndexMapping[i]]; + } else { + cachedNullCounts[i] = -1L; + } + } + return cachedNullCounts; + } + + @Override + public boolean hasColumnStatistics(int fieldIndex) { + return reversedStatsIndexMapping[fieldIndex] != -1; + } + + /** + * Get the underlying memory segment. + * + * @return The memory segment + */ + public MemorySegment getSegment() { + return segment; + } + + /** + * Get the position in the memory segment. + * + * @return The position + */ + public int getPosition() { + return position; + } + + /** + * Get the total size of the statistics data. + * + * @return The size in bytes + */ + public int getSize() { + return size; + } + + /** + * Check if minimum values are available. + * + * @return true if minimum values are available + */ + public boolean hasMinValues() { + return minValuesSize > 0; + } + + /** + * Check if maximum values are available. + * + * @return true if maximum values are available + */ + public boolean hasMaxValues() { + return maxValuesSize > 0; + } + + /** + * Get the row type for this statistics. + * + * @return The row type + */ + public RowType getRowType() { + return rowType; + } + + @Override + public int getSchemaId() { + return schemaId; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("DefaultLogRecordBatchStatistics{"); + sb.append(", hasMinValues=").append(hasMinValues()); + sb.append(", hasMaxValues=").append(hasMaxValues()); + sb.append(", size=").append(size); + if (statsIndexMapping != null) { + sb.append(", statisticsColumns=").append(Arrays.toString(statsIndexMapping)); + } + sb.append('}'); + return sb.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + DefaultLogRecordBatchStatistics that = (DefaultLogRecordBatchStatistics) o; + + if (position != that.position) { + return false; + } + if (size != that.size) { + return false; + } + if (minValuesOffset != that.minValuesOffset) { + return false; + } + if (maxValuesOffset != that.maxValuesOffset) { + return false; + } + if (minValuesSize != that.minValuesSize) { + return false; + } + if (maxValuesSize != that.maxValuesSize) { + return false; + } + if (!Arrays.equals(statsIndexMapping, that.statsIndexMapping)) { + return false; + } + if (!Arrays.equals(statsNullCounts, that.statsNullCounts)) { + return false; + } + if (!rowType.equals(that.rowType)) { + return false; + } + return segment.equals(that.segment); + } + + @Override + public int hashCode() { + int result = segment.hashCode(); + result = 31 * result + position; + result = 31 * result + size; + result = 31 * result + rowType.hashCode(); + result = 31 * result + Arrays.hashCode(statsIndexMapping); + result = 31 * result + Arrays.hashCode(statsNullCounts); + result = 31 * result + minValuesOffset; + result = 31 * result + maxValuesOffset; + result = 31 * result + minValuesSize; + result = 31 * result + maxValuesSize; + return result; + } + + /** + * A wrapper that provides a full row view of projected statistics data. + * + *

This class solves the problem of accessing statistics data that only contains a subset of + * columns from the original table schema. The underlying statistics storage is optimized to + * only store columns that have statistics collected (projected view), but external consumers + * expect to access statistics using the original table's field indexes (full row view). + * + *

Background: Statistics are often collected for only a subset of columns to save + * storage space. For example, if a table has columns [id, name, age, salary, department] but + * statistics are only collected for [name, age, salary], the underlying storage contains only 3 + * columns. However, external code may want to access statistics for "age" using its original + * field index (2 in the full schema), not its position in the statistics row (1 in the + * projected schema). + * + *

Mapping Arrays: + * + *

    + *
  • {@code statsIndexMapping}: Maps statistics column positions to original schema field + * indexes. For the example above: [1, 2, 3] (name=1, age=2, salary=3 in original schema) + *
  • {@code reversedStatsIndexMapping}: Reverse mapping from original field indexes to + * statistics positions. For the example: [-1, 0, 1, 2, -1] where -1 means no statistics + * available + *
+ * + *

Usage Pattern: + * + *

{@code
+     * // External code can access using original field indexes
+     * statistics.getMinValues().getInt(2);  // Gets age minimum (field 2 in original schema)
+     * // Internally maps to position 1 in the underlying statistics row
+     * }
+ * + *

Error Handling: Accessing a field that doesn't have statistics collected will throw + * an {@link IllegalArgumentException} with a descriptive message. + */ + private static class FullRowWrapper implements InternalRow { + + private final InternalRow internalRow; + + private final int[] reversedStatsIndexMapping; + + FullRowWrapper(InternalRow internalRow, int[] reversedStatsIndexMapping) { + this.internalRow = internalRow; + this.reversedStatsIndexMapping = reversedStatsIndexMapping; + } + + private void ensureColumnExists(int pos) { + checkArgument( + pos >= 0 && pos < reversedStatsIndexMapping.length, + "Column index out of range."); + checkArgument( + this.reversedStatsIndexMapping[pos] >= 0, + "Column index not available in underlying row data."); + } + + @Override + public int getFieldCount() { + return reversedStatsIndexMapping.length; + } + + @Override + public boolean isNullAt(int pos) { + ensureColumnExists(pos); + return internalRow.isNullAt(reversedStatsIndexMapping[pos]); + } + + @Override + public boolean getBoolean(int pos) { + ensureColumnExists(pos); + return internalRow.getBoolean(reversedStatsIndexMapping[pos]); + } + + @Override + public byte getByte(int pos) { + ensureColumnExists(pos); + return internalRow.getByte(reversedStatsIndexMapping[pos]); + } + + @Override + public short getShort(int pos) { + ensureColumnExists(pos); + return internalRow.getShort(reversedStatsIndexMapping[pos]); + } + + @Override + public int getInt(int pos) { + ensureColumnExists(pos); + return internalRow.getInt(reversedStatsIndexMapping[pos]); + } + + @Override + public long getLong(int pos) { + ensureColumnExists(pos); + return internalRow.getLong(reversedStatsIndexMapping[pos]); + } + + @Override + public float getFloat(int pos) { + ensureColumnExists(pos); + return internalRow.getFloat(reversedStatsIndexMapping[pos]); + } + + @Override + public double getDouble(int pos) { + ensureColumnExists(pos); + return internalRow.getDouble(reversedStatsIndexMapping[pos]); + } + + @Override + public BinaryString getChar(int pos, int length) { + ensureColumnExists(pos); + return internalRow.getChar(reversedStatsIndexMapping[pos], length); + } + + @Override + public BinaryString getString(int pos) { + ensureColumnExists(pos); + return internalRow.getString(reversedStatsIndexMapping[pos]); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + ensureColumnExists(pos); + return internalRow.getDecimal(reversedStatsIndexMapping[pos], precision, scale); + } + + @Override + public TimestampNtz getTimestampNtz(int pos, int precision) { + ensureColumnExists(pos); + return internalRow.getTimestampNtz(reversedStatsIndexMapping[pos], precision); + } + + @Override + public TimestampLtz getTimestampLtz(int pos, int precision) { + ensureColumnExists(pos); + return internalRow.getTimestampLtz(reversedStatsIndexMapping[pos], precision); + } + + @Override + public byte[] getBinary(int pos, int length) { + ensureColumnExists(pos); + return internalRow.getBinary(reversedStatsIndexMapping[pos], length); + } + + @Override + public byte[] getBytes(int pos) { + ensureColumnExists(pos); + return internalRow.getBytes(reversedStatsIndexMapping[pos]); + } + + @Override + public InternalArray getArray(int pos) { + ensureColumnExists(pos); + return internalRow.getArray(reversedStatsIndexMapping[pos]); + } + + @Override + public InternalMap getMap(int pos) { + ensureColumnExists(pos); + return internalRow.getMap(reversedStatsIndexMapping[pos]); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + ensureColumnExists(pos); + return internalRow.getRow(reversedStatsIndexMapping[pos], numFields); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/record/FileLogInputStream.java b/fluss-common/src/main/java/org/apache/fluss/record/FileLogInputStream.java index 9be7e459ad..fca1a2c49c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/FileLogInputStream.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/FileLogInputStream.java @@ -19,14 +19,19 @@ import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; import org.apache.fluss.utils.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.FileChannel; import java.util.Objects; +import java.util.Optional; import static org.apache.fluss.record.LogRecordBatchFormat.BASE_OFFSET_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.HEADER_SIZE_UP_TO_MAGIC; @@ -42,6 +47,8 @@ /** A log input stream which is backed by a {@link FileChannel}. */ public class FileLogInputStream implements LogInputStream { + private static final Logger LOG = LoggerFactory.getLogger(FileLogInputStream.class); + private int position; private final int end; private final FileLogRecords fileRecords; @@ -93,8 +100,11 @@ public static class FileChannelLogRecordBatch implements LogRecordBatch { protected final int position; protected final int batchSize; - private LogRecordBatch fullBatch; - private LogRecordBatch batchHeader; + private DefaultLogRecordBatch fullBatch; + private DefaultLogRecordBatch batchHeader; + + // Cache for statistics to avoid repeated parsing + private Optional cachedStatistics = null; FileChannelLogRecordBatch( long offset, byte magic, FileLogRecords fileRecords, int position, int batchSize) { @@ -184,7 +194,7 @@ public int sizeInBytes() { return LOG_OVERHEAD + batchSize; } - private LogRecordBatch toMemoryRecordBatch(ByteBuffer buffer) { + private DefaultLogRecordBatch toMemoryRecordBatch(ByteBuffer buffer) { DefaultLogRecordBatch records = new DefaultLogRecordBatch(); records.pointTo(MemorySegment.wrap(buffer.array()), 0); return records; @@ -202,7 +212,7 @@ protected LogRecordBatch loadFullBatch() { return fullBatch; } - protected LogRecordBatch loadBatchHeader() { + protected DefaultLogRecordBatch loadBatchHeader() { if (fullBatch != null) { return fullBatch; } @@ -228,8 +238,9 @@ protected ByteBuffer loadByteBufferWithSize(int size, int position, String descr } } - private LogRecordBatch loadBatchWithSize(int size, String description) { - return toMemoryRecordBatch(loadByteBufferWithSize(size, position, description)); + private DefaultLogRecordBatch loadBatchWithSize(int size, String description) { + ByteBuffer buffer = loadByteBufferWithSize(size, position, description); + return toMemoryRecordBatch(buffer); } @Override @@ -273,5 +284,51 @@ public String toString() { + batchSize + ")"; } + + @Override + public Optional getStatistics(ReadContext context) { + if (context == null) { + return Optional.empty(); + } + + if (cachedStatistics != null) { + return cachedStatistics; + } + + cachedStatistics = parseStatistics(context); + return cachedStatistics; + } + + private Optional parseStatistics(ReadContext context) { + if (magic < LogRecordBatchFormat.LOG_MAGIC_VALUE_V1) { + return Optional.empty(); + } + + try { + RowType rowType = context.getRowType(schemaId()); + if (rowType == null) { + return Optional.empty(); + } + + DefaultLogRecordBatch header = loadBatchHeader(); + int statisticsLength = header.getStatisticsLength(); + if (statisticsLength == 0) { + return Optional.empty(); + } + + int statsDataOffset = LogRecordBatchFormat.statisticsDataOffset(magic); + ByteBuffer statisticsData = + loadByteBufferWithSize( + statisticsLength, position + statsDataOffset, "statistics"); + + LogRecordBatchStatistics parsedStatistics = + LogRecordBatchStatisticsParser.parseStatistics( + statisticsData.array(), rowType, schemaId()); + return Optional.ofNullable(parsedStatistics); + } catch (Exception e) { + LOG.error("Failed to load statistics for record batch at position {}", position, e); + return Optional.empty(); + } + } } } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java b/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java index 7a4cfe55d4..1ba9592157 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/FileLogProjection.java @@ -57,15 +57,17 @@ import static org.apache.fluss.record.LogRecordBatchFormat.LENGTH_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V0; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V1; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_OVERHEAD; import static org.apache.fluss.record.LogRecordBatchFormat.MAGIC_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.V0_RECORD_BATCH_HEADER_SIZE; import static org.apache.fluss.record.LogRecordBatchFormat.V1_RECORD_BATCH_HEADER_SIZE; -import static org.apache.fluss.record.LogRecordBatchFormat.arrowChangeTypeOffset; +import static org.apache.fluss.record.LogRecordBatchFormat.V2_RECORD_BATCH_HEADER_SIZE; import static org.apache.fluss.record.LogRecordBatchFormat.attributeOffset; import static org.apache.fluss.record.LogRecordBatchFormat.recordBatchHeaderSize; import static org.apache.fluss.record.LogRecordBatchFormat.recordsCountOffset; import static org.apache.fluss.record.LogRecordBatchFormat.schemaIdOffset; +import static org.apache.fluss.record.LogRecordBatchFormat.statisticsLengthOffset; import static org.apache.fluss.utils.FileUtils.readFully; import static org.apache.fluss.utils.FileUtils.readFullyOrFail; import static org.apache.fluss.utils.Preconditions.checkState; @@ -92,7 +94,7 @@ public class FileLogProjection { * Buffer to read log records batch header. V1 is larger than V0, so use V1 head buffer can read * V0 header even if there is no enough bytes in log file. */ - private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(V1_RECORD_BATCH_HEADER_SIZE); + private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(V2_RECORD_BATCH_HEADER_SIZE); private final ByteBuffer arrowHeaderBuffer = ByteBuffer.allocate(ARROW_HEADER_SIZE); private ByteBuffer arrowMetadataBuffer; @@ -179,14 +181,21 @@ public BytesViewLogRecords project(FileChannel channel, int start, int end, int boolean isAppendOnly = (logHeaderBuffer.get(attributeOffset(magic)) & APPEND_ONLY_FLAG_MASK) > 0; + // For V1+, skip statistics data between header and records + int statisticsLength = 0; + if (magic >= LogRecordBatchFormat.LOG_MAGIC_VALUE_V1) { + statisticsLength = logHeaderBuffer.getInt(statisticsLengthOffset(magic)); + } + int recordsStartOffset = recordBatchHeaderSize + statisticsLength; + final int changeTypeBytes; final long arrowHeaderOffset; if (isAppendOnly) { changeTypeBytes = 0; - arrowHeaderOffset = position + recordBatchHeaderSize; + arrowHeaderOffset = position + recordsStartOffset; } else { changeTypeBytes = logHeaderBuffer.getInt(recordsCountOffset(magic)); - arrowHeaderOffset = position + recordBatchHeaderSize + changeTypeBytes; + arrowHeaderOffset = position + recordsStartOffset + changeTypeBytes; } // read arrow header @@ -217,7 +226,7 @@ public BytesViewLogRecords project(FileChannel channel, int start, int end, int recordBatchHeaderSize + changeTypeBytes + currentProjection.arrowMetadataLength - + (int) arrowBodyLength; // safe to cast to int + + (int) arrowBodyLength; if (newBatchSizeInBytes > maxBytes) { // the remaining bytes in the file are not enough to read a full batch return new BytesViewLogRecords(builder.build()); @@ -236,6 +245,10 @@ public BytesViewLogRecords project(FileChannel channel, int start, int end, int // 4. update and copy log batch header logHeaderBuffer.position(LENGTH_OFFSET); logHeaderBuffer.putInt(newBatchSizeInBytes - LOG_OVERHEAD); + + // For V1+ format, clear statistics information since projection removes statistics + LogRecordBatchFormat.clearStatisticsFromHeader(logHeaderBuffer, magic); + logHeaderBuffer.rewind(); // the logHeader can't be reused, as it will be sent to network byte[] logHeader = new byte[recordBatchHeaderSize]; @@ -244,7 +257,7 @@ public BytesViewLogRecords project(FileChannel channel, int start, int end, int // 5. build log records builder.addBytes(logHeader); if (!isAppendOnly) { - builder.addBytes(channel, position + arrowChangeTypeOffset(magic), changeTypeBytes); + builder.addBytes(channel, position + recordsStartOffset, changeTypeBytes); } builder.addBytes(headerMetadata); final long bufferOffset = arrowHeaderOffset + ARROW_HEADER_SIZE + arrowMetadataSize; @@ -370,7 +383,7 @@ private static BitSet fillBitSet(int length, boolean value) { /** * Read log header fully or fail with EOFException if there is no enough bytes to read a full - * log header. This handles different log header size for magic v0 and v1. + * log header. This handles different log header size for magic v0, v1 and v2. */ static void readLogHeaderFullyOrFail(FileChannel channel, ByteBuffer buffer, int position) throws IOException { @@ -394,6 +407,12 @@ static void readLogHeaderFullyOrFail(FileChannel channel, ByteBuffer buffer, int "Failed to read v1 log header from file channel `%s`. Expected to read %d bytes, " + "but reached end of file after reading %d bytes. Started read from position %d.", channel, V1_RECORD_BATCH_HEADER_SIZE, size, position)); + } else if (magic == LOG_MAGIC_VALUE_V2 && size < V2_RECORD_BATCH_HEADER_SIZE) { + throw new EOFException( + String.format( + "Failed to read v2 log header from file channel `%s`. Expected to read %d bytes, " + + "but reached end of file after reading %d bytes. Started read from position %d.", + channel, V2_RECORD_BATCH_HEADER_SIZE, size, position)); } } } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatch.java b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatch.java index 06536e5821..d1cec1c9e4 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatch.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatch.java @@ -28,6 +28,7 @@ import javax.annotation.Nullable; import java.util.Iterator; +import java.util.Optional; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V0; import static org.apache.fluss.record.LogRecordBatchFormat.NO_WRITER_ID; @@ -40,12 +41,8 @@ @PublicEvolving public interface LogRecordBatch { /** - * The current "magic" value. Even though we already support LOG_MAGIC_VALUE_V1, for - * compatibility reasons — specifically, a higher-version Fluss Client (which supports - * LOG_MAGIC_VALUE_V1) cannot write to a lower-version Fluss Server (which only supports - * LOG_MAGIC_VALUE_V0) — we are unable to guarantee compatibility at this time. Therefore, we - * will keep the current log magic value set to LOG_MAGIC_VALUE_V0 for now, and only upgrade it - * to LOG_MAGIC_VALUE_V1 once the compatibility issue is resolved. + * The base "magic" value used when no statistics are needed. When statistics collection is + * enabled, V1 should be used instead. */ byte CURRENT_LOG_MAGIC_VALUE = LOG_MAGIC_VALUE_V0; @@ -56,6 +53,17 @@ public interface LogRecordBatch { */ boolean isValid(); + /** + * Get the statistics of this record batch using the provided read context. + * + *

This method can deserialize statistics when the read context provides the necessary schema + * information. + * + * @param context The read context that provides schema information + * @return Optional containing the statistics if available and valid + */ + Optional getStatistics(ReadContext context); + /** Raise an exception if the checksum is not valid. */ void ensureValid(); diff --git a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchFormat.java b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchFormat.java index d0f42c052c..9ba630b35a 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchFormat.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchFormat.java @@ -19,6 +19,8 @@ import org.apache.fluss.exception.OutOfOrderSequenceException; +import java.nio.ByteBuffer; + /** The format of Fluss how to organize and storage a {@link LogRecordBatch}. */ public class LogRecordBatchFormat { @@ -33,7 +35,7 @@ public class LogRecordBatchFormat { /** * Used to indicate an unknown leaderEpoch, which will be the case when the record set is first - * created by the writer or the magic lower than V1. + * created by the writer or the magic lower than V2. */ public static final int NO_LEADER_EPOCH = -1; @@ -57,33 +59,43 @@ public class LogRecordBatchFormat { public static final int LOG_OVERHEAD = LENGTH_OFFSET + LENGTH_LENGTH; public static final int HEADER_SIZE_UP_TO_MAGIC = MAGIC_OFFSET + MAGIC_LENGTH; + // Statistics format version + public static final byte STATISTICS_VERSION = 1; + // ---------------------------------------------------------------------------------------- - // Format of Magic Version: V1 + // Format of Magic Version: V2 // ---------------------------------------------------------------------------------------- /** - * LogRecordBatch implementation for magic 1 (V1). The schema of {@link LogRecordBatch} is given + * LogRecordBatch implementation for magic 2 (V2). The schema of {@link LogRecordBatch} is given * below: * - *

    - * RecordBatch => - *
  • BaseOffset => Int64 - *
  • Length => Int32 - *
  • Magic => Int8 - *
  • CommitTimestamp => Int64 - *
  • LeaderEpoch => Int32 - *
  • CRC => Uint32 - *
  • SchemaId => Int16 - *
  • Attributes => Int8 - *
  • LastOffsetDelta => Int32 - *
  • WriterID => Int64 - *
  • SequenceID => Int32 - *
  • RecordCount => Int32 - *
  • Records => [Record] - *
+ *
+     * +----------------+------------------+------------------+
+     * | BaseOffset     | Length           | Magic            |
+     * | (8 bytes)      | (4 bytes)        | (1 byte)         |
+     * +----------------+------------------+------------------+
+     * | CommitTimestamp                   | LeaderEpoch      |
+     * | (8 bytes)                        | (4 bytes)        |
+     * +----------------------------------+------------------+
+     * | CRC            | SchemaId         | Attributes       |
+     * | (4 bytes)      | (2 bytes)        | (1 byte)         |
+     * +----------------+------------------+------------------+
+     * | LastOffsetDelta                   | WriterID         |
+     * | (4 bytes)                        | (8 bytes)        |
+     * +----------------------------------+------------------+
+     * | BatchSequence  | RecordCount      | StatisticsLength |
+     * | (4 bytes)      | (4 bytes)        | (4 bytes)        |
+     * +----------------+------------------+------------------+
+     * | Statistics Data (optional)                           |
+     * | (variable length, only if StatisticsLength > 0)  |
+     * +------------------------------------------------------+
+     * | Records Data (variable length)                       |
+     * +------------------------------------------------------+
+     * 
* - *

Newly added field in LogRecordBatch header of magic V1 is LeaderEpoch, which used to build - * a consistent leaderEpoch cache across different tabletServers. + *

V2 extends V1 by adding the LeaderEpoch field, which is used to build a consistent + * leaderEpoch cache across different tabletServers. * *

The CRC covers the data from the schemaId to the end of the batch (i.e. all the bytes that * follow the CRC). It is located after the magic byte, which means that clients must parse the @@ -108,13 +120,86 @@ public class LogRecordBatchFormat { * ------------------------------------------ * * - * @since 0.7 + * @since 1.0 + */ + public static final byte LOG_MAGIC_VALUE_V2 = 2; + + private static final int STATISTICS_LENGTH_LENGTH = 4; + + private static final int V2_COMMIT_TIMESTAMP_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH; + private static final int V2_LEADER_EPOCH_OFFSET = + V2_COMMIT_TIMESTAMP_OFFSET + COMMIT_TIMESTAMP_LENGTH; + private static final int V2_CRC_OFFSET = V2_LEADER_EPOCH_OFFSET + LEADER_EPOCH_LENGTH; + private static final int V2_SCHEMA_ID_OFFSET = V2_CRC_OFFSET + CRC_LENGTH; + private static final int V2_ATTRIBUTES_OFFSET = V2_SCHEMA_ID_OFFSET + SCHEMA_ID_LENGTH; + private static final int V2_LAST_OFFSET_DELTA_OFFSET = V2_ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH; + private static final int V2_WRITE_CLIENT_ID_OFFSET = + V2_LAST_OFFSET_DELTA_OFFSET + LAST_OFFSET_DELTA_LENGTH; + private static final int V2_BATCH_SEQUENCE_OFFSET = + V2_WRITE_CLIENT_ID_OFFSET + WRITE_CLIENT_ID_LENGTH; + private static final int V2_RECORDS_COUNT_OFFSET = + V2_BATCH_SEQUENCE_OFFSET + BATCH_SEQUENCE_LENGTH; + private static final int V2_STATISTICS_LENGTH_OFFSET = + V2_RECORDS_COUNT_OFFSET + RECORDS_COUNT_LENGTH; + private static final int V2_STATISTICS_DATA_OFFSET = + V2_STATISTICS_LENGTH_OFFSET + STATISTICS_LENGTH_LENGTH; + + // V2 record batch header size (fixed part, without statistics data) + public static final int V2_RECORD_BATCH_HEADER_SIZE = V2_STATISTICS_DATA_OFFSET; + + // ---------------------------------------------------------------------------------------- + // Format of Magic Version: V1 + // ---------------------------------------------------------------------------------------- + + /** + * LogRecordBatch implementation for magic 1 (V1). The schema of {@link LogRecordBatch} is given + * below: + * + *

+     * +----------------+------------------+------------------+
+     * | BaseOffset     | Length           | Magic            |
+     * | (8 bytes)      | (4 bytes)        | (1 byte)         |
+     * +----------------+------------------+------------------+
+     * | CommitTimestamp                   | CRC              |
+     * | (8 bytes)                        | (4 bytes)        |
+     * +----------------------------------+------------------+
+     * | SchemaId       | Attributes       | LastOffsetDelta  |
+     * | (2 bytes)      | (1 byte)         | (4 bytes)        |
+     * +----------------+------------------+------------------+
+     * | WriterID                          | BatchSequence    |
+     * | (8 bytes)                        | (4 bytes)        |
+     * +----------------------------------+------------------+
+     * | RecordCount    | StatisticsLength |                  |
+     * | (4 bytes)      | (4 bytes)        |                  |
+     * +----------------+------------------+------------------+
+     * | Statistics Data (optional)                           |
+     * | (variable length, only if StatisticsLength > 0)  |
+     * +------------------------------------------------------+
+     * | Records Data (variable length)                       |
+     * +------------------------------------------------------+
+     * 
+ * + *

V1 introduces statistics support for filter pushdown optimization. The statistics include: + * - Row count (already available in RecordCount) - Min values for each column - Max values for + * each column - Null counts for each column + * + *

The StatisticsLength field indicates the length of the statistics data. If + * StatisticsLength is 0, no statistics are present. The Statistics data is placed between the + * header and the Records section. + * + *

The current attributes are given below: + * + *

+     * ------------------------------------------
+     * |  Unused (1-7)   |  AppendOnly Flag (0) |
+     * ------------------------------------------
+     * 
+ * + * @since 1.0 */ public static final byte LOG_MAGIC_VALUE_V1 = 1; - private static final int V1_LEADER_EPOCH_OFFSET = - COMMIT_TIMESTAMP_OFFSET + COMMIT_TIMESTAMP_LENGTH; - private static final int V1_CRC_OFFSET = V1_LEADER_EPOCH_OFFSET + LEADER_EPOCH_LENGTH; + private static final int V1_CRC_OFFSET = COMMIT_TIMESTAMP_OFFSET + COMMIT_TIMESTAMP_LENGTH; private static final int V1_SCHEMA_ID_OFFSET = V1_CRC_OFFSET + CRC_LENGTH; private static final int V1_ATTRIBUTES_OFFSET = V1_SCHEMA_ID_OFFSET + SCHEMA_ID_LENGTH; private static final int V1_LAST_OFFSET_DELTA_OFFSET = V1_ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH; @@ -124,10 +209,13 @@ public class LogRecordBatchFormat { V1_WRITE_CLIENT_ID_OFFSET + WRITE_CLIENT_ID_LENGTH; private static final int V1_RECORDS_COUNT_OFFSET = V1_BATCH_SEQUENCE_OFFSET + BATCH_SEQUENCE_LENGTH; - private static final int V1_RECORDS_OFFSET = V1_RECORDS_COUNT_OFFSET + RECORDS_COUNT_LENGTH; + private static final int V1_STATISTICS_LENGTH_OFFSET = + V1_RECORDS_COUNT_OFFSET + RECORDS_COUNT_LENGTH; + private static final int V1_STATISTICS_DATA_OFFSET = + V1_STATISTICS_LENGTH_OFFSET + STATISTICS_LENGTH_LENGTH; - public static final int V1_RECORD_BATCH_HEADER_SIZE = V1_RECORDS_OFFSET; - private static final int V1_ARROW_CHANGETYPE_OFFSET = V1_RECORD_BATCH_HEADER_SIZE; + // V1 record batch header size (fixed part, without statistics data) + public static final int V1_RECORD_BATCH_HEADER_SIZE = V1_STATISTICS_DATA_OFFSET; // ---------------------------------------------------------------------------------------- // Format of Magic Version: V0 @@ -178,25 +266,31 @@ public class LogRecordBatchFormat { private static final int V0_RECORDS_OFFSET = V0_RECORDS_COUNT_OFFSET + RECORDS_COUNT_LENGTH; public static final int V0_RECORD_BATCH_HEADER_SIZE = V0_RECORDS_OFFSET; - private static final int V0_ARROW_CHANGETYPE_OFFSET = V0_RECORD_BATCH_HEADER_SIZE; // ---------------------------------------------------------------------------------------- // Static Methods // ---------------------------------------------------------------------------------------- public static int leaderEpochOffset(byte magic) { - if (magic == LOG_MAGIC_VALUE_V1) { - return V1_LEADER_EPOCH_OFFSET; + switch (magic) { + case LOG_MAGIC_VALUE_V0: + case LOG_MAGIC_VALUE_V1: + throw new UnsupportedOperationException("Leader epoch is not supported in V0/V1"); + case LOG_MAGIC_VALUE_V2: + return V2_LEADER_EPOCH_OFFSET; + default: + throw new IllegalArgumentException("Unsupported magic value " + magic); } - throw new IllegalArgumentException("Unsupported magic value " + magic); } public static int crcOffset(byte magic) { switch (magic) { - case LOG_MAGIC_VALUE_V1: - return V1_CRC_OFFSET; case LOG_MAGIC_VALUE_V0: return V0_CRC_OFFSET; + case LOG_MAGIC_VALUE_V1: + return V1_CRC_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_CRC_OFFSET; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } @@ -204,10 +298,12 @@ public static int crcOffset(byte magic) { public static int schemaIdOffset(byte magic) { switch (magic) { - case LOG_MAGIC_VALUE_V1: - return V1_SCHEMA_ID_OFFSET; case LOG_MAGIC_VALUE_V0: return V0_SCHEMA_ID_OFFSET; + case LOG_MAGIC_VALUE_V1: + return V1_SCHEMA_ID_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_SCHEMA_ID_OFFSET; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } @@ -215,10 +311,12 @@ public static int schemaIdOffset(byte magic) { public static int attributeOffset(byte magic) { switch (magic) { - case LOG_MAGIC_VALUE_V1: - return V1_ATTRIBUTES_OFFSET; case LOG_MAGIC_VALUE_V0: return V0_ATTRIBUTES_OFFSET; + case LOG_MAGIC_VALUE_V1: + return V1_ATTRIBUTES_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_ATTRIBUTES_OFFSET; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } @@ -226,10 +324,12 @@ public static int attributeOffset(byte magic) { public static int lastOffsetDeltaOffset(byte magic) { switch (magic) { - case LOG_MAGIC_VALUE_V1: - return V1_LAST_OFFSET_DELTA_OFFSET; case LOG_MAGIC_VALUE_V0: return V0_LAST_OFFSET_DELTA_OFFSET; + case LOG_MAGIC_VALUE_V1: + return V1_LAST_OFFSET_DELTA_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_LAST_OFFSET_DELTA_OFFSET; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } @@ -237,10 +337,12 @@ public static int lastOffsetDeltaOffset(byte magic) { public static int writeClientIdOffset(byte magic) { switch (magic) { - case LOG_MAGIC_VALUE_V1: - return V1_WRITE_CLIENT_ID_OFFSET; case LOG_MAGIC_VALUE_V0: return V0_WRITE_CLIENT_ID_OFFSET; + case LOG_MAGIC_VALUE_V1: + return V1_WRITE_CLIENT_ID_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_WRITE_CLIENT_ID_OFFSET; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } @@ -248,10 +350,12 @@ public static int writeClientIdOffset(byte magic) { public static int batchSequenceOffset(byte magic) { switch (magic) { - case LOG_MAGIC_VALUE_V1: - return V1_BATCH_SEQUENCE_OFFSET; case LOG_MAGIC_VALUE_V0: return V0_BATCH_SEQUENCE_OFFSET; + case LOG_MAGIC_VALUE_V1: + return V1_BATCH_SEQUENCE_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_BATCH_SEQUENCE_OFFSET; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } @@ -259,10 +363,12 @@ public static int batchSequenceOffset(byte magic) { public static int recordsCountOffset(byte magic) { switch (magic) { - case LOG_MAGIC_VALUE_V1: - return V1_RECORDS_COUNT_OFFSET; case LOG_MAGIC_VALUE_V0: return V0_RECORDS_COUNT_OFFSET; + case LOG_MAGIC_VALUE_V1: + return V1_RECORDS_COUNT_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_RECORDS_COUNT_OFFSET; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } @@ -270,23 +376,70 @@ public static int recordsCountOffset(byte magic) { public static int recordBatchHeaderSize(byte magic) { switch (magic) { - case LOG_MAGIC_VALUE_V1: - return V1_RECORD_BATCH_HEADER_SIZE; case LOG_MAGIC_VALUE_V0: return V0_RECORD_BATCH_HEADER_SIZE; + case LOG_MAGIC_VALUE_V1: + return V1_RECORD_BATCH_HEADER_SIZE; + case LOG_MAGIC_VALUE_V2: + return V2_RECORD_BATCH_HEADER_SIZE; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } } - public static int arrowChangeTypeOffset(byte magic) { + /** + * Get the statistics length field offset for the given magic version. Only available for V1 and + * later. + */ + public static int statisticsLengthOffset(byte magic) { switch (magic) { + case LOG_MAGIC_VALUE_V0: + throw new UnsupportedOperationException( + "Statistics not supported in magic version " + magic); case LOG_MAGIC_VALUE_V1: - return V1_ARROW_CHANGETYPE_OFFSET; + return V1_STATISTICS_LENGTH_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_STATISTICS_LENGTH_OFFSET; + default: + throw new IllegalArgumentException("Unsupported magic value " + magic); + } + } + + /** + * Get the statistics data offset for the given magic version. For V1+, statistics data is + * placed right after the fixed header. + */ + public static int statisticsDataOffset(byte magic) { + switch (magic) { case LOG_MAGIC_VALUE_V0: - return V0_ARROW_CHANGETYPE_OFFSET; + throw new UnsupportedOperationException( + "Statistics not supported in magic version " + magic); + case LOG_MAGIC_VALUE_V1: + return V1_STATISTICS_DATA_OFFSET; + case LOG_MAGIC_VALUE_V2: + return V2_STATISTICS_DATA_OFFSET; default: throw new IllegalArgumentException("Unsupported magic value " + magic); } } + + /** + * Clear statistics information from a V1+ log record batch header buffer. This method modifies + * the header buffer in-place to set StatisticsLength field to 0 (no statistics). + * + *

This method should only be called for V1+ format batches. For V0, this method has no + * effect. + * + * @param headerBuffer the header buffer to modify (must have little-endian byte order) + * @param magic the magic byte indicating the batch format version + */ + public static void clearStatisticsFromHeader(ByteBuffer headerBuffer, byte magic) { + if (magic < LOG_MAGIC_VALUE_V1) { + return; + } + + // Set StatisticsLength to 0 (no statistics) + headerBuffer.position(statisticsLengthOffset(magic)); + headerBuffer.putInt(0); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatistics.java b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatistics.java new file mode 100644 index 0000000000..30f4d984b6 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatistics.java @@ -0,0 +1,59 @@ +/* + * 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.record; + +import org.apache.fluss.row.InternalRow; + +/** Statistics information of {@link LogRecordBatch LogRecordBatch}. */ +public interface LogRecordBatchStatistics { + + /** + * Get the minimum values as an InternalRow. + * + * @return The minimum values + */ + InternalRow getMinValues(); + + /** + * Get the maximum values as an InternalRow. + * + * @return The maximum values + */ + InternalRow getMaxValues(); + + /** + * Get the null counts for each field. + * + * @return Array of null counts + */ + Long[] getNullCounts(); + + /** + * Whether the statistics information for a specific field is available. + * + * @return true if statistics information is available + */ + boolean hasColumnStatistics(int fieldIndex); + + /** + * Get the schema id. + * + * @return The schema id + */ + int getSchemaId(); +} diff --git a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsCollector.java b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsCollector.java new file mode 100644 index 0000000000..8c71fbc784 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsCollector.java @@ -0,0 +1,232 @@ +/* + * 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.record; + +import org.apache.fluss.memory.OutputView; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DecimalType; +import org.apache.fluss.types.LocalZonedTimestampType; +import org.apache.fluss.types.RowType; +import org.apache.fluss.types.TimestampType; + +import java.io.IOException; +import java.util.Arrays; +import java.util.function.Function; + +/** + * Collector for {@link LogRecordBatchStatistics} that accumulates statistics during record batch + * construction. Manages statistics data in memory arrays and supports schema-aware statistics + * format. + */ +public class LogRecordBatchStatisticsCollector { + + private final RowType rowType; + private final int[] statsIndexMapping; + + // Statistics arrays (only for columns that need statistics) + private final Object[] minValues; + private final Object[] maxValues; + private final Long[] nullCounts; + + private final LogRecordBatchStatisticsWriter statisticsWriter; + + public LogRecordBatchStatisticsCollector(RowType rowType, int[] statsIndexMapping) { + this.rowType = rowType; + this.statsIndexMapping = statsIndexMapping; + + // Initialize statistics arrays + this.minValues = new Object[statsIndexMapping.length]; + this.maxValues = new Object[statsIndexMapping.length]; + this.nullCounts = new Long[statsIndexMapping.length]; + + this.statisticsWriter = new LogRecordBatchStatisticsWriter(rowType, statsIndexMapping); + + Arrays.fill(nullCounts, 0L); + } + + /** + * Process a row and update statistics for configured columns only. + * + * @param row The row to process + */ + public void processRow(InternalRow row) { + for (int statIndex = 0; statIndex < statsIndexMapping.length; statIndex++) { + int fullRowIndex = statsIndexMapping[statIndex]; + if (row.isNullAt(fullRowIndex)) { + nullCounts[statIndex]++; + } else { + updateMinMax(statIndex, fullRowIndex, row); + } + } + } + + /** + * Write the collected statistics to an OutputView. + * + * @param outputView The target output view + * @return The number of bytes written, or 0 if no statistics collected + * @throws IOException If writing fails + */ + public int writeStatistics(OutputView outputView) throws IOException { + return statisticsWriter.writeStatistics( + GenericRow.of(minValues), GenericRow.of(maxValues), nullCounts, outputView); + } + + /** + * Get the row type used by this collector. + * + * @return The row type + */ + public RowType getRowType() { + return rowType; + } + + /** + * Estimate the size in bytes that would be required to serialize the currently collected + * statistics. This method provides an efficient way to calculate statistics size without + * actually writing the data to memory. + * + * @return The estimated number of bytes required for serialization + */ + public int estimatedSizeInBytes() { + return statisticsWriter.estimatedSizeInBytes( + GenericRow.of(minValues), GenericRow.of(maxValues)); + } + + /** Reset the collector to collect new statistics. */ + public void reset() { + Arrays.fill(nullCounts, 0L); + Arrays.fill(minValues, null); + Arrays.fill(maxValues, null); + } + + /** + * Update min/max values for a specific field. + * + * @param statsIndex the index in the statistics arrays + * @param schemaIndex the index in the full schema + * @param row the row being processed + */ + private void updateMinMax(int statsIndex, int schemaIndex, InternalRow row) { + DataType fieldType = rowType.getTypeAt(schemaIndex); + + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + boolean boolValue = row.getBoolean(schemaIndex); + updateMinMaxInternal(statsIndex, boolValue, Boolean::compare); + break; + case TINYINT: + byte byteValue = row.getByte(schemaIndex); + updateMinMaxInternal(statsIndex, byteValue, Byte::compare); + break; + case SMALLINT: + short shortValue = row.getShort(schemaIndex); + updateMinMaxInternal(statsIndex, shortValue, Short::compare); + break; + case INTEGER: + int intValue = row.getInt(schemaIndex); + updateMinMaxInternal(statsIndex, intValue, Integer::compare); + break; + case BIGINT: + long longValue = row.getLong(schemaIndex); + updateMinMaxInternal(statsIndex, longValue, Long::compare); + break; + case FLOAT: + float floatValue = row.getFloat(schemaIndex); + updateMinMaxInternal(statsIndex, floatValue, Float::compare); + break; + case DOUBLE: + double doubleValue = row.getDouble(schemaIndex); + updateMinMaxInternal(statsIndex, doubleValue, Double::compare); + break; + case STRING: + BinaryString stringValue = row.getString(schemaIndex); + updateMinMaxInternal( + statsIndex, stringValue, BinaryString::compareTo, BinaryString::copy); + break; + case DECIMAL: + DecimalType decimalType = (DecimalType) fieldType; + Decimal decimalValue = + row.getDecimal( + schemaIndex, decimalType.getPrecision(), decimalType.getScale()); + updateMinMaxInternal(statsIndex, decimalValue, Decimal::compareTo); + break; + case DATE: + int dateValue = row.getInt(schemaIndex); + updateMinMaxInternal(statsIndex, dateValue, Integer::compare); + break; + case TIME_WITHOUT_TIME_ZONE: + int timeValue = row.getInt(schemaIndex); + updateMinMaxInternal(statsIndex, timeValue, Integer::compare); + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) fieldType; + TimestampNtz timestampNtzValue = + row.getTimestampNtz(schemaIndex, timestampType.getPrecision()); + updateMinMaxInternal(statsIndex, timestampNtzValue, TimestampNtz::compareTo); + break; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType localZonedTimestampType = + (LocalZonedTimestampType) fieldType; + TimestampLtz timestampLtzValue = + row.getTimestampLtz(schemaIndex, localZonedTimestampType.getPrecision()); + updateMinMaxInternal(statsIndex, timestampLtzValue, TimestampLtz::compareTo); + break; + default: + // For unsupported types, don't collect min/max + break; + } + } + + // Unified generic method for all min/max updates with optional value transformation + private void updateMinMaxInternal( + int statsIndex, + T value, + java.util.Comparator comparator, + Function valueTransformer) { + if (minValues[statsIndex] == null) { + minValues[statsIndex] = valueTransformer.apply(value); + } else { + T currentMin = (T) minValues[statsIndex]; + if (comparator.compare(value, currentMin) < 0) { + minValues[statsIndex] = valueTransformer.apply(value); + } + } + + if (maxValues[statsIndex] == null) { + maxValues[statsIndex] = valueTransformer.apply(value); + } else { + T currentMax = (T) maxValues[statsIndex]; + if (comparator.compare(value, currentMax) > 0) { + maxValues[statsIndex] = valueTransformer.apply(value); + } + } + } + + // Convenience method for values that don't need transformation + private void updateMinMaxInternal( + int statsIndex, T value, java.util.Comparator comparator) { + updateMinMaxInternal(statsIndex, value, comparator, Function.identity()); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsParser.java b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsParser.java new file mode 100644 index 0000000000..69feb7e580 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsParser.java @@ -0,0 +1,384 @@ +/* + * 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.record; + +import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.types.RowType; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.nio.ByteBuffer; + +import static org.apache.fluss.record.LogRecordBatchFormat.STATISTICS_VERSION; + +/** + * A parser for LogRecordBatch statistics that reads statistical metadata directly from various + * memory sources (MemorySegment, ByteBuffer, byte array) without creating intermediate heap + * objects. + * + *

This parser supports the schema-aware statistics format and provides functionality to: + * + *

    + *
  • Parse complete statistics data into DefaultLogRecordBatchStatistics objects + *
  • Validate statistics data integrity and format compatibility + *
  • Calculate statistics data size from headers + *
+ * + *

The statistics format includes: + * + *

    + *
  • Version byte for format compatibility + *
  • Column count and index mapping for statistics columns + *
  • Null counts for each statistics column + *
  • Min/max values stored as variable-length binary data + *
+ */ +public class LogRecordBatchStatisticsParser { + + private static final Logger LOG = LoggerFactory.getLogger(LogRecordBatchStatisticsParser.class); + + // Fixed offsets within the statistics binary format. + // Format: Version(1B) | ColumnCount(2B) | ColumnIndexes(2*N B) | NullCounts(4*N B) + // | MinValuesSize(4B) | MinValues(...) | MaxValuesSize(4B) | MaxValues(...) + private static final int VERSION_OFFSET = 0; + private static final int COLUMN_COUNT_OFFSET = 1; + private static final int COLUMN_INDEXES_OFFSET = 3; + + /** Returns the byte offset where null counts begin, given N statistics columns. */ + private static int nullCountsOffset(int columnCount) { + return COLUMN_INDEXES_OFFSET + 2 * columnCount; + } + + /** Returns the byte offset where the min-values size field begins. */ + private static int minValuesSizeOffset(int columnCount) { + return nullCountsOffset(columnCount) + 4 * columnCount; + } + + /** + * Parse statistics data from a memory segment using the schema-aware format. + * + *

This method reads the complete statistics structure including version, column mappings, + * null counts, and min/max value boundaries directly from memory without copying data. The + * returned object maintains references to the original memory segment for efficient lazy + * evaluation of min/max values. + * + * @param segment The memory segment containing the serialized statistics data + * @param position The byte position in the segment where statistics data begins + * @param rowType The row type schema used to interpret column types and validate structure + * @param schemaId The schema identifier for version compatibility and data interpretation + * @return A DefaultLogRecordBatchStatistics object with zero-copy access to the data, or null + * if the data is invalid, corrupted, or has incompatible version + */ + @Nullable + public static DefaultLogRecordBatchStatistics parseStatistics( + MemorySegment segment, int position, RowType rowType, int schemaId) { + + try { + // Read version at fixed offset + byte version = segment.get(position + VERSION_OFFSET); + if (version != STATISTICS_VERSION) { + return null; + } + + // Read statistics column count at fixed offset + short statisticsColumnCount = segment.getShort(position + COLUMN_COUNT_OFFSET); + if (statisticsColumnCount <= 0) { + return null; + } + + // Read statistics column indexes at fixed offset + int indexesStart = position + COLUMN_INDEXES_OFFSET; + int[] statsIndexMapping = new int[statisticsColumnCount]; + for (int i = 0; i < statisticsColumnCount; i++) { + statsIndexMapping[i] = segment.getShort(indexesStart + 2 * i); + } + + // Read null counts at fixed offset + int nullCountsStart = position + nullCountsOffset(statisticsColumnCount); + Long[] nullCounts = new Long[statisticsColumnCount]; + for (int i = 0; i < statisticsColumnCount; i++) { + nullCounts[i] = (long) segment.getInt(nullCountsStart + 4 * i); + } + + // Read min values size at fixed offset + int minSizeFieldOffset = minValuesSizeOffset(statisticsColumnCount); + int minValuesSize = segment.getInt(position + minSizeFieldOffset); + + // Min values data starts right after the min-values size field + int minValuesOffset = minSizeFieldOffset + 4; + + // Max values size field follows min values data + int maxSizeFieldPos = position + minValuesOffset + minValuesSize; + int maxValuesSize = segment.getInt(maxSizeFieldPos); + + // Max values data starts right after the max-values size field + int maxValuesOffset = minValuesOffset + minValuesSize + 4; + + return new DefaultLogRecordBatchStatistics( + segment, + position, + maxValuesOffset + maxValuesSize, + rowType, + schemaId, + nullCounts, + minValuesOffset, + maxValuesOffset, + minValuesSize, + maxValuesSize, + statsIndexMapping); + } catch (Exception e) { + LOG.warn("Failed to parse statistics for schema {}", schemaId, e); + return null; + } + } + + /** + * Parse statistics data from a ByteBuffer by wrapping it as a MemorySegment. + * + *

This is a convenience method that wraps the ByteBuffer in a MemorySegment and delegates to + * the primary parsing method. The ByteBuffer's position is not modified. Note that heap-backed + * ByteBuffers incur a data copy during wrapping; direct ByteBuffers are wrapped without + * copying. + * + * @param buffer The ByteBuffer containing the serialized statistics data, must not be null + * @param rowType The row type schema used to interpret column types and validate structure + * @param schemaId The schema identifier for version compatibility and data interpretation + * @return A DefaultLogRecordBatchStatistics object, or null if the buffer is null/empty or + * contains invalid data + */ + @Nullable + public static DefaultLogRecordBatchStatistics parseStatistics( + ByteBuffer buffer, RowType rowType, int schemaId) { + if (buffer == null || buffer.remaining() == 0) { + return null; + } + + MemorySegment segment = wrapByteBuffer(buffer); + return parseStatistics(segment, 0, rowType, schemaId); + } + + /** + * Parse statistics data from a byte array by wrapping it as a heap memory segment. + * + *

This is a convenience method that wraps the byte array in a MemorySegment and delegates to + * the primary parsing method. The original byte array is not modified. + * + * @param buffer The byte array containing the serialized statistics data, must not be null + * @param rowType The row type schema used to interpret column types and validate structure + * @param schemaId The schema identifier for version compatibility and data interpretation + * @return A DefaultLogRecordBatchStatistics object with zero-copy access to the data, or null + * if the array is null/empty or contains invalid data + */ + @Nullable + public static DefaultLogRecordBatchStatistics parseStatistics( + byte[] buffer, RowType rowType, int schemaId) { + if (buffer == null || buffer.length == 0) { + return null; + } + + MemorySegment segment = MemorySegment.wrap(buffer); + return parseStatistics(segment, 0, rowType, schemaId); + } + + /** + * Validates whether the data in a memory segment represents well-formed statistics. + * + *

This method performs fast validation by checking: + * + *

    + *
  • Minimum data size requirements (at least 3 bytes for version + column count) + *
  • Statistics format version compatibility + *
  • Statistics column count against row type field count (must be positive and <= field + * count) + *
+ * + *

This is a lightweight validation that does not fully parse the statistics data. + * + * @param segment The memory segment to validate, must not be null + * @param position The byte position in the segment where statistics data begins + * @param size The size of the data region to validate, must be >= 3 + * @param rowType The row type schema to validate column count against + * @return true if the data appears to contain valid statistics format, false otherwise + */ + public static boolean isValidStatistics( + MemorySegment segment, int position, int size, RowType rowType) { + if (segment == null || size < 3) { + return false; + } + + try { + // Check version at fixed offset + byte version = segment.get(position + VERSION_OFFSET); + if (version != STATISTICS_VERSION) { + return false; + } + + // Check statistics column count at fixed offset + int statisticsColumnCount = segment.getShort(position + COLUMN_COUNT_OFFSET); + return statisticsColumnCount <= rowType.getFieldCount() && statisticsColumnCount > 0; + + } catch (Exception e) { + return false; + } + } + + /** + * Validates whether a byte array contains well-formed statistics data. + * + *

This is a convenience method that wraps the byte array in a MemorySegment and delegates to + * the primary validation method. + * + * @param data The byte array to validate, must not be null and at least 3 bytes long + * @param rowType The row type schema to validate column count against + * @return true if the array contains valid statistics format, false otherwise + */ + public static boolean isValidStatistics(byte[] data, RowType rowType) { + if (data == null || data.length < 3) { + return false; + } + + MemorySegment segment = MemorySegment.wrap(data); + return isValidStatistics(segment, 0, data.length, rowType); + } + + /** + * Validates whether a ByteBuffer contains well-formed statistics data. + * + *

This is a convenience method that wraps the ByteBuffer as an off-heap MemorySegment and + * delegates to the primary validation method. The ByteBuffer's position is not modified. + * + * @param buffer The ByteBuffer to validate, must not be null and have at least 3 remaining + * bytes + * @param rowType The row type schema to validate column count against + * @return true if the buffer contains valid statistics format, false otherwise + */ + public static boolean isValidStatistics(ByteBuffer buffer, RowType rowType) { + if (buffer == null || buffer.remaining() < 3) { + return false; + } + + MemorySegment segment = wrapByteBuffer(buffer); + return isValidStatistics(segment, 0, buffer.remaining(), rowType); + } + + /** + * Calculates the total size of statistics data by parsing the header and size fields. + * + *

This method reads through the statistics structure to determine the complete size + * including header, column mappings, null counts, and variable-length min/max values. It does + * not validate data integrity beyond basic size requirements. + * + * @param segment The memory segment containing the statistics data, must not be null + * @param position The byte position in the segment where statistics data begins + * @param size The size of the available data region, must be >= 3 bytes + * @return The total size in bytes of the complete statistics data structure, or -1 if the data + * is insufficient or malformed + */ + public static int getStatisticsSize(MemorySegment segment, int position, int size) { + if (segment == null || size < 3) { + return -1; + } + + try { + // Read column count at fixed offset (skip version byte) + short statisticsColumnCount = segment.getShort(position + COLUMN_COUNT_OFFSET); + if (statisticsColumnCount <= 0) { + return -1; + } + + // Calculate offset to min-values size field using fixed layout + int minSizeFieldOffset = minValuesSizeOffset(statisticsColumnCount); + int minValuesSize = segment.getInt(position + minSizeFieldOffset); + + // Max-values size field follows min values data + int maxSizeFieldOffset = minSizeFieldOffset + 4 + minValuesSize; + int maxValuesSize = segment.getInt(position + maxSizeFieldOffset); + + // Total size: up to end of max values data + return maxSizeFieldOffset + 4 + maxValuesSize; + + } catch (Exception e) { + return -1; + } + } + + /** + * Calculates the total size of statistics data from a byte array. + * + *

This is a convenience method that wraps the byte array in a MemorySegment and delegates to + * the primary size calculation method. + * + * @param data The byte array containing the statistics data, must not be null and at least 3 + * bytes + * @return The total size in bytes of the complete statistics data structure, or -1 if the array + * is insufficient or contains malformed data + */ + public static int getStatisticsSize(byte[] data) { + if (data == null || data.length < 3) { + return -1; + } + + MemorySegment segment = MemorySegment.wrap(data); + return getStatisticsSize(segment, 0, data.length); + } + + /** + * Calculates the total size of statistics data from a ByteBuffer. + * + *

This is a convenience method that wraps the ByteBuffer as an off-heap MemorySegment and + * delegates to the primary size calculation method. The ByteBuffer's position is not modified. + * + * @param buffer The ByteBuffer containing the statistics data, must not be null with at least 3 + * remaining bytes + * @return The total size in bytes of the complete statistics data structure, or -1 if the + * buffer is insufficient or contains malformed data + */ + public static int getStatisticsSize(ByteBuffer buffer) { + if (buffer == null || buffer.remaining() < 3) { + return -1; + } + + MemorySegment segment = wrapByteBuffer(buffer); + return getStatisticsSize(segment, 0, buffer.remaining()); + } + + /** + * Wraps a ByteBuffer into a MemorySegment, handling both direct and heap buffers correctly. + * + *

Note: For heap-backed ByteBuffers, this method copies the remaining bytes into a new byte + * array because heap ByteBuffers cannot be wrapped directly as off-heap memory segments. Direct + * ByteBuffers are wrapped without copying. + * + * @param buffer The ByteBuffer to wrap, must not be null + * @return A MemorySegment wrapping the buffer's data + */ + private static MemorySegment wrapByteBuffer(ByteBuffer buffer) { + if (buffer.isDirect()) { + return MemorySegment.wrapOffHeapMemory(buffer); + } else { + byte[] bytes = new byte[buffer.remaining()]; + int pos = buffer.position(); + buffer.get(bytes); + buffer.position(pos); + return MemorySegment.wrap(bytes); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsWriter.java b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsWriter.java new file mode 100644 index 0000000000..4e6d6d6de9 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/record/LogRecordBatchStatisticsWriter.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.record; + +import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.memory.OutputView; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.aligned.AlignedRow; +import org.apache.fluss.row.aligned.AlignedRowWriter; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DecimalType; +import org.apache.fluss.types.LocalZonedTimestampType; +import org.apache.fluss.types.RowType; +import org.apache.fluss.types.TimestampType; + +import java.io.IOException; + +import static org.apache.fluss.record.LogRecordBatchFormat.STATISTICS_VERSION; + +/** + * A high-performance writer for LogRecordBatch statistics that efficiently serializes statistical + * information directly to memory streams without creating intermediate heap objects. + * + *

This writer provides schema-aware statistics serialization capabilities, supporting selective + * column statistics based on index mappings. It can write min/max values, null counts, and other + * statistical metadata for log record batches in a compact binary format. + * + *

Binary Format Structure:

+ * + *
+ * [Version(1 byte)] [Column Count(2 bytes)] [Column Indexes(2*N bytes)]
+ * [Null Counts(4*N bytes)] [Min Values Row] [Max Values Row]
+ * 
+ * + *

Usage Example:

+ * + *
+ * RowType rowType = ...;
+ * int[] statsMapping = {0, 2, 5}; // Only collect stats for columns 0, 2, 5
+ * LogRecordBatchStatisticsWriter writer = new LogRecordBatchStatisticsWriter(rowType, statsMapping);
+ *
+ * InternalRow minValues = ...;
+ * InternalRow maxValues = ...;
+ * Long[] nullCounts = ...;
+ *
+ * int bytesWritten = writer.writeStatistics(minValues, maxValues, nullCounts, outputView);
+ * 
+ * + *

Thread Safety: This class is NOT thread-safe. Each thread should use its own instance. + * + *

Memory Management: The writer reuses internal buffers where possible. The internal + * {@link AlignedRowWriter} is created once and reused across writes to avoid repeated allocations. + * + * @see LogRecordBatchStatistics + * @see LogRecordBatchStatisticsCollector + * @see AlignedRow + */ +public class LogRecordBatchStatisticsWriter { + + /** Assumed size for variable-length fields (STRING, BYTES, non-compact DECIMAL, etc.). */ + private static final int VARIABLE_LENGTH_FIELD_ESTIMATE = 16; + + private final RowType rowType; + private final int[] statsIndexMapping; + private final RowType statsRowType; + + /** Cached rough estimate of a single row's serialized size. -1 means not yet computed. */ + private int cachedRowSizeEstimate = -1; + + /** Reusable AlignedRow and AlignedRowWriter to avoid per-write allocations. */ + private final AlignedRow reusableRow; + + private final AlignedRowWriter reusableRowWriter; + + public LogRecordBatchStatisticsWriter(RowType rowType, int[] statsIndexMapping) { + this.rowType = rowType; + this.statsIndexMapping = statsIndexMapping; + RowType.Builder statsRowTypeBuilder = RowType.builder(); + for (int fullRowIndex : statsIndexMapping) { + statsRowTypeBuilder.field( + rowType.getFieldNames().get(fullRowIndex), rowType.getTypeAt(fullRowIndex)); + } + this.statsRowType = statsRowTypeBuilder.build(); + + // Create reusable AlignedRow and AlignedRowWriter once + int fieldCount = statsRowType.getFieldCount(); + this.reusableRow = new AlignedRow(fieldCount); + this.reusableRowWriter = new AlignedRowWriter(reusableRow, fieldCount * 64); + } + + /** + * Write statistics to an OutputView in schema-aware format. + * + * @param minValues The minimum values as InternalRow, can be null + * @param maxValues The maximum values as InternalRow, can be null + * @param nullCounts The null counts array + * @param outputView The target output view + * @return The number of bytes written + * @throws IOException If writing fails + */ + public int writeStatistics( + InternalRow minValues, InternalRow maxValues, Long[] nullCounts, OutputView outputView) + throws IOException { + + int totalBytesWritten = 0; + + // Write version (1 byte) + outputView.writeByte(STATISTICS_VERSION); + totalBytesWritten += 1; + + // Write statistics column count (2 bytes) + outputView.writeShort(statsIndexMapping.length); + totalBytesWritten += 2; + + // Write statistics column indexes (2 bytes per index) + for (int fullRowIndex : statsIndexMapping) { + outputView.writeShort(fullRowIndex); + totalBytesWritten += 2; + } + + // Write null counts for statistics columns only (4 bytes per count) + for (Long count : nullCounts) { + long nullCount = count != null ? count : 0; + outputView.writeInt((int) nullCount); + totalBytesWritten += 4; + } + + // Write min values + int minRowBytes = writeRowData(minValues, outputView); + totalBytesWritten += minRowBytes; + + // Write max values + int maxRowBytes = writeRowData(maxValues, outputView); + totalBytesWritten += maxRowBytes; + + return totalBytesWritten; + } + + private int writeRowData(InternalRow row, OutputView outputView) throws IOException { + if (row != null) { + AlignedRow binaryRowData = convertToAlignedRow(row); + int rowSize = binaryRowData.getSizeInBytes(); + outputView.writeInt(rowSize); + // The reusable writer always produces heap-backed segments, so getArray() is safe. + MemorySegment segment = binaryRowData.getSegments()[0]; + outputView.write(segment.getArray(), binaryRowData.getOffset(), rowSize); + return 4 + rowSize; + } else { + outputView.writeInt(0); + return 4; + } + } + + /** + * Converts an InternalRow to AlignedRow using the internal reusable AlignedRowWriter. Always + * goes through the writer path to guarantee the result is backed by a heap MemorySegment, which + * is required by {@link #writeRowData} (it calls {@link MemorySegment#getArray()}). + */ + private AlignedRow convertToAlignedRow(InternalRow row) { + // Always use the reusable writer to ensure heap-backed segments. + // Do NOT shortcut with "instanceof AlignedRow" — an AlignedRow may be backed by an + // off-heap MemorySegment, and getArray() would throw on off-heap segments. + reusableRowWriter.reset(); + int fieldCount = statsRowType.getFieldCount(); + for (int i = 0; i < fieldCount; i++) { + if (row.isNullAt(i)) { + reusableRowWriter.setNullAt(i); + } else { + DataType fieldType = statsRowType.getTypeAt(i); + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + reusableRowWriter.writeBoolean(i, row.getBoolean(i)); + break; + case TINYINT: + reusableRowWriter.writeByte(i, row.getByte(i)); + break; + case SMALLINT: + reusableRowWriter.writeShort(i, row.getShort(i)); + break; + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + reusableRowWriter.writeInt(i, row.getInt(i)); + break; + case BIGINT: + reusableRowWriter.writeLong(i, row.getLong(i)); + break; + case FLOAT: + reusableRowWriter.writeFloat(i, row.getFloat(i)); + break; + case DOUBLE: + reusableRowWriter.writeDouble(i, row.getDouble(i)); + break; + case STRING: + reusableRowWriter.writeString(i, row.getString(i)); + break; + case DECIMAL: + DecimalType decimalType = (DecimalType) fieldType; + reusableRowWriter.writeDecimal( + i, + row.getDecimal( + i, decimalType.getPrecision(), decimalType.getScale()), + decimalType.getPrecision()); + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) fieldType; + reusableRowWriter.writeTimestampNtz( + i, + row.getTimestampNtz(i, timestampType.getPrecision()), + timestampType.getPrecision()); + break; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType ltzType = (LocalZonedTimestampType) fieldType; + reusableRowWriter.writeTimestampLtz( + i, + row.getTimestampLtz(i, ltzType.getPrecision()), + ltzType.getPrecision()); + break; + default: + throw new UnsupportedOperationException( + "Statistics collection is not supported for type: " + + fieldType.getTypeRoot()); + } + } + } + reusableRowWriter.complete(); + return reusableRow; + } + + /** + * Returns a heuristic estimate of the serialized size in bytes for the given statistics data. + * + *

Note: This is a rough estimate, not a guaranteed upper bound. For fixed-length + * types the size is exact, but for variable-length types (STRING, BYTES, non-compact DECIMAL, + * etc.) a constant estimate ({@value VARIABLE_LENGTH_FIELD_ESTIMATE} bytes) is assumed. As a + * result, the estimate may underestimate the actual size when rows contain large + * variable-length values. + * + * @param minValues The minimum values as InternalRow, can be null + * @param maxValues The maximum values as InternalRow, can be null + * @return The estimated number of bytes required for serialization + */ + public int estimatedSizeInBytes(InternalRow minValues, InternalRow maxValues) { + + int totalEstimatedBytes = 0; + + // Fixed size header components: + // Version (1 byte) + Column count (2 bytes) + totalEstimatedBytes += 3; + + // Column indexes (2 bytes per index) + totalEstimatedBytes += statsIndexMapping.length * 2; + + // Null counts (4 bytes per count) + totalEstimatedBytes += statsIndexMapping.length * 4; + + // Estimate min values size: 4 bytes for size field + row data + totalEstimatedBytes += 4 + (minValues != null ? getRowSizeEstimate() : 0); + + // Estimate max values size: 4 bytes for size field + row data + totalEstimatedBytes += 4 + (maxValues != null ? getRowSizeEstimate() : 0); + + return totalEstimatedBytes; + } + + /** + * Returns a cached rough estimate of the serialized row size based on the stats row type. The + * fixed-length part size is exact; variable-length fields use a constant estimate. + */ + private int getRowSizeEstimate() { + if (cachedRowSizeEstimate < 0) { + int fieldCount = statsRowType.getFieldCount(); + // Fixed-length part: null bits + 8 bytes per field + int estimate = AlignedRow.calculateFixPartSizeInBytes(fieldCount); + // Add variable-length field estimates + for (int i = 0; i < fieldCount; i++) { + if (!AlignedRow.isInFixedLengthPart(statsRowType.getTypeAt(i))) { + estimate += VARIABLE_LENGTH_FIELD_ESTIMATE; + } + } + cachedRowSizeEstimate = estimate; + } + return cachedRowSizeEstimate; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/record/MemoryLogRecordsArrowBuilder.java b/fluss-common/src/main/java/org/apache/fluss/record/MemoryLogRecordsArrowBuilder.java index 6624f14bd1..3276771d0b 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/MemoryLogRecordsArrowBuilder.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/MemoryLogRecordsArrowBuilder.java @@ -27,16 +27,21 @@ import org.apache.fluss.row.arrow.ArrowWriter; import org.apache.fluss.utils.crc.Crc32C; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + import java.io.IOException; -import static org.apache.fluss.record.LogRecordBatch.CURRENT_LOG_MAGIC_VALUE; import static org.apache.fluss.record.LogRecordBatchFormat.BASE_OFFSET_LENGTH; import static org.apache.fluss.record.LogRecordBatchFormat.LENGTH_LENGTH; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V0; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V1; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; import static org.apache.fluss.record.LogRecordBatchFormat.NO_BATCH_SEQUENCE; import static org.apache.fluss.record.LogRecordBatchFormat.NO_LEADER_EPOCH; import static org.apache.fluss.record.LogRecordBatchFormat.NO_WRITER_ID; -import static org.apache.fluss.record.LogRecordBatchFormat.arrowChangeTypeOffset; import static org.apache.fluss.record.LogRecordBatchFormat.crcOffset; import static org.apache.fluss.record.LogRecordBatchFormat.recordBatchHeaderSize; import static org.apache.fluss.record.LogRecordBatchFormat.schemaIdOffset; @@ -46,6 +51,7 @@ /** Builder for {@link MemoryLogRecords} of log records in {@link LogFormat#ARROW} format. */ public class MemoryLogRecordsArrowBuilder implements AutoCloseable { private static final int BUILDER_DEFAULT_OFFSET = 0; + private static final Logger LOG = LoggerFactory.getLogger(MemoryLogRecordsArrowBuilder.class); private final long baseLogOffset; private final int schemaId; @@ -56,6 +62,7 @@ public class MemoryLogRecordsArrowBuilder implements AutoCloseable { private final MemorySegment firstSegment; private final AbstractPagedOutputView pagedOutputView; private final boolean appendOnly; + @Nullable private final LogRecordBatchStatisticsCollector statisticsCollector; private volatile MultiBytesView bytesView = null; @@ -67,6 +74,8 @@ public class MemoryLogRecordsArrowBuilder implements AutoCloseable { private boolean reCalculateSizeInBytes = false; private boolean resetBatchHeader = false; private boolean aborted = false; + // Length of statistics bytes written directly to pagedOutputView (V1+) + private int statisticsBytesLength = 0; private MemoryLogRecordsArrowBuilder( long baseLogOffset, @@ -74,7 +83,8 @@ private MemoryLogRecordsArrowBuilder( byte magic, ArrowWriter arrowWriter, AbstractPagedOutputView pagedOutputView, - boolean appendOnly) { + boolean appendOnly, + @Nullable LogRecordBatchStatisticsCollector statisticsCollector) { this.appendOnly = appendOnly; checkArgument( schemaId <= Short.MAX_VALUE, @@ -91,15 +101,34 @@ private MemoryLogRecordsArrowBuilder( this.pagedOutputView = pagedOutputView; this.firstSegment = pagedOutputView.getCurrentSegment(); - int arrowChangeTypeOffset = arrowChangeTypeOffset(magic); + int headerSize = recordBatchHeaderSize(magic); checkArgument( - firstSegment.size() >= arrowChangeTypeOffset, + firstSegment.size() >= headerSize, "The size of first segment of pagedOutputView is too small, need at least " - + arrowChangeTypeOffset + + headerSize + " bytes."); - this.changeTypeWriter = new ChangeTypeVectorWriter(firstSegment, arrowChangeTypeOffset); - this.estimatedSizeInBytes = recordBatchHeaderSize(magic); + this.changeTypeWriter = new ChangeTypeVectorWriter(firstSegment, headerSize); + this.estimatedSizeInBytes = headerSize; this.recordCount = 0; + this.statisticsCollector = statisticsCollector; + } + + @VisibleForTesting + public static MemoryLogRecordsArrowBuilder builder( + long baseLogOffset, + byte magic, + int schemaId, + ArrowWriter arrowWriter, + AbstractPagedOutputView outputView, + LogRecordBatchStatisticsCollector statisticsCollector) { + return new MemoryLogRecordsArrowBuilder( + baseLogOffset, + schemaId, + magic, + arrowWriter, + outputView, + false, + statisticsCollector); } @VisibleForTesting @@ -110,7 +139,7 @@ public static MemoryLogRecordsArrowBuilder builder( ArrowWriter arrowWriter, AbstractPagedOutputView outputView) { return new MemoryLogRecordsArrowBuilder( - baseLogOffset, schemaId, magic, arrowWriter, outputView, false); + baseLogOffset, schemaId, magic, arrowWriter, outputView, false, null); } /** Builder with limited write size and the memory segment used to serialize records. */ @@ -118,14 +147,18 @@ public static MemoryLogRecordsArrowBuilder builder( int schemaId, ArrowWriter arrowWriter, AbstractPagedOutputView outputView, - boolean appendOnly) { + boolean appendOnly, + LogRecordBatchStatisticsCollector statisticsCollector) { + // Use V1 when statistics collector is provided, V0 otherwise + byte magic = statisticsCollector != null ? LOG_MAGIC_VALUE_V1 : LOG_MAGIC_VALUE_V0; return new MemoryLogRecordsArrowBuilder( BUILDER_DEFAULT_OFFSET, schemaId, - CURRENT_LOG_MAGIC_VALUE, + magic, arrowWriter, outputView, - appendOnly); + appendOnly, + statisticsCollector); } public MultiBytesView build() throws IOException { @@ -141,10 +174,54 @@ public MultiBytesView build() throws IOException { return bytesView; } - // serialize the arrow batch to dynamically allocated memory segments - arrowWriter.serializeToOutputView( - pagedOutputView, arrowChangeTypeOffset(magic) + changeTypeWriter.sizeInBytes()); + int headerSize = recordBatchHeaderSize(magic); recordCount = arrowWriter.getRecordsCount(); + int changeTypeSize = changeTypeWriter.sizeInBytes(); + + // For V1+ with statistics, write everything sequentially to pagedOutputView: + // [header] [statistics] [changeTypes] [arrow data] + // This makes CRC computation zero-copy over contiguous memory segments. + if (magic >= LOG_MAGIC_VALUE_V1 && statisticsCollector != null && recordCount > 0) { + // Save changeType bytes before they get overwritten. The changeType data lives + // in firstSegment at offset headerSize, which is the same memory backing + // pagedOutputView — so writing statistics there would clobber it. + byte[] changeTypeBytes = new byte[changeTypeSize]; + firstSegment.get(headerSize, changeTypeBytes, 0, changeTypeSize); + + // Position pagedOutputView right after the header + pagedOutputView.setPosition(headerSize); + + // Write statistics directly to pagedOutputView (no temp byte[]) + try { + statisticsBytesLength = statisticsCollector.writeStatistics(pagedOutputView); + } catch (Exception e) { + LOG.error("Failed to serialize statistics for record batch", e); + statisticsBytesLength = 0; + // Rewind to undo any partial writes from writeStatistics(). + // This is safe because statistics data is typically small (a few hundred + // bytes) and the first page is usually 1MB+, so no page boundary has + // been crossed and setPosition() can rewind within the same page. + pagedOutputView.setPosition(headerSize); + } + + // Write saved changeType bytes to pagedOutputView + pagedOutputView.write(changeTypeBytes); + + // Write arrow data to pagedOutputView at current position. + // Use the no-position overload since pages may have advanced. + arrowWriter.serializeToOutputView(pagedOutputView); + } else { + // V0 path or no stats: layout is [header] [changeTypes] [arrow data] + // changeTypes are already in firstSegment at headerSize offset + arrowWriter.serializeToOutputView(pagedOutputView, headerSize + changeTypeSize); + } + + // Reset the statistics collector for reuse + if (statisticsCollector != null) { + statisticsCollector.reset(); + } + + // Build MultiBytesView from contiguous pagedOutputView segments (zero-copy) bytesView = MultiBytesView.builder() .addMemorySegmentByteViewList(pagedOutputView.getWrittenSegments()) @@ -152,6 +229,7 @@ public MultiBytesView build() throws IOException { arrowWriter.recycle(writerEpoch); writeBatchHeader(); + return bytesView; } @@ -184,6 +262,10 @@ public void append(ChangeType changeType, InternalRow row) throws Exception { if (!appendOnly) { changeTypeWriter.writeChangeType(changeType); } + // Collect statistics for the row if enabled + if (statisticsCollector != null) { + statisticsCollector.processRow(row); + } reCalculateSizeInBytes = true; } @@ -241,9 +323,13 @@ public int estimatedSizeInBytes() { if (reCalculateSizeInBytes) { // make size in bytes up-to-date estimatedSizeInBytes = - arrowChangeTypeOffset(magic) + recordBatchHeaderSize(magic) + changeTypeWriter.sizeInBytes() + arrowWriter.estimatedSizeInBytes(); + // For V1+, add estimated statistics size (placed between header and records) + if (magic >= LOG_MAGIC_VALUE_V1 && statisticsCollector != null) { + estimatedSizeInBytes += statisticsCollector.estimatedSizeInBytes(); + } } reCalculateSizeInBytes = false; @@ -265,7 +351,7 @@ private void writeBatchHeader() throws IOException { outputView.writeLong(0); // write empty leaderEpoch which will be overridden on server side - if (magic >= LOG_MAGIC_VALUE_V1) { + if (magic >= LOG_MAGIC_VALUE_V2) { outputView.writeInt(NO_LEADER_EPOCH); } @@ -273,8 +359,15 @@ private void writeBatchHeader() throws IOException { outputView.writeUnsignedInt(0); // write schema id outputView.writeShort((short) schemaId); - // write attributes (currently only appendOnly flag) - outputView.writeBoolean(appendOnly); + + // write attributes (appendOnly flag) + byte attributes = 0; + if (appendOnly) { + attributes |= 0x01; // set appendOnly flag + } + + outputView.writeByte(attributes); + // write lastOffsetDelta if (recordCount > 0) { outputView.writeInt(recordCount - 1); @@ -287,7 +380,13 @@ private void writeBatchHeader() throws IOException { outputView.writeInt(batchSequence); outputView.writeInt(recordCount); - // Update crc. + // For V1+, write statistics length + if (magic >= LOG_MAGIC_VALUE_V1) { + outputView.writeInt(statisticsBytesLength); + } + + // Update crc - CRC covers from schemaId to end of the batch. + // All data is contiguous in pagedOutputView, so we can compute CRC directly. long crc = Crc32C.compute(pagedOutputView.getWrittenSegments(), schemaIdOffset(magic)); outputView.setPosition(crcOffset(magic)); outputView.writeUnsignedInt(crc); diff --git a/fluss-common/src/main/java/org/apache/fluss/record/MemoryLogRecordsRowBuilder.java b/fluss-common/src/main/java/org/apache/fluss/record/MemoryLogRecordsRowBuilder.java index 127503aa02..f31fd41e0b 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/MemoryLogRecordsRowBuilder.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/MemoryLogRecordsRowBuilder.java @@ -28,7 +28,7 @@ import static org.apache.fluss.record.LogRecordBatchFormat.BASE_OFFSET_LENGTH; import static org.apache.fluss.record.LogRecordBatchFormat.LENGTH_LENGTH; -import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V1; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; import static org.apache.fluss.record.LogRecordBatchFormat.NO_BATCH_SEQUENCE; import static org.apache.fluss.record.LogRecordBatchFormat.NO_LEADER_EPOCH; import static org.apache.fluss.record.LogRecordBatchFormat.NO_WRITER_ID; @@ -196,7 +196,7 @@ private void writeBatchHeader() throws IOException { outputView.writeLong(0); // write empty leaderEpoch which will be overridden on server side - if (magic >= LOG_MAGIC_VALUE_V1) { + if (magic >= LOG_MAGIC_VALUE_V2) { outputView.writeInt(NO_LEADER_EPOCH); } diff --git a/fluss-common/src/main/java/org/apache/fluss/record/bytesview/MultiBytesView.java b/fluss-common/src/main/java/org/apache/fluss/record/bytesview/MultiBytesView.java index 010b9b42d8..7462d5a109 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/bytesview/MultiBytesView.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/bytesview/MultiBytesView.java @@ -85,14 +85,26 @@ public static class Builder { private final List views = new ArrayList<>(); private FileRegionBytesView lastFileRegionView = null; - /** Adds a bytes section from a byte array. */ + /** + * Adds a bytes section from a byte array. + * + * @param bytes the byte array to add as a bytes view + * @return this builder instance for method chaining + */ public Builder addBytes(byte[] bytes) { views.add(new ByteBufBytesView(bytes)); lastFileRegionView = null; return this; } - /** Adds a bytes section from a range of {@link MemorySegment}. */ + /** + * Adds a bytes section from a range of {@link MemorySegment}. + * + * @param memorySegment the memory segment to read bytes from + * @param position the starting position in the memory segment + * @param size the number of bytes to read from the memory segment + * @return this builder instance for method chaining + */ public Builder addBytes(MemorySegment memorySegment, int position, int size) { views.add(new MemorySegmentBytesView(memorySegment, position, size)); lastFileRegionView = null; @@ -105,7 +117,16 @@ public Builder addMemorySegmentByteViewList(List bytesVi return this; } - /** Adds a bytes section from a range of {@link FileChannel}. */ + /** + * Adds a bytes section from a range of {@link FileChannel}. If this file region is + * continuous with the last added file region view from the same channel, they will be + * merged to improve file read performance. + * + * @param fileChannel the file channel to read bytes from + * @param position the starting position in the file channel + * @param size the number of bytes to read from the file channel + * @return this builder instance for method chaining + */ public Builder addBytes(FileChannel fileChannel, long position, int size) { if (lastFileRegionView != null && lastFileRegionView.fileChannel == fileChannel @@ -125,6 +146,10 @@ public Builder addBytes(FileChannel fileChannel, long position, int size) { return this; } + public boolean isEmpty() { + return views.isEmpty(); + } + /** Builds a {@link MultiBytesView}. */ public MultiBytesView build() { return new MultiBytesView(views.toArray(new BytesView[0])); diff --git a/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java index 0bc5ac071d..f8a399a20f 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java @@ -36,6 +36,7 @@ import org.apache.fluss.types.DataType; import org.apache.fluss.types.DecimalType; import org.apache.fluss.types.LocalZonedTimestampType; +import org.apache.fluss.types.RowType; import org.apache.fluss.types.TimestampType; import javax.annotation.Nullable; @@ -476,6 +477,101 @@ public int hashCode() { return BinarySegmentUtils.hashByWords(segments, offset, sizeInBytes); } + /** + * Creates an AlignedRow from an InternalRow using the provided RowType. If the input row is + * already an AlignedRow, it returns the original row for efficiency. Otherwise, it converts the + * row to AlignedRow format. + * + * @param rowType The type schema for the row data + * @param row The InternalRow to convert, can be null + * @return An AlignedRow instance, or null if input row is null + */ + public static AlignedRow from(RowType rowType, @Nullable InternalRow row) { + if (row == null) { + return null; + } + + // If the input is already an AlignedRow, return it directly for efficiency + if (row instanceof AlignedRow) { + return (AlignedRow) row; + } + + // Convert other InternalRow types to AlignedRow + int fieldCount = rowType.getFieldCount(); + AlignedRow alignedRow = new AlignedRow(fieldCount); + // Allocate sufficient space for variable-length fields + // 64 bytes per field should be sufficient for most cases + int initialSize = fieldCount * 64; + AlignedRowWriter writer = new AlignedRowWriter(alignedRow, initialSize); + writer.reset(); + + for (int i = 0; i < fieldCount; i++) { + if (row.isNullAt(i)) { + writer.setNullAt(i); + } else { + DataType fieldType = rowType.getTypeAt(i); + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + writer.writeBoolean(i, row.getBoolean(i)); + break; + case TINYINT: + writer.writeByte(i, row.getByte(i)); + break; + case SMALLINT: + writer.writeShort(i, row.getShort(i)); + break; + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + writer.writeInt(i, row.getInt(i)); + break; + case BIGINT: + writer.writeLong(i, row.getLong(i)); + break; + case FLOAT: + writer.writeFloat(i, row.getFloat(i)); + break; + case DOUBLE: + writer.writeDouble(i, row.getDouble(i)); + break; + case STRING: + writer.writeString(i, row.getString(i)); + break; + case DECIMAL: + DecimalType decimalType = (DecimalType) fieldType; + writer.writeDecimal( + i, + row.getDecimal( + i, decimalType.getPrecision(), decimalType.getScale()), + decimalType.getPrecision()); + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) fieldType; + writer.writeTimestampNtz( + i, + row.getTimestampNtz(i, timestampType.getPrecision()), + timestampType.getPrecision()); + break; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType localZonedTimestampType = + (LocalZonedTimestampType) fieldType; + writer.writeTimestampLtz( + i, + row.getTimestampLtz(i, localZonedTimestampType.getPrecision()), + localZonedTimestampType.getPrecision()); + break; + default: + throw new UnsupportedOperationException( + "Statistics collection is not supported for type: " + + fieldType.getTypeRoot()); + } + } + } + + writer.complete(); + return alignedRow; + } + public static AlignedRow singleColumn(@Nullable Integer i) { AlignedRow row = new AlignedRow(1); AlignedRowWriter writer = new AlignedRowWriter(row); @@ -508,7 +604,7 @@ public static AlignedRow singleColumn(@Nullable BinaryString string) { } /** - * If it is a fixed-length field, we can call this BinaryRowData's setXX method for in-place + * If it is a fixed-length field, we can call this AlignedRow's setXX method for in-place * updates. If it is variable-length field, can't use this method, because the underlying data * is stored continuously. */ diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriter.java index ebc0a40438..a07329a0b1 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/ArrowWriter.java @@ -77,6 +77,9 @@ public class ArrowWriter implements AutoCloseable { /** Container that holds a set of vectors for the rows. */ final VectorSchemaRoot root; + /** The schema of the rows. */ + private final RowType schema; + /** * An array of writers which are responsible for the serialization of each column of the rows. */ @@ -114,6 +117,7 @@ public class ArrowWriter implements AutoCloseable { ArrowCompressionInfo compressionInfo, ArrowCompressionRatioEstimator compressionRatioEstimator) { this.writerKey = writerKey; + this.schema = schema; this.root = VectorSchemaRoot.create(ArrowUtils.toArrowSchema(schema), allocator); this.provider = checkNotNull(provider); this.compressionCodec = compressionInfo.createCompressionCodec(); @@ -143,6 +147,10 @@ public int getWriteLimitInBytes() { return writeLimitInBytes; } + public RowType getSchema() { + return schema; + } + public boolean isFull() { if (recordsCount > 0 && recordsCount >= estimatedMaxRecordsCount) { root.setRowCount(recordsCount); @@ -243,6 +251,16 @@ public int serializeToOutputView(AbstractPagedOutputView outputView, int positio // Whether there is any record to write, we need to advance the position to make sure the // batch header will be written in outputView. outputView.setPosition(position); + return serializeToOutputView(outputView); + } + + /** + * Serializes the current row batch to Arrow format at the current position of the output view + * and returns the written size in bytes. Unlike {@link #serializeToOutputView( + * AbstractPagedOutputView, int)}, this method does not call {@code setPosition} and can be used + * when pages have already been advanced (e.g., after writing statistics data). + */ + public int serializeToOutputView(AbstractPagedOutputView outputView) throws IOException { if (recordsCount == 0) { return 0; } diff --git a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeChecks.java b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeChecks.java index 15ec476a31..ac598cee47 100644 --- a/fluss-common/src/main/java/org/apache/fluss/types/DataTypeChecks.java +++ b/fluss-common/src/main/java/org/apache/fluss/types/DataTypeChecks.java @@ -62,6 +62,31 @@ public static boolean equalsWithFieldId(DataType original, DataType that) { return that.accept(new DataTypeEqualsWithFieldId(original)); } + /** + * Check if the given data type is supported for statistics collection. Uses a whitelist + * approach to ensure only types with meaningful statistical properties are included. + * + * @param dataType the data type to check + * @return true if the data type is supported for statistics, false otherwise + */ + public static boolean isSupportedStatisticsType(DataType dataType) { + return dataType.isAnyOf( + DataTypeRoot.BOOLEAN, + DataTypeRoot.TINYINT, + DataTypeRoot.SMALLINT, + DataTypeRoot.INTEGER, + DataTypeRoot.BIGINT, + DataTypeRoot.FLOAT, + DataTypeRoot.DOUBLE, + DataTypeRoot.STRING, + DataTypeRoot.CHAR, + DataTypeRoot.DECIMAL, + DataTypeRoot.DATE, + DataTypeRoot.TIME_WITHOUT_TIME_ZONE, + DataTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, + DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + } + private DataTypeChecks() { // no instantiation } diff --git a/fluss-common/src/test/java/org/apache/fluss/config/StatisticsColumnsConfigTest.java b/fluss-common/src/test/java/org/apache/fluss/config/StatisticsColumnsConfigTest.java new file mode 100644 index 0000000000..a3cb491f64 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/config/StatisticsColumnsConfigTest.java @@ -0,0 +1,158 @@ +/* + * 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.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link StatisticsColumnsConfig}. */ +class StatisticsColumnsConfigTest { + + @Test + void testDisabledMode() { + StatisticsColumnsConfig config = StatisticsColumnsConfig.disabled(); + assertThat(config.getMode()).isEqualTo(StatisticsColumnsConfig.Mode.DISABLED); + assertThat(config.getColumns()).isEmpty(); + assertThat(config.isEnabled()).isFalse(); + } + + @Test + void testAllMode() { + StatisticsColumnsConfig config = StatisticsColumnsConfig.all(); + assertThat(config.getMode()).isEqualTo(StatisticsColumnsConfig.Mode.ALL); + assertThat(config.getColumns()).isEmpty(); + assertThat(config.isEnabled()).isTrue(); + } + + @Test + void testSpecifiedMode() { + List columns = Arrays.asList("col1", "col2", "col3"); + StatisticsColumnsConfig config = StatisticsColumnsConfig.of(columns); + assertThat(config.getMode()).isEqualTo(StatisticsColumnsConfig.Mode.SPECIFIED); + assertThat(config.getColumns()).containsExactly("col1", "col2", "col3"); + assertThat(config.isEnabled()).isTrue(); + } + + @Test + void testSpecifiedModeWithEmptyList() { + StatisticsColumnsConfig config = StatisticsColumnsConfig.of(Collections.emptyList()); + assertThat(config.getMode()).isEqualTo(StatisticsColumnsConfig.Mode.SPECIFIED); + assertThat(config.getColumns()).isEmpty(); + assertThat(config.isEnabled()).isTrue(); + } + + @Test + void testSpecifiedModeColumnsAreUnmodifiable() { + List columns = Arrays.asList("col1", "col2"); + StatisticsColumnsConfig config = StatisticsColumnsConfig.of(columns); + assertThatThrownBy(() -> config.getColumns().add("col3")) + .isInstanceOf(UnsupportedOperationException.class); + } + + @Test + void testOfWithNullThrows() { + assertThatThrownBy(() -> StatisticsColumnsConfig.of(null)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("columns must not be null"); + } + + @Test + void testDisabledSingleton() { + StatisticsColumnsConfig config1 = StatisticsColumnsConfig.disabled(); + StatisticsColumnsConfig config2 = StatisticsColumnsConfig.disabled(); + assertThat(config1).isSameAs(config2); + } + + @Test + void testAllSingleton() { + StatisticsColumnsConfig config1 = StatisticsColumnsConfig.all(); + StatisticsColumnsConfig config2 = StatisticsColumnsConfig.all(); + assertThat(config1).isSameAs(config2); + } + + @Test + void testEqualsAndHashCode() { + // same mode and columns + StatisticsColumnsConfig specified1 = + StatisticsColumnsConfig.of(Arrays.asList("col1", "col2")); + StatisticsColumnsConfig specified2 = + StatisticsColumnsConfig.of(Arrays.asList("col1", "col2")); + assertThat(specified1).isEqualTo(specified2); + assertThat(specified1.hashCode()).isEqualTo(specified2.hashCode()); + + // same instance + assertThat(specified1).isEqualTo(specified1); + + // null and different type + assertThat(specified1).isNotEqualTo(null); + assertThat(specified1).isNotEqualTo("not a config"); + + // different columns + StatisticsColumnsConfig specified3 = + StatisticsColumnsConfig.of(Arrays.asList("col1", "col3")); + assertThat(specified1).isNotEqualTo(specified3); + + // different modes + assertThat(StatisticsColumnsConfig.disabled()).isNotEqualTo(StatisticsColumnsConfig.all()); + assertThat(StatisticsColumnsConfig.disabled()).isNotEqualTo(specified1); + assertThat(StatisticsColumnsConfig.all()).isNotEqualTo(specified1); + + // disabled equals disabled + assertThat(StatisticsColumnsConfig.disabled()) + .isEqualTo(StatisticsColumnsConfig.disabled()); + assertThat(StatisticsColumnsConfig.disabled().hashCode()) + .isEqualTo(StatisticsColumnsConfig.disabled().hashCode()); + + // all equals all + assertThat(StatisticsColumnsConfig.all()).isEqualTo(StatisticsColumnsConfig.all()); + assertThat(StatisticsColumnsConfig.all().hashCode()) + .isEqualTo(StatisticsColumnsConfig.all().hashCode()); + } + + @Test + void testToString() { + assertThat(StatisticsColumnsConfig.disabled().toString()) + .isEqualTo("StatisticsColumnsConfig{DISABLED}"); + + assertThat(StatisticsColumnsConfig.all().toString()) + .isEqualTo("StatisticsColumnsConfig{ALL}"); + + StatisticsColumnsConfig specified = + StatisticsColumnsConfig.of(Arrays.asList("col1", "col2")); + assertThat(specified.toString()) + .isEqualTo("StatisticsColumnsConfig{SPECIFIED: [col1, col2]}"); + } + + @Test + void testSingleColumnSpecified() { + StatisticsColumnsConfig config = + StatisticsColumnsConfig.of(Collections.singletonList("single_col")); + assertThat(config.getMode()).isEqualTo(StatisticsColumnsConfig.Mode.SPECIFIED); + assertThat(config.getColumns()).containsExactly("single_col"); + assertThat(config.isEnabled()).isTrue(); + assertThat(config.toString()).isEqualTo("StatisticsColumnsConfig{SPECIFIED: [single_col]}"); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/config/StatisticsConfigUtilsTest.java b/fluss-common/src/test/java/org/apache/fluss/config/StatisticsConfigUtilsTest.java new file mode 100644 index 0000000000..04e5f90070 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/config/StatisticsConfigUtilsTest.java @@ -0,0 +1,183 @@ +/* + * 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.exception.InvalidConfigException; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.types.DataTypes; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link StatisticsConfigUtils}. */ +class StatisticsConfigUtilsTest { + + private static final Schema TEST_SCHEMA = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .column("data", DataTypes.BYTES()) + .column("tags", DataTypes.ARRAY(DataTypes.STRING())) + .column("metadata", DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())) + .column("nested", DataTypes.ROW(DataTypes.FIELD("f1", DataTypes.INT()))) + .build(); + + @Test + void testValidateWithWildcard() { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(TEST_SCHEMA) + .distributedBy(3) + .property(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "*") + .build(); + + assertThatNoException() + .isThrownBy(() -> StatisticsConfigUtils.validateStatisticsConfig(descriptor)); + } + + @Test + void testValidateWithNotSet() { + // When the property is not set, statistics is disabled - no validation needed + TableDescriptor descriptor = + TableDescriptor.builder().schema(TEST_SCHEMA).distributedBy(3).build(); + + assertThatNoException() + .isThrownBy(() -> StatisticsConfigUtils.validateStatisticsConfig(descriptor)); + } + + @Test + void testValidateWithSpecificColumns() { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(TEST_SCHEMA) + .distributedBy(3) + .property(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "id,name") + .build(); + + assertThatNoException() + .isThrownBy(() -> StatisticsConfigUtils.validateStatisticsConfig(descriptor)); + } + + @Test + void testValidateWithNonExistentColumn() { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(TEST_SCHEMA) + .distributedBy(3) + .property(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "nonexistent") + .build(); + + assertThatThrownBy(() -> StatisticsConfigUtils.validateStatisticsConfig(descriptor)) + .isInstanceOf(InvalidConfigException.class) + .hasMessageContaining("does not exist in table schema"); + } + + @Test + void testValidateWithBinaryColumn() { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(TEST_SCHEMA) + .distributedBy(3) + .property(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "data") + .build(); + + assertThatThrownBy(() -> StatisticsConfigUtils.validateStatisticsConfig(descriptor)) + .isInstanceOf(InvalidConfigException.class) + .hasMessageContaining("is not supported for statistics collection"); + } + + @Test + void testValidateWithArrayColumn() { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(TEST_SCHEMA) + .distributedBy(3) + .property(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "tags") + .build(); + + assertThatThrownBy(() -> StatisticsConfigUtils.validateStatisticsConfig(descriptor)) + .isInstanceOf(InvalidConfigException.class) + .hasMessageContaining("is not supported for statistics collection"); + } + + @Test + void testValidateWithMapColumn() { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(TEST_SCHEMA) + .distributedBy(3) + .property(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "metadata") + .build(); + + assertThatThrownBy(() -> StatisticsConfigUtils.validateStatisticsConfig(descriptor)) + .isInstanceOf(InvalidConfigException.class) + .hasMessageContaining("is not supported for statistics collection"); + } + + @Test + void testValidateWithRowColumn() { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(TEST_SCHEMA) + .distributedBy(3) + .property(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "nested") + .build(); + + assertThatThrownBy(() -> StatisticsConfigUtils.validateStatisticsConfig(descriptor)) + .isInstanceOf(InvalidConfigException.class) + .hasMessageContaining("is not supported for statistics collection"); + } + + @Test + void testTableConfigStatisticsColumnsDisabled() { + // Not set -> DISABLED + Configuration config = new Configuration(); + TableConfig tableConfig = new TableConfig(config); + StatisticsColumnsConfig columnsConfig = tableConfig.getStatisticsColumns(); + + assertThat(columnsConfig.getMode()).isEqualTo(StatisticsColumnsConfig.Mode.DISABLED); + assertThat(columnsConfig.isEnabled()).isFalse(); + } + + @Test + void testTableConfigStatisticsColumnsAll() { + Configuration config = new Configuration(); + config.setString(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "*"); + TableConfig tableConfig = new TableConfig(config); + StatisticsColumnsConfig columnsConfig = tableConfig.getStatisticsColumns(); + + assertThat(columnsConfig.getMode()).isEqualTo(StatisticsColumnsConfig.Mode.ALL); + assertThat(columnsConfig.isEnabled()).isTrue(); + } + + @Test + void testTableConfigStatisticsColumnsSpecified() { + Configuration config = new Configuration(); + config.setString(ConfigOptions.TABLE_STATISTICS_COLUMNS.key(), "id,name"); + TableConfig tableConfig = new TableConfig(config); + StatisticsColumnsConfig columnsConfig = tableConfig.getStatisticsColumns(); + + assertThat(columnsConfig.getMode()).isEqualTo(StatisticsColumnsConfig.Mode.SPECIFIED); + assertThat(columnsConfig.isEnabled()).isTrue(); + assertThat(columnsConfig.getColumns()).containsExactly("id", "name"); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchFormatTest.java b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchFormatTest.java index 3f83aa7a53..2be1b78baf 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchFormatTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchFormatTest.java @@ -23,7 +23,6 @@ import static org.apache.fluss.record.LogRecordBatchFormat.LENGTH_OFFSET; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_OVERHEAD; import static org.apache.fluss.record.LogRecordBatchFormat.MAGIC_OFFSET; -import static org.apache.fluss.record.LogRecordBatchFormat.arrowChangeTypeOffset; import static org.apache.fluss.record.LogRecordBatchFormat.attributeOffset; import static org.apache.fluss.record.LogRecordBatchFormat.batchSequenceOffset; import static org.apache.fluss.record.LogRecordBatchFormat.crcOffset; @@ -32,6 +31,8 @@ import static org.apache.fluss.record.LogRecordBatchFormat.recordBatchHeaderSize; import static org.apache.fluss.record.LogRecordBatchFormat.recordsCountOffset; import static org.apache.fluss.record.LogRecordBatchFormat.schemaIdOffset; +import static org.apache.fluss.record.LogRecordBatchFormat.statisticsDataOffset; +import static org.apache.fluss.record.LogRecordBatchFormat.statisticsLengthOffset; import static org.apache.fluss.record.LogRecordBatchFormat.writeClientIdOffset; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -51,8 +52,8 @@ void testCommonParam() { void testLogRecordBatchFormatForMagicV0() { byte magic = (byte) 0; assertThatThrownBy(() -> leaderEpochOffset(magic)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Unsupported magic value 0"); + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Leader epoch is not supported in V0/V1"); assertThat(crcOffset(magic)).isEqualTo(21); assertThat(schemaIdOffset(magic)).isEqualTo(25); assertThat(attributeOffset(magic)).isEqualTo(27); @@ -61,12 +62,36 @@ void testLogRecordBatchFormatForMagicV0() { assertThat(batchSequenceOffset(magic)).isEqualTo(40); assertThat(recordsCountOffset(magic)).isEqualTo(44); assertThat(recordBatchHeaderSize(magic)).isEqualTo(48); - assertThat(arrowChangeTypeOffset(magic)).isEqualTo(48); + + // V0 does not support statistics + assertThatThrownBy(() -> statisticsLengthOffset(magic)) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> statisticsDataOffset(magic)) + .isInstanceOf(UnsupportedOperationException.class); } @Test void testLogRecordBatchFormatForMagicV1() { byte magic = (byte) 1; + // V1 does not have leaderEpoch + assertThatThrownBy(() -> leaderEpochOffset(magic)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Leader epoch is not supported in V0/V1"); + assertThat(crcOffset(magic)).isEqualTo(21); + assertThat(schemaIdOffset(magic)).isEqualTo(25); + assertThat(attributeOffset(magic)).isEqualTo(27); + assertThat(lastOffsetDeltaOffset(magic)).isEqualTo(28); + assertThat(writeClientIdOffset(magic)).isEqualTo(32); + assertThat(batchSequenceOffset(magic)).isEqualTo(40); + assertThat(recordsCountOffset(magic)).isEqualTo(44); + assertThat(statisticsLengthOffset(magic)).isEqualTo(48); + assertThat(statisticsDataOffset(magic)).isEqualTo(52); + assertThat(recordBatchHeaderSize(magic)).isEqualTo(52); + } + + @Test + void testLogRecordBatchFormatForMagicV2() { + byte magic = (byte) 2; assertThat(leaderEpochOffset(magic)).isEqualTo(21); assertThat(crcOffset(magic)).isEqualTo(25); assertThat(schemaIdOffset(magic)).isEqualTo(29); @@ -75,7 +100,8 @@ void testLogRecordBatchFormatForMagicV1() { assertThat(writeClientIdOffset(magic)).isEqualTo(36); assertThat(batchSequenceOffset(magic)).isEqualTo(44); assertThat(recordsCountOffset(magic)).isEqualTo(48); - assertThat(recordBatchHeaderSize(magic)).isEqualTo(52); - assertThat(arrowChangeTypeOffset(magic)).isEqualTo(52); + assertThat(statisticsLengthOffset(magic)).isEqualTo(52); + assertThat(statisticsDataOffset(magic)).isEqualTo(56); + assertThat(recordBatchHeaderSize(magic)).isEqualTo(56); } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsCollectorTest.java b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsCollectorTest.java new file mode 100644 index 0000000000..21c4a08582 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsCollectorTest.java @@ -0,0 +1,403 @@ +/* + * 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.record; + +import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.memory.MemorySegmentOutputView; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.testutils.DataTestUtils; +import org.apache.fluss.types.DataField; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link LogRecordBatchStatisticsCollector}. */ +public class LogRecordBatchStatisticsCollectorTest { + + private LogRecordBatchStatisticsCollector collector; + private RowType testRowType; + + @BeforeEach + void setUp() { + testRowType = TestData.STATISTICS_MIXED_TYPE_ROW_TYPE; + collector = + new LogRecordBatchStatisticsCollector( + testRowType, + LogRecordBatchStatisticsTestUtils.createAllColumnsStatsMapping( + testRowType)); + } + + @Test + void testProcessRowAndWriteStatistics() throws IOException { + // Process single row + InternalRow row = DataTestUtils.row(TestData.STATISTICS_MIXED_TYPE_DATA.get(0)); + collector.processRow(row); + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + int bytesWritten = collector.writeStatistics(new MemorySegmentOutputView(segment)); + assertThat(bytesWritten).isGreaterThan(0); + } + + @Test + void testProcessMultipleRowsAndVerifyStatistics() throws IOException { + for (Object[] data : TestData.STATISTICS_MIXED_TYPE_DATA) { + collector.processRow(DataTestUtils.row(data)); + } + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + int bytesWritten = collector.writeStatistics(new MemorySegmentOutputView(segment)); + assertThat(bytesWritten).isGreaterThan(0); + + DefaultLogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, testRowType, 1); + assertThat(statistics).isNotNull(); + + InternalRow minValues = statistics.getMinValues(); + InternalRow maxValues = statistics.getMaxValues(); + + // Verify min/max values based on MIXED_TYPE_DATA + assertThat(minValues.getInt(0)).isEqualTo(1); + assertThat(maxValues.getInt(0)).isEqualTo(5); + assertThat(minValues.getString(1)).isEqualTo(BinaryString.fromString("a")); + assertThat(maxValues.getString(1)).isEqualTo(BinaryString.fromString("e")); + assertThat(minValues.getDouble(2)).isEqualTo(8.9); + assertThat(maxValues.getDouble(2)).isEqualTo(30.1); + assertThat(minValues.getBoolean(3)).isEqualTo(false); + assertThat(maxValues.getBoolean(3)).isEqualTo(true); + assertThat(minValues.getLong(4)).isEqualTo(100L); + assertThat(maxValues.getLong(4)).isEqualTo(300L); + assertThat(minValues.getFloat(5)).isEqualTo(1.23f); + assertThat(maxValues.getFloat(5)).isEqualTo(5.67f); + + // All null counts should be 0 + for (int i = 0; i < 6; i++) { + assertThat(statistics.getNullCounts()[i]).isEqualTo(0L); + } + } + + @Test + void testResetAndReprocess() throws IOException { + // Process initial data + for (Object[] data : TestData.STATISTICS_MIXED_TYPE_DATA) { + collector.processRow(DataTestUtils.row(data)); + } + + // Reset and process new data + collector.reset(); + List newData = + Arrays.asList( + new Object[] {10, "x", 100.0, false, 500L, 8.9f}, + new Object[] {15, "y", 200.0, true, 600L, 9.1f}); + + for (Object[] data : newData) { + collector.processRow(DataTestUtils.row(data)); + } + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + collector.writeStatistics(new MemorySegmentOutputView(segment)); + + DefaultLogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, testRowType, 1); + + // Verify values from new data only + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(10); + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(15); + } + + @Test + void testProcessNullValues() throws IOException { + RowType nullableRowType = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE())); + + LogRecordBatchStatisticsCollector nullCollector = + new LogRecordBatchStatisticsCollector( + nullableRowType, + LogRecordBatchStatisticsTestUtils.createAllColumnsStatsMapping( + nullableRowType)); + + List dataWithNulls = + Arrays.asList( + new Object[] {1, "a", 10.5}, + new Object[] {null, "b", 20.3}, + new Object[] {3, null, 15.7}, + new Object[] {2, "c", null}); + + for (Object[] data : dataWithNulls) { + nullCollector.processRow(DataTestUtils.row(data)); + } + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + nullCollector.writeStatistics(new MemorySegmentOutputView(segment)); + + DefaultLogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, nullableRowType, 1); + + // Verify null counts + assertThat(statistics.getNullCounts()[0]).isEqualTo(1L); + assertThat(statistics.getNullCounts()[1]).isEqualTo(1L); + assertThat(statistics.getNullCounts()[2]).isEqualTo(1L); + + // Verify min/max values exclude nulls + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(1); + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(3); + } + + @Test + void testPartialStatsIndexMapping() throws IOException { + RowType fullRowType = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE())); + + int[] partialStatsMapping = new int[] {0, 2}; // Skip column 1 + + LogRecordBatchStatisticsCollector partialCollector = + new LogRecordBatchStatisticsCollector(fullRowType, partialStatsMapping); + + List data = + Arrays.asList( + new Object[] {1, "a", 10.5}, + new Object[] {5, "b", 5.3}, + new Object[] {3, "c", 15.7}); + + for (Object[] rowData : data) { + partialCollector.processRow(DataTestUtils.row(rowData)); + } + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + partialCollector.writeStatistics(new MemorySegmentOutputView(segment)); + + DefaultLogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, fullRowType, 1); + + // Verify statistics for collected columns + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(1); + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(5); + assertThat(statistics.getMinValues().getDouble(2)).isEqualTo(5.3); + assertThat(statistics.getMaxValues().getDouble(2)).isEqualTo(15.7); + } + + @Test + void testDifferentDataTypes() throws IOException { + RowType comprehensiveRowType = + DataTypes.ROW( + new DataField("boolean_val", DataTypes.BOOLEAN()), + new DataField("byte_val", DataTypes.TINYINT()), + new DataField("short_val", DataTypes.SMALLINT()), + new DataField("int_val", DataTypes.INT()), + new DataField("long_val", DataTypes.BIGINT()), + new DataField("float_val", DataTypes.FLOAT()), + new DataField("double_val", DataTypes.DOUBLE()), + new DataField("string_val", DataTypes.STRING())); + + LogRecordBatchStatisticsCollector typeCollector = + new LogRecordBatchStatisticsCollector( + comprehensiveRowType, + LogRecordBatchStatisticsTestUtils.createAllColumnsStatsMapping( + comprehensiveRowType)); + + List comprehensiveData = + Arrays.asList( + new Object[] {true, (byte) 1, (short) 100, 1000, 10000L, 1.1f, 1.11, "a"}, + new Object[] {false, (byte) 5, (short) 50, 500, 5000L, 5.5f, 5.55, "z"}, + new Object[] {true, (byte) 3, (short) 200, 2000, 20000L, 2.2f, 2.22, "m"}); + + for (Object[] data : comprehensiveData) { + typeCollector.processRow(DataTestUtils.row(data)); + } + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + typeCollector.writeStatistics(new MemorySegmentOutputView(segment)); + + DefaultLogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, comprehensiveRowType, 1); + + InternalRow minValues = statistics.getMinValues(); + InternalRow maxValues = statistics.getMaxValues(); + + assertThat(minValues.getBoolean(0)).isEqualTo(false); + assertThat(maxValues.getBoolean(0)).isEqualTo(true); + assertThat(minValues.getByte(1)).isEqualTo((byte) 1); + assertThat(maxValues.getByte(1)).isEqualTo((byte) 5); + assertThat(minValues.getShort(2)).isEqualTo((short) 50); + assertThat(maxValues.getShort(2)).isEqualTo((short) 200); + assertThat(minValues.getInt(3)).isEqualTo(500); + assertThat(maxValues.getInt(3)).isEqualTo(2000); + assertThat(minValues.getLong(4)).isEqualTo(5000L); + assertThat(maxValues.getLong(4)).isEqualTo(20000L); + assertThat(minValues.getFloat(5)).isEqualTo(1.1f); + assertThat(maxValues.getFloat(5)).isEqualTo(5.5f); + assertThat(minValues.getDouble(6)).isEqualTo(1.11); + assertThat(maxValues.getDouble(6)).isEqualTo(5.55); + assertThat(minValues.getString(7)).isEqualTo(BinaryString.fromString("a")); + assertThat(maxValues.getString(7)).isEqualTo(BinaryString.fromString("z")); + } + + @Test + void testEstimatedSizeInBytesAccuracy() throws IOException { + for (Object[] data : TestData.STATISTICS_MIXED_TYPE_DATA) { + collector.processRow(DataTestUtils.row(data)); + } + + int estimatedSize = collector.estimatedSizeInBytes(); + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + int actualSize = collector.writeStatistics(new MemorySegmentOutputView(segment)); + + assertThat(estimatedSize).isGreaterThan(0); + // estimatedSizeInBytes() is a heuristic estimate, not a guaranteed upper bound. + // It may underestimate for rows with large variable-length values (STRING, BYTES, etc.) + // since it uses a fixed constant for variable-length fields. We verify the estimate + // is reasonably close to the actual size (within 2x). + assertThat((double) estimatedSize) + .as("Heuristic estimate should be reasonably close to actual size") + .isGreaterThan(actualSize * 0.5) + .isLessThan(actualSize * 2.0); + } + + @Test + void testZeroRows() throws IOException { + // Create a collector but do NOT call processRow() + RowType rowType = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE())); + + LogRecordBatchStatisticsCollector zeroRowCollector = + new LogRecordBatchStatisticsCollector( + rowType, + LogRecordBatchStatisticsTestUtils.createAllColumnsStatsMapping(rowType)); + + // Write statistics without processing any rows + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + int bytesWritten = zeroRowCollector.writeStatistics(new MemorySegmentOutputView(segment)); + assertThat(bytesWritten).isGreaterThan(0); + + // Parse back and verify + DefaultLogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, rowType, 1); + assertThat(statistics).isNotNull(); + + // All null counts should be 0 + Long[] nullCounts = statistics.getNullCounts(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + assertThat(nullCounts[i]).isEqualTo(0L); + } + + // Min/max values row exists but all fields should be null since no rows were processed + InternalRow minValues = statistics.getMinValues(); + InternalRow maxValues = statistics.getMaxValues(); + assertThat(minValues).isNotNull(); + assertThat(maxValues).isNotNull(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + assertThat(minValues.isNullAt(i)).isTrue(); + assertThat(maxValues.isNullAt(i)).isTrue(); + } + } + + @Test + void testAllNullColumn() throws IOException { + // Create data where column 2 (value) has ALL null values + RowType rowType = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE())); + + LogRecordBatchStatisticsCollector allNullCollector = + new LogRecordBatchStatisticsCollector( + rowType, + LogRecordBatchStatisticsTestUtils.createAllColumnsStatsMapping(rowType)); + + List dataWithAllNullColumn = + Arrays.asList( + new Object[] {1, "a", null}, + new Object[] {3, "c", null}, + new Object[] {5, "e", null}, + new Object[] {2, "b", null}); + + for (Object[] data : dataWithAllNullColumn) { + allNullCollector.processRow(DataTestUtils.row(data)); + } + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + allNullCollector.writeStatistics(new MemorySegmentOutputView(segment)); + + DefaultLogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, rowType, 1); + assertThat(statistics).isNotNull(); + + // Column 2 (value) null count should equal total row count + assertThat(statistics.getNullCounts()[2]).isEqualTo(dataWithAllNullColumn.size()); + + // Column 2 min/max should be null (all values were null) + InternalRow minValues = statistics.getMinValues(); + InternalRow maxValues = statistics.getMaxValues(); + assertThat(minValues).isNotNull(); + assertThat(maxValues).isNotNull(); + assertThat(minValues.isNullAt(2)).isTrue(); + assertThat(maxValues.isNullAt(2)).isTrue(); + + // Other columns should have correct min/max + assertThat(statistics.getNullCounts()[0]).isEqualTo(0L); + assertThat(statistics.getNullCounts()[1]).isEqualTo(0L); + assertThat(minValues.getInt(0)).isEqualTo(1); + assertThat(maxValues.getInt(0)).isEqualTo(5); + assertThat(minValues.getString(1)).isEqualTo(BinaryString.fromString("a")); + assertThat(maxValues.getString(1)).isEqualTo(BinaryString.fromString("e")); + } + + @Test + void testLargeDataset() throws IOException { + LogRecordBatchStatisticsCollector largeCollector = + new LogRecordBatchStatisticsCollector( + testRowType, + LogRecordBatchStatisticsTestUtils.createAllColumnsStatsMapping( + testRowType)); + + for (int i = 0; i < 1000; i++) { + Object[] data = {i, "row" + i, (double) i, i % 2 == 0, (long) i, (float) i}; + largeCollector.processRow(DataTestUtils.row(data)); + } + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + int bytesWritten = largeCollector.writeStatistics(new MemorySegmentOutputView(segment)); + assertThat(bytesWritten).isGreaterThan(0); + + DefaultLogRecordBatchStatistics statistics = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, testRowType, 1); + + // Verify min/max for large dataset + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(0); + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(999); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsParserTest.java b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsParserTest.java new file mode 100644 index 0000000000..f0b97393d0 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsParserTest.java @@ -0,0 +1,268 @@ +/* + * 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.record; + +import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.memory.MemorySegmentOutputView; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.indexed.IndexedRow; +import org.apache.fluss.row.indexed.IndexedRowWriter; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.IntType; +import org.apache.fluss.types.RowType; +import org.apache.fluss.types.StringType; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for LogRecordBatchStatisticsParser. */ +public class LogRecordBatchStatisticsParserTest { + + // Helper method to create stats index mapping for all columns + private static int[] createAllColumnsStatsMapping(RowType rowType) { + int[] statsIndexMapping = new int[rowType.getFieldCount()]; + for (int i = 0; i < statsIndexMapping.length; i++) { + statsIndexMapping[i] = i; + } + return statsIndexMapping; + } + + @Test + public void testParseStatisticsFromByteArray() throws Exception { + RowType rowType = DataTypes.ROW(new IntType(false), new StringType(false)); + + // Create test data + Long[] nullCounts = new Long[] {10L, 0L}; + IndexedRow minValues = createTestIndexedRow(rowType, -100, "aaa"); + IndexedRow maxValues = createTestIndexedRow(rowType, 999, "zzz"); + + // Write statistics using writer + LogRecordBatchStatisticsWriter writer = + new LogRecordBatchStatisticsWriter(rowType, createAllColumnsStatsMapping(rowType)); + byte[] writtenData = writeStatisticsForTest(writer, minValues, maxValues, nullCounts); + + // Parse using byte array + DefaultLogRecordBatchStatistics parsedStats = + LogRecordBatchStatisticsParser.parseStatistics( + MemorySegment.wrap(writtenData), 0, rowType, DEFAULT_SCHEMA_ID); + + // Verify parsing results - just basic validation + assertThat(parsedStats).isNotNull(); + assertThat(parsedStats.getNullCounts()).isNotNull(); + assertThat(parsedStats.getNullCounts()).hasSize(2); + } + + @Test + public void testParseStatisticsFromByteBuffer() throws Exception { + RowType rowType = DataTypes.ROW(new IntType(false), new StringType(false)); + + // Create test data + Long[] nullCounts = new Long[] {3L, 7L}; + IndexedRow minValues = createTestIndexedRow(rowType, 0, "first"); + IndexedRow maxValues = createTestIndexedRow(rowType, 500, "last"); + + // Write statistics using writer + LogRecordBatchStatisticsWriter writer = + new LogRecordBatchStatisticsWriter(rowType, createAllColumnsStatsMapping(rowType)); + byte[] writtenData = writeStatisticsForTest(writer, minValues, maxValues, nullCounts); + + // Parse using direct ByteBuffer + ByteBuffer buffer = ByteBuffer.allocateDirect(writtenData.length); + buffer.put(writtenData); + buffer.flip(); + DefaultLogRecordBatchStatistics parsedStats = + LogRecordBatchStatisticsParser.parseStatistics(buffer, rowType, DEFAULT_SCHEMA_ID); + + // Verify parsing results - just basic validation + assertThat(parsedStats).isNotNull(); + assertThat(parsedStats.getNullCounts()).isNotNull(); + assertThat(parsedStats.getNullCounts()).hasSize(2); + } + + @Test + public void testParseStatisticsFromMemorySegment() throws Exception { + RowType rowType = DataTypes.ROW(new IntType(false), new StringType(false)); + + // Create test data + Long[] nullCounts = new Long[] {1L, 2L}; + IndexedRow minValues = createTestIndexedRow(rowType, 42, "test"); + IndexedRow maxValues = createTestIndexedRow(rowType, 84, "testing"); + + // Write statistics using writer + LogRecordBatchStatisticsWriter writer = + new LogRecordBatchStatisticsWriter(rowType, createAllColumnsStatsMapping(rowType)); + byte[] writtenData = writeStatisticsForTest(writer, minValues, maxValues, nullCounts); + + // Parse using MemorySegment + MemorySegment segment = MemorySegment.wrap(writtenData); + DefaultLogRecordBatchStatistics parsedStats = + LogRecordBatchStatisticsParser.parseStatistics( + segment, 0, rowType, DEFAULT_SCHEMA_ID); + + // Verify parsing results - just basic validation + assertThat(parsedStats).isNotNull(); + assertThat(parsedStats.getNullCounts()).isNotNull(); + assertThat(parsedStats.getNullCounts()).hasSize(2); + } + + @Test + public void testIsValidStatistics() throws Exception { + RowType rowType = DataTypes.ROW(new IntType(false), new StringType(false)); + + // Create valid test statistics + Long[] nullCounts = new Long[] {1L, 2L}; + IndexedRow minValues = createTestIndexedRow(rowType, 1, "min"); + IndexedRow maxValues = createTestIndexedRow(rowType, 100, "max"); + + LogRecordBatchStatisticsWriter writer = + new LogRecordBatchStatisticsWriter(rowType, createAllColumnsStatsMapping(rowType)); + byte[] writtenData = writeStatisticsForTest(writer, minValues, maxValues, nullCounts); + + // Test with valid data + assertThat(LogRecordBatchStatisticsParser.isValidStatistics(writtenData, rowType)).isTrue(); + + // Test with direct ByteBuffer + ByteBuffer buffer = ByteBuffer.allocateDirect(writtenData.length); + buffer.put(writtenData); + buffer.flip(); + assertThat(LogRecordBatchStatisticsParser.isValidStatistics(buffer, rowType)).isTrue(); + + // Test with empty data + assertThat(LogRecordBatchStatisticsParser.isValidStatistics(new byte[0], rowType)) + .isFalse(); + + // Test with null data + assertThat(LogRecordBatchStatisticsParser.isValidStatistics((byte[]) null, rowType)) + .isFalse(); + assertThat(LogRecordBatchStatisticsParser.isValidStatistics((ByteBuffer) null, rowType)) + .isFalse(); + } + + @Test + public void testGetStatisticsSize() throws Exception { + RowType rowType = DataTypes.ROW(new IntType(false), new StringType(false)); + + // Create test statistics + Long[] nullCounts = new Long[] {1L, 2L}; + IndexedRow minValues = createTestIndexedRow(rowType, 1, "a"); + IndexedRow maxValues = createTestIndexedRow(rowType, 100, "z"); + + LogRecordBatchStatisticsWriter writer = + new LogRecordBatchStatisticsWriter(rowType, createAllColumnsStatsMapping(rowType)); + byte[] writtenData = writeStatisticsForTest(writer, minValues, maxValues, nullCounts); + + // Test with byte array + int size = LogRecordBatchStatisticsParser.getStatisticsSize(writtenData); + assertThat(size).isEqualTo(writtenData.length); + + // Test with direct ByteBuffer + ByteBuffer buffer = ByteBuffer.allocateDirect(writtenData.length); + buffer.put(writtenData); + buffer.flip(); + size = LogRecordBatchStatisticsParser.getStatisticsSize(buffer); + assertThat(size).isEqualTo(writtenData.length); + } + + @Test + public void testGetStatisticsSizeWithInvalidData() { + // Test with invalid data + assertThat(LogRecordBatchStatisticsParser.getStatisticsSize(ByteBuffer.allocateDirect(0))) + .isEqualTo(-1); + + // Test with null data + assertThat(LogRecordBatchStatisticsParser.getStatisticsSize((byte[]) null)).isEqualTo(-1); + assertThat(LogRecordBatchStatisticsParser.getStatisticsSize((ByteBuffer) null)) + .isEqualTo(-1); + } + + @Test + public void testBasicFunctionality() throws Exception { + // Test basic write and parse cycle without detailed verification + RowType rowType = DataTypes.ROW(new IntType(false)); + + Long[] nullCounts = new Long[] {0L}; + IndexedRow minValues = createTestIndexedRow(rowType, 1); + IndexedRow maxValues = createTestIndexedRow(rowType, 100); + + LogRecordBatchStatisticsWriter writer = + new LogRecordBatchStatisticsWriter(rowType, createAllColumnsStatsMapping(rowType)); + byte[] writtenData = writeStatisticsForTest(writer, minValues, maxValues, nullCounts); + + // Verify we can parse it back + DefaultLogRecordBatchStatistics parsedStats = + LogRecordBatchStatisticsParser.parseStatistics( + MemorySegment.wrap(writtenData), 0, rowType, DEFAULT_SCHEMA_ID); + + assertThat(parsedStats).isNotNull(); + assertThat(parsedStats.getNullCounts()).isNotNull(); + assertThat(parsedStats.hasMinValues()).isTrue(); + assertThat(parsedStats.hasMaxValues()).isTrue(); + } + + // Helper methods for creating test data + private IndexedRow createTestIndexedRow(RowType rowType, Object... values) throws IOException { + IndexedRowWriter writer = + new IndexedRowWriter(rowType.getChildren().toArray(new DataType[0])); + writer.reset(); + + for (int i = 0; i < values.length; i++) { + if (values[i] == null) { + writer.setNullAt(i); + } else { + DataType fieldType = rowType.getTypeAt(i); + if (fieldType instanceof StringType) { + // Convert String to BinaryString for StringType fields + writer.writeString(BinaryString.fromString((String) values[i])); + } else if (fieldType instanceof IntType) { + // Write int values directly + writer.writeInt((Integer) values[i]); + } + } + } + + IndexedRow row = new IndexedRow(rowType.getChildren().toArray(new DataType[0])); + row.pointTo(writer.segment(), 0, writer.position()); + return row; + } + + private byte[] writeStatisticsForTest( + LogRecordBatchStatisticsWriter writer, + InternalRow minValues, + InternalRow maxValues, + Long[] nullCounts) + throws IOException { + + // Allocate enough memory for statistics + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + int bytesWritten = + writer.writeStatistics( + minValues, maxValues, nullCounts, new MemorySegmentOutputView(segment)); + + // Copy the written data to a byte array + byte[] result = new byte[bytesWritten]; + segment.get(0, result, 0, bytesWritten); + return result; + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsTest.java b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsTest.java new file mode 100644 index 0000000000..89eeda925c --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsTest.java @@ -0,0 +1,461 @@ +/* + * 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.record; + +import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.memory.MemorySegmentOutputView; +import org.apache.fluss.metadata.LogFormat; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.aligned.AlignedRow; +import org.apache.fluss.testutils.DataTestUtils; +import org.apache.fluss.types.DataField; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.Optional; + +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V0; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V1; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; +import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; +import static org.apache.fluss.record.TestData.TEST_SCHEMA_GETTER; +import static org.apache.fluss.testutils.DataTestUtils.createRecordsWithoutBaseLogOffset; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * Test for getStatistics method of DefaultLogRecordBatch, FileChannelLogRecordBatch, and + * DefaultLogRecordBatchStatistics. + */ +public class LogRecordBatchStatisticsTest extends LogTestBase { + + private @TempDir File tempDir; + private static final int SCHEMA_ID = 1; + + // ==================== DefaultLogRecordBatch Tests ==================== + + @Test + void testDefaultLogRecordBatchGetStatisticsWithValidData() throws Exception { + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.STATISTICS_WITH_BOOLEAN_DATA, + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + 0L, + DEFAULT_SCHEMA_ID); + + LogRecordBatch batch = memoryLogRecords.batches().iterator().next(); + assertThat(batch).isInstanceOf(DefaultLogRecordBatch.class); + assertThat(batch.magic()).isEqualTo(LOG_MAGIC_VALUE_V2); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + DEFAULT_SCHEMA_ID, + TEST_SCHEMA_GETTER)) { + Optional statisticsOpt = batch.getStatistics(readContext); + assertThat(statisticsOpt).isPresent(); + + LogRecordBatchStatistics statistics = statisticsOpt.get(); + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(1); + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(5); + assertThat(statistics.getMinValues().getDouble(2)).isEqualTo(8.9); + assertThat(statistics.getMaxValues().getDouble(2)).isEqualTo(30.1); + assertThat(statistics.getMinValues().getBoolean(3)).isEqualTo(false); + assertThat(statistics.getMaxValues().getBoolean(3)).isEqualTo(true); + + // All null counts should be 0 + for (int i = 0; i < 4; i++) { + assertThat(statistics.getNullCounts()[i]).isEqualTo(0); + } + } + } + + @Test + void testDefaultLogRecordBatchGetStatisticsWithNullValues() throws Exception { + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.STATISTICS_WITH_NULLS_DATA, + TestData.STATISTICS_WITH_NULLS_ROW_TYPE, + 0L, + DEFAULT_SCHEMA_ID); + + LogRecordBatch batch = memoryLogRecords.batches().iterator().next(); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + TestData.STATISTICS_WITH_NULLS_ROW_TYPE, + DEFAULT_SCHEMA_ID, + TEST_SCHEMA_GETTER)) { + LogRecordBatchStatistics statistics = batch.getStatistics(readContext).get(); + assertThat(statistics.getNullCounts()[0]).isEqualTo(1); + assertThat(statistics.getNullCounts()[1]).isEqualTo(1); + assertThat(statistics.getNullCounts()[2]).isEqualTo(1); + } + } + + @Test + void testDefaultLogRecordBatchGetStatisticsEdgeCases() throws Exception { + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.STATISTICS_WITH_BOOLEAN_DATA, + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + 0L, + DEFAULT_SCHEMA_ID); + + LogRecordBatch batch = memoryLogRecords.batches().iterator().next(); + + // Test with null context + assertThat(batch.getStatistics(null)).isEmpty(); + + // Test with invalid schema ID + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, 999, TEST_SCHEMA_GETTER)) { + assertThat(batch.getStatistics(readContext)).isEmpty(); + } + } + + @ParameterizedTest + @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1}) + void testDefaultLogRecordBatchGetStatisticsWithOldMagicVersions(byte magic) throws Exception { + MemoryLogRecords memoryLogRecords = + createRecordsWithoutBaseLogOffset( + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + DEFAULT_SCHEMA_ID, + 0L, + -1L, + magic, + Collections.singletonList(TestData.STATISTICS_WITH_BOOLEAN_DATA.get(0)), + LogFormat.ARROW); + + LogRecordBatch batch = memoryLogRecords.batches().iterator().next(); + assertThat(batch.magic()).isEqualTo(magic); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + DEFAULT_SCHEMA_ID, + TEST_SCHEMA_GETTER)) { + assertThat(batch.getStatistics(readContext)).isEmpty(); + } + } + + @Test + void testDefaultLogRecordBatchGetStatisticsCaching() throws Exception { + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.STATISTICS_WITH_BOOLEAN_DATA, + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + 0L, + DEFAULT_SCHEMA_ID); + + LogRecordBatch batch = memoryLogRecords.batches().iterator().next(); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + DEFAULT_SCHEMA_ID, + TEST_SCHEMA_GETTER)) { + Optional stats1 = batch.getStatistics(readContext); + Optional stats2 = batch.getStatistics(readContext); + assertThat(stats2.get()).isSameAs(stats1.get()); + } + } + + // ==================== FileChannelLogRecordBatch Tests ==================== + + @Test + void testFileChannelLogRecordBatchGetStatisticsWithValidData() throws Exception { + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.STATISTICS_WITH_BOOLEAN_DATA, + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + 0L, + DEFAULT_SCHEMA_ID); + + try (FileLogRecords fileLogRecords = FileLogRecords.open(new File(tempDir, "test.tmp"))) { + fileLogRecords.append(memoryLogRecords); + fileLogRecords.flush(); + + FileLogInputStream logInputStream = + new FileLogInputStream(fileLogRecords, 0, fileLogRecords.sizeInBytes()); + FileLogInputStream.FileChannelLogRecordBatch batch = logInputStream.nextBatch(); + assertThat(batch.magic()).isEqualTo(LOG_MAGIC_VALUE_V2); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + DEFAULT_SCHEMA_ID, + TEST_SCHEMA_GETTER)) { + LogRecordBatchStatistics statistics = batch.getStatistics(readContext).get(); + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(1); + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(5); + } + } + } + + @Test + void testDefaultVsFileChannelLogRecordBatchStatisticsConsistency() throws Exception { + MemoryLogRecords memoryLogRecords = + LogRecordBatchStatisticsTestUtils.createLogRecordsWithStatistics( + TestData.STATISTICS_WITH_BOOLEAN_DATA, + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + 0L, + DEFAULT_SCHEMA_ID); + + LogRecordBatch defaultBatch = memoryLogRecords.batches().iterator().next(); + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + TestData.STATISTICS_WITH_BOOLEAN_ROW_TYPE, + DEFAULT_SCHEMA_ID, + TEST_SCHEMA_GETTER)) { + LogRecordBatchStatistics defaultStats = defaultBatch.getStatistics(readContext).get(); + + try (FileLogRecords fileLogRecords = + FileLogRecords.open(new File(tempDir, "consistency.tmp"))) { + fileLogRecords.append(memoryLogRecords); + fileLogRecords.flush(); + + FileLogInputStream logInputStream = + new FileLogInputStream(fileLogRecords, 0, fileLogRecords.sizeInBytes()); + FileLogInputStream.FileChannelLogRecordBatch fileBatch = logInputStream.nextBatch(); + LogRecordBatchStatistics fileStats = fileBatch.getStatistics(readContext).get(); + + // Verify both implementations return the same statistics + assertThat(fileStats.getMinValues().getInt(0)) + .isEqualTo(defaultStats.getMinValues().getInt(0)); + assertThat(fileStats.getMaxValues().getInt(0)) + .isEqualTo(defaultStats.getMaxValues().getInt(0)); + + for (int i = 0; i < fileStats.getNullCounts().length; i++) { + assertThat(fileStats.getNullCounts()[i]) + .isEqualTo(defaultStats.getNullCounts()[i]); + } + } + } + } + + // ==================== DefaultLogRecordBatchStatistics Tests ==================== + + @Test + void testDefaultLogRecordBatchStatisticsBasicMethods() { + MemorySegment segment = MemorySegment.allocateHeapMemory(512); + Long[] nullCounts = new Long[] {0L, 1L, 2L}; + int[] statsIndexMapping = new int[] {0, 1, 2}; + + DefaultLogRecordBatchStatistics stats = + new DefaultLogRecordBatchStatistics( + segment, + 0, + 100, + TestData.STATISTICS_BASIC_ROW_TYPE, + SCHEMA_ID, + nullCounts, + 50, + 80, + 20, + 15, + statsIndexMapping); + + assertThat(stats.getSegment()).isEqualTo(segment); + assertThat(stats.getSchemaId()).isEqualTo(SCHEMA_ID); + assertThat(stats.getNullCounts()).isEqualTo(nullCounts); + assertThat(stats.hasMinValues()).isTrue(); + assertThat(stats.hasMaxValues()).isTrue(); + assertThat(stats.hasColumnStatistics(0)).isTrue(); + assertThat(stats.getNullCounts()[0]).isEqualTo(0L); + assertThat(stats.getNullCounts()[1]).isEqualTo(1L); + } + + @Test + void testDefaultLogRecordBatchStatisticsPartialColumns() { + MemorySegment segment = MemorySegment.allocateHeapMemory(512); + Long[] nullCounts = new Long[] {0L, 2L}; + int[] statsIndexMapping = new int[] {0, 2}; // Skip column 1 + + DefaultLogRecordBatchStatistics stats = + new DefaultLogRecordBatchStatistics( + segment, + 0, + 100, + TestData.STATISTICS_BASIC_ROW_TYPE, + SCHEMA_ID, + nullCounts, + 50, + 80, + 20, + 15, + statsIndexMapping); + + assertThat(stats.hasColumnStatistics(0)).isTrue(); + assertThat(stats.hasColumnStatistics(1)).isFalse(); + assertThat(stats.hasColumnStatistics(2)).isTrue(); + } + + @Test + void testDefaultLogRecordBatchStatisticsNoMinMaxValues() { + MemorySegment segment = MemorySegment.allocateHeapMemory(512); + Long[] nullCounts = new Long[] {0L, 1L, 2L}; + int[] statsIndexMapping = new int[] {0, 1, 2}; + + assertThatThrownBy( + () -> + new DefaultLogRecordBatchStatistics( + segment, + 0, + 50, + TestData.STATISTICS_BASIC_ROW_TYPE, + SCHEMA_ID, + nullCounts, + 0, + 0, + 0, + 10, + statsIndexMapping)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("minValuesSize must be positive"); + + assertThatThrownBy( + () -> + new DefaultLogRecordBatchStatistics( + segment, + 0, + 50, + TestData.STATISTICS_BASIC_ROW_TYPE, + SCHEMA_ID, + nullCounts, + 0, + 0, + 10, + 0, + statsIndexMapping)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("maxValuesSize must be positive"); + } + + @Test + void testDefaultLogRecordBatchStatisticsWithSerializedData() throws IOException { + InternalRow minRow = DataTestUtils.row(new Object[] {1, "a", 10.5}); + InternalRow maxRow = DataTestUtils.row(new Object[] {100, "z", 99.9}); + Long[] nullCounts = new Long[] {0L, 2L, 1L}; + int[] statsIndexMapping = new int[] {0, 1, 2}; + + LogRecordBatchStatisticsWriter writer = + new LogRecordBatchStatisticsWriter( + TestData.STATISTICS_BASIC_ROW_TYPE, statsIndexMapping); + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + MemorySegmentOutputView outputView = new MemorySegmentOutputView(segment); + + writer.writeStatistics( + AlignedRow.from(TestData.STATISTICS_BASIC_ROW_TYPE, minRow), + AlignedRow.from(TestData.STATISTICS_BASIC_ROW_TYPE, maxRow), + nullCounts, + outputView); + + DefaultLogRecordBatchStatistics parsedStats = + LogRecordBatchStatisticsParser.parseStatistics( + segment, 0, TestData.STATISTICS_BASIC_ROW_TYPE, SCHEMA_ID); + + assertThat(parsedStats.getSchemaId()).isEqualTo(SCHEMA_ID); + assertThat(parsedStats.getNullCounts()).isEqualTo(nullCounts); + assertThat(parsedStats.getMinValues().getInt(0)).isEqualTo(1); + assertThat(parsedStats.getMaxValues().getInt(0)).isEqualTo(100); + } + + @Test + void testPartialStatisticsWrapperThrowsForUnavailableColumns() throws IOException { + InternalRow minRow = DataTestUtils.row(1, 10.5); + InternalRow maxRow = DataTestUtils.row(100, 99.9); + Long[] nullCounts = new Long[] {0L, 1L}; + int[] statsIndexMapping = new int[] {0, 2}; // Skip column 1 + + RowType fullRowType = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE())); + + LogRecordBatchStatisticsWriter writer = + new LogRecordBatchStatisticsWriter(fullRowType, statsIndexMapping); + + MemorySegment segment = MemorySegment.allocateHeapMemory(1024); + MemorySegmentOutputView outputView = new MemorySegmentOutputView(segment); + + writer.writeStatistics(minRow, maxRow, nullCounts, outputView); + + DefaultLogRecordBatchStatistics parsedStats = + LogRecordBatchStatisticsParser.parseStatistics(segment, 0, fullRowType, 0); + + // Valid accesses work + assertThat(parsedStats.getMinValues().getInt(0)).isEqualTo(1); + assertThat(parsedStats.getMinValues().getDouble(2)).isEqualTo(10.5); + + // Accessing unavailable column throws + assertThatThrownBy(() -> parsedStats.getMinValues().getString(1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Column index not available"); + } + + @Test + void testDefaultLogRecordBatchStatisticsEqualsAndHashCode() { + MemorySegment segment = MemorySegment.allocateHeapMemory(512); + Long[] nullCounts = new Long[] {0L, 1L, 2L}; + int[] statsIndexMapping = new int[] {0, 1, 2}; + + DefaultLogRecordBatchStatistics stats1 = + new DefaultLogRecordBatchStatistics( + segment, + 0, + 100, + TestData.STATISTICS_BASIC_ROW_TYPE, + SCHEMA_ID, + nullCounts, + 50, + 80, + 20, + 15, + statsIndexMapping); + + DefaultLogRecordBatchStatistics stats2 = + new DefaultLogRecordBatchStatistics( + segment, + 0, + 100, + TestData.STATISTICS_BASIC_ROW_TYPE, + SCHEMA_ID, + nullCounts, + 50, + 80, + 20, + 15, + statsIndexMapping); + + assertThat(stats1).isEqualTo(stats2); + assertThat(stats1.hashCode()).isEqualTo(stats2.hashCode()); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsTestUtils.java b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsTestUtils.java new file mode 100644 index 0000000000..5e42b39c2b --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/record/LogRecordBatchStatisticsTestUtils.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.record; + +import org.apache.fluss.memory.ManagedPagedOutputView; +import org.apache.fluss.memory.TestingMemorySegmentPool; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.arrow.ArrowWriter; +import org.apache.fluss.row.arrow.ArrowWriterPool; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.RootAllocator; +import org.apache.fluss.testutils.DataTestUtils; +import org.apache.fluss.types.RowType; + +import java.util.List; + +import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; +import static org.apache.fluss.record.LogRecordBatch.CURRENT_LOG_MAGIC_VALUE; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; + +/** + * Test utility class for generating LogRecordBatch with statistics. This utility class can be + * reused in other test cases. + */ +public class LogRecordBatchStatisticsTestUtils { + + // Helper method to create stats index mapping for all columns + public static int[] createAllColumnsStatsMapping(RowType rowType) { + int[] statsIndexMapping = new int[rowType.getFieldCount()]; + for (int i = 0; i < statsIndexMapping.length; i++) { + statsIndexMapping[i] = i; + } + return statsIndexMapping; + } + + /** + * Create a reusable utility method for generating LogRecordBatch with statistics. This method + * can be reused in other test cases. + * + * @param data Test data + * @param rowType Row type + * @param baseOffset Base offset + * @param schemaId Schema ID + * @return MemoryLogRecords containing statistics + * @throws Exception If creation fails + */ + public static MemoryLogRecords createLogRecordsWithStatistics( + List data, RowType rowType, long baseOffset, int schemaId) throws Exception { + + try (BufferAllocator allocator = new RootAllocator(Long.MAX_VALUE); + ArrowWriterPool writerPool = new ArrowWriterPool(allocator)) { + + ArrowWriter writer = + writerPool.getOrCreateWriter( + 1L, schemaId, Integer.MAX_VALUE, rowType, DEFAULT_COMPRESSION); + + // Create statistics collector for the writer's schema + LogRecordBatchStatisticsCollector statisticsCollector = + new LogRecordBatchStatisticsCollector( + writer.getSchema(), createAllColumnsStatsMapping(writer.getSchema())); + + MemoryLogRecordsArrowBuilder builder = + MemoryLogRecordsArrowBuilder.builder( + baseOffset, + LOG_MAGIC_VALUE_V2, + schemaId, + writer, + new ManagedPagedOutputView(new TestingMemorySegmentPool(10 * 1024)), + statisticsCollector); + + // Convert data to InternalRow and add to builder + List rows = + data.stream() + .map(DataTestUtils::row) + .collect(java.util.stream.Collectors.toList()); + for (InternalRow row : rows) { + builder.append(ChangeType.APPEND_ONLY, row); + } + + builder.setWriterState(1L, 0); + builder.close(); + + MemoryLogRecords memoryLogRecords = MemoryLogRecords.pointToBytesView(builder.build()); + memoryLogRecords.ensureValid(CURRENT_LOG_MAGIC_VALUE); + + return memoryLogRecords; + } + } + + /** + * Create a reusable utility method for generating LogRecordBatch with statistics (using default + * parameters). + * + * @param data Test data + * @param rowType Row type + * @return MemoryLogRecords containing statistics + * @throws Exception If creation fails + */ + public static MemoryLogRecords createLogRecordsWithStatistics( + List data, RowType rowType) throws Exception { + return createLogRecordsWithStatistics(data, rowType, 0L, 1); + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/record/MemoryLogRecordsArrowBuilderTest.java b/fluss-common/src/test/java/org/apache/fluss/record/MemoryLogRecordsArrowBuilderTest.java index e72b44f140..63b3f52dbb 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/MemoryLogRecordsArrowBuilderTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/MemoryLogRecordsArrowBuilderTest.java @@ -30,6 +30,9 @@ import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.RootAllocator; import org.apache.fluss.testutils.DataTestUtils; +import org.apache.fluss.types.DataField; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; import org.apache.commons.lang3.RandomUtils; @@ -48,6 +51,7 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; @@ -55,6 +59,8 @@ import static org.apache.fluss.record.LogRecordBatch.CURRENT_LOG_MAGIC_VALUE; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V0; import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V1; +import static org.apache.fluss.record.LogRecordBatchFormat.LOG_MAGIC_VALUE_V2; +import static org.apache.fluss.record.LogRecordBatchStatisticsTestUtils.createAllColumnsStatsMapping; import static org.apache.fluss.record.TestData.DATA1; import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; @@ -108,7 +114,7 @@ void testAppendWithEmptyRecord(byte recordBatchMagic, int expectedBatchSize) thr } @ParameterizedTest - @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1}) + @ValueSource(bytes = {LOG_MAGIC_VALUE_V0, LOG_MAGIC_VALUE_V1, LOG_MAGIC_VALUE_V2}) void testAppend(byte recordBatchMagic) throws Exception { int maxSizeInBytes = 1024; ArrowWriter writer = @@ -351,6 +357,297 @@ void testResetWriterState() throws Exception { assertThat(recordBatch.batchSequence()).isEqualTo(1); } + @Test + void testStatisticsWriteAndRead() throws Exception { + // Create test data with different data types to test statistics collection + List testData = + Arrays.asList( + new Object[] {1, "a", 10.5}, + new Object[] {2, "b", 20.7}, + new Object[] {3, "c", 15.2}, + new Object[] {4, "d", 30.1}, + new Object[] {5, "e", 8.9}); + + // Create row type for test data + RowType testRowType = + new RowType( + Arrays.asList( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE()))); + + // Create ArrowWriter and builder + ArrowWriter writer = + provider.getOrCreateWriter( + 1L, DEFAULT_SCHEMA_ID, 1024 * 10, testRowType, NO_COMPRESSION); + MemoryLogRecordsArrowBuilder builder = + createMemoryLogRecordsArrowBuilder(0, writer, 10, 1024 * 10, LOG_MAGIC_VALUE_V2); + + // Append test data + List changeTypes = + testData.stream().map(row -> ChangeType.APPEND_ONLY).collect(Collectors.toList()); + List rows = + testData.stream().map(DataTestUtils::row).collect(Collectors.toList()); + + for (int i = 0; i < testData.size(); i++) { + builder.append(changeTypes.get(i), rows.get(i)); + } + + // Set writer state and close + builder.setWriterState(1L, 0); + builder.close(); + + // Build and create MemoryLogRecords + MemoryLogRecords records = MemoryLogRecords.pointToBytesView(builder.build()); + + // Verify basic properties + assertThat(records.sizeInBytes()).isGreaterThan(0); + Iterator iterator = records.batches().iterator(); + assertThat(iterator.hasNext()).isTrue(); + LogRecordBatch batch = iterator.next(); + assertThat(iterator.hasNext()).isFalse(); + + // Verify batch properties + assertThat(batch.getRecordCount()).isEqualTo(testData.size()); + assertThat(batch.baseLogOffset()).isEqualTo(0); + assertThat(batch.lastLogOffset()).isEqualTo(testData.size() - 1); + assertThat(batch.nextLogOffset()).isEqualTo(testData.size()); + assertThat(batch.writerId()).isEqualTo(1L); + assertThat(batch.batchSequence()).isEqualTo(0); + assertThat(batch.magic()).isEqualTo(LogRecordBatchFormat.LOG_MAGIC_VALUE_V2); + + // Create read context + LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + testRowType, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER); + + // Test statistics reading + Optional statisticsOpt = batch.getStatistics(readContext); + assertThat(statisticsOpt).isPresent(); + + LogRecordBatchStatistics statistics = statisticsOpt.get(); + assertThat(statistics.getMinValues()).isNotNull(); + assertThat(statistics.getMaxValues()).isNotNull(); + assertThat(statistics.getNullCounts()).isNotNull(); + + // Verify statistics for each field + // Field 0: id (INT) + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(1); // min id + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(5); // max id + assertThat(statistics.getNullCounts()[0]).isEqualTo(0); // no nulls + + // Field 1: name (STRING) + assertThat(statistics.getMinValues().getString(1).toString()).isEqualTo("a"); // min name + assertThat(statistics.getMaxValues().getString(1).toString()).isEqualTo("e"); // max name + assertThat(statistics.getNullCounts()[1]).isEqualTo(0); // no nulls + + // Field 2: value (DOUBLE) + assertThat(statistics.getMinValues().getDouble(2)).isEqualTo(8.9); // min value + assertThat(statistics.getMaxValues().getDouble(2)).isEqualTo(30.1); // max value + assertThat(statistics.getNullCounts()[2]).isEqualTo(0); // no nulls + + // Test record reading and verify data integrity + try (CloseableIterator recordIterator = batch.records(readContext)) { + assertThat(recordIterator.hasNext()).isTrue(); + int recordCount = 0; + while (recordIterator.hasNext()) { + LogRecord record = recordIterator.next(); + assertThat(record).isNotNull(); + assertThat(record.getChangeType()).isEqualTo(ChangeType.APPEND_ONLY); + assertThat(record.logOffset()).isEqualTo(recordCount); + + InternalRow row = record.getRow(); + assertThat(row).isNotNull(); + assertThat(row.getFieldCount()).isEqualTo(3); + + // Verify data matches original test data + Object[] originalData = testData.get(recordCount); + assertThat(row.getInt(0)).isEqualTo(originalData[0]); // id + assertThat(row.getString(1).toString()).isEqualTo(originalData[1]); // name + assertThat(row.getDouble(2)).isEqualTo(originalData[2]); // value + + recordCount++; + } + assertThat(recordCount).isEqualTo(testData.size()); + } + + // Close read context + readContext.close(); + } + + @Test + void testStatisticsWithNullValues() throws Exception { + // Create test data with null values + List testData = + Arrays.asList( + new Object[] {1, "a", 10.5}, + new Object[] {null, "b", 20.7}, + new Object[] {3, null, 15.2}, + new Object[] {4, "d", null}, + new Object[] {5, "e", 8.9}); + + // Create row type for test data + RowType testRowType = + new RowType( + Arrays.asList( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE()))); + + // Create ArrowWriter and builder + ArrowWriter writer = + provider.getOrCreateWriter( + 1L, DEFAULT_SCHEMA_ID, 1024 * 10, testRowType, NO_COMPRESSION); + MemoryLogRecordsArrowBuilder builder = + createMemoryLogRecordsArrowBuilder(0, writer, 10, 1024 * 10, LOG_MAGIC_VALUE_V2); + + // Append test data + List changeTypes = + testData.stream().map(row -> ChangeType.APPEND_ONLY).collect(Collectors.toList()); + List rows = + testData.stream().map(DataTestUtils::row).collect(Collectors.toList()); + + for (int i = 0; i < testData.size(); i++) { + builder.append(changeTypes.get(i), rows.get(i)); + } + + // Set writer state and close + builder.setWriterState(1L, 0); + builder.close(); + + // Build and create MemoryLogRecords + MemoryLogRecords records = MemoryLogRecords.pointToBytesView(builder.build()); + + // Get the batch + LogRecordBatch batch = records.batches().iterator().next(); + + // Create read context + LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + testRowType, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER); + + // Test statistics reading + Optional statisticsOpt = batch.getStatistics(readContext); + assertThat(statisticsOpt).isPresent(); + + LogRecordBatchStatistics statistics = statisticsOpt.get(); + + // Verify null counts + assertThat(statistics.getNullCounts()[0]).isEqualTo(1); // one null in id field + assertThat(statistics.getNullCounts()[1]).isEqualTo(1); // one null in name field + assertThat(statistics.getNullCounts()[2]).isEqualTo(1); // one null in value field + + // Verify min/max values (should exclude nulls) + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(1); // min id (excluding null) + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(5); // max id (excluding null) + assertThat(statistics.getMinValues().getDouble(2)) + .isEqualTo(8.9); // min value (excluding null) + assertThat(statistics.getMaxValues().getDouble(2)) + .isEqualTo(20.7); // max value (excluding null) + + // Close read context + readContext.close(); + } + + @Test + void testStatisticsWithDifferentChangeTypes() throws Exception { + // Create test data with different change types + List testData = + Arrays.asList(new Object[] {1, "a"}, new Object[] {2, "b"}, new Object[] {3, "c"}); + + List changeTypes = + Arrays.asList(ChangeType.APPEND_ONLY, ChangeType.UPDATE_AFTER, ChangeType.DELETE); + + // Create row type for test data + RowType testRowType = + new RowType( + Arrays.asList( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()))); + + // Create ArrowWriter and builder (non-append-only mode) + ArrowWriter writer = + provider.getOrCreateWriter( + 1L, DEFAULT_SCHEMA_ID, 1024 * 10, testRowType, NO_COMPRESSION); + + // Create statistics collector for the writer's schema + LogRecordBatchStatisticsCollector statisticsCollector = + new LogRecordBatchStatisticsCollector( + writer.getSchema(), + LogRecordBatchStatisticsTestUtils.createAllColumnsStatsMapping( + writer.getSchema())); + + MemoryLogRecordsArrowBuilder builder = + MemoryLogRecordsArrowBuilder.builder( + 0, + LOG_MAGIC_VALUE_V2, + DEFAULT_SCHEMA_ID, + writer, + new ManagedPagedOutputView(new TestingMemorySegmentPool(1024 * 10)), + statisticsCollector); + + // Append test data with different change types + List rows = + testData.stream().map(DataTestUtils::row).collect(Collectors.toList()); + + for (int i = 0; i < testData.size(); i++) { + builder.append(changeTypes.get(i), rows.get(i)); + } + + // Set writer state and close + builder.setWriterState(1L, 0); + builder.close(); + + // Build and create MemoryLogRecords + MemoryLogRecords records = MemoryLogRecords.pointToBytesView(builder.build()); + + // Get the batch + LogRecordBatch batch = records.batches().iterator().next(); + + // Create read context + LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + testRowType, DEFAULT_SCHEMA_ID, TEST_SCHEMA_GETTER); + + // Test statistics reading + Optional statisticsOpt = batch.getStatistics(readContext); + assertThat(statisticsOpt).isPresent(); + + LogRecordBatchStatistics statistics = statisticsOpt.get(); + + // Verify statistics + assertThat(statistics.getMinValues().getInt(0)).isEqualTo(1); + assertThat(statistics.getMaxValues().getInt(0)).isEqualTo(3); + assertThat(statistics.getNullCounts()[0]).isEqualTo(0); + + // Test record reading and verify change types + try (CloseableIterator recordIterator = batch.records(readContext)) { + assertThat(recordIterator.hasNext()).isTrue(); + int recordCount = 0; + while (recordIterator.hasNext()) { + LogRecord record = recordIterator.next(); + assertThat(record).isNotNull(); + assertThat(record.getChangeType()).isEqualTo(changeTypes.get(recordCount)); + assertThat(record.logOffset()).isEqualTo(recordCount); + + InternalRow row = record.getRow(); + assertThat(row).isNotNull(); + assertThat(row.getFieldCount()).isEqualTo(2); + + // Verify data matches original test data + Object[] originalData = testData.get(recordCount); + assertThat(row.getInt(0)).isEqualTo(originalData[0]); // id + assertThat(row.getString(1).toString()).isEqualTo(originalData[1]); // name + + recordCount++; + } + assertThat(recordCount).isEqualTo(testData.size()); + } + + // Close read context + readContext.close(); + } + private static List compressionInfos() { return Arrays.asList( new ArrowCompressionInfo(ArrowCompressionType.LZ4_FRAME, -1), @@ -362,6 +659,7 @@ private static Collection magicAndExpectedBatchSize() { List params = new ArrayList<>(); params.add(Arguments.arguments(LOG_MAGIC_VALUE_V0, 48)); params.add(Arguments.arguments(LOG_MAGIC_VALUE_V1, 52)); + params.add(Arguments.arguments(LOG_MAGIC_VALUE_V2, 56)); return params; } @@ -377,11 +675,18 @@ private MemoryLogRecordsArrowBuilder createMemoryLogRecordsArrowBuilder( new MemorySize((long) maxPages * pageSizeInBytes)); conf.set(ConfigOptions.CLIENT_WRITER_BUFFER_PAGE_SIZE, new MemorySize(pageSizeInBytes)); conf.set(ConfigOptions.CLIENT_WRITER_BATCH_SIZE, new MemorySize(pageSizeInBytes)); + + // Create statistics collector for the writer's schema + LogRecordBatchStatisticsCollector statisticsCollector = + new LogRecordBatchStatisticsCollector( + writer.getSchema(), createAllColumnsStatsMapping(writer.getSchema())); + return MemoryLogRecordsArrowBuilder.builder( baseOffset, recordBatchMagic, DEFAULT_SCHEMA_ID, writer, - new ManagedPagedOutputView(new TestingMemorySegmentPool(pageSizeInBytes))); + new ManagedPagedOutputView(new TestingMemorySegmentPool(pageSizeInBytes)), + statisticsCollector); } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java index 28ab9ececc..1925aadb86 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java @@ -260,6 +260,94 @@ public final class TestData { // ---------------------------- data3 table info end ------------------------------ + // ------------------- Statistics test data and schemas begin ---------------------- + // Schema with mixed types for comprehensive statistics testing + public static final RowType STATISTICS_MIXED_TYPE_ROW_TYPE = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE()), + new DataField("flag", DataTypes.BOOLEAN()), + new DataField("bigint_val", DataTypes.BIGINT()), + new DataField("float_val", DataTypes.FLOAT())); + + public static final List STATISTICS_MIXED_TYPE_DATA = + Arrays.asList( + new Object[] {1, "a", 10.5, true, 100L, 1.23f}, + new Object[] {2, "b", 20.3, false, 200L, 2.34f}, + new Object[] {3, "c", 15.7, true, 150L, 3.45f}, + new Object[] {4, "d", 8.9, false, 300L, 4.56f}, + new Object[] {5, "e", 30.1, true, 250L, 5.67f}); + + // Schema with basic types for statistics testing + public static final RowType STATISTICS_BASIC_ROW_TYPE = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE())); + + public static final List STATISTICS_BASIC_DATA = + Arrays.asList( + new Object[] {1, "a", 10.5}, + new Object[] {2, "b", 20.3}, + new Object[] {3, "c", 15.7}, + new Object[] {4, "d", 8.9}, + new Object[] {5, "e", 30.1}); + + // Schema with all common types including boolean + public static final RowType STATISTICS_WITH_BOOLEAN_ROW_TYPE = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE()), + new DataField("flag", DataTypes.BOOLEAN())); + + public static final List STATISTICS_WITH_BOOLEAN_DATA = + Arrays.asList( + new Object[] {1, "a", 10.5, true}, + new Object[] {2, "b", 20.3, false}, + new Object[] {3, "c", 15.7, true}, + new Object[] {4, "d", 8.9, false}, + new Object[] {5, "e", 30.1, true}); + + // Schema with nullable columns for statistics testing + public static final RowType STATISTICS_WITH_NULLS_ROW_TYPE = + DataTypes.ROW( + new DataField("id", DataTypes.INT()), + new DataField("name", DataTypes.STRING()), + new DataField("value", DataTypes.DOUBLE())); + + public static final List STATISTICS_WITH_NULLS_DATA = + Arrays.asList( + new Object[] {1, "a", 10.5}, + new Object[] {null, "b", 20.3}, + new Object[] {3, null, 15.7}, + new Object[] {4, "d", null}, + new Object[] {5, "e", 30.1}); + + // Schema for filter testing + public static final RowType FILTER_TEST_ROW_TYPE = + DataTypes.ROW( + DataTypes.FIELD("id", DataTypes.BIGINT()), + DataTypes.FIELD("name", DataTypes.STRING()), + DataTypes.FIELD("score", DataTypes.DOUBLE())); + + public static final Schema FILTER_TEST_SCHEMA = + Schema.newBuilder() + .fromColumns( + Arrays.asList( + new Schema.Column("id", DataTypes.BIGINT(), null, 1), + new Schema.Column("name", DataTypes.STRING(), null, 2), + new Schema.Column("score", DataTypes.DOUBLE(), null, 3))) + .build(); + + public static final List FILTER_TEST_DATA = + Arrays.asList( + new Object[] {1L, "a", 1.0}, + new Object[] {5L, "m", 50.0}, + new Object[] {10L, "z", 100.0}); + // ------------------- Statistics test data and schemas end ------------------------ + public static final TestingSchemaGetter TEST_SCHEMA_GETTER = new TestingSchemaGetter(DEFAULT_SCHEMA_ID, DATA2_SCHEMA); } diff --git a/fluss-common/src/test/java/org/apache/fluss/row/aligned/AlignedRowTest.java b/fluss-common/src/test/java/org/apache/fluss/row/aligned/AlignedRowTest.java index cb79d559bd..ea3d87138a 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/aligned/AlignedRowTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/aligned/AlignedRowTest.java @@ -29,6 +29,7 @@ import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DataTypes; +import org.apache.fluss.types.RowType; import org.junit.jupiter.api.Test; @@ -44,6 +45,7 @@ import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.ALIGNED; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link AlignedRow}. */ class AlignedRowTest { @@ -816,4 +818,266 @@ public void testAlignedArrayGetNestedArray() { assertThat(nestedArray2.getInt(1)).isEqualTo(50); assertThat(nestedArray2.getInt(2)).isEqualTo(60); } + + @Test + public void testSingleColumnInteger() { + // non-null integer + AlignedRow row = AlignedRow.singleColumn(42); + assertThat(row.getFieldCount()).isEqualTo(1); + assertThat(row.isNullAt(0)).isFalse(); + assertThat(row.getInt(0)).isEqualTo(42); + + // null integer + AlignedRow nullRow = AlignedRow.singleColumn((Integer) null); + assertThat(nullRow.getFieldCount()).isEqualTo(1); + assertThat(nullRow.isNullAt(0)).isTrue(); + } + + @Test + public void testSingleColumnBinaryString() { + // non-null BinaryString + BinaryString binaryString = BinaryString.fromString("binary"); + AlignedRow row = AlignedRow.singleColumn(binaryString); + assertThat(row.getFieldCount()).isEqualTo(1); + assertThat(row.isNullAt(0)).isFalse(); + assertThat(row.getString(0)).isEqualTo(binaryString); + + // null BinaryString + AlignedRow nullRow = AlignedRow.singleColumn((BinaryString) null); + assertThat(nullRow.getFieldCount()).isEqualTo(1); + assertThat(nullRow.isNullAt(0)).isTrue(); + } + + @Test + public void testIsInFixedLengthPart() { + // fixed-length types + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.BOOLEAN())).isTrue(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.TINYINT())).isTrue(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.SMALLINT())).isTrue(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.INT())).isTrue(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.DATE())).isTrue(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.TIME())).isTrue(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.BIGINT())).isTrue(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.FLOAT())).isTrue(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.DOUBLE())).isTrue(); + + // compact decimal (precision <= 18) + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.DECIMAL(10, 2))).isTrue(); + // non-compact decimal (precision > 18) + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.DECIMAL(25, 5))).isFalse(); + + // compact timestamp (precision <= 3) + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.TIMESTAMP(3))).isTrue(); + // non-compact timestamp (precision > 3) + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.TIMESTAMP(9))).isFalse(); + + // compact timestamp_ltz (precision <= 3) + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.TIMESTAMP_LTZ(3))).isTrue(); + // non-compact timestamp_ltz (precision > 3) + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.TIMESTAMP_LTZ(9))).isFalse(); + + // variable-length types + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.STRING())).isFalse(); + assertThat(AlignedRow.isInFixedLengthPart(DataTypes.BYTES())).isFalse(); + } + + @Test + public void testFromWithNullInput() { + RowType rowType = RowType.of(DataTypes.INT(), DataTypes.STRING()); + assertThat(AlignedRow.from(rowType, null)).isNull(); + } + + @Test + public void testFromWithAlignedRowInput() { + RowType rowType = RowType.of(DataTypes.INT(), DataTypes.STRING()); + + AlignedRow original = new AlignedRow(2); + AlignedRowWriter writer = new AlignedRowWriter(original); + writer.writeInt(0, 42); + writer.writeString(1, BinaryString.fromString("pass-through")); + writer.complete(); + + // should return the same instance + AlignedRow result = AlignedRow.from(rowType, original); + assertThat(result).isSameAs(original); + } + + @Test + public void testFromWithGenericRowAllTypes() { + RowType rowType = + RowType.of( + DataTypes.BOOLEAN(), + DataTypes.TINYINT(), + DataTypes.SMALLINT(), + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.STRING(), + DataTypes.DECIMAL(5, 2), + DataTypes.DATE(), + DataTypes.TIME(), + DataTypes.TIMESTAMP(3), + DataTypes.TIMESTAMP_LTZ(3)); + + GenericRow genericRow = + GenericRow.of( + true, + (byte) 10, + (short) 200, + 3000, + 40000L, + 1.5f, + 2.5d, + BinaryString.fromString("from-test"), + Decimal.fromUnscaledLong(12345, 5, 2), + 18000, // date as days since epoch + 43200000, // time as millis since midnight + TimestampNtz.fromMillis(1609459200000L), + TimestampLtz.fromEpochMillis(1609459200000L)); + + AlignedRow result = AlignedRow.from(rowType, genericRow); + assertThat(result).isNotNull(); + assertThat(result.getBoolean(0)).isTrue(); + assertThat(result.getByte(1)).isEqualTo((byte) 10); + assertThat(result.getShort(2)).isEqualTo((short) 200); + assertThat(result.getInt(3)).isEqualTo(3000); + assertThat(result.getLong(4)).isEqualTo(40000L); + assertThat(result.getFloat(5)).isEqualTo(1.5f); + assertThat(result.getDouble(6)).isEqualTo(2.5d); + assertThat(result.getString(7).toString()).isEqualTo("from-test"); + assertThat(result.getDecimal(8, 5, 2).toString()).isEqualTo("123.45"); + assertThat(result.getInt(9)).isEqualTo(18000); + assertThat(result.getInt(10)).isEqualTo(43200000); + assertThat(result.getTimestampNtz(11, 3).getMillisecond()).isEqualTo(1609459200000L); + assertThat(result.getTimestampLtz(12, 3).getEpochMillisecond()).isEqualTo(1609459200000L); + } + + @Test + public void testFromWithNullFields() { + RowType rowType = RowType.of(DataTypes.INT(), DataTypes.STRING(), DataTypes.DOUBLE()); + + GenericRow genericRow = GenericRow.of(null, null, null); + + AlignedRow result = AlignedRow.from(rowType, genericRow); + assertThat(result).isNotNull(); + assertThat(result.isNullAt(0)).isTrue(); + assertThat(result.isNullAt(1)).isTrue(); + assertThat(result.isNullAt(2)).isTrue(); + } + + @Test + public void testFromWithNonCompactTimestamp() { + RowType rowType = RowType.of(DataTypes.TIMESTAMP(9), DataTypes.TIMESTAMP_LTZ(9)); + + TimestampNtz timestampNtz = TimestampNtz.fromMillis(1609459200000L, 123456); + TimestampLtz timestampLtz = + TimestampLtz.fromInstant( + LocalDateTime.of(2021, 1, 1, 0, 0, 0, 123456789).toInstant(ZoneOffset.UTC)); + + GenericRow genericRow = GenericRow.of(timestampNtz, timestampLtz); + + AlignedRow result = AlignedRow.from(rowType, genericRow); + assertThat(result).isNotNull(); + assertThat(result.getTimestampNtz(0, 9).getMillisecond()).isEqualTo(1609459200000L); + assertThat(result.getTimestampNtz(0, 9).getNanoOfMillisecond()).isEqualTo(123456); + assertThat(result.getTimestampLtz(1, 9).toString()) + .isEqualTo("2021-01-01T00:00:00.123456789Z"); + } + + @Test + public void testFromWithNonCompactDecimal() { + RowType rowType = RowType.of(DataTypes.DECIMAL(25, 5)); + + Decimal decimal = + Decimal.fromBigDecimal(new BigDecimal("12345678901234567890.12345"), 25, 5); + GenericRow genericRow = GenericRow.of(decimal); + + AlignedRow result = AlignedRow.from(rowType, genericRow); + assertThat(result).isNotNull(); + assertThat(result.getDecimal(0, 25, 5).toBigDecimal()) + .isEqualTo(new BigDecimal("12345678901234567890.12345")); + } + + @Test + public void testFromWithUnsupportedType() { + RowType rowType = RowType.of(DataTypes.BYTES()); + GenericRow genericRow = GenericRow.of(new byte[] {1, 2, 3}); + + assertThatThrownBy(() -> AlignedRow.from(rowType, genericRow)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Statistics collection is not supported for type"); + } + + @Test + public void testSetTimestampNtzInPlace() { + // non-compact timestamp set in-place + int precision = 9; + TimestampNtz timestamp1 = TimestampNtz.fromMillis(1609459200000L, 111111); + TimestampNtz timestamp2 = TimestampNtz.fromMillis(1609459300000L, 222222); + + AlignedRow row = new AlignedRow(2); + AlignedRowWriter writer = new AlignedRowWriter(row); + writer.writeTimestampNtz(0, timestamp1, precision); + writer.setNullAt(1); + writer.complete(); + + assertThat(row.getTimestampNtz(0, precision).getMillisecond()).isEqualTo(1609459200000L); + assertThat(row.getTimestampNtz(0, precision).getNanoOfMillisecond()).isEqualTo(111111); + + // in-place update + row.setTimestampNtz(0, timestamp2, precision); + assertThat(row.getTimestampNtz(0, precision).getMillisecond()).isEqualTo(1609459300000L); + assertThat(row.getTimestampNtz(0, precision).getNanoOfMillisecond()).isEqualTo(222222); + + // set to null + row.setTimestampNtz(0, null, precision); + assertThat(row.isNullAt(0)).isTrue(); + } + + @Test + public void testSetDecimalNullNonCompact() { + int precision = 25; + int scale = 5; + Decimal decimal = Decimal.fromBigDecimal(new BigDecimal("99.99"), precision, scale); + + AlignedRow row = new AlignedRow(1); + AlignedRowWriter writer = new AlignedRowWriter(row); + writer.writeDecimal(0, decimal, precision); + writer.complete(); + + assertThat(row.getDecimal(0, precision, scale).toBigDecimal()) + .isEqualByComparingTo(new BigDecimal("99.99")); + + // set to null in-place + row.setDecimal(0, null, precision); + assertThat(row.isNullAt(0)).isTrue(); + } + + @Test + public void testSetTimestampLtzNullNonCompact() { + int precision = 9; + TimestampLtz timestamp = + TimestampLtz.fromInstant( + LocalDateTime.of(2021, 1, 1, 0, 0, 0, 500000000).toInstant(ZoneOffset.UTC)); + + AlignedRow row = new AlignedRow(1); + AlignedRowWriter writer = new AlignedRowWriter(row); + writer.writeTimestampLtz(0, timestamp, precision); + writer.complete(); + + assertThat(row.getTimestampLtz(0, precision)).isNotNull(); + + // set to null in-place + row.setTimestampLtz(0, null, precision); + assertThat(row.isNullAt(0)).isTrue(); + } + + @Test + public void testCalculateFixPartSizeInBytes() { + assertThat(AlignedRow.calculateFixPartSizeInBytes(0)).isEqualTo(8); + assertThat(AlignedRow.calculateFixPartSizeInBytes(1)).isEqualTo(16); + assertThat(AlignedRow.calculateFixPartSizeInBytes(56)).isEqualTo(8 + 56 * 8); + assertThat(AlignedRow.calculateFixPartSizeInBytes(57)).isEqualTo(16 + 57 * 8); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/row/arrow/ArrowReaderWriterTest.java b/fluss-common/src/test/java/org/apache/fluss/row/arrow/ArrowReaderWriterTest.java index eab2734639..3190fa251b 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/arrow/ArrowReaderWriterTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/arrow/ArrowReaderWriterTest.java @@ -46,7 +46,7 @@ import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; import static org.apache.fluss.compression.ArrowCompressionInfo.NO_COMPRESSION; import static org.apache.fluss.record.LogRecordBatch.CURRENT_LOG_MAGIC_VALUE; -import static org.apache.fluss.record.LogRecordBatchFormat.arrowChangeTypeOffset; +import static org.apache.fluss.record.LogRecordBatchFormat.recordBatchHeaderSize; import static org.apache.fluss.record.TestData.DATA1; import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; import static org.apache.fluss.row.BinaryString.fromString; @@ -200,13 +200,13 @@ void testReaderWriter() throws IOException { // skip arrow batch header. int size = writer.serializeToOutputView( - pagedOutputView, arrowChangeTypeOffset(CURRENT_LOG_MAGIC_VALUE)); + pagedOutputView, recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE)); int heapMemorySize = Math.max(size, writer.estimatedSizeInBytes()); MemorySegment segment = MemorySegment.allocateHeapMemory(heapMemorySize); assertThat(pagedOutputView.getWrittenSegments().size()).isEqualTo(1); MemorySegment firstSegment = pagedOutputView.getCurrentSegment(); - firstSegment.copyTo(arrowChangeTypeOffset(CURRENT_LOG_MAGIC_VALUE), segment, 0, size); + firstSegment.copyTo(recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE), segment, 0, size); ArrowReader reader = ArrowUtils.createArrowReader(segment, 0, size, root, allocator, rowType); @@ -313,14 +313,14 @@ void testArrayWriterWithManyElements() throws IOException { new ManagedPagedOutputView(new TestingMemorySegmentPool(64 * 1024)); int size = writer.serializeToOutputView( - pagedOutputView, arrowChangeTypeOffset(CURRENT_LOG_MAGIC_VALUE)); + pagedOutputView, recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE)); assertThat(size).isGreaterThan(0); // Verify the data can be read back correctly int heapMemorySize = Math.max(size, writer.estimatedSizeInBytes()); MemorySegment segment = MemorySegment.allocateHeapMemory(heapMemorySize); MemorySegment firstSegment = pagedOutputView.getCurrentSegment(); - firstSegment.copyTo(arrowChangeTypeOffset(CURRENT_LOG_MAGIC_VALUE), segment, 0, size); + firstSegment.copyTo(recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE), segment, 0, size); ArrowReader reader = ArrowUtils.createArrowReader(segment, 0, size, root, allocator, rowType); @@ -381,14 +381,14 @@ void testMapWriterWithManyEntries() throws IOException { new ManagedPagedOutputView(new TestingMemorySegmentPool(64 * 1024)); int size = writer.serializeToOutputView( - pagedOutputView, arrowChangeTypeOffset(CURRENT_LOG_MAGIC_VALUE)); + pagedOutputView, recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE)); assertThat(size).isGreaterThan(0); // Verify the data can be read back correctly int heapMemorySize = Math.max(size, writer.estimatedSizeInBytes()); MemorySegment segment = MemorySegment.allocateHeapMemory(heapMemorySize); MemorySegment firstSegment = pagedOutputView.getCurrentSegment(); - firstSegment.copyTo(arrowChangeTypeOffset(CURRENT_LOG_MAGIC_VALUE), segment, 0, size); + firstSegment.copyTo(recordBatchHeaderSize(CURRENT_LOG_MAGIC_VALUE), segment, 0, size); ArrowReader reader = ArrowUtils.createArrowReader(segment, 0, size, root, allocator, rowType); 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 7c6040d8c4..0ce2b3b464 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 @@ -24,6 +24,7 @@ import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.StatisticsConfigUtils; import org.apache.fluss.config.cluster.AlterConfig; import org.apache.fluss.config.cluster.AlterConfigOpType; import org.apache.fluss.exception.ApiException; @@ -403,6 +404,9 @@ public CompletableFuture createTable(CreateTableRequest req tableDescriptor = applySystemDefaults(tableDescriptor, lakeCatalogContainer.getDataLakeFormat()); + // validate statistics configuration + StatisticsConfigUtils.validateStatisticsConfig(tableDescriptor); + // the distribution and bucket count must be set now //noinspection OptionalGetWithoutIsPresent int bucketCount = tableDescriptor.getTableDistribution().get().getBucketCount().get(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/wal/ArrowWalBuilder.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/wal/ArrowWalBuilder.java index d8cd39872f..b46a5fd102 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/wal/ArrowWalBuilder.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/wal/ArrowWalBuilder.java @@ -40,7 +40,7 @@ public ArrowWalBuilder(int schemaId, ArrowWriter writer, MemorySegmentPool memor this.memorySegmentPool = memorySegmentPool; this.outputView = new ManagedPagedOutputView(memorySegmentPool); this.recordsBuilder = - MemoryLogRecordsArrowBuilder.builder(schemaId, writer, outputView, false); + MemoryLogRecordsArrowBuilder.builder(schemaId, writer, outputView, false, null); } @Override diff --git a/website/docs/_configs/_partial_config.mdx b/website/docs/_configs/_partial_config.mdx index 7bb8e0a663..5f171c8564 100644 --- a/website/docs/_configs/_partial_config.mdx +++ b/website/docs/_configs/_partial_config.mdx @@ -276,6 +276,7 @@ | Key | Default | Type | Description | | :--- | :--- | :--- | :--- | | `table.replication.factor` | `none` | String | The replication factor for the log of the new table. When it's not set, Fluss will use the cluster's default replication factor configured by default.replication.factor. It should be a positive number and not larger than the number of tablet servers in the Fluss cluster. A value larger than the number of tablet servers in Fluss cluster will result in an error when the new table is created. | +| `table.statistics.columns` | `none` | String | Specifies which columns to collect statistics (min, max, null count) for in log table batches. Use `*` to collect statistics for all supported columns, or specify a comma-separated list of column names (e.g., `col1,col2`). Supported types: BOOLEAN, TINYINT, SMALLINT, INTEGER, BIGINT, FLOAT, DOUBLE, STRING, CHAR, DECIMAL, DATE, TIME, TIMESTAMP, TIMESTAMP_LTZ. Unsupported types are automatically excluded. By default, no column statistics are collected. Note: Enabling column statistics upgrades the log batch format to V1. Downstream consumers must be running Fluss v1.0 or later to parse the extended batch format. | | `table.log.format` | `ARROW` | LogFormat | The format of the log records in log store. The default value is `arrow`. The supported formats are `arrow`, `indexed` and `compacted`. | | `table.log.arrow.compression.type` | `ZSTD` | ArrowCompressionType | The compression type of the log records if the log format is set to `ARROW`. The candidate compression type is [NONE, LZ4_FRAME, ZSTD] | | `table.log.arrow.compression.zstd.level` | `3` | Integer | The compression level of ZSTD for the log records if the log format is set to `ARROW` and the compression type is set to `ZSTD`. The valid range is 1 to 22. | diff --git a/website/docs/engine-flink/options.md b/website/docs/engine-flink/options.md index 200c8bb5ce..6f04256450 100644 --- a/website/docs/engine-flink/options.md +++ b/website/docs/engine-flink/options.md @@ -74,6 +74,7 @@ See more details about [ALTER TABLE ... SET](engine-flink/ddl.md#set-properties) | table.auto-partition.num-retention | Integer | 7 | The number of history partitions to retain for auto created partitions in each check for auto partition. For example, if the current check time is 2024-11-11, time-unit is DAY, and the value is configured as 3, then the history partitions 20241108, 20241109, 20241110 will be retained. The partitions earlier than 20241108 will be deleted. The default value is 7, which means that 7 partitions will be retained. | | table.auto-partition.time-zone | String | the system time zone | The time zone for auto partitions, which is by default the same as the system time zone. | | table.replication.factor | Integer | (None) | The replication factor for the log of the new table. When it's not set, Fluss will use the cluster's default replication factor configured by default.replication.factor. It should be a positive number and not larger than the number of tablet servers in the Fluss cluster. A value larger than the number of tablet servers in Fluss cluster will result in an error when the new table is created. | +| table.statistics.columns | String | (None) | Specifies which columns to collect statistics (min, max, null count) for in log table batches. Use `*` to collect statistics for all supported columns, or specify a comma-separated list of column names (e.g., `col1,col2`). Supported types: BOOLEAN, TINYINT, SMALLINT, INTEGER, BIGINT, FLOAT, DOUBLE, STRING, CHAR, DECIMAL, DATE, TIME, TIMESTAMP, TIMESTAMP_LTZ. Unsupported types (BYTES, BINARY, ARRAY, MAP, ROW) are automatically excluded. By default, this option is not set and no column statistics are collected.

**Compatibility Note:** Enabling column statistics upgrades the log batch format to V1. Downstream consumers (e.g., Flink jobs) must be running Fluss v1.0 or later to parse the extended batch format. Ensure all downstream jobs are upgraded before enabling this option, otherwise consumer jobs will fail when parsing V1 format logs. | | table.log.format | Enum | ARROW | The format of the log records in log store. The default value is `ARROW`. The supported formats are `ARROW`, `INDEXED` and `COMPACTED`. | | table.log.arrow.compression.type | Enum | ZSTD | The compression type of the log records if the log format is set to `ARROW`. The candidate compression type is `NONE`, `LZ4_FRAME`, `ZSTD`. The default value is `ZSTD`. | | table.log.arrow.compression.zstd.level | Integer | 3 | The compression level of the log records if the log format is set to `ARROW` and the compression type is set to `ZSTD`. The valid range is 1 to 22. The default value is 3. |