From cfef6c8d4cf289ad49371417e53164d6f4d4f0e9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 21 Jan 2026 17:24:10 -0700 Subject: [PATCH 1/7] docs: add Iceberg public API documentation to contributor guide Add documentation detailing all Comet classes and methods that form the public API used by Apache Iceberg. This helps contributors understand which APIs may affect Iceberg integration and need backward compatibility considerations. The documentation covers: - org.apache.comet.parquet: FileReader, RowGroupReader, ReadOptions, ParquetColumnSpec, column readers, BatchReader, Native JNI methods - org.apache.comet: CometSchemaImporter - org.apache.comet.vector: CometVector - org.apache.comet.shaded.arrow: RootAllocator, ValueVector Co-Authored-By: Claude Opus 4.5 --- .../contributor-guide/iceberg_public_api.md | 325 ++++++++++++++++++ docs/source/contributor-guide/index.md | 1 + 2 files changed, 326 insertions(+) create mode 100644 docs/source/contributor-guide/iceberg_public_api.md diff --git a/docs/source/contributor-guide/iceberg_public_api.md b/docs/source/contributor-guide/iceberg_public_api.md new file mode 100644 index 0000000000..48e35388a1 --- /dev/null +++ b/docs/source/contributor-guide/iceberg_public_api.md @@ -0,0 +1,325 @@ + + +# Public API for Apache Iceberg Integration + +This document describes the Comet classes and methods that form the public API used by +[Apache Iceberg](https://iceberg.apache.org/). These APIs enable Iceberg to leverage Comet's +native Parquet reader for vectorized reads in Spark. + +**Important**: Changes to these APIs may break Iceberg's Comet integration. Contributors should +exercise caution when modifying these classes and consider backward compatibility. + +## Overview + +Iceberg uses Comet's Parquet reading infrastructure to accelerate table scans. The integration +uses two approaches: + +1. **Hybrid Reader**: Native Parquet decoding with JVM-based I/O (requires building Iceberg + from source with Comet patches) +2. **Native Reader**: Fully-native Iceberg scans using iceberg-rust + +This document focuses on the Hybrid Reader API, which is used when Iceberg is configured with +`spark.sql.iceberg.parquet.reader-type=COMET`. + +## Package: `org.apache.comet.parquet` + +### FileReader + +Main class for reading Parquet files with native decoding. + +```java +// Constructor +public FileReader( + WrappedInputFile inputFile, + ReadOptions options, + Map properties, + Long start, + Long length, + byte[] fileEncryptionKey, + byte[] fileAADPrefix +) throws IOException + +// Methods used by Iceberg +public void setRequestedSchemaFromSpecs(List specs) +public RowGroupReader readNextRowGroup() throws IOException +public void skipNextRowGroup() +public void close() throws IOException +``` + +### RowGroupReader + +Provides access to row group data. + +```java +// Methods used by Iceberg +public long getRowCount() +``` + +### ReadOptions + +Configuration for Parquet read operations. + +```java +// Builder pattern +public static Builder builder(Configuration conf) + +public class Builder { + public ReadOptions build() +} +``` + +### WrappedInputFile + +Wrapper that adapts Iceberg's `InputFile` interface to Comet's file reading infrastructure. + +```java +// Constructor +public WrappedInputFile(org.apache.iceberg.io.InputFile inputFile) +``` + +### ParquetColumnSpec + +Specification describing a Parquet column's schema information. + +```java +// Constructor +public ParquetColumnSpec( + int fieldId, + String[] path, + String physicalType, + int typeLength, + boolean isRepeated, + int maxDefinitionLevel, + int maxRepetitionLevel, + String logicalTypeName, + Map logicalTypeParams +) + +// Getters used by Iceberg +public int getFieldId() +public String[] getPath() +public String getPhysicalType() +public int getTypeLength() +public int getMaxDefinitionLevel() +public int getMaxRepetitionLevel() +public String getLogicalTypeName() +public Map getLogicalTypeParams() +``` + +### AbstractColumnReader + +Base class for column readers. + +```java +// Methods used by Iceberg +public void setBatchSize(int batchSize) +public void close() +``` + +### ColumnReader + +Column reader for regular Parquet columns (extends `AbstractColumnReader`). + +```java +// Methods used by Iceberg +public void setPageReader(PageReader pageReader) throws IOException +``` + +### BatchReader + +Coordinates reading batches across multiple column readers. + +```java +// Constructor +public BatchReader(AbstractColumnReader[] columnReaders) + +// Methods used by Iceberg +public void setSparkSchema(StructType schema) +public AbstractColumnReader[] getColumnReaders() +public void nextBatch(int batchSize) +``` + +### MetadataColumnReader + +Reader for metadata columns (used for Iceberg's delete and position columns). + +```java +// Constructor +public MetadataColumnReader( + DataType sparkType, + ColumnDescriptor descriptor, + boolean useDecimal128, + boolean isConstant +) + +// Methods used by Iceberg +public void readBatch(int total) +public CometVector currentBatch() + +// Protected field accessed by subclasses +protected long nativeHandle +``` + +### ConstantColumnReader + +Reader for columns with constant/default values (extends `MetadataColumnReader`). + +```java +// Constructor +public ConstantColumnReader( + DataType sparkType, + ColumnDescriptor descriptor, + Object value, + boolean useDecimal128 +) +``` + +### Native + +JNI interface for native operations. + +```java +// Static methods used by Iceberg +public static void resetBatch(long nativeHandle) +public static void setIsDeleted(long nativeHandle, boolean[] isDeleted) +public static void setPosition(long nativeHandle, long position, int total) +``` + +### TypeUtil + +Utilities for Parquet type conversions. + +```java +// Methods used by Iceberg +public static ColumnDescriptor convertToParquet(StructField sparkField) +``` + +### Utils + +General utility methods. + +```java +// Methods used by Iceberg +public static AbstractColumnReader getColumnReader( + DataType sparkType, + ColumnDescriptor descriptor, + CometSchemaImporter importer, + int batchSize, + boolean useDecimal128, + boolean isConstant +) +``` + +## Package: `org.apache.comet` + +### CometSchemaImporter + +Imports and converts schemas between Arrow and Spark formats. + +```java +// Constructor +public CometSchemaImporter(RootAllocator allocator) + +// Methods used by Iceberg +public void close() +``` + +## Package: `org.apache.comet.vector` + +### CometVector + +Base class for Comet's columnar vectors (extends Spark's `ColumnVector`). + +```java +// Constructor +public CometVector(DataType type, boolean useDecimal128) + +// Abstract methods that subclasses must implement +public abstract int numValues() +public abstract ValueVector getValueVector() +public abstract CometVector slice(int offset, int length) +public abstract void setNumNulls(int numNulls) +public abstract void setNumValues(int numValues) + +// Inherited from Spark ColumnVector - commonly overridden +public abstract void close() +public abstract boolean hasNull() +public abstract int numNulls() +public abstract boolean isNullAt(int rowId) +public abstract boolean getBoolean(int rowId) +// ... other type-specific getters +``` + +## Package: `org.apache.comet.shaded.arrow.memory` + +### RootAllocator + +Arrow memory allocator (shaded to avoid conflicts). + +```java +// Constructor used by Iceberg +public RootAllocator() +``` + +## Package: `org.apache.comet.shaded.arrow.vector` + +### ValueVector + +Arrow's base vector interface (shaded). Used as return type in `CometVector.getValueVector()`. + +## How Iceberg Uses These APIs + +### Parquet File Reading Flow + +1. Iceberg creates a `WrappedInputFile` from its `InputFile` +2. Creates `ReadOptions` via builder pattern +3. Instantiates `FileReader` with the wrapped input file +4. Converts Parquet `ColumnDescriptor`s to `ParquetColumnSpec`s using `CometTypeUtils` +5. Calls `setRequestedSchemaFromSpecs()` to specify which columns to read +6. Iterates through row groups via `readNextRowGroup()` and `skipNextRowGroup()` + +### Column Reading Flow + +1. Creates `CometSchemaImporter` with a `RootAllocator` +2. Uses `Utils.getColumnReader()` to create appropriate column readers +3. Calls `reset()` and `setPageReader()` for each row group +4. Uses `BatchReader` to coordinate reading batches across all columns +5. Retrieves results via `delegate().currentBatch()` + +### Metadata Columns + +Iceberg uses `MetadataColumnReader` subclasses for special columns: + +- **Delete tracking**: Uses `Native.setIsDeleted()` to mark deleted rows +- **Position tracking**: Uses `Native.setPosition()` to track row positions + +## Compatibility Considerations + +When modifying these APIs, consider: + +1. **Constructor signatures**: Adding required parameters breaks Iceberg +2. **Method signatures**: Changing return types or parameters breaks Iceberg +3. **Protected fields**: `MetadataColumnReader.nativeHandle` is accessed by Iceberg subclasses +4. **Shaded dependencies**: Arrow classes are shaded under `org.apache.comet.shaded` + +## Testing Iceberg Integration + +See the [Iceberg user guide](../user-guide/latest/iceberg.md) for instructions on testing +Comet with Iceberg. diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index db3270b6af..c603645830 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -29,6 +29,7 @@ Arrow FFI JVM Shuffle Native Shuffle Parquet Scans +Iceberg Public API Development Guide Debugging Guide Benchmarking Guide From 2a0e7d1f29c2e8b86e0f6b6dcef965d0d0080d7e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 21 Jan 2026 17:31:08 -0700 Subject: [PATCH 2/7] feat: add @IcebergApi annotation to mark public API used by Iceberg Add a custom Java annotation @IcebergApi to mark all classes, methods, constructors, and fields that form the public API used by Apache Iceberg. This makes it easy to identify which APIs need backward compatibility considerations when making changes. The annotation is applied to: - org.apache.comet.parquet: FileReader, RowGroupReader, ReadOptions, WrappedInputFile, ParquetColumnSpec, AbstractColumnReader, ColumnReader, BatchReader, MetadataColumnReader, ConstantColumnReader, Native, TypeUtil, Utils - org.apache.comet: CometSchemaImporter - org.apache.comet.vector: CometVector Co-Authored-By: Claude Opus 4.5 --- .../org/apache/comet/CometSchemaImporter.java | 2 + .../java/org/apache/comet/IcebergApi.java | 44 +++++++++++++++++++ .../comet/parquet/AbstractColumnReader.java | 5 +++ .../org/apache/comet/parquet/BatchReader.java | 7 +++ .../apache/comet/parquet/ColumnReader.java | 5 +++ .../comet/parquet/ConstantColumnReader.java | 5 +++ .../org/apache/comet/parquet/FileReader.java | 8 ++++ .../comet/parquet/MetadataColumnReader.java | 7 +++ .../java/org/apache/comet/parquet/Native.java | 5 +++ .../comet/parquet/ParquetColumnSpec.java | 12 +++++ .../org/apache/comet/parquet/ReadOptions.java | 6 +++ .../apache/comet/parquet/RowGroupReader.java | 4 ++ .../org/apache/comet/parquet/TypeUtil.java | 3 ++ .../java/org/apache/comet/parquet/Utils.java | 6 +++ .../comet/parquet/WrappedInputFile.java | 4 ++ .../org/apache/comet/vector/CometVector.java | 9 ++++ .../contributor-guide/iceberg_public_api.md | 3 ++ 17 files changed, 135 insertions(+) create mode 100644 common/src/main/java/org/apache/comet/IcebergApi.java diff --git a/common/src/main/java/org/apache/comet/CometSchemaImporter.java b/common/src/main/java/org/apache/comet/CometSchemaImporter.java index 7dc4b75db6..4841f16f19 100644 --- a/common/src/main/java/org/apache/comet/CometSchemaImporter.java +++ b/common/src/main/java/org/apache/comet/CometSchemaImporter.java @@ -23,7 +23,9 @@ import org.apache.arrow.memory.BufferAllocator; /** This is a simple wrapper around SchemaImporter to make it accessible from Java Arrow. */ +@IcebergApi public class CometSchemaImporter extends AbstractCometSchemaImporter { + @IcebergApi public CometSchemaImporter(BufferAllocator allocator) { super(allocator); } diff --git a/common/src/main/java/org/apache/comet/IcebergApi.java b/common/src/main/java/org/apache/comet/IcebergApi.java new file mode 100644 index 0000000000..ce1182bcf5 --- /dev/null +++ b/common/src/main/java/org/apache/comet/IcebergApi.java @@ -0,0 +1,44 @@ +/* + * 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.comet; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Indicates that the annotated element is part of the public API used by Apache Iceberg. + * + *

This annotation marks classes, methods, constructors, and fields that form the contract + * between Comet and Iceberg. Changes to these APIs may break Iceberg's Comet integration, + * so contributors should exercise caution and consider backward compatibility when modifying + * annotated elements. + * + *

The Iceberg integration uses Comet's native Parquet reader for accelerated vectorized reads. + * See the contributor guide documentation for details on how Iceberg uses these APIs. + * + * @see Apache Iceberg + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.METHOD, ElementType.CONSTRUCTOR, ElementType.FIELD}) +public @interface IcebergApi {} diff --git a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java index 3768bff56b..1355252118 100644 --- a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,6 +33,7 @@ import org.apache.comet.vector.CometVector; /** Base class for Comet Parquet column reader implementations. */ +@IcebergApi public abstract class AbstractColumnReader implements AutoCloseable { protected static final Logger LOG = LoggerFactory.getLogger(AbstractColumnReader.class); @@ -96,6 +99,7 @@ String getPath() { /** * Set the batch size of this reader to be 'batchSize'. Also initializes the native column reader. */ + @IcebergApi public void setBatchSize(int batchSize) { assert nativeHandle == 0 : "Native column reader shouldn't be initialized before " + "'setBatchSize' is called"; @@ -113,6 +117,7 @@ public void setBatchSize(int batchSize) { /** Returns the {@link CometVector} read by this reader. */ public abstract CometVector currentBatch(); + @IcebergApi @Override public void close() { if (nativeHandle != 0) { diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java index edac28ec1b..22b5a03cf7 100644 --- a/common/src/main/java/org/apache/comet/parquet/BatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.io.Closeable; import java.io.IOException; import java.net.URI; @@ -87,6 +89,7 @@ * } * */ +@IcebergApi public class BatchReader extends RecordReader implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); protected static final BufferAllocator ALLOCATOR = new RootAllocator(); @@ -189,6 +192,7 @@ public BatchReader( * @deprecated since 0.10.0, will be removed in 0.11.0. * @see Comet Issue #2079 */ + @IcebergApi public BatchReader(AbstractColumnReader[] columnReaders) { // Todo: set useDecimal128 and useLazyMaterialization int numColumns = columnReaders.length; @@ -387,6 +391,7 @@ public void init() throws URISyntaxException, IOException { * @deprecated since 0.10.0, will be removed in 0.11.0. * @see Comet Issue #2079 */ + @IcebergApi public void setSparkSchema(StructType schema) { this.sparkSchema = schema; } @@ -395,6 +400,7 @@ public void setSparkSchema(StructType schema) { * @deprecated since 0.10.0, will be removed in 0.11.0. * @see Comet Issue #2079 */ + @IcebergApi public AbstractColumnReader[] getColumnReaders() { return columnReaders; } @@ -498,6 +504,7 @@ public boolean nextBatch() throws IOException { return nextBatch(batchSize); } + @IcebergApi public boolean nextBatch(int batchSize) { long totalDecodeTime = 0, totalLoadTime = 0; for (int i = 0; i < columnReaders.length; i++) { diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 968da1959c..89f91864d0 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.io.IOException; import java.nio.ByteBuffer; @@ -50,6 +52,7 @@ import org.apache.comet.vector.CometPlainVector; import org.apache.comet.vector.CometVector; +@IcebergApi public class ColumnReader extends AbstractColumnReader { protected static final Logger LOG = LoggerFactory.getLogger(ColumnReader.class); protected final BufferAllocator ALLOCATOR = new RootAllocator(); @@ -114,6 +117,7 @@ public class ColumnReader extends AbstractColumnReader { * @deprecated since 0.10.0, will be removed in 0.11.0. * @see Comet Issue #2079 */ + @IcebergApi public void setPageReader(PageReader pageReader) throws IOException { this.pageReader = pageReader; @@ -129,6 +133,7 @@ public void setPageReader(PageReader pageReader) throws IOException { } /** This method is called from Apache Iceberg. */ + @IcebergApi public void setRowGroupReader(RowGroupReader rowGroupReader, ParquetColumnSpec columnSpec) throws IOException { ColumnDescriptor descriptor = Utils.buildColumnDescriptor(columnSpec); diff --git a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java index b8fc49a175..95cd8d9641 100644 --- a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.math.BigInteger; import org.apache.parquet.column.ColumnDescriptor; @@ -31,6 +33,7 @@ * A column reader that always return constant vectors. Used for reading partition columns, for * instance. */ +@IcebergApi public class ConstantColumnReader extends MetadataColumnReader { /** Whether all the values in this constant column are nulls */ private boolean isNull; @@ -56,6 +59,7 @@ public class ConstantColumnReader extends MetadataColumnReader { * @deprecated since 0.10.0, will be removed in 0.11.0. * @see Comet Issue #2079 */ + @IcebergApi public ConstantColumnReader( DataType type, ColumnDescriptor descriptor, Object value, boolean useDecimal128) { super(type, descriptor, useDecimal128, true); @@ -63,6 +67,7 @@ public ConstantColumnReader( } // Used by Iceberg + @IcebergApi public ConstantColumnReader( DataType type, ParquetColumnSpec spec, Object value, boolean useDecimal128) { super(type, spec, useDecimal128, true); diff --git a/common/src/main/java/org/apache/comet/parquet/FileReader.java b/common/src/main/java/org/apache/comet/parquet/FileReader.java index fa0d81f13e..ecf92f405f 100644 --- a/common/src/main/java/org/apache/comet/parquet/FileReader.java +++ b/common/src/main/java/org/apache/comet/parquet/FileReader.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.io.Closeable; import java.io.IOException; import java.io.InputStream; @@ -101,6 +103,7 @@ * A Parquet file reader. Mostly followed {@code ParquetFileReader} in {@code parquet-mr}, but with * customizations & optimizations for Comet. */ +@IcebergApi public class FileReader implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); @@ -135,6 +138,7 @@ public class FileReader implements Closeable { } /** This constructor is called from Apache Iceberg. */ + @IcebergApi public FileReader( WrappedInputFile file, ReadOptions cometOptions, @@ -258,6 +262,7 @@ public void setRequestedSchema(List projection) { } /** This method is called from Apache Iceberg. */ + @IcebergApi public void setRequestedSchemaFromSpecs(List specList) { paths.clear(); for (ParquetColumnSpec colSpec : specList) { @@ -336,6 +341,7 @@ public long getFilteredRecordCount() { } /** Skips the next row group. Returns false if there's no row group to skip. Otherwise, true. */ + @IcebergApi public boolean skipNextRowGroup() { return advanceToNextBlock(); } @@ -344,6 +350,7 @@ public boolean skipNextRowGroup() { * Returns the next row group to read (after applying row group filtering), or null if there's no * more row group. */ + @IcebergApi public RowGroupReader readNextRowGroup() throws IOException { if (currentBlock == blocks.size()) { return null; @@ -864,6 +871,7 @@ public void closeStream() throws IOException { } } + @IcebergApi @Override public void close() throws IOException { try { diff --git a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java index 6240c8c8c5..cbe0f369d2 100644 --- a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import org.apache.arrow.c.ArrowArray; import org.apache.arrow.c.ArrowSchema; import org.apache.arrow.c.Data; @@ -32,6 +34,7 @@ import org.apache.comet.vector.CometVector; /** A metadata column reader that can be extended by {@link RowIndexColumnReader} etc. */ +@IcebergApi public class MetadataColumnReader extends AbstractColumnReader { private final BufferAllocator allocator = new RootAllocator(); @@ -46,6 +49,7 @@ public class MetadataColumnReader extends AbstractColumnReader { * @deprecated since 0.10.0, will be made package private in 0.11.0. * @see Comet Issue #2079 */ + @IcebergApi public MetadataColumnReader( DataType type, ColumnDescriptor descriptor, boolean useDecimal128, boolean isConstant) { // TODO: should we handle legacy dates & timestamps for metadata columns? @@ -55,6 +59,7 @@ public MetadataColumnReader( } // Used by Iceberg + @IcebergApi public MetadataColumnReader( DataType type, ParquetColumnSpec spec, boolean useDecimal128, boolean isConstant) { // TODO: should we handle legacy dates & timestamps for metadata columns? @@ -69,6 +74,7 @@ public void setBatchSize(int batchSize) { super.setBatchSize(batchSize); } + @IcebergApi @Override public void readBatch(int total) { if (vector == null) { @@ -90,6 +96,7 @@ void setNumNulls(int total) { vector.setNumNulls(total); } + @IcebergApi @Override public CometVector currentBatch() { return vector; diff --git a/common/src/main/java/org/apache/comet/parquet/Native.java b/common/src/main/java/org/apache/comet/parquet/Native.java index baca63b005..467abf9f17 100644 --- a/common/src/main/java/org/apache/comet/parquet/Native.java +++ b/common/src/main/java/org/apache/comet/parquet/Native.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.nio.ByteBuffer; import java.util.Map; @@ -143,6 +145,7 @@ public static native void setPageV2( * * @param handle the handle to the native Parquet column reader */ + @IcebergApi public static native void resetBatch(long handle); /** @@ -221,12 +224,14 @@ public static native void setPageV2( public static native void setDecimal(long handle, byte[] value); /** Set position of row index vector for Iceberg Metadata Column */ + @IcebergApi public static native void setPosition(long handle, long value, int size); /** Set row index vector for Spark row index metadata column and return vector size */ public static native int setIndices(long handle, long offset, int size, long[] indices); /** Set deleted info for Iceberg Metadata Column */ + @IcebergApi public static native void setIsDeleted(long handle, boolean[] isDeleted); /** diff --git a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java index 805aaa033d..704fa3e3f1 100644 --- a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java +++ b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.util.Map; /** @@ -27,6 +29,7 @@ * passing of Column descriptors between Comet and Iceberg. This is required because Iceberg shades * Parquet, changing the package of Parquet classes and making then incompatible with Comet. */ +@IcebergApi public class ParquetColumnSpec { private final int fieldId; @@ -41,6 +44,7 @@ public class ParquetColumnSpec { private String logicalTypeName; private Map logicalTypeParams; + @IcebergApi public ParquetColumnSpec( int fieldId, String[] path, @@ -62,18 +66,22 @@ public ParquetColumnSpec( this.logicalTypeParams = logicalTypeParams; } + @IcebergApi public int getFieldId() { return fieldId; } + @IcebergApi public String[] getPath() { return path; } + @IcebergApi public String getPhysicalType() { return physicalType; } + @IcebergApi public int getTypeLength() { return typeLength; } @@ -82,18 +90,22 @@ public boolean isRepeated() { return isRepeated; } + @IcebergApi public int getMaxRepetitionLevel() { return maxRepetitionLevel; } + @IcebergApi public int getMaxDefinitionLevel() { return maxDefinitionLevel; } + @IcebergApi public String getLogicalTypeName() { return logicalTypeName; } + @IcebergApi public Map getLogicalTypeParams() { return logicalTypeParams; } diff --git a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java index b2889f3895..5375b1bcac 100644 --- a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java +++ b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,6 +35,7 @@ * *

TODO: merge this with {@link org.apache.parquet.HadoopReadOptions} once PARQUET-2203 is done. */ +@IcebergApi public class ReadOptions { private static final Logger LOG = LoggerFactory.getLogger(ReadOptions.class); @@ -86,10 +89,12 @@ public boolean adjustReadRangesSkew() { return adjustReadRangeSkew; } + @IcebergApi public static Builder builder(Configuration conf) { return new Builder(conf); } + @IcebergApi public static class Builder { private final Configuration conf; @@ -134,6 +139,7 @@ public Builder adjustReadRangeSkew(boolean adjustReadRangeSkew) { return this; } + @IcebergApi public ReadOptions build() { return new ReadOptions( parallelIOEnabled, diff --git a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java index 0ddda61b44..dae566f9a1 100644 --- a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java +++ b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -29,6 +31,7 @@ import org.apache.parquet.column.page.PageReader; import org.apache.parquet.internal.filter2.columnindex.RowRanges; +@IcebergApi public class RowGroupReader implements PageReadStore { private final Map readers = new HashMap<>(); private final long rowCount; @@ -47,6 +50,7 @@ public RowGroupReader(long rowCount, long rowIndexOffset) { this.rowIndexOffset = -1; } + @IcebergApi @Override public long getRowCount() { return rowCount; diff --git a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java index 889e2baf50..d3d23e3ffd 100644 --- a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java +++ b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.util.Arrays; import org.apache.parquet.column.ColumnDescriptor; @@ -41,6 +43,7 @@ public class TypeUtil { * @deprecated since 0.10.0, will be removed in 0.11.0. * @see Comet Issue #2079 */ + @IcebergApi public static ColumnDescriptor convertToParquet(StructField field) { Type.Repetition repetition; int maxDefinitionLevel; diff --git a/common/src/main/java/org/apache/comet/parquet/Utils.java b/common/src/main/java/org/apache/comet/parquet/Utils.java index 7fb2eac5b6..af2865a4e6 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.util.HashMap; import java.util.Map; @@ -34,6 +36,7 @@ public class Utils { /** This method is called from Apache Iceberg. */ + @IcebergApi public static ColumnReader getColumnReader( DataType type, ParquetColumnSpec columnSpec, @@ -61,6 +64,7 @@ public static ColumnReader getColumnReader( * instead. * @see Comet Issue #2079 */ + @IcebergApi public static ColumnReader getColumnReader( DataType type, ColumnDescriptor descriptor, @@ -293,6 +297,7 @@ static int getTimeUnitId(LogicalTypeAnnotation.TimeUnit tu) { } } + @IcebergApi public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpec) { PrimitiveType.PrimitiveTypeName primType = PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType()); @@ -458,6 +463,7 @@ private static LogicalTypeAnnotation reconstructLogicalType( } } + @IcebergApi public static ParquetColumnSpec descriptorToParquetColumnSpec(ColumnDescriptor descriptor) { String[] path = descriptor.getPath(); diff --git a/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java b/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java index 666d4c2e7b..bd5448c1d4 100644 --- a/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java +++ b/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import org.apache.comet.IcebergApi; + import java.io.IOException; import java.io.InputStream; import java.lang.reflect.Method; @@ -30,9 +32,11 @@ * Wraps an Object that possibly implements the methods of a Parquet InputFile (but is not a Parquet * InputFile). Such an object` exists, for instance, in Iceberg's InputFile */ +@IcebergApi public class WrappedInputFile implements InputFile { Object wrapped; + @IcebergApi public WrappedInputFile(Object inputFile) { this.wrapped = inputFile; } diff --git a/common/src/main/java/org/apache/comet/vector/CometVector.java b/common/src/main/java/org/apache/comet/vector/CometVector.java index a1f75696f6..be50d3b7a9 100644 --- a/common/src/main/java/org/apache/comet/vector/CometVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometVector.java @@ -19,6 +19,8 @@ package org.apache.comet.vector; +import org.apache.comet.IcebergApi; + import java.math.BigDecimal; import java.math.BigInteger; @@ -40,6 +42,7 @@ import org.apache.spark.unsafe.types.UTF8String; /** Base class for all Comet column vector implementations. */ +@IcebergApi public abstract class CometVector extends ColumnVector { private static final int DECIMAL_BYTE_WIDTH = 16; private final byte[] DECIMAL_BYTES = new byte[DECIMAL_BYTE_WIDTH]; @@ -58,6 +61,7 @@ public abstract class CometVector extends ColumnVector { } } + @IcebergApi protected CometVector(DataType type, boolean useDecimal128) { super(type); this.useDecimal128 = useDecimal128; @@ -67,15 +71,18 @@ protected CometVector(DataType type, boolean useDecimal128) { * Sets the number of nulls in this vector to be 'numNulls'. This is used when the vector is * reused across batches. */ + @IcebergApi public abstract void setNumNulls(int numNulls); /** * Sets the number of values (including both nulls and non-nulls) in this vector to be * 'numValues'. This is used when the vector is reused across batches. */ + @IcebergApi public abstract void setNumValues(int numValues); /** Returns the number of values in this vector. */ + @IcebergApi public abstract int numValues(); /** Whether the elements of this vector are of fixed length. */ @@ -215,6 +222,7 @@ public DictionaryProvider getDictionaryProvider() { throw new UnsupportedOperationException("Not implemented"); } + @IcebergApi public abstract ValueVector getValueVector(); /** @@ -224,6 +232,7 @@ public DictionaryProvider getDictionaryProvider() { * @param length the length of the new vector * @return the new vector */ + @IcebergApi public abstract CometVector slice(int offset, int length); /** diff --git a/docs/source/contributor-guide/iceberg_public_api.md b/docs/source/contributor-guide/iceberg_public_api.md index 48e35388a1..d26bac23de 100644 --- a/docs/source/contributor-guide/iceberg_public_api.md +++ b/docs/source/contributor-guide/iceberg_public_api.md @@ -26,6 +26,9 @@ native Parquet reader for vectorized reads in Spark. **Important**: Changes to these APIs may break Iceberg's Comet integration. Contributors should exercise caution when modifying these classes and consider backward compatibility. +All classes and methods documented here are marked with the `@IcebergApi` annotation +(`org.apache.comet.IcebergApi`) to make them easily identifiable in the source code. + ## Overview Iceberg uses Comet's Parquet reading infrastructure to accelerate table scans. The integration From 6b4a47d32a775b75b6626f25580e257034b2c3be Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 21 Jan 2026 17:35:08 -0700 Subject: [PATCH 3/7] fix: add missing @IcebergApi annotations Add annotations to: - AbstractColumnReader.nativeHandle (protected field accessed by Iceberg subclasses) - AbstractCometSchemaImporter.close() (called by Iceberg) Also update documentation to include these APIs. Co-Authored-By: Claude Opus 4.5 --- .../arrow/c/AbstractCometSchemaImporter.java | 3 +++ .../apache/comet/parquet/AbstractColumnReader.java | 1 + .../source/contributor-guide/iceberg_public_api.md | 14 ++++++++++++++ 3 files changed, 18 insertions(+) diff --git a/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java b/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java index f35196356f..c3bb1cbf18 100644 --- a/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java +++ b/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java @@ -19,6 +19,8 @@ package org.apache.arrow.c; +import org.apache.comet.IcebergApi; + import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.types.pojo.Field; @@ -67,6 +69,7 @@ public FieldVector importVector(ArrowArray array, ArrowSchema schema) { return vector; } + @IcebergApi public void close() { provider.close(); } diff --git a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java index 1355252118..7289de8226 100644 --- a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java @@ -64,6 +64,7 @@ public abstract class AbstractColumnReader implements AutoCloseable { protected int batchSize; /** A pointer to the native implementation of ColumnReader. */ + @IcebergApi protected long nativeHandle; AbstractColumnReader( diff --git a/docs/source/contributor-guide/iceberg_public_api.md b/docs/source/contributor-guide/iceberg_public_api.md index d26bac23de..bf459d3df2 100644 --- a/docs/source/contributor-guide/iceberg_public_api.md +++ b/docs/source/contributor-guide/iceberg_public_api.md @@ -131,6 +131,9 @@ public Map getLogicalTypeParams() Base class for column readers. ```java +// Protected field accessed by Iceberg subclasses +protected long nativeHandle + // Methods used by Iceberg public void setBatchSize(int batchSize) public void close() @@ -240,6 +243,17 @@ Imports and converts schemas between Arrow and Spark formats. // Constructor public CometSchemaImporter(RootAllocator allocator) +// Methods used by Iceberg (inherited from AbstractCometSchemaImporter) +public void close() +``` + +## Package: `org.apache.arrow.c` + +### AbstractCometSchemaImporter + +Base class for `CometSchemaImporter`. + +```java // Methods used by Iceberg public void close() ``` From 7b58965d41caafbd34dcf587266e99eedcbc7d95 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 21 Jan 2026 17:53:54 -0700 Subject: [PATCH 4/7] format --- .../org/apache/arrow/c/AbstractCometSchemaImporter.java | 4 ++-- common/src/main/java/org/apache/comet/IcebergApi.java | 6 +++--- .../java/org/apache/comet/parquet/AbstractColumnReader.java | 6 ++---- .../src/main/java/org/apache/comet/parquet/BatchReader.java | 3 +-- .../main/java/org/apache/comet/parquet/ColumnReader.java | 3 +-- .../java/org/apache/comet/parquet/ConstantColumnReader.java | 4 ++-- .../src/main/java/org/apache/comet/parquet/FileReader.java | 4 ++-- .../java/org/apache/comet/parquet/MetadataColumnReader.java | 3 +-- common/src/main/java/org/apache/comet/parquet/Native.java | 3 +-- .../java/org/apache/comet/parquet/ParquetColumnSpec.java | 4 ++-- .../src/main/java/org/apache/comet/parquet/ReadOptions.java | 3 +-- .../main/java/org/apache/comet/parquet/RowGroupReader.java | 4 ++-- common/src/main/java/org/apache/comet/parquet/TypeUtil.java | 3 +-- common/src/main/java/org/apache/comet/parquet/Utils.java | 3 +-- .../java/org/apache/comet/parquet/WrappedInputFile.java | 4 ++-- .../src/main/java/org/apache/comet/vector/CometVector.java | 4 ++-- 16 files changed, 26 insertions(+), 35 deletions(-) diff --git a/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java b/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java index c3bb1cbf18..49ce92a408 100644 --- a/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java +++ b/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java @@ -19,12 +19,12 @@ package org.apache.arrow.c; -import org.apache.comet.IcebergApi; - import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.types.pojo.Field; +import org.apache.comet.IcebergApi; + /** This is a simple wrapper around SchemaImporter to make it accessible from Java Arrow. */ public abstract class AbstractCometSchemaImporter { private final BufferAllocator allocator; diff --git a/common/src/main/java/org/apache/comet/IcebergApi.java b/common/src/main/java/org/apache/comet/IcebergApi.java index ce1182bcf5..915fd87848 100644 --- a/common/src/main/java/org/apache/comet/IcebergApi.java +++ b/common/src/main/java/org/apache/comet/IcebergApi.java @@ -29,9 +29,9 @@ * Indicates that the annotated element is part of the public API used by Apache Iceberg. * *

This annotation marks classes, methods, constructors, and fields that form the contract - * between Comet and Iceberg. Changes to these APIs may break Iceberg's Comet integration, - * so contributors should exercise caution and consider backward compatibility when modifying - * annotated elements. + * between Comet and Iceberg. Changes to these APIs may break Iceberg's Comet integration, so + * contributors should exercise caution and consider backward compatibility when modifying annotated + * elements. * *

The Iceberg integration uses Comet's native Parquet reader for accelerated vectorized reads. * See the contributor guide documentation for details on how Iceberg uses these APIs. diff --git a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java index 7289de8226..f8385f41b6 100644 --- a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,6 +28,7 @@ import org.apache.spark.sql.types.TimestampNTZType$; import org.apache.comet.CometConf; +import org.apache.comet.IcebergApi; import org.apache.comet.vector.CometVector; /** Base class for Comet Parquet column reader implementations. */ @@ -64,8 +63,7 @@ public abstract class AbstractColumnReader implements AutoCloseable { protected int batchSize; /** A pointer to the native implementation of ColumnReader. */ - @IcebergApi - protected long nativeHandle; + @IcebergApi protected long nativeHandle; AbstractColumnReader( DataType type, diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java index 22b5a03cf7..57c5d26f3b 100644 --- a/common/src/main/java/org/apache/comet/parquet/BatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.io.Closeable; import java.io.IOException; import java.net.URI; @@ -67,6 +65,7 @@ import org.apache.comet.CometConf; import org.apache.comet.CometSchemaImporter; +import org.apache.comet.IcebergApi; import org.apache.comet.shims.ShimBatchReader; import org.apache.comet.shims.ShimFileFormat; import org.apache.comet.vector.CometVector; diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 89f91864d0..9def88e427 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.io.IOException; import java.nio.ByteBuffer; @@ -46,6 +44,7 @@ import org.apache.comet.CometConf; import org.apache.comet.CometSchemaImporter; +import org.apache.comet.IcebergApi; import org.apache.comet.vector.CometDecodedVector; import org.apache.comet.vector.CometDictionary; import org.apache.comet.vector.CometDictionaryVector; diff --git a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java index 95cd8d9641..58f68543bd 100644 --- a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.math.BigInteger; import org.apache.parquet.column.ColumnDescriptor; @@ -29,6 +27,8 @@ import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.types.UTF8String; +import org.apache.comet.IcebergApi; + /** * A column reader that always return constant vectors. Used for reading partition columns, for * instance. diff --git a/common/src/main/java/org/apache/comet/parquet/FileReader.java b/common/src/main/java/org/apache/comet/parquet/FileReader.java index ecf92f405f..80c214fc7c 100644 --- a/common/src/main/java/org/apache/comet/parquet/FileReader.java +++ b/common/src/main/java/org/apache/comet/parquet/FileReader.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.io.Closeable; import java.io.IOException; import java.io.InputStream; @@ -92,6 +90,8 @@ import org.apache.parquet.schema.PrimitiveType; import org.apache.spark.sql.execution.metric.SQLMetric; +import org.apache.comet.IcebergApi; + import static org.apache.parquet.hadoop.ParquetFileWriter.EFMAGIC; import static org.apache.parquet.hadoop.ParquetFileWriter.MAGIC; diff --git a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java index cbe0f369d2..f20c450e15 100644 --- a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import org.apache.arrow.c.ArrowArray; import org.apache.arrow.c.ArrowSchema; import org.apache.arrow.c.Data; @@ -30,6 +28,7 @@ import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.types.DataType; +import org.apache.comet.IcebergApi; import org.apache.comet.vector.CometPlainVector; import org.apache.comet.vector.CometVector; diff --git a/common/src/main/java/org/apache/comet/parquet/Native.java b/common/src/main/java/org/apache/comet/parquet/Native.java index 467abf9f17..babd0d392c 100644 --- a/common/src/main/java/org/apache/comet/parquet/Native.java +++ b/common/src/main/java/org/apache/comet/parquet/Native.java @@ -19,11 +19,10 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.nio.ByteBuffer; import java.util.Map; +import org.apache.comet.IcebergApi; import org.apache.comet.NativeBase; public final class Native extends NativeBase { diff --git a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java index 704fa3e3f1..95fed362d3 100644 --- a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java +++ b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java @@ -19,10 +19,10 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.util.Map; +import org.apache.comet.IcebergApi; + /** * Parquet ColumnSpec encapsulates the information withing a Parquet ColumnDescriptor. Utility * methods can convert from and to a ColumnDescriptor The only purpose of this class is to allow diff --git a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java index 5375b1bcac..6363aea31d 100644 --- a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java +++ b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,6 +27,7 @@ import org.apache.spark.launcher.SparkLauncher; import org.apache.comet.CometConf; +import org.apache.comet.IcebergApi; /** * Comet specific Parquet related read options. diff --git a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java index dae566f9a1..0ca7478b7b 100644 --- a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java +++ b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -31,6 +29,8 @@ import org.apache.parquet.column.page.PageReader; import org.apache.parquet.internal.filter2.columnindex.RowRanges; +import org.apache.comet.IcebergApi; + @IcebergApi public class RowGroupReader implements PageReadStore { private final Map readers = new HashMap<>(); diff --git a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java index d3d23e3ffd..818c828cfa 100644 --- a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java +++ b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.util.Arrays; import org.apache.parquet.column.ColumnDescriptor; @@ -32,6 +30,7 @@ import org.apache.spark.sql.types.*; import org.apache.comet.CometConf; +import org.apache.comet.IcebergApi; import static org.apache.comet.parquet.Utils.descriptorToParquetColumnSpec; diff --git a/common/src/main/java/org/apache/comet/parquet/Utils.java b/common/src/main/java/org/apache/comet/parquet/Utils.java index af2865a4e6..c1b6fb2c1d 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.util.HashMap; import java.util.Map; @@ -32,6 +30,7 @@ import org.apache.spark.sql.types.*; import org.apache.comet.CometSchemaImporter; +import org.apache.comet.IcebergApi; public class Utils { diff --git a/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java b/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java index bd5448c1d4..9b5e50ddb4 100644 --- a/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java +++ b/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java @@ -19,8 +19,6 @@ package org.apache.comet.parquet; -import org.apache.comet.IcebergApi; - import java.io.IOException; import java.io.InputStream; import java.lang.reflect.Method; @@ -28,6 +26,8 @@ import org.apache.parquet.io.InputFile; import org.apache.parquet.io.SeekableInputStream; +import org.apache.comet.IcebergApi; + /** * Wraps an Object that possibly implements the methods of a Parquet InputFile (but is not a Parquet * InputFile). Such an object` exists, for instance, in Iceberg's InputFile diff --git a/common/src/main/java/org/apache/comet/vector/CometVector.java b/common/src/main/java/org/apache/comet/vector/CometVector.java index be50d3b7a9..bc1545ac3b 100644 --- a/common/src/main/java/org/apache/comet/vector/CometVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometVector.java @@ -19,8 +19,6 @@ package org.apache.comet.vector; -import org.apache.comet.IcebergApi; - import java.math.BigDecimal; import java.math.BigInteger; @@ -41,6 +39,8 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.types.UTF8String; +import org.apache.comet.IcebergApi; + /** Base class for all Comet column vector implementations. */ @IcebergApi public abstract class CometVector extends ColumnVector { From 8798b1d2d57367da84a06bb056c6fdcbe5812709 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 22 Jan 2026 10:03:42 -0700 Subject: [PATCH 5/7] test: add iceberg-public-api module with API stability tests Add a new Maven module containing dedicated unit tests for all @IcebergApi annotated classes, ensuring the public API contract with Apache Iceberg remains stable and tested. Key changes: - Add iceberg-public-api module with 169 tests covering all @IcebergApi classes - Fix CometVector constructor visibility (protected -> public) to match API annotation - Add IcebergApiVerificationTest for reflection-based API verification - Add tests for FileReader, BatchReader, ColumnReader, Native, TypeUtil, Utils - Add tests for CometVector, CometSchemaImporter, WrappedInputFile Co-Authored-By: Claude Opus 4.5 --- .../org/apache/comet/vector/CometVector.java | 2 +- iceberg-public-api/pom.xml | 178 +++++++++ .../comet/iceberg/api/AbstractApiTest.java | 111 ++++++ .../api/IcebergApiVerificationTest.java | 376 ++++++++++++++++++ .../comet/iceberg/api/ParquetTestHelper.java | 162 ++++++++ .../parquet/AbstractColumnReaderApiTest.java | 98 +++++ .../api/parquet/BatchReaderApiTest.java | 100 +++++ .../api/parquet/ColumnReaderApiTest.java | 93 +++++ .../parquet/ConstantColumnReaderApiTest.java | 76 ++++ .../api/parquet/FileReaderApiTest.java | 183 +++++++++ .../parquet/MetadataColumnReaderApiTest.java | 92 +++++ .../iceberg/api/parquet/NativeApiTest.java | 195 +++++++++ .../api/parquet/ParquetColumnSpecApiTest.java | 210 ++++++++++ .../api/parquet/ReadOptionsApiTest.java | 123 ++++++ .../api/parquet/RowGroupReaderApiTest.java | 155 ++++++++ .../iceberg/api/parquet/TypeUtilApiTest.java | 207 ++++++++++ .../iceberg/api/parquet/UtilsApiTest.java | 371 +++++++++++++++++ .../api/parquet/WrappedInputFileApiTest.java | 138 +++++++ .../schema/CometSchemaImporterApiTest.java | 116 ++++++ .../api/vector/CometVectorApiTest.java | 179 +++++++++ .../src/test/resources/log4j2.properties | 40 ++ pom.xml | 1 + 22 files changed, 3205 insertions(+), 1 deletion(-) create mode 100644 iceberg-public-api/pom.xml create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/AbstractApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/IcebergApiVerificationTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/ParquetTestHelper.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/AbstractColumnReaderApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/BatchReaderApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ColumnReaderApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ConstantColumnReaderApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/FileReaderApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/MetadataColumnReaderApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/NativeApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ParquetColumnSpecApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ReadOptionsApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/RowGroupReaderApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/TypeUtilApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/UtilsApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/WrappedInputFileApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/schema/CometSchemaImporterApiTest.java create mode 100644 iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/vector/CometVectorApiTest.java create mode 100644 iceberg-public-api/src/test/resources/log4j2.properties diff --git a/common/src/main/java/org/apache/comet/vector/CometVector.java b/common/src/main/java/org/apache/comet/vector/CometVector.java index bc1545ac3b..6dda765d51 100644 --- a/common/src/main/java/org/apache/comet/vector/CometVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometVector.java @@ -62,7 +62,7 @@ public abstract class CometVector extends ColumnVector { } @IcebergApi - protected CometVector(DataType type, boolean useDecimal128) { + public CometVector(DataType type, boolean useDecimal128) { super(type); this.useDecimal128 = useDecimal128; } diff --git a/iceberg-public-api/pom.xml b/iceberg-public-api/pom.xml new file mode 100644 index 0000000000..50a05447ab --- /dev/null +++ b/iceberg-public-api/pom.xml @@ -0,0 +1,178 @@ + + + + + + + 4.0.0 + + org.apache.datafusion + comet-parent-spark${spark.version.short}_${scala.binary.version} + 0.13.0-SNAPSHOT + ../pom.xml + + + comet-iceberg-public-api-spark${spark.version.short}_${scala.binary.version} + comet-iceberg-public-api + Tests for Comet's public API used by Apache Iceberg + + + + true + true + + + + + + org.apache.datafusion + comet-common-spark${spark.version.short}_${scala.binary.version} + ${project.version} + test + + + + + junit + junit + test + + + org.assertj + assertj-core + test + + + + + org.apache.parquet + parquet-hadoop + test + + + org.apache.parquet + parquet-column + test + + + + + org.apache.hadoop + hadoop-client-minicluster + test + + + commons-logging + commons-logging + + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + test + + + + + org.apache.arrow + arrow-vector + test + + + org.apache.arrow + arrow-memory-unsafe + test + + + org.apache.arrow + arrow-c-data + test + + + + + src/test/java + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-test-source + generate-test-sources + + add-test-source + + + + src/test/java + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + true + false + + + + test-compile + test-compile + + testCompile + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + file:src/test/resources/log4j2.properties + + false + + + + com.diffplug.spotless + spotless-maven-plugin + + + + check + + compile + + + + + + + diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/AbstractApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/AbstractApiTest.java new file mode 100644 index 0000000000..c44057ee6a --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/AbstractApiTest.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.iceberg.api; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; + +import org.junit.After; +import org.junit.Before; + +import org.apache.comet.IcebergApi; + +/** + * Base class for Iceberg API tests. Provides common utilities for testing annotated API elements. + */ +public abstract class AbstractApiTest { + + protected Path tempDir; + + @Before + public void setUp() throws IOException { + tempDir = Files.createTempDirectory("iceberg-api-test"); + } + + @After + public void tearDown() throws IOException { + if (tempDir != null && Files.exists(tempDir)) { + Files.walk(tempDir).sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } + } + + /** Checks if a class has the @IcebergApi annotation. */ + protected static boolean hasIcebergApiAnnotation(Class clazz) { + return clazz.isAnnotationPresent(IcebergApi.class); + } + + /** Checks if a method has the @IcebergApi annotation. */ + protected static boolean hasIcebergApiAnnotation(Method method) { + return method.isAnnotationPresent(IcebergApi.class); + } + + /** Checks if a constructor has the @IcebergApi annotation. */ + protected static boolean hasIcebergApiAnnotation(Constructor constructor) { + return constructor.isAnnotationPresent(IcebergApi.class); + } + + /** Checks if a field has the @IcebergApi annotation. */ + protected static boolean hasIcebergApiAnnotation(Field field) { + return field.isAnnotationPresent(IcebergApi.class); + } + + /** Checks if a class is public. */ + protected static boolean isPublic(Class clazz) { + return Modifier.isPublic(clazz.getModifiers()); + } + + /** Checks if a method is public. */ + protected static boolean isPublic(Method method) { + return Modifier.isPublic(method.getModifiers()); + } + + /** Checks if a constructor is public. */ + protected static boolean isPublic(Constructor constructor) { + return Modifier.isPublic(constructor.getModifiers()); + } + + /** Checks if a field is public or protected. */ + protected static boolean isAccessible(Field field) { + int modifiers = field.getModifiers(); + return Modifier.isPublic(modifiers) || Modifier.isProtected(modifiers); + } + + /** Checks if native library is available. */ + protected static boolean isNativeLibraryAvailable() { + try { + Class.forName("org.apache.comet.NativeBase"); + return true; + } catch (Throwable t) { + return false; + } + } + + /** Creates a temp file path for testing. */ + protected String createTempFilePath(String name) { + return tempDir.resolve(name).toString(); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/IcebergApiVerificationTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/IcebergApiVerificationTest.java new file mode 100644 index 0000000000..be761e3874 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/IcebergApiVerificationTest.java @@ -0,0 +1,376 @@ +/* + * 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.comet.iceberg.api; + +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.junit.Test; + +import org.apache.arrow.c.AbstractCometSchemaImporter; + +import org.apache.comet.CometSchemaImporter; +import org.apache.comet.IcebergApi; +import org.apache.comet.parquet.*; +import org.apache.comet.vector.CometVector; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Verifies that all @IcebergApi annotated elements are properly accessible. This test uses + * reflection to scan for all annotated elements and ensures they meet the public API contract. + */ +public class IcebergApiVerificationTest extends AbstractApiTest { + + /** List of all classes that should have @IcebergApi annotations. */ + private static final List> ICEBERG_API_CLASSES = + Arrays.asList( + // Parquet classes + FileReader.class, + BatchReader.class, + ColumnReader.class, + ConstantColumnReader.class, + MetadataColumnReader.class, + AbstractColumnReader.class, + RowGroupReader.class, + ParquetColumnSpec.class, + ReadOptions.class, + ReadOptions.Builder.class, + Utils.class, + TypeUtil.class, + WrappedInputFile.class, + Native.class, + // Vector classes + CometVector.class, + // Schema classes + CometSchemaImporter.class, + AbstractCometSchemaImporter.class); + + @Test + public void testIcebergApiAnnotationIsRetainedAtRuntime() { + Retention retention = IcebergApi.class.getAnnotation(Retention.class); + assertThat(retention).isNotNull(); + assertThat(retention.value()).isEqualTo(RetentionPolicy.RUNTIME); + } + + @Test + public void testAllExpectedClassesHaveIcebergApiAnnotation() { + List missingAnnotations = new ArrayList<>(); + + for (Class clazz : ICEBERG_API_CLASSES) { + if (!hasIcebergApiAnnotation(clazz)) { + // Check if any methods, constructors, or fields have the annotation + boolean hasAnyAnnotation = false; + + for (Method method : clazz.getDeclaredMethods()) { + if (hasIcebergApiAnnotation(method)) { + hasAnyAnnotation = true; + break; + } + } + + if (!hasAnyAnnotation) { + for (Constructor constructor : clazz.getDeclaredConstructors()) { + if (hasIcebergApiAnnotation(constructor)) { + hasAnyAnnotation = true; + break; + } + } + } + + if (!hasAnyAnnotation) { + for (Field field : clazz.getDeclaredFields()) { + if (hasIcebergApiAnnotation(field)) { + hasAnyAnnotation = true; + break; + } + } + } + + if (!hasAnyAnnotation) { + missingAnnotations.add(clazz.getName()); + } + } + } + + assertThat(missingAnnotations).as("Classes without @IcebergApi annotation").isEmpty(); + } + + @Test + public void testAnnotatedClassesArePublic() { + List nonPublicClasses = new ArrayList<>(); + + for (Class clazz : ICEBERG_API_CLASSES) { + if (hasIcebergApiAnnotation(clazz) && !isPublic(clazz)) { + nonPublicClasses.add(clazz.getName()); + } + } + + assertThat(nonPublicClasses).as("@IcebergApi annotated classes that are not public").isEmpty(); + } + + @Test + public void testAnnotatedMethodsArePublic() { + List nonPublicMethods = new ArrayList<>(); + + for (Class clazz : ICEBERG_API_CLASSES) { + for (Method method : clazz.getDeclaredMethods()) { + if (hasIcebergApiAnnotation(method) && !isPublic(method)) { + nonPublicMethods.add(clazz.getSimpleName() + "." + method.getName()); + } + } + } + + assertThat(nonPublicMethods).as("@IcebergApi annotated methods that are not public").isEmpty(); + } + + @Test + public void testAnnotatedConstructorsArePublic() { + List nonPublicConstructors = new ArrayList<>(); + + for (Class clazz : ICEBERG_API_CLASSES) { + for (Constructor constructor : clazz.getDeclaredConstructors()) { + if (hasIcebergApiAnnotation(constructor) && !isPublic(constructor)) { + nonPublicConstructors.add( + clazz.getSimpleName() + "(" + Arrays.toString(constructor.getParameterTypes()) + ")"); + } + } + } + + assertThat(nonPublicConstructors) + .as("@IcebergApi annotated constructors that are not public") + .isEmpty(); + } + + @Test + public void testAnnotatedFieldsAreAccessible() { + List inaccessibleFields = new ArrayList<>(); + + for (Class clazz : ICEBERG_API_CLASSES) { + for (Field field : clazz.getDeclaredFields()) { + if (hasIcebergApiAnnotation(field) && !isAccessible(field)) { + inaccessibleFields.add(clazz.getSimpleName() + "." + field.getName()); + } + } + } + + assertThat(inaccessibleFields) + .as("@IcebergApi annotated fields that are not accessible") + .isEmpty(); + } + + @Test + public void testFileReaderHasExpectedPublicApi() { + Class clazz = FileReader.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + + // Check for expected public methods + assertMethodExists(clazz, "readNextRowGroup"); + assertMethodExists(clazz, "skipNextRowGroup"); + assertMethodExists(clazz, "setRequestedSchemaFromSpecs", List.class); + assertMethodExists(clazz, "close"); + } + + @Test + public void testBatchReaderHasExpectedPublicApi() { + Class clazz = BatchReader.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + + // Check for expected public methods + assertMethodExists(clazz, "setSparkSchema", org.apache.spark.sql.types.StructType.class); + assertMethodExists(clazz, "getColumnReaders"); + assertMethodExists(clazz, "nextBatch", int.class); + + // Check for expected constructor + assertConstructorExists(clazz, AbstractColumnReader[].class); + } + + @Test + public void testColumnReaderHasExpectedPublicApi() { + Class clazz = ColumnReader.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + + // Check for expected public methods + assertMethodExists(clazz, "setPageReader", org.apache.parquet.column.page.PageReader.class); + assertMethodExists(clazz, "setRowGroupReader", RowGroupReader.class, ParquetColumnSpec.class); + } + + @Test + public void testParquetColumnSpecHasExpectedPublicApi() { + Class clazz = ParquetColumnSpec.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + + // Check for expected getter methods + assertMethodExists(clazz, "getFieldId"); + assertMethodExists(clazz, "getPath"); + assertMethodExists(clazz, "getPhysicalType"); + assertMethodExists(clazz, "getTypeLength"); + assertMethodExists(clazz, "getMaxRepetitionLevel"); + assertMethodExists(clazz, "getMaxDefinitionLevel"); + assertMethodExists(clazz, "getLogicalTypeName"); + assertMethodExists(clazz, "getLogicalTypeParams"); + } + + @Test + public void testReadOptionsHasExpectedPublicApi() { + Class clazz = ReadOptions.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + + // Check for builder method + assertMethodExists(clazz, "builder", org.apache.hadoop.conf.Configuration.class); + + // Check Builder class + Class builderClass = ReadOptions.Builder.class; + assertThat(hasIcebergApiAnnotation(builderClass)).isTrue(); + assertMethodExists(builderClass, "build"); + } + + @Test + public void testRowGroupReaderHasExpectedPublicApi() { + Class clazz = RowGroupReader.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + + assertMethodExists(clazz, "getRowCount"); + } + + @Test + public void testUtilsHasExpectedPublicApi() { + Class clazz = Utils.class; + + // Check for expected static methods + assertStaticMethodExists( + clazz, + "getColumnReader", + org.apache.spark.sql.types.DataType.class, + ParquetColumnSpec.class, + CometSchemaImporter.class, + int.class, + boolean.class, + boolean.class, + boolean.class); + + assertStaticMethodExists(clazz, "buildColumnDescriptor", ParquetColumnSpec.class); + assertStaticMethodExists( + clazz, "descriptorToParquetColumnSpec", org.apache.parquet.column.ColumnDescriptor.class); + } + + @Test + public void testTypeUtilHasExpectedPublicApi() { + Class clazz = TypeUtil.class; + + assertStaticMethodExists( + clazz, "convertToParquet", org.apache.spark.sql.types.StructField.class); + } + + @Test + public void testCometVectorHasExpectedPublicApi() { + Class clazz = CometVector.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + assertThat(Modifier.isAbstract(clazz.getModifiers())).isTrue(); + + // Check for expected methods + assertMethodExists(clazz, "setNumNulls", int.class); + assertMethodExists(clazz, "setNumValues", int.class); + assertMethodExists(clazz, "numValues"); + assertMethodExists(clazz, "getValueVector"); + assertMethodExists(clazz, "slice", int.class, int.class); + } + + @Test + public void testCometSchemaImporterHasExpectedPublicApi() { + Class clazz = CometSchemaImporter.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + + // Check for expected constructor + assertConstructorExists(clazz, org.apache.arrow.memory.BufferAllocator.class); + } + + @Test + public void testWrappedInputFileHasExpectedPublicApi() { + Class clazz = WrappedInputFile.class; + assertThat(hasIcebergApiAnnotation(clazz)).isTrue(); + assertThat(isPublic(clazz)).isTrue(); + + // Check for expected constructor + assertConstructorExists(clazz, Object.class); + } + + @Test + public void testNativeHasExpectedPublicApi() { + Class clazz = Native.class; + + // Check for expected native methods used by Iceberg + assertStaticMethodExists(clazz, "resetBatch", long.class); + assertStaticMethodExists(clazz, "setPosition", long.class, long.class, int.class); + assertStaticMethodExists(clazz, "setIsDeleted", long.class, boolean[].class); + } + + private void assertMethodExists(Class clazz, String methodName, Class... paramTypes) { + try { + Method method = clazz.getMethod(methodName, paramTypes); + assertThat(method).isNotNull(); + } catch (NoSuchMethodException e) { + throw new AssertionError( + "Expected method " + methodName + " not found in " + clazz.getSimpleName(), e); + } + } + + private void assertStaticMethodExists(Class clazz, String methodName, Class... paramTypes) { + try { + Method method = clazz.getMethod(methodName, paramTypes); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())) + .as(methodName + " should be static") + .isTrue(); + } catch (NoSuchMethodException e) { + throw new AssertionError( + "Expected static method " + methodName + " not found in " + clazz.getSimpleName(), e); + } + } + + private void assertConstructorExists(Class clazz, Class... paramTypes) { + try { + Constructor constructor = clazz.getConstructor(paramTypes); + assertThat(constructor).isNotNull(); + } catch (NoSuchMethodException e) { + throw new AssertionError( + "Expected constructor with params " + + Arrays.toString(paramTypes) + + " not found in " + + clazz.getSimpleName(), + e); + } + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/ParquetTestHelper.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/ParquetTestHelper.java new file mode 100644 index 0000000000..ea57fde570 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/ParquetTestHelper.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.iceberg.api; + +import java.io.IOException; +import java.nio.file.Path; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.SimpleGroupFactory; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.example.GroupWriteSupport; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; + +/** Helper class for creating test Parquet files. */ +public class ParquetTestHelper { + + /** Schema for a simple test file with int and string columns. */ + public static final String SIMPLE_SCHEMA = + "message test_schema {" + + " required int32 id;" + + " optional binary name (UTF8);" + + " optional int64 value;" + + "}"; + + /** Schema for testing decimal types. */ + public static final String DECIMAL_SCHEMA = + "message decimal_schema {" + + " required int32 id;" + + " optional fixed_len_byte_array(16) decimal_col (DECIMAL(38,10));" + + "}"; + + /** Schema for testing timestamp types. */ + public static final String TIMESTAMP_SCHEMA = + "message timestamp_schema {" + + " required int32 id;" + + " optional int64 ts_col (TIMESTAMP(MICROS,true));" + + "}"; + + private final Configuration conf; + + public ParquetTestHelper() { + this.conf = new Configuration(); + } + + public ParquetTestHelper(Configuration conf) { + this.conf = conf; + } + + public Configuration getConfiguration() { + return conf; + } + + /** + * Creates a simple Parquet file with the given number of rows. + * + * @param filePath path to write the file + * @param numRows number of rows to write + * @return the path to the created file + */ + public String createSimpleParquetFile(Path filePath, int numRows) throws IOException { + return createSimpleParquetFile(filePath.toString(), numRows); + } + + /** + * Creates a simple Parquet file with the given number of rows. + * + * @param filePath path to write the file + * @param numRows number of rows to write + * @return the path to the created file + */ + public String createSimpleParquetFile(String filePath, int numRows) throws IOException { + MessageType schema = MessageTypeParser.parseMessageType(SIMPLE_SCHEMA); + return createParquetFile( + filePath, schema, numRows, (factory, i) -> createSimpleRow(factory, i)); + } + + /** + * Creates a Parquet file with the given schema and row creator. + * + * @param filePath path to write the file + * @param schema the Parquet schema + * @param numRows number of rows to write + * @param rowCreator function to create each row + * @return the path to the created file + */ + public String createParquetFile( + String filePath, MessageType schema, int numRows, RowCreator rowCreator) throws IOException { + org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(filePath); + + GroupWriteSupport.setSchema(schema, conf); + SimpleGroupFactory factory = new SimpleGroupFactory(schema); + + try (ParquetWriter writer = + new ParquetWriter<>( + hadoopPath, + new GroupWriteSupport(), + CompressionCodecName.UNCOMPRESSED, + ParquetWriter.DEFAULT_BLOCK_SIZE, + ParquetWriter.DEFAULT_PAGE_SIZE, + ParquetWriter.DEFAULT_PAGE_SIZE, + ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, + ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, + ParquetProperties.WriterVersion.PARQUET_1_0, + conf)) { + + for (int i = 0; i < numRows; i++) { + writer.write(rowCreator.createRow(factory, i)); + } + } + + return filePath; + } + + private Group createSimpleRow(SimpleGroupFactory factory, int rowIndex) { + Group group = factory.newGroup(); + group.add("id", rowIndex); + if (rowIndex % 3 != 0) { // Make some values null + group.add("name", "name_" + rowIndex); + } + if (rowIndex % 2 == 0) { + group.add("value", (long) rowIndex * 100); + } + return group; + } + + /** Deletes the file at the given path if it exists. */ + public void deleteFile(String filePath) throws IOException { + org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(filePath); + FileSystem fs = hadoopPath.getFileSystem(conf); + if (fs.exists(hadoopPath)) { + fs.delete(hadoopPath, false); + } + } + + /** Functional interface for creating rows. */ + @FunctionalInterface + public interface RowCreator { + Group createRow(SimpleGroupFactory factory, int rowIndex); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/AbstractColumnReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/AbstractColumnReaderApiTest.java new file mode 100644 index 0000000000..39ea3166c2 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/AbstractColumnReaderApiTest.java @@ -0,0 +1,98 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; + +import org.junit.Test; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.AbstractColumnReader; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the AbstractColumnReader public API. */ +public class AbstractColumnReaderApiTest extends AbstractApiTest { + + @Test + public void testAbstractColumnReaderHasIcebergApiAnnotation() { + assertThat(hasIcebergApiAnnotation(AbstractColumnReader.class)).isTrue(); + } + + @Test + public void testAbstractColumnReaderIsPublic() { + assertThat(isPublic(AbstractColumnReader.class)).isTrue(); + } + + @Test + public void testAbstractColumnReaderIsAbstract() { + assertThat(Modifier.isAbstract(AbstractColumnReader.class.getModifiers())).isTrue(); + } + + @Test + public void testImplementsAutoCloseable() { + assertThat(AutoCloseable.class.isAssignableFrom(AbstractColumnReader.class)).isTrue(); + } + + @Test + public void testSetBatchSizeMethodExists() throws NoSuchMethodException { + Method method = AbstractColumnReader.class.getMethod("setBatchSize", int.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testCloseMethodExists() throws NoSuchMethodException { + Method method = AbstractColumnReader.class.getMethod("close"); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testReadBatchMethodExists() throws NoSuchMethodException { + Method method = AbstractColumnReader.class.getMethod("readBatch", int.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); + } + + @Test + public void testCurrentBatchMethodExists() throws NoSuchMethodException { + Method method = AbstractColumnReader.class.getMethod("currentBatch"); + assertThat(method).isNotNull(); + assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("CometVector"); + } + + @Test + public void testNativeHandleFieldExists() throws NoSuchFieldException { + Field field = AbstractColumnReader.class.getDeclaredField("nativeHandle"); + assertThat(field).isNotNull(); + assertThat(hasIcebergApiAnnotation(field)).isTrue(); + assertThat(Modifier.isProtected(field.getModifiers())).isTrue(); + } + + @Test + public void testNativeHandleFieldType() throws NoSuchFieldException { + Field field = AbstractColumnReader.class.getDeclaredField("nativeHandle"); + assertThat(field.getType()).isEqualTo(long.class); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/BatchReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/BatchReaderApiTest.java new file mode 100644 index 0000000000..949a7b4a78 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/BatchReaderApiTest.java @@ -0,0 +1,100 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; + +import org.junit.Test; + +import org.apache.spark.sql.types.StructType; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.AbstractColumnReader; +import org.apache.comet.parquet.BatchReader; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the BatchReader public API. */ +public class BatchReaderApiTest extends AbstractApiTest { + + @Test + public void testBatchReaderHasIcebergApiAnnotation() { + assertThat(hasIcebergApiAnnotation(BatchReader.class)).isTrue(); + } + + @Test + public void testBatchReaderIsPublic() { + assertThat(isPublic(BatchReader.class)).isTrue(); + } + + @Test + public void testConstructorWithColumnReadersExists() throws NoSuchMethodException { + Constructor constructor = BatchReader.class.getConstructor(AbstractColumnReader[].class); + assertThat(constructor).isNotNull(); + assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); + } + + @Test + public void testSetSparkSchemaMethodExists() throws NoSuchMethodException { + Method method = BatchReader.class.getMethod("setSparkSchema", StructType.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testGetColumnReadersMethodExists() throws NoSuchMethodException { + Method method = BatchReader.class.getMethod("getColumnReaders"); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + assertThat(method.getReturnType()).isEqualTo(AbstractColumnReader[].class); + } + + @Test + public void testNextBatchWithSizeMethodExists() throws NoSuchMethodException { + Method method = BatchReader.class.getMethod("nextBatch", int.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + assertThat(method.getReturnType()).isEqualTo(boolean.class); + } + + @Test + public void testCurrentBatchMethodExists() throws NoSuchMethodException { + Method method = BatchReader.class.getMethod("currentBatch"); + assertThat(method).isNotNull(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("ColumnarBatch"); + } + + @Test + public void testCloseMethodExists() throws NoSuchMethodException { + Method method = BatchReader.class.getMethod("close"); + assertThat(method).isNotNull(); + } + + @Test + public void testImplementsCloseable() { + assertThat(java.io.Closeable.class.isAssignableFrom(BatchReader.class)).isTrue(); + } + + @Test + public void testExtendsRecordReader() { + assertThat(BatchReader.class.getSuperclass().getSimpleName()).isEqualTo("RecordReader"); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ColumnReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ColumnReaderApiTest.java new file mode 100644 index 0000000000..4b851f6f5d --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ColumnReaderApiTest.java @@ -0,0 +1,93 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.lang.reflect.Method; + +import org.junit.Test; + +import org.apache.parquet.column.page.PageReader; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.ColumnReader; +import org.apache.comet.parquet.ParquetColumnSpec; +import org.apache.comet.parquet.RowGroupReader; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the ColumnReader public API. Uses reflection for signature verification. */ +public class ColumnReaderApiTest extends AbstractApiTest { + + @Test + public void testColumnReaderHasIcebergApiAnnotation() { + assertThat(hasIcebergApiAnnotation(ColumnReader.class)).isTrue(); + } + + @Test + public void testColumnReaderIsPublic() { + assertThat(isPublic(ColumnReader.class)).isTrue(); + } + + @Test + public void testSetPageReaderMethodExists() throws NoSuchMethodException { + Method method = ColumnReader.class.getMethod("setPageReader", PageReader.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testSetRowGroupReaderMethodExists() throws NoSuchMethodException { + Method method = + ColumnReader.class.getMethod( + "setRowGroupReader", RowGroupReader.class, ParquetColumnSpec.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testExtendsAbstractColumnReader() { + assertThat(ColumnReader.class.getSuperclass().getSimpleName()) + .isEqualTo("AbstractColumnReader"); + } + + @Test + public void testImplementsAutoCloseable() { + assertThat(AutoCloseable.class.isAssignableFrom(ColumnReader.class)).isTrue(); + } + + @Test + public void testReadBatchMethodExists() throws NoSuchMethodException { + Method method = ColumnReader.class.getMethod("readBatch", int.class); + assertThat(method).isNotNull(); + } + + @Test + public void testCurrentBatchMethodExists() throws NoSuchMethodException { + Method method = ColumnReader.class.getMethod("currentBatch"); + assertThat(method).isNotNull(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("CometVector"); + } + + @Test + public void testCloseMethodExists() throws NoSuchMethodException { + Method method = ColumnReader.class.getMethod("close"); + assertThat(method).isNotNull(); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ConstantColumnReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ConstantColumnReaderApiTest.java new file mode 100644 index 0000000000..a2ba79f7fb --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ConstantColumnReaderApiTest.java @@ -0,0 +1,76 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.lang.reflect.Constructor; + +import org.junit.Test; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.ConstantColumnReader; +import org.apache.comet.parquet.MetadataColumnReader; +import org.apache.comet.parquet.ParquetColumnSpec; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the ConstantColumnReader public API. */ +public class ConstantColumnReaderApiTest extends AbstractApiTest { + + @Test + public void testConstantColumnReaderHasIcebergApiAnnotation() { + assertThat(hasIcebergApiAnnotation(ConstantColumnReader.class)).isTrue(); + } + + @Test + public void testConstantColumnReaderIsPublic() { + assertThat(isPublic(ConstantColumnReader.class)).isTrue(); + } + + @Test + public void testExtendsMetadataColumnReader() { + assertThat(MetadataColumnReader.class.isAssignableFrom(ConstantColumnReader.class)).isTrue(); + } + + @Test + public void testConstructorWithColumnDescriptorExists() throws NoSuchMethodException { + Constructor constructor = + ConstantColumnReader.class.getConstructor( + DataType.class, ColumnDescriptor.class, Object.class, boolean.class); + assertThat(constructor).isNotNull(); + assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); + } + + @Test + public void testConstructorWithParquetColumnSpecExists() throws NoSuchMethodException { + Constructor constructor = + ConstantColumnReader.class.getConstructor( + DataType.class, ParquetColumnSpec.class, Object.class, boolean.class); + assertThat(constructor).isNotNull(); + assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); + } + + @Test + public void testImplementsAutoCloseable() { + assertThat(AutoCloseable.class.isAssignableFrom(ConstantColumnReader.class)).isTrue(); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/FileReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/FileReaderApiTest.java new file mode 100644 index 0000000000..9fca5685fe --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/FileReaderApiTest.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.comet.iceberg.api.parquet; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.iceberg.api.ParquetTestHelper; +import org.apache.comet.parquet.*; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the FileReader public API. */ +public class FileReaderApiTest extends AbstractApiTest { + + private ParquetTestHelper helper; + + @Override + @Before + public void setUp() throws IOException { + super.setUp(); + helper = new ParquetTestHelper(); + } + + @Test + public void testFileReaderConstructorWithWrappedInputFile() throws IOException { + String filePath = createTempFilePath("test_wrapped.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + // Create a mock object that has getLength and newStream methods + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + Map properties = new HashMap<>(); + + try (FileReader reader = + new FileReader(wrappedFile, readOptions, properties, null, null, null, null)) { + assertThat(reader).isNotNull(); + } + } + + @Test + public void testReadNextRowGroup() throws IOException { + String filePath = createTempFilePath("test_row_group.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + + try (FileReader reader = + new FileReader(wrappedFile, readOptions, new HashMap<>(), null, null, null, null)) { + + RowGroupReader rowGroup = reader.readNextRowGroup(); + assertThat(rowGroup).isNotNull(); + assertThat(rowGroup.getRowCount()).isEqualTo(100); + + // Try to read another row group (should be null since small file) + RowGroupReader nextRowGroup = reader.readNextRowGroup(); + assertThat(nextRowGroup).isNull(); + } + } + + @Test + public void testSkipNextRowGroup() throws IOException { + String filePath = createTempFilePath("test_skip.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + + try (FileReader reader = + new FileReader(wrappedFile, readOptions, new HashMap<>(), null, null, null, null)) { + + boolean skipped = reader.skipNextRowGroup(); + assertThat(skipped).isTrue(); + + // Try to skip another row group + boolean skippedAgain = reader.skipNextRowGroup(); + assertThat(skippedAgain).isFalse(); + } + } + + @Test + public void testSetRequestedSchemaFromSpecs() throws IOException { + String filePath = createTempFilePath("test_schema_spec.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + + try (FileReader reader = + new FileReader(wrappedFile, readOptions, new HashMap<>(), null, null, null, null)) { + + // Create column specs for only the 'id' column + List specs = new ArrayList<>(); + specs.add(new ParquetColumnSpec(1, new String[] {"id"}, "INT32", 0, false, 0, 0, null, null)); + + // Set the requested schema + reader.setRequestedSchemaFromSpecs(specs); + + // Read row group with only the requested columns + RowGroupReader rowGroup = reader.readNextRowGroup(); + assertThat(rowGroup).isNotNull(); + } + } + + @Test + public void testClose() throws IOException { + String filePath = createTempFilePath("test_close.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + + FileReader reader = + new FileReader(wrappedFile, readOptions, new HashMap<>(), null, null, null, null); + reader.close(); + // No exception should be thrown + } + + /** Mock InputFile for testing WrappedInputFile. */ + private static class MockInputFile { + private final String filePath; + private final Configuration conf; + + MockInputFile(String filePath, Configuration conf) { + this.filePath = filePath; + this.conf = conf; + } + + public long getLength() throws IOException { + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(filePath); + org.apache.hadoop.fs.FileSystem fs = path.getFileSystem(conf); + return fs.getFileStatus(path).getLen(); + } + + public org.apache.parquet.io.SeekableInputStream newStream() throws IOException { + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(filePath); + org.apache.hadoop.fs.FileSystem fs = path.getFileSystem(conf); + return org.apache.parquet.hadoop.util.HadoopStreams.wrap(fs.open(path)); + } + + @Override + public String toString() { + return filePath; + } + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/MetadataColumnReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/MetadataColumnReaderApiTest.java new file mode 100644 index 0000000000..5f72a8bf6c --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/MetadataColumnReaderApiTest.java @@ -0,0 +1,92 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; + +import org.junit.Test; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.types.DataType; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.AbstractColumnReader; +import org.apache.comet.parquet.MetadataColumnReader; +import org.apache.comet.parquet.ParquetColumnSpec; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the MetadataColumnReader public API. */ +public class MetadataColumnReaderApiTest extends AbstractApiTest { + + @Test + public void testMetadataColumnReaderHasIcebergApiAnnotation() { + assertThat(hasIcebergApiAnnotation(MetadataColumnReader.class)).isTrue(); + } + + @Test + public void testMetadataColumnReaderIsPublic() { + assertThat(isPublic(MetadataColumnReader.class)).isTrue(); + } + + @Test + public void testExtendsAbstractColumnReader() { + assertThat(AbstractColumnReader.class.isAssignableFrom(MetadataColumnReader.class)).isTrue(); + } + + @Test + public void testConstructorWithColumnDescriptorExists() throws NoSuchMethodException { + Constructor constructor = + MetadataColumnReader.class.getConstructor( + DataType.class, ColumnDescriptor.class, boolean.class, boolean.class); + assertThat(constructor).isNotNull(); + assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); + } + + @Test + public void testConstructorWithParquetColumnSpecExists() throws NoSuchMethodException { + Constructor constructor = + MetadataColumnReader.class.getConstructor( + DataType.class, ParquetColumnSpec.class, boolean.class, boolean.class); + assertThat(constructor).isNotNull(); + assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); + } + + @Test + public void testReadBatchMethodExists() throws NoSuchMethodException { + Method method = MetadataColumnReader.class.getMethod("readBatch", int.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testCurrentBatchMethodExists() throws NoSuchMethodException { + Method method = MetadataColumnReader.class.getMethod("currentBatch"); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("CometVector"); + } + + @Test + public void testImplementsAutoCloseable() { + assertThat(AutoCloseable.class.isAssignableFrom(MetadataColumnReader.class)).isTrue(); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/NativeApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/NativeApiTest.java new file mode 100644 index 0000000000..4d682067f4 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/NativeApiTest.java @@ -0,0 +1,195 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; + +import org.junit.Test; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.Native; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Tests for the Native public API. These tests verify method signatures exist using reflection + * since actual invocation requires native library. + */ +public class NativeApiTest extends AbstractApiTest { + + @Test + public void testNativeClassIsPublic() { + assertThat(isPublic(Native.class)).isTrue(); + } + + @Test + public void testNativeClassIsFinal() { + assertThat(Modifier.isFinal(Native.class.getModifiers())).isTrue(); + } + + @Test + public void testResetBatchMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("resetBatch", long.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testSetPositionMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("setPosition", long.class, long.class, int.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testSetIsDeletedMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("setIsDeleted", long.class, boolean[].class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } + + @Test + public void testInitColumnReaderMethodExists() throws NoSuchMethodException { + Method method = + Native.class.getMethod( + "initColumnReader", + int.class, // physicalTypeId + int.class, // logicalTypeId + int.class, // expectedPhysicalTypeId + String[].class, // path + int.class, // maxDl + int.class, // maxRl + int.class, // bitWidth + int.class, // expectedBitWidth + boolean.class, // isSigned + int.class, // typeLength + int.class, // precision + int.class, // expectedPrecision + int.class, // scale + int.class, // expectedScale + int.class, // tu + boolean.class, // isAdjustedUtc + int.class, // batchSize + boolean.class, // useDecimal128 + boolean.class); // useLegacyDateTimestampOrNTZ + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + assertThat(method.getReturnType()).isEqualTo(long.class); + } + + @Test + public void testCloseColumnReaderMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("closeColumnReader", long.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testReadBatchMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("readBatch", long.class, int.class, int.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + assertThat(method.getReturnType()).isEqualTo(int[].class); + } + + @Test + public void testCurrentBatchMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("currentBatch", long.class, long.class, long.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testSetDictionaryPageMethodExists() throws NoSuchMethodException { + Method method = + Native.class.getMethod("setDictionaryPage", long.class, int.class, byte[].class, int.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testSetPageV1MethodExists() throws NoSuchMethodException { + Method method = + Native.class.getMethod("setPageV1", long.class, int.class, byte[].class, int.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testSetNullMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("setNull", long.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testSetBooleanMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("setBoolean", long.class, boolean.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testSetIntMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("setInt", long.class, int.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testSetLongMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("setLong", long.class, long.class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testSetBinaryMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("setBinary", long.class, byte[].class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } + + @Test + public void testSetDecimalMethodExists() throws NoSuchMethodException { + Method method = Native.class.getMethod("setDecimal", long.class, byte[].class); + assertThat(method).isNotNull(); + assertThat(Modifier.isStatic(method.getModifiers())).isTrue(); + assertThat(Modifier.isNative(method.getModifiers())).isTrue(); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ParquetColumnSpecApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ParquetColumnSpecApiTest.java new file mode 100644 index 0000000000..e9d9c93dc2 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ParquetColumnSpecApiTest.java @@ -0,0 +1,210 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.util.HashMap; +import java.util.Map; + +import org.junit.Test; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.ParquetColumnSpec; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the ParquetColumnSpec public API. */ +public class ParquetColumnSpecApiTest extends AbstractApiTest { + + @Test + public void testConstructorAndGetters() { + int fieldId = 1; + String[] path = new String[] {"parent", "child"}; + String physicalType = "INT64"; + int typeLength = 0; + boolean isRepeated = false; + int maxDefinitionLevel = 2; + int maxRepetitionLevel = 1; + String logicalTypeName = "TimestampLogicalTypeAnnotation"; + Map logicalTypeParams = new HashMap<>(); + logicalTypeParams.put("isAdjustedToUTC", "true"); + logicalTypeParams.put("unit", "MICROS"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + fieldId, + path, + physicalType, + typeLength, + isRepeated, + maxDefinitionLevel, + maxRepetitionLevel, + logicalTypeName, + logicalTypeParams); + + assertThat(spec.getFieldId()).isEqualTo(fieldId); + assertThat(spec.getPath()).isEqualTo(path); + assertThat(spec.getPhysicalType()).isEqualTo(physicalType); + assertThat(spec.getTypeLength()).isEqualTo(typeLength); + assertThat(spec.getMaxDefinitionLevel()).isEqualTo(maxDefinitionLevel); + assertThat(spec.getMaxRepetitionLevel()).isEqualTo(maxRepetitionLevel); + assertThat(spec.getLogicalTypeName()).isEqualTo(logicalTypeName); + assertThat(spec.getLogicalTypeParams()).isEqualTo(logicalTypeParams); + } + + @Test + public void testSimpleInt32Column() { + ParquetColumnSpec spec = + new ParquetColumnSpec( + 1, + new String[] {"id"}, + "INT32", + 0, + false, + 0, // required column + 0, + null, + null); + + assertThat(spec.getFieldId()).isEqualTo(1); + assertThat(spec.getPath()).containsExactly("id"); + assertThat(spec.getPhysicalType()).isEqualTo("INT32"); + assertThat(spec.getTypeLength()).isEqualTo(0); + assertThat(spec.getMaxDefinitionLevel()).isEqualTo(0); + assertThat(spec.getMaxRepetitionLevel()).isEqualTo(0); + assertThat(spec.getLogicalTypeName()).isNull(); + assertThat(spec.getLogicalTypeParams()).isNull(); + } + + @Test + public void testOptionalStringColumn() { + ParquetColumnSpec spec = + new ParquetColumnSpec( + 2, + new String[] {"name"}, + "BINARY", + 0, + false, + 1, // optional column + 0, + "StringLogicalTypeAnnotation", + new HashMap<>()); + + assertThat(spec.getFieldId()).isEqualTo(2); + assertThat(spec.getPath()).containsExactly("name"); + assertThat(spec.getPhysicalType()).isEqualTo("BINARY"); + assertThat(spec.getMaxDefinitionLevel()).isEqualTo(1); + assertThat(spec.getMaxRepetitionLevel()).isEqualTo(0); + assertThat(spec.getLogicalTypeName()).isEqualTo("StringLogicalTypeAnnotation"); + } + + @Test + public void testDecimalColumn() { + Map params = new HashMap<>(); + params.put("precision", "38"); + params.put("scale", "10"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 3, + new String[] {"amount"}, + "FIXED_LEN_BYTE_ARRAY", + 16, + false, + 1, + 0, + "DecimalLogicalTypeAnnotation", + params); + + assertThat(spec.getFieldId()).isEqualTo(3); + assertThat(spec.getPath()).containsExactly("amount"); + assertThat(spec.getPhysicalType()).isEqualTo("FIXED_LEN_BYTE_ARRAY"); + assertThat(spec.getTypeLength()).isEqualTo(16); + assertThat(spec.getLogicalTypeName()).isEqualTo("DecimalLogicalTypeAnnotation"); + assertThat(spec.getLogicalTypeParams()).containsEntry("precision", "38"); + assertThat(spec.getLogicalTypeParams()).containsEntry("scale", "10"); + } + + @Test + public void testNestedColumn() { + ParquetColumnSpec spec = + new ParquetColumnSpec( + 4, new String[] {"struct", "nested", "field"}, "INT64", 0, false, 3, 0, null, null); + + assertThat(spec.getPath()).containsExactly("struct", "nested", "field"); + assertThat(spec.getMaxDefinitionLevel()).isEqualTo(3); + } + + @Test + public void testRepeatedColumn() { + ParquetColumnSpec spec = + new ParquetColumnSpec( + 5, new String[] {"list", "element"}, "INT32", 0, true, 2, 1, null, null); + + assertThat(spec.getPath()).containsExactly("list", "element"); + assertThat(spec.getMaxDefinitionLevel()).isEqualTo(2); + assertThat(spec.getMaxRepetitionLevel()).isEqualTo(1); + } + + @Test + public void testTimestampColumn() { + Map params = new HashMap<>(); + params.put("isAdjustedToUTC", "true"); + params.put("unit", "MICROS"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 6, + new String[] {"created_at"}, + "INT64", + 0, + false, + 1, + 0, + "TimestampLogicalTypeAnnotation", + params); + + assertThat(spec.getLogicalTypeName()).isEqualTo("TimestampLogicalTypeAnnotation"); + assertThat(spec.getLogicalTypeParams()).containsEntry("isAdjustedToUTC", "true"); + assertThat(spec.getLogicalTypeParams()).containsEntry("unit", "MICROS"); + } + + @Test + public void testIntLogicalTypeColumn() { + Map params = new HashMap<>(); + params.put("isSigned", "true"); + params.put("bitWidth", "16"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 7, + new String[] {"small_int"}, + "INT32", + 0, + false, + 1, + 0, + "IntLogicalTypeAnnotation", + params); + + assertThat(spec.getLogicalTypeName()).isEqualTo("IntLogicalTypeAnnotation"); + assertThat(spec.getLogicalTypeParams()).containsEntry("isSigned", "true"); + assertThat(spec.getLogicalTypeParams()).containsEntry("bitWidth", "16"); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ReadOptionsApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ReadOptionsApiTest.java new file mode 100644 index 0000000000..c0ea91aa4d --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/ReadOptionsApiTest.java @@ -0,0 +1,123 @@ +/* + * 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.comet.iceberg.api.parquet; + +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.ReadOptions; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the ReadOptions public API. */ +public class ReadOptionsApiTest extends AbstractApiTest { + + @Test + public void testBuilderCreation() { + Configuration conf = new Configuration(); + ReadOptions.Builder builder = ReadOptions.builder(conf); + assertThat(builder).isNotNull(); + } + + @Test + public void testBuildDefaultOptions() { + Configuration conf = new Configuration(); + ReadOptions options = ReadOptions.builder(conf).build(); + + assertThat(options).isNotNull(); + } + + @Test + public void testBuilderWithParallelIO() { + Configuration conf = new Configuration(); + ReadOptions options = + ReadOptions.builder(conf).enableParallelIO(true).withParallelIOThreadPoolSize(8).build(); + + assertThat(options).isNotNull(); + assertThat(options.isParallelIOEnabled()).isTrue(); + assertThat(options.parallelIOThreadPoolSize()).isEqualTo(8); + } + + @Test + public void testBuilderWithIOMergeRanges() { + Configuration conf = new Configuration(); + ReadOptions options = + ReadOptions.builder(conf) + .enableIOMergeRanges(true) + .withIOMergeRangesDelta(1024 * 1024) + .build(); + + assertThat(options).isNotNull(); + assertThat(options.isIOMergeRangesEnabled()).isTrue(); + assertThat(options.getIOMergeRangesDelta()).isEqualTo(1024 * 1024); + } + + @Test + public void testBuilderWithAdjustReadRangeSkew() { + Configuration conf = new Configuration(); + ReadOptions options = ReadOptions.builder(conf).adjustReadRangeSkew(true).build(); + + assertThat(options).isNotNull(); + assertThat(options.adjustReadRangesSkew()).isTrue(); + } + + @Test + public void testBuilderChaining() { + Configuration conf = new Configuration(); + ReadOptions options = + ReadOptions.builder(conf) + .enableParallelIO(true) + .withParallelIOThreadPoolSize(4) + .enableIOMergeRanges(true) + .withIOMergeRangesDelta(512 * 1024) + .adjustReadRangeSkew(false) + .build(); + + assertThat(options).isNotNull(); + assertThat(options.isParallelIOEnabled()).isTrue(); + assertThat(options.parallelIOThreadPoolSize()).isEqualTo(4); + assertThat(options.isIOMergeRangesEnabled()).isTrue(); + assertThat(options.getIOMergeRangesDelta()).isEqualTo(512 * 1024); + assertThat(options.adjustReadRangesSkew()).isFalse(); + } + + @Test + public void testBuilderWithDisabledOptions() { + Configuration conf = new Configuration(); + ReadOptions options = + ReadOptions.builder(conf).enableParallelIO(false).enableIOMergeRanges(false).build(); + + assertThat(options).isNotNull(); + assertThat(options.isParallelIOEnabled()).isFalse(); + assertThat(options.isIOMergeRangesEnabled()).isFalse(); + } + + @Test + public void testS3ConfigConstants() { + // Verify S3-related constants are accessible + assertThat(ReadOptions.S3A_MAX_EXPECTED_PARALLELISM).isEqualTo(32); + assertThat(ReadOptions.S3A_MAXIMUM_CONNECTIONS).isEqualTo("fs.s3a.connection.maximum"); + assertThat(ReadOptions.S3A_DEFAULT_MAX_HTTP_CONNECTIONS).isEqualTo(96); + assertThat(ReadOptions.S3A_READAHEAD_RANGE).isEqualTo("fs.s3a.readahead.range"); + assertThat(ReadOptions.COMET_DEFAULT_READAHEAD_RANGE).isEqualTo(1024 * 1024); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/RowGroupReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/RowGroupReaderApiTest.java new file mode 100644 index 0000000000..519067153b --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/RowGroupReaderApiTest.java @@ -0,0 +1,155 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.io.IOException; +import java.util.HashMap; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.iceberg.api.ParquetTestHelper; +import org.apache.comet.parquet.*; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the RowGroupReader public API. */ +public class RowGroupReaderApiTest extends AbstractApiTest { + + private ParquetTestHelper helper; + + @Override + @Before + public void setUp() throws IOException { + super.setUp(); + helper = new ParquetTestHelper(); + } + + @Test + public void testGetRowCount() throws IOException { + String filePath = createTempFilePath("test_row_count.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + + try (FileReader reader = + new FileReader(wrappedFile, readOptions, new HashMap<>(), null, null, null, null)) { + + RowGroupReader rowGroup = reader.readNextRowGroup(); + assertThat(rowGroup).isNotNull(); + assertThat(rowGroup.getRowCount()).isEqualTo(100); + } + } + + @Test + public void testRowGroupReaderIsPageReadStore() throws IOException { + String filePath = createTempFilePath("test_page_read_store.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + + try (FileReader reader = + new FileReader(wrappedFile, readOptions, new HashMap<>(), null, null, null, null)) { + + RowGroupReader rowGroup = reader.readNextRowGroup(); + assertThat(rowGroup).isNotNull(); + + // RowGroupReader implements PageReadStore + assertThat(rowGroup).isInstanceOf(org.apache.parquet.column.page.PageReadStore.class); + } + } + + @Test + public void testGetRowIndexes() throws IOException { + String filePath = createTempFilePath("test_row_indexes.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + + try (FileReader reader = + new FileReader(wrappedFile, readOptions, new HashMap<>(), null, null, null, null)) { + + RowGroupReader rowGroup = reader.readNextRowGroup(); + assertThat(rowGroup).isNotNull(); + + // getRowIndexes() returns Optional + assertThat(rowGroup.getRowIndexes()).isNotNull(); + } + } + + @Test + public void testGetRowIndexOffset() throws IOException { + String filePath = createTempFilePath("test_row_index_offset.parquet"); + helper.createSimpleParquetFile(filePath, 100); + + MockInputFile mockInputFile = new MockInputFile(filePath, helper.getConfiguration()); + WrappedInputFile wrappedFile = new WrappedInputFile(mockInputFile); + + ReadOptions readOptions = ReadOptions.builder(helper.getConfiguration()).build(); + + try (FileReader reader = + new FileReader(wrappedFile, readOptions, new HashMap<>(), null, null, null, null)) { + + RowGroupReader rowGroup = reader.readNextRowGroup(); + assertThat(rowGroup).isNotNull(); + + // getRowIndexOffset() returns Optional + assertThat(rowGroup.getRowIndexOffset()).isNotNull(); + } + } + + /** Mock InputFile for testing WrappedInputFile. */ + private static class MockInputFile { + private final String filePath; + private final org.apache.hadoop.conf.Configuration conf; + + MockInputFile(String filePath, org.apache.hadoop.conf.Configuration conf) { + this.filePath = filePath; + this.conf = conf; + } + + public long getLength() throws IOException { + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(filePath); + org.apache.hadoop.fs.FileSystem fs = path.getFileSystem(conf); + return fs.getFileStatus(path).getLen(); + } + + public org.apache.parquet.io.SeekableInputStream newStream() throws IOException { + org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(filePath); + org.apache.hadoop.fs.FileSystem fs = path.getFileSystem(conf); + return org.apache.parquet.hadoop.util.HadoopStreams.wrap(fs.open(path)); + } + + @Override + public String toString() { + return filePath; + } + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/TypeUtilApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/TypeUtilApiTest.java new file mode 100644 index 0000000000..958600b032 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/TypeUtilApiTest.java @@ -0,0 +1,207 @@ +/* + * 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.comet.iceberg.api.parquet; + +import org.junit.Test; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.spark.sql.types.*; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.TypeUtil; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the TypeUtil public API. */ +public class TypeUtilApiTest extends AbstractApiTest { + + @Test + public void testConvertBooleanType() { + StructField field = new StructField("bool_col", DataTypes.BooleanType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPath()).containsExactly("bool_col"); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.BOOLEAN); + } + + @Test + public void testConvertIntegerType() { + StructField field = new StructField("int_col", DataTypes.IntegerType, false, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPath()).containsExactly("int_col"); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.INT32); + assertThat(descriptor.getMaxDefinitionLevel()).isEqualTo(0); // required field + } + + @Test + public void testConvertLongType() { + StructField field = new StructField("long_col", DataTypes.LongType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.INT64); + assertThat(descriptor.getMaxDefinitionLevel()).isEqualTo(1); // nullable field + } + + @Test + public void testConvertFloatType() { + StructField field = new StructField("float_col", DataTypes.FloatType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.FLOAT); + } + + @Test + public void testConvertDoubleType() { + StructField field = new StructField("double_col", DataTypes.DoubleType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.DOUBLE); + } + + @Test + public void testConvertStringType() { + StructField field = new StructField("string_col", DataTypes.StringType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.BINARY); + assertThat(descriptor.getPrimitiveType().getLogicalTypeAnnotation()) + .isInstanceOf(LogicalTypeAnnotation.StringLogicalTypeAnnotation.class); + } + + @Test + public void testConvertBinaryType() { + StructField field = new StructField("binary_col", DataTypes.BinaryType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.BINARY); + assertThat(descriptor.getPrimitiveType().getLogicalTypeAnnotation()).isNull(); + } + + @Test + public void testConvertDateType() { + StructField field = new StructField("date_col", DataTypes.DateType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.INT32); + assertThat(descriptor.getPrimitiveType().getLogicalTypeAnnotation()) + .isInstanceOf(LogicalTypeAnnotation.DateLogicalTypeAnnotation.class); + } + + @Test + public void testConvertTimestampType() { + StructField field = + new StructField("timestamp_col", DataTypes.TimestampType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.INT64); + assertThat(descriptor.getPrimitiveType().getLogicalTypeAnnotation()) + .isInstanceOf(LogicalTypeAnnotation.TimestampLogicalTypeAnnotation.class); + + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation tsAnnotation = + (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + assertThat(tsAnnotation.isAdjustedToUTC()).isTrue(); + assertThat(tsAnnotation.getUnit()).isEqualTo(LogicalTypeAnnotation.TimeUnit.MICROS); + } + + @Test + public void testConvertByteType() { + StructField field = new StructField("byte_col", DataTypes.ByteType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.INT32); + + LogicalTypeAnnotation.IntLogicalTypeAnnotation intAnnotation = + (LogicalTypeAnnotation.IntLogicalTypeAnnotation) + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + assertThat(intAnnotation.getBitWidth()).isEqualTo(8); + assertThat(intAnnotation.isSigned()).isTrue(); + } + + @Test + public void testConvertShortType() { + StructField field = new StructField("short_col", DataTypes.ShortType, true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.INT32); + + LogicalTypeAnnotation.IntLogicalTypeAnnotation intAnnotation = + (LogicalTypeAnnotation.IntLogicalTypeAnnotation) + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + assertThat(intAnnotation.getBitWidth()).isEqualTo(16); + assertThat(intAnnotation.isSigned()).isTrue(); + } + + @Test + public void testConvertDecimalType() { + StructField field = + new StructField("decimal_col", new DecimalType(38, 10), true, Metadata.empty()); + ColumnDescriptor descriptor = TypeUtil.convertToParquet(field); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY); + assertThat(descriptor.getPrimitiveType().getTypeLength()).isEqualTo(16); + + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decAnnotation = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + assertThat(decAnnotation.getPrecision()).isEqualTo(38); + assertThat(decAnnotation.getScale()).isEqualTo(10); + } + + @Test + public void testConvertNullableVsRequired() { + StructField nullableField = + new StructField("nullable_col", DataTypes.IntegerType, true, Metadata.empty()); + ColumnDescriptor nullableDescriptor = TypeUtil.convertToParquet(nullableField); + assertThat(nullableDescriptor.getMaxDefinitionLevel()).isEqualTo(1); + + StructField requiredField = + new StructField("required_col", DataTypes.IntegerType, false, Metadata.empty()); + ColumnDescriptor requiredDescriptor = TypeUtil.convertToParquet(requiredField); + assertThat(requiredDescriptor.getMaxDefinitionLevel()).isEqualTo(0); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/UtilsApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/UtilsApiTest.java new file mode 100644 index 0000000000..63e3c99a74 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/UtilsApiTest.java @@ -0,0 +1,371 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.util.HashMap; +import java.util.Map; + +import org.junit.Test; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.ParquetColumnSpec; +import org.apache.comet.parquet.Utils; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the Utils public API. */ +public class UtilsApiTest extends AbstractApiTest { + + @Test + public void testBuildColumnDescriptorFromSimpleInt32Spec() { + ParquetColumnSpec spec = + new ParquetColumnSpec( + 1, + new String[] {"id"}, + "INT32", + 0, + false, + 0, // required + 0, + null, + null); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPath()).containsExactly("id"); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.INT32); + assertThat(descriptor.getMaxDefinitionLevel()).isEqualTo(0); + assertThat(descriptor.getMaxRepetitionLevel()).isEqualTo(0); + } + + @Test + public void testBuildColumnDescriptorFromOptionalBinarySpec() { + ParquetColumnSpec spec = + new ParquetColumnSpec( + 2, + new String[] {"name"}, + "BINARY", + 0, + false, + 1, // optional + 0, + "StringLogicalTypeAnnotation", + new HashMap<>()); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPath()).containsExactly("name"); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.BINARY); + assertThat(descriptor.getMaxDefinitionLevel()).isEqualTo(1); + assertThat(descriptor.getPrimitiveType().getLogicalTypeAnnotation()) + .isInstanceOf(LogicalTypeAnnotation.StringLogicalTypeAnnotation.class); + } + + @Test + public void testBuildColumnDescriptorFromDecimalSpec() { + Map params = new HashMap<>(); + params.put("precision", "38"); + params.put("scale", "10"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 3, + new String[] {"amount"}, + "FIXED_LEN_BYTE_ARRAY", + 16, + false, + 1, + 0, + "DecimalLogicalTypeAnnotation", + params); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPrimitiveType().getPrimitiveTypeName()) + .isEqualTo(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY); + assertThat(descriptor.getPrimitiveType().getTypeLength()).isEqualTo(16); + + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decAnnotation = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + assertThat(decAnnotation.getPrecision()).isEqualTo(38); + assertThat(decAnnotation.getScale()).isEqualTo(10); + } + + @Test + public void testBuildColumnDescriptorFromTimestampSpec() { + Map params = new HashMap<>(); + params.put("isAdjustedToUTC", "true"); + params.put("unit", "MICROS"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 4, + new String[] {"created_at"}, + "INT64", + 0, + false, + 1, + 0, + "TimestampLogicalTypeAnnotation", + params); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + + assertThat(descriptor).isNotNull(); + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation tsAnnotation = + (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + assertThat(tsAnnotation.isAdjustedToUTC()).isTrue(); + assertThat(tsAnnotation.getUnit()).isEqualTo(LogicalTypeAnnotation.TimeUnit.MICROS); + } + + @Test + public void testBuildColumnDescriptorFromIntLogicalTypeSpec() { + Map params = new HashMap<>(); + params.put("isSigned", "true"); + params.put("bitWidth", "16"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 5, + new String[] {"small_int"}, + "INT32", + 0, + false, + 1, + 0, + "IntLogicalTypeAnnotation", + params); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + + assertThat(descriptor).isNotNull(); + LogicalTypeAnnotation.IntLogicalTypeAnnotation intAnnotation = + (LogicalTypeAnnotation.IntLogicalTypeAnnotation) + descriptor.getPrimitiveType().getLogicalTypeAnnotation(); + assertThat(intAnnotation.isSigned()).isTrue(); + assertThat(intAnnotation.getBitWidth()).isEqualTo(16); + } + + @Test + public void testBuildColumnDescriptorFromRepeatedSpec() { + ParquetColumnSpec spec = + new ParquetColumnSpec( + 6, new String[] {"list", "element"}, "INT32", 0, true, 2, 1, null, null); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + + assertThat(descriptor).isNotNull(); + assertThat(descriptor.getPath()).containsExactly("list", "element"); + assertThat(descriptor.getMaxDefinitionLevel()).isEqualTo(2); + assertThat(descriptor.getMaxRepetitionLevel()).isEqualTo(1); + assertThat(descriptor.getPrimitiveType().getRepetition()).isEqualTo(Type.Repetition.REPEATED); + } + + @Test + public void testDescriptorToParquetColumnSpec() { + // Create a ColumnDescriptor + PrimitiveType primitiveType = + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .as(LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .id(42) + .named("timestamp_col"); + + ColumnDescriptor descriptor = + new ColumnDescriptor(new String[] {"timestamp_col"}, primitiveType, 0, 1); + + ParquetColumnSpec spec = Utils.descriptorToParquetColumnSpec(descriptor); + + assertThat(spec).isNotNull(); + assertThat(spec.getFieldId()).isEqualTo(42); + assertThat(spec.getPath()).containsExactly("timestamp_col"); + assertThat(spec.getPhysicalType()).isEqualTo("INT64"); + assertThat(spec.getMaxDefinitionLevel()).isEqualTo(1); + assertThat(spec.getMaxRepetitionLevel()).isEqualTo(0); + assertThat(spec.getLogicalTypeName()).isEqualTo("TimestampLogicalTypeAnnotation"); + assertThat(spec.getLogicalTypeParams()).containsEntry("isAdjustedToUTC", "true"); + assertThat(spec.getLogicalTypeParams()).containsEntry("unit", "MICROS"); + } + + @Test + public void testDescriptorToParquetColumnSpecForDecimal() { + PrimitiveType primitiveType = + Types.primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.OPTIONAL) + .length(16) + .as(LogicalTypeAnnotation.decimalType(10, 38)) + .id(10) + .named("decimal_col"); + + ColumnDescriptor descriptor = + new ColumnDescriptor(new String[] {"decimal_col"}, primitiveType, 0, 1); + + ParquetColumnSpec spec = Utils.descriptorToParquetColumnSpec(descriptor); + + assertThat(spec).isNotNull(); + assertThat(spec.getPhysicalType()).isEqualTo("FIXED_LEN_BYTE_ARRAY"); + assertThat(spec.getTypeLength()).isEqualTo(16); + assertThat(spec.getLogicalTypeName()).isEqualTo("DecimalLogicalTypeAnnotation"); + assertThat(spec.getLogicalTypeParams()).containsEntry("precision", "38"); + assertThat(spec.getLogicalTypeParams()).containsEntry("scale", "10"); + } + + @Test + public void testRoundTripConversion() { + // Create a ParquetColumnSpec + Map params = new HashMap<>(); + params.put("precision", "18"); + params.put("scale", "2"); + + ParquetColumnSpec originalSpec = + new ParquetColumnSpec( + 100, + new String[] {"price"}, + "FIXED_LEN_BYTE_ARRAY", + 16, + false, + 1, + 0, + "DecimalLogicalTypeAnnotation", + params); + + // Convert to ColumnDescriptor and back + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(originalSpec); + ParquetColumnSpec roundTrippedSpec = Utils.descriptorToParquetColumnSpec(descriptor); + + assertThat(roundTrippedSpec.getFieldId()).isEqualTo(originalSpec.getFieldId()); + assertThat(roundTrippedSpec.getPath()).isEqualTo(originalSpec.getPath()); + assertThat(roundTrippedSpec.getPhysicalType()).isEqualTo(originalSpec.getPhysicalType()); + assertThat(roundTrippedSpec.getTypeLength()).isEqualTo(originalSpec.getTypeLength()); + assertThat(roundTrippedSpec.getMaxDefinitionLevel()) + .isEqualTo(originalSpec.getMaxDefinitionLevel()); + assertThat(roundTrippedSpec.getMaxRepetitionLevel()) + .isEqualTo(originalSpec.getMaxRepetitionLevel()); + assertThat(roundTrippedSpec.getLogicalTypeName()).isEqualTo(originalSpec.getLogicalTypeName()); + } + + @Test + public void testBuildColumnDescriptorForAllLogicalTypes() { + // Test DATE + ParquetColumnSpec dateSpec = + new ParquetColumnSpec( + 1, + new String[] {"date_col"}, + "INT32", + 0, + false, + 1, + 0, + "DateLogicalTypeAnnotation", + null); + ColumnDescriptor dateDescriptor = Utils.buildColumnDescriptor(dateSpec); + assertThat(dateDescriptor.getPrimitiveType().getLogicalTypeAnnotation()) + .isInstanceOf(LogicalTypeAnnotation.DateLogicalTypeAnnotation.class); + + // Test ENUM + ParquetColumnSpec enumSpec = + new ParquetColumnSpec( + 2, + new String[] {"enum_col"}, + "BINARY", + 0, + false, + 1, + 0, + "EnumLogicalTypeAnnotation", + null); + ColumnDescriptor enumDescriptor = Utils.buildColumnDescriptor(enumSpec); + assertThat(enumDescriptor.getPrimitiveType().getLogicalTypeAnnotation()) + .isInstanceOf(LogicalTypeAnnotation.EnumLogicalTypeAnnotation.class); + + // Test JSON + ParquetColumnSpec jsonSpec = + new ParquetColumnSpec( + 3, + new String[] {"json_col"}, + "BINARY", + 0, + false, + 1, + 0, + "JsonLogicalTypeAnnotation", + null); + ColumnDescriptor jsonDescriptor = Utils.buildColumnDescriptor(jsonSpec); + assertThat(jsonDescriptor.getPrimitiveType().getLogicalTypeAnnotation()) + .isInstanceOf(LogicalTypeAnnotation.JsonLogicalTypeAnnotation.class); + + // Test UUID + ParquetColumnSpec uuidSpec = + new ParquetColumnSpec( + 4, + new String[] {"uuid_col"}, + "FIXED_LEN_BYTE_ARRAY", + 16, + false, + 1, + 0, + "UUIDLogicalTypeAnnotation", + null); + ColumnDescriptor uuidDescriptor = Utils.buildColumnDescriptor(uuidSpec); + assertThat(uuidDescriptor.getPrimitiveType().getLogicalTypeAnnotation()) + .isInstanceOf(LogicalTypeAnnotation.UUIDLogicalTypeAnnotation.class); + + // Note: LIST and MAP are group types, not applicable to primitive column descriptors + // They would require a different API or nested structure to test properly + } + + @Test + public void testBuildColumnDescriptorForTimeLogicalType() { + Map params = new HashMap<>(); + params.put("isAdjustedToUTC", "false"); + params.put("unit", "NANOS"); + + ParquetColumnSpec timeSpec = + new ParquetColumnSpec( + 7, + new String[] {"time_col"}, + "INT64", + 0, + false, + 1, + 0, + "TimeLogicalTypeAnnotation", + params); + ColumnDescriptor timeDescriptor = Utils.buildColumnDescriptor(timeSpec); + + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeAnnotation = + (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) + timeDescriptor.getPrimitiveType().getLogicalTypeAnnotation(); + assertThat(timeAnnotation.isAdjustedToUTC()).isFalse(); + assertThat(timeAnnotation.getUnit()).isEqualTo(LogicalTypeAnnotation.TimeUnit.NANOS); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/WrappedInputFileApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/WrappedInputFileApiTest.java new file mode 100644 index 0000000000..980791fe7c --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/WrappedInputFileApiTest.java @@ -0,0 +1,138 @@ +/* + * 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.comet.iceberg.api.parquet; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; + +import org.junit.Test; + +import org.apache.parquet.io.InputFile; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.parquet.WrappedInputFile; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the WrappedInputFile public API. */ +public class WrappedInputFileApiTest extends AbstractApiTest { + + @Test + public void testWrappedInputFileHasIcebergApiAnnotation() { + assertThat(hasIcebergApiAnnotation(WrappedInputFile.class)).isTrue(); + } + + @Test + public void testWrappedInputFileIsPublic() { + assertThat(isPublic(WrappedInputFile.class)).isTrue(); + } + + @Test + public void testImplementsInputFile() { + assertThat(InputFile.class.isAssignableFrom(WrappedInputFile.class)).isTrue(); + } + + @Test + public void testConstructorExists() throws NoSuchMethodException { + Constructor constructor = WrappedInputFile.class.getConstructor(Object.class); + assertThat(constructor).isNotNull(); + assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); + } + + @Test + public void testGetLengthMethodExists() throws NoSuchMethodException { + Method method = WrappedInputFile.class.getMethod("getLength"); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(long.class); + } + + @Test + public void testNewStreamMethodExists() throws NoSuchMethodException { + Method method = WrappedInputFile.class.getMethod("newStream"); + assertThat(method).isNotNull(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("SeekableInputStream"); + } + + @Test + public void testToStringMethodExists() throws NoSuchMethodException { + Method method = WrappedInputFile.class.getMethod("toString"); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(String.class); + } + + @Test + public void testCanWrapMockObject() { + MockInputFile mockFile = new MockInputFile(100L); + WrappedInputFile wrappedFile = new WrappedInputFile(mockFile); + assertThat(wrappedFile).isNotNull(); + } + + @Test + public void testGetLengthDelegatesToWrappedObject() throws IOException { + MockInputFile mockFile = new MockInputFile(12345L); + WrappedInputFile wrappedFile = new WrappedInputFile(mockFile); + assertThat(wrappedFile.getLength()).isEqualTo(12345L); + } + + @Test + public void testToStringDelegatesToWrappedObject() { + MockInputFile mockFile = new MockInputFile(100L); + WrappedInputFile wrappedFile = new WrappedInputFile(mockFile); + assertThat(wrappedFile.toString()).isEqualTo("MockInputFile"); + } + + @Test + public void testNewStreamDelegatesToWrappedObject() throws IOException { + MockInputFile mockFile = new MockInputFile(100L); + WrappedInputFile wrappedFile = new WrappedInputFile(mockFile); + + // newStream should return a SeekableInputStream + org.apache.parquet.io.SeekableInputStream stream = wrappedFile.newStream(); + assertThat(stream).isNotNull(); + stream.close(); + } + + /** Mock InputFile that simulates Iceberg's InputFile interface. */ + private static class MockInputFile { + private final long length; + + MockInputFile(long length) { + this.length = length; + } + + public long getLength() { + return length; + } + + public InputStream newStream() { + // Return a simple ByteArrayInputStream for testing + byte[] data = new byte[(int) length]; + return new ByteArrayInputStream(data); + } + + @Override + public String toString() { + return "MockInputFile"; + } + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/schema/CometSchemaImporterApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/schema/CometSchemaImporterApiTest.java new file mode 100644 index 0000000000..ee94105d47 --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/schema/CometSchemaImporterApiTest.java @@ -0,0 +1,116 @@ +/* + * 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.comet.iceberg.api.schema; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; + +import org.junit.Test; + +import org.apache.arrow.c.AbstractCometSchemaImporter; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; + +import org.apache.comet.CometSchemaImporter; +import org.apache.comet.iceberg.api.AbstractApiTest; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the CometSchemaImporter public API. */ +public class CometSchemaImporterApiTest extends AbstractApiTest { + + @Test + public void testCometSchemaImporterHasIcebergApiAnnotation() { + assertThat(hasIcebergApiAnnotation(CometSchemaImporter.class)).isTrue(); + } + + @Test + public void testCometSchemaImporterIsPublic() { + assertThat(isPublic(CometSchemaImporter.class)).isTrue(); + } + + @Test + public void testExtendsAbstractCometSchemaImporter() { + assertThat(AbstractCometSchemaImporter.class.isAssignableFrom(CometSchemaImporter.class)) + .isTrue(); + } + + @Test + public void testConstructorExists() throws NoSuchMethodException { + Constructor constructor = CometSchemaImporter.class.getConstructor(BufferAllocator.class); + assertThat(constructor).isNotNull(); + assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); + } + + @Test + public void testCanInstantiate() { + try (BufferAllocator allocator = new RootAllocator()) { + CometSchemaImporter importer = new CometSchemaImporter(allocator); + assertThat(importer).isNotNull(); + importer.close(); + } + } + + @Test + public void testCloseMethodExists() throws NoSuchMethodException { + Method method = CometSchemaImporter.class.getMethod("close"); + assertThat(method).isNotNull(); + } + + @Test + public void testGetAllocatorMethodExists() throws NoSuchMethodException { + Method method = CometSchemaImporter.class.getMethod("getAllocator"); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(BufferAllocator.class); + } + + @Test + public void testGetProviderMethodExists() throws NoSuchMethodException { + Method method = CometSchemaImporter.class.getMethod("getProvider"); + assertThat(method).isNotNull(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("CDataDictionaryProvider"); + } + + @Test + public void testGetAllocatorReturnsCorrectValue() { + try (BufferAllocator allocator = new RootAllocator()) { + CometSchemaImporter importer = new CometSchemaImporter(allocator); + assertThat(importer.getAllocator()).isSameAs(allocator); + importer.close(); + } + } + + @Test + public void testGetProviderIsNotNull() { + try (BufferAllocator allocator = new RootAllocator()) { + CometSchemaImporter importer = new CometSchemaImporter(allocator); + assertThat(importer.getProvider()).isNotNull(); + importer.close(); + } + } + + @Test + public void testAbstractCometSchemaImporterCloseMethodHasAnnotation() + throws NoSuchMethodException { + Method method = AbstractCometSchemaImporter.class.getMethod("close"); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + } +} diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/vector/CometVectorApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/vector/CometVectorApiTest.java new file mode 100644 index 0000000000..3924e4c84b --- /dev/null +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/vector/CometVectorApiTest.java @@ -0,0 +1,179 @@ +/* + * 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.comet.iceberg.api.vector; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; + +import org.junit.Test; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.vectorized.ColumnVector; + +import org.apache.comet.iceberg.api.AbstractApiTest; +import org.apache.comet.vector.CometVector; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the CometVector public API. */ +public class CometVectorApiTest extends AbstractApiTest { + + @Test + public void testCometVectorHasIcebergApiAnnotation() { + assertThat(hasIcebergApiAnnotation(CometVector.class)).isTrue(); + } + + @Test + public void testCometVectorIsPublic() { + assertThat(isPublic(CometVector.class)).isTrue(); + } + + @Test + public void testCometVectorIsAbstract() { + assertThat(Modifier.isAbstract(CometVector.class.getModifiers())).isTrue(); + } + + @Test + public void testExtendsColumnVector() { + assertThat(ColumnVector.class.isAssignableFrom(CometVector.class)).isTrue(); + } + + @Test + public void testPublicConstructorExists() throws NoSuchMethodException { + Constructor constructor = + CometVector.class.getDeclaredConstructor(DataType.class, boolean.class); + assertThat(constructor).isNotNull(); + assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); + assertThat(Modifier.isPublic(constructor.getModifiers())).isTrue(); + } + + @Test + public void testSetNumNullsMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("setNumNulls", int.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); + } + + @Test + public void testSetNumValuesMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("setNumValues", int.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); + } + + @Test + public void testNumValuesMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("numValues"); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); + assertThat(method.getReturnType()).isEqualTo(int.class); + } + + @Test + public void testGetValueVectorMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getValueVector"); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("ValueVector"); + } + + @Test + public void testSliceMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("slice", int.class, int.class); + assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); + assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); + assertThat(method.getReturnType()).isEqualTo(CometVector.class); + } + + @Test + public void testCloseMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("close"); + assertThat(method).isNotNull(); + } + + @Test + public void testIsNullAtMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("isNullAt", int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(boolean.class); + } + + @Test + public void testGetBooleanMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getBoolean", int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(boolean.class); + } + + @Test + public void testGetIntMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getInt", int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(int.class); + } + + @Test + public void testGetLongMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getLong", int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(long.class); + } + + @Test + public void testGetFloatMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getFloat", int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(float.class); + } + + @Test + public void testGetDoubleMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getDouble", int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(double.class); + } + + @Test + public void testGetBinaryMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getBinary", int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType()).isEqualTo(byte[].class); + } + + @Test + public void testGetDecimalMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getDecimal", int.class, int.class, int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("Decimal"); + } + + @Test + public void testGetUTF8StringMethodExists() throws NoSuchMethodException { + Method method = CometVector.class.getMethod("getUTF8String", int.class); + assertThat(method).isNotNull(); + assertThat(method.getReturnType().getSimpleName()).isEqualTo("UTF8String"); + } +} diff --git a/iceberg-public-api/src/test/resources/log4j2.properties b/iceberg-public-api/src/test/resources/log4j2.properties new file mode 100644 index 0000000000..1251d38e36 --- /dev/null +++ b/iceberg-public-api/src/test/resources/log4j2.properties @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# Set everything to be logged to the file target/unit-tests.log +rootLogger.level = info +rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} + +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Tests that launch java subprocesses can set the "test.appender" system property to +# "console" to avoid having the child process's logs overwrite the unit test's +# log file. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %t: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +logger.jetty.name = org.sparkproject.jetty +logger.jetty.level = warn + diff --git a/pom.xml b/pom.xml index ab42aa773e..6a909266e4 100644 --- a/pom.xml +++ b/pom.xml @@ -39,6 +39,7 @@ under the License. spark spark-integration fuzz-testing + iceberg-public-api From 640dd03b17838edac6fc9beed8bde9b383888d82 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 22 Jan 2026 13:47:01 -0700 Subject: [PATCH 6/7] trigger CI From a4ca88532b79d355399ab68f67c2b0a798fbf209 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 23 Jan 2026 10:18:43 -0700 Subject: [PATCH 7/7] Address review comments - Fix documentation: skipNextRowGroup returns boolean, WrappedInputFile takes Object, update Utils.getColumnReader signature, fix TypeUtil reference, clarify Native.resetBatch() - Use try-with-resources for Files.walk() in AbstractApiTest - Remove unnecessary isNotNull() assertions in test files (getMethod/getConstructor throw if not found) - Add @IcebergApi to BatchReader.close() and ReadOptions.Builder constructor Co-Authored-By: Claude Opus 4.5 --- .../org/apache/comet/parquet/BatchReader.java | 1 + .../org/apache/comet/parquet/ReadOptions.java | 1 + .../contributor-guide/iceberg_public_api.md | 17 +++++++++-------- .../comet/iceberg/api/AbstractApiTest.java | 4 +++- .../parquet/AbstractColumnReaderApiTest.java | 5 ----- .../iceberg/api/parquet/BatchReaderApiTest.java | 7 +------ 6 files changed, 15 insertions(+), 20 deletions(-) diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java index 57c5d26f3b..857a1de3c1 100644 --- a/common/src/main/java/org/apache/comet/parquet/BatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -530,6 +530,7 @@ public boolean nextBatch(int batchSize) { return true; } + @IcebergApi @Override public void close() throws IOException { if (columnReaders != null) { diff --git a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java index 6363aea31d..ec5c16ce8e 100644 --- a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java +++ b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java @@ -148,6 +148,7 @@ public ReadOptions build() { adjustReadRangeSkew); } + @IcebergApi public Builder(Configuration conf) { this.conf = conf; this.parallelIOEnabled = diff --git a/docs/source/contributor-guide/iceberg_public_api.md b/docs/source/contributor-guide/iceberg_public_api.md index bf459d3df2..051b22459b 100644 --- a/docs/source/contributor-guide/iceberg_public_api.md +++ b/docs/source/contributor-guide/iceberg_public_api.md @@ -62,7 +62,7 @@ public FileReader( // Methods used by Iceberg public void setRequestedSchemaFromSpecs(List specs) public RowGroupReader readNextRowGroup() throws IOException -public void skipNextRowGroup() +public boolean skipNextRowGroup() public void close() throws IOException ``` @@ -94,7 +94,7 @@ Wrapper that adapts Iceberg's `InputFile` interface to Comet's file reading infr ```java // Constructor -public WrappedInputFile(org.apache.iceberg.io.InputFile inputFile) +public WrappedInputFile(Object inputFile) ``` ### ParquetColumnSpec @@ -223,13 +223,14 @@ General utility methods. ```java // Methods used by Iceberg -public static AbstractColumnReader getColumnReader( - DataType sparkType, - ColumnDescriptor descriptor, +public static ColumnReader getColumnReader( + DataType type, + ParquetColumnSpec columnSpec, CometSchemaImporter importer, int batchSize, boolean useDecimal128, - boolean isConstant + boolean useLazyMaterialization, + boolean useLegacyTimestamp ) ``` @@ -308,7 +309,7 @@ Arrow's base vector interface (shaded). Used as return type in `CometVector.getV 1. Iceberg creates a `WrappedInputFile` from its `InputFile` 2. Creates `ReadOptions` via builder pattern 3. Instantiates `FileReader` with the wrapped input file -4. Converts Parquet `ColumnDescriptor`s to `ParquetColumnSpec`s using `CometTypeUtils` +4. Converts Parquet `ColumnDescriptor`s to `ParquetColumnSpec`s using `TypeUtil` 5. Calls `setRequestedSchemaFromSpecs()` to specify which columns to read 6. Iterates through row groups via `readNextRowGroup()` and `skipNextRowGroup()` @@ -316,7 +317,7 @@ Arrow's base vector interface (shaded). Used as return type in `CometVector.getV 1. Creates `CometSchemaImporter` with a `RootAllocator` 2. Uses `Utils.getColumnReader()` to create appropriate column readers -3. Calls `reset()` and `setPageReader()` for each row group +3. Calls `Native.resetBatch()` and `setPageReader()` for each row group 4. Uses `BatchReader` to coordinate reading batches across all columns 5. Retrieves results via `delegate().currentBatch()` diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/AbstractApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/AbstractApiTest.java index c44057ee6a..4ba7b2b091 100644 --- a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/AbstractApiTest.java +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/AbstractApiTest.java @@ -49,7 +49,9 @@ public void setUp() throws IOException { @After public void tearDown() throws IOException { if (tempDir != null && Files.exists(tempDir)) { - Files.walk(tempDir).sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + try (var stream = Files.walk(tempDir)) { + stream.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } } } diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/AbstractColumnReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/AbstractColumnReaderApiTest.java index 39ea3166c2..6e111c710f 100644 --- a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/AbstractColumnReaderApiTest.java +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/AbstractColumnReaderApiTest.java @@ -56,28 +56,24 @@ public void testImplementsAutoCloseable() { @Test public void testSetBatchSizeMethodExists() throws NoSuchMethodException { Method method = AbstractColumnReader.class.getMethod("setBatchSize", int.class); - assertThat(method).isNotNull(); assertThat(hasIcebergApiAnnotation(method)).isTrue(); } @Test public void testCloseMethodExists() throws NoSuchMethodException { Method method = AbstractColumnReader.class.getMethod("close"); - assertThat(method).isNotNull(); assertThat(hasIcebergApiAnnotation(method)).isTrue(); } @Test public void testReadBatchMethodExists() throws NoSuchMethodException { Method method = AbstractColumnReader.class.getMethod("readBatch", int.class); - assertThat(method).isNotNull(); assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); } @Test public void testCurrentBatchMethodExists() throws NoSuchMethodException { Method method = AbstractColumnReader.class.getMethod("currentBatch"); - assertThat(method).isNotNull(); assertThat(Modifier.isAbstract(method.getModifiers())).isTrue(); assertThat(method.getReturnType().getSimpleName()).isEqualTo("CometVector"); } @@ -85,7 +81,6 @@ public void testCurrentBatchMethodExists() throws NoSuchMethodException { @Test public void testNativeHandleFieldExists() throws NoSuchFieldException { Field field = AbstractColumnReader.class.getDeclaredField("nativeHandle"); - assertThat(field).isNotNull(); assertThat(hasIcebergApiAnnotation(field)).isTrue(); assertThat(Modifier.isProtected(field.getModifiers())).isTrue(); } diff --git a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/BatchReaderApiTest.java b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/BatchReaderApiTest.java index 949a7b4a78..fc673b7d20 100644 --- a/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/BatchReaderApiTest.java +++ b/iceberg-public-api/src/test/java/org/apache/comet/iceberg/api/parquet/BatchReaderApiTest.java @@ -48,21 +48,18 @@ public void testBatchReaderIsPublic() { @Test public void testConstructorWithColumnReadersExists() throws NoSuchMethodException { Constructor constructor = BatchReader.class.getConstructor(AbstractColumnReader[].class); - assertThat(constructor).isNotNull(); assertThat(hasIcebergApiAnnotation(constructor)).isTrue(); } @Test public void testSetSparkSchemaMethodExists() throws NoSuchMethodException { Method method = BatchReader.class.getMethod("setSparkSchema", StructType.class); - assertThat(method).isNotNull(); assertThat(hasIcebergApiAnnotation(method)).isTrue(); } @Test public void testGetColumnReadersMethodExists() throws NoSuchMethodException { Method method = BatchReader.class.getMethod("getColumnReaders"); - assertThat(method).isNotNull(); assertThat(hasIcebergApiAnnotation(method)).isTrue(); assertThat(method.getReturnType()).isEqualTo(AbstractColumnReader[].class); } @@ -70,7 +67,6 @@ public void testGetColumnReadersMethodExists() throws NoSuchMethodException { @Test public void testNextBatchWithSizeMethodExists() throws NoSuchMethodException { Method method = BatchReader.class.getMethod("nextBatch", int.class); - assertThat(method).isNotNull(); assertThat(hasIcebergApiAnnotation(method)).isTrue(); assertThat(method.getReturnType()).isEqualTo(boolean.class); } @@ -78,14 +74,13 @@ public void testNextBatchWithSizeMethodExists() throws NoSuchMethodException { @Test public void testCurrentBatchMethodExists() throws NoSuchMethodException { Method method = BatchReader.class.getMethod("currentBatch"); - assertThat(method).isNotNull(); assertThat(method.getReturnType().getSimpleName()).isEqualTo("ColumnarBatch"); } @Test public void testCloseMethodExists() throws NoSuchMethodException { Method method = BatchReader.class.getMethod("close"); - assertThat(method).isNotNull(); + assertThat(hasIcebergApiAnnotation(method)).isTrue(); } @Test