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..49ce92a408 100644 --- a/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java +++ b/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java @@ -23,6 +23,8 @@ 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; @@ -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/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..915fd87848 --- /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..f8385f41b6 100644 --- a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java @@ -28,9 +28,11 @@ 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. */ +@IcebergApi public abstract class AbstractColumnReader implements AutoCloseable { protected static final Logger LOG = LoggerFactory.getLogger(AbstractColumnReader.class); @@ -61,7 +63,7 @@ public abstract class AbstractColumnReader implements AutoCloseable { protected int batchSize; /** A pointer to the native implementation of ColumnReader. */ - protected long nativeHandle; + @IcebergApi protected long nativeHandle; AbstractColumnReader( DataType type, @@ -96,6 +98,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 +116,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..d591454596 100644 --- a/common/src/main/java/org/apache/comet/parquet/BatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -65,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; @@ -87,6 +88,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(); @@ -186,9 +188,9 @@ 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; @@ -384,17 +386,17 @@ 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; } /** - * @deprecated since 0.10.0, will be removed in 0.11.0. * @see Comet Issue #2079 */ + @IcebergApi public AbstractColumnReader[] getColumnReaders() { return columnReaders; } @@ -498,6 +500,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++) { @@ -524,6 +527,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/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 968da1959c..70c2532f1f 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -44,12 +44,14 @@ 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; 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(); @@ -111,9 +113,9 @@ public class ColumnReader extends AbstractColumnReader { * Set the page reader for a new column chunk to read. Expects to call `readBatch` after this. * * @param pageReader the page reader for the new column chunk - * @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 +131,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..1d4ec84c2c 100644 --- a/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ConstantColumnReader.java @@ -27,10 +27,13 @@ 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. */ +@IcebergApi public class ConstantColumnReader extends MetadataColumnReader { /** Whether all the values in this constant column are nulls */ private boolean isNull; @@ -53,9 +56,9 @@ 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 +66,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..80c214fc7c 100644 --- a/common/src/main/java/org/apache/comet/parquet/FileReader.java +++ b/common/src/main/java/org/apache/comet/parquet/FileReader.java @@ -90,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; @@ -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..ace1ab4164 100644 --- a/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/MetadataColumnReader.java @@ -28,10 +28,12 @@ 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; /** A metadata column reader that can be extended by {@link RowIndexColumnReader} etc. */ +@IcebergApi public class MetadataColumnReader extends AbstractColumnReader { private final BufferAllocator allocator = new RootAllocator(); @@ -43,9 +45,9 @@ public class MetadataColumnReader extends AbstractColumnReader { private boolean isConstant; /** - * @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 +57,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 +72,7 @@ public void setBatchSize(int batchSize) { super.setBatchSize(batchSize); } + @IcebergApi @Override public void readBatch(int total) { if (vector == null) { @@ -90,6 +94,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..babd0d392c 100644 --- a/common/src/main/java/org/apache/comet/parquet/Native.java +++ b/common/src/main/java/org/apache/comet/parquet/Native.java @@ -22,6 +22,7 @@ import java.nio.ByteBuffer; import java.util.Map; +import org.apache.comet.IcebergApi; import org.apache.comet.NativeBase; public final class Native extends NativeBase { @@ -143,6 +144,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 +223,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..95fed362d3 100644 --- a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java +++ b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java @@ -21,12 +21,15 @@ 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 * 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..ec5c16ce8e 100644 --- a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java +++ b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java @@ -27,12 +27,14 @@ import org.apache.spark.launcher.SparkLauncher; import org.apache.comet.CometConf; +import org.apache.comet.IcebergApi; /** * Comet specific Parquet related read options. * *

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 +88,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 +138,7 @@ public Builder adjustReadRangeSkew(boolean adjustReadRangeSkew) { return this; } + @IcebergApi public ReadOptions build() { return new ReadOptions( parallelIOEnabled, @@ -143,6 +148,7 @@ public ReadOptions build() { adjustReadRangeSkew); } + @IcebergApi public Builder(Configuration conf) { this.conf = conf; this.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..0ca7478b7b 100644 --- a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java +++ b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java @@ -29,6 +29,9 @@ 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<>(); 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..cb5f4997dd 100644 --- a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java +++ b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java @@ -30,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; @@ -38,9 +39,9 @@ public class TypeUtil { /** * Converts the input Spark 'field' into a Parquet column descriptor. * - * @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..87845ae760 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -30,10 +30,12 @@ import org.apache.spark.sql.types.*; import org.apache.comet.CometSchemaImporter; +import org.apache.comet.IcebergApi; public class Utils { /** This method is called from Apache Iceberg. */ + @IcebergApi public static ColumnReader getColumnReader( DataType type, ParquetColumnSpec columnSpec, @@ -57,10 +59,9 @@ public static ColumnReader getColumnReader( /** * This method is called from Apache Iceberg. * - * @deprecated since 0.10.0, will be removed in 0.11.0; use getColumnReader with ParquetColumnSpec - * instead. * @see Comet Issue #2079 */ + @IcebergApi public static ColumnReader getColumnReader( DataType type, ColumnDescriptor descriptor, @@ -293,6 +294,7 @@ static int getTimeUnitId(LogicalTypeAnnotation.TimeUnit tu) { } } + @IcebergApi public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpec) { PrimitiveType.PrimitiveTypeName primType = PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType()); @@ -458,6 +460,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..9b5e50ddb4 100644 --- a/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java +++ b/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java @@ -26,13 +26,17 @@ 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 */ +@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..6dda765d51 100644 --- a/common/src/main/java/org/apache/comet/vector/CometVector.java +++ b/common/src/main/java/org/apache/comet/vector/CometVector.java @@ -39,7 +39,10 @@ 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 { private static final int DECIMAL_BYTE_WIDTH = 16; private final byte[] DECIMAL_BYTES = new byte[DECIMAL_BYTE_WIDTH]; @@ -58,7 +61,8 @@ public abstract class CometVector extends ColumnVector { } } - protected CometVector(DataType type, boolean useDecimal128) { + @IcebergApi + public 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); /**