Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions .baseline/checkstyle/checkstyle-suppressions.xml
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,8 @@

<!-- Suppress checks for CometColumnReader -->
<suppress files="org.apache.iceberg.spark.data.vectorized.CometColumnReader" checks="IllegalImport"/>
<!-- Suppress checks for CometDeletedColumnVector -->
<suppress files="org.apache.iceberg.spark.data.vectorized.CometDeletedColumnVector" checks="IllegalImport"/>

<!-- Suppress TestClassNamingConvention for main source files -->
<suppress files=".*[/\\]src[/\\]main[/\\].*" id="TestClassNamingConvention" />
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,11 @@
import java.util.Map;
import org.apache.iceberg.arrow.vectorized.BaseBatchReader;
import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader;
import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader.DeletedVectorReader;
import org.apache.iceberg.data.DeleteFilter;
import org.apache.iceberg.parquet.VectorizedReader;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.util.Pair;
import org.apache.parquet.column.page.PageReadStore;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.hadoop.metadata.ColumnPath;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.vectorized.ColumnVector;
import org.apache.spark.sql.vectorized.ColumnarBatch;

Expand All @@ -40,31 +36,15 @@
* populated via delegated read calls to {@linkplain VectorizedArrowReader VectorReader(s)}.
*/
public class ColumnarBatchReader extends BaseBatchReader<ColumnarBatch> {
private final boolean hasIsDeletedColumn;
private DeleteFilter<InternalRow> deletes = null;
private long rowStartPosInBatch = 0;

public ColumnarBatchReader(List<VectorizedReader<?>> readers) {
super(readers);
this.hasIsDeletedColumn =
readers.stream().anyMatch(reader -> reader instanceof DeletedVectorReader);
}

@Override
public void setRowGroupInfo(
PageReadStore pageStore, Map<ColumnPath, ColumnChunkMetaData> metaData) {
super.setRowGroupInfo(pageStore, metaData);
this.rowStartPosInBatch =
pageStore
.getRowIndexOffset()
.orElseThrow(
() ->
new IllegalArgumentException(
"PageReadStore does not contain row index offset"));
}

public void setDeleteFilter(DeleteFilter<InternalRow> deleteFilter) {
this.deletes = deleteFilter;
}

@Override
Expand All @@ -73,9 +53,7 @@ public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) {
closeVectors();
}

ColumnarBatch columnarBatch = new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch();
rowStartPosInBatch += numRowsToRead;
return columnarBatch;
return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch();
}

private class ColumnBatchLoader {
Expand All @@ -89,43 +67,12 @@ private class ColumnBatchLoader {

ColumnarBatch loadDataToColumnBatch() {
ColumnVector[] vectors = readDataToColumnVectors();
int numLiveRows = batchSize;

if (hasIsDeletedColumn) {
boolean[] isDeleted = buildIsDeleted(vectors);
for (ColumnVector vector : vectors) {
if (vector instanceof DeletedColumnVector) {
((DeletedColumnVector) vector).setValue(isDeleted);
}
}
} else {
Pair<int[], Integer> pair = buildRowIdMapping(vectors);
if (pair != null) {
int[] rowIdMapping = pair.first();
numLiveRows = pair.second();
for (int i = 0; i < vectors.length; i++) {
vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping);
}
}
}

if (deletes != null && deletes.hasEqDeletes()) {
vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors);
}

ColumnarBatch batch = new ColumnarBatch(vectors);
batch.setNumRows(numLiveRows);
batch.setNumRows(batchSize);
return batch;
}

private boolean[] buildIsDeleted(ColumnVector[] vectors) {
return ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, batchSize);
}

private Pair<int[], Integer> buildRowIdMapping(ColumnVector[] vectors) {
return ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, batchSize);
}

ColumnVector[] readDataToColumnVectors() {
ColumnVector[] arrowColumnVectors = new ColumnVector[readers.length];

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,15 +25,12 @@
import org.apache.comet.parquet.AbstractColumnReader;
import org.apache.comet.parquet.BatchReader;
import org.apache.iceberg.Schema;
import org.apache.iceberg.data.DeleteFilter;
import org.apache.iceberg.parquet.VectorizedReader;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.spark.SparkSchemaUtil;
import org.apache.iceberg.util.Pair;
import org.apache.parquet.column.page.PageReadStore;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.hadoop.metadata.ColumnPath;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.vectorized.ColumnVector;
import org.apache.spark.sql.vectorized.ColumnarBatch;

Expand All @@ -46,7 +43,6 @@
class CometColumnarBatchReader implements VectorizedReader<ColumnarBatch> {

private final CometColumnReader[] readers;
private final boolean hasIsDeletedColumn;

// The delegated BatchReader on the Comet side does the real work of loading a batch of rows.
// The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call
Expand All @@ -56,14 +52,10 @@ class CometColumnarBatchReader implements VectorizedReader<ColumnarBatch> {
// DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is
// available.
private final BatchReader delegate;
private DeleteFilter<InternalRow> deletes = null;
private long rowStartPosInBatch = 0;

CometColumnarBatchReader(List<VectorizedReader<?>> readers, Schema schema) {
this.readers =
readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new);
this.hasIsDeletedColumn =
readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader);

AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()];
this.delegate = new BatchReader(abstractColumnReaders);
Expand All @@ -89,25 +81,11 @@ public void setRowGroupInfo(
for (int i = 0; i < readers.length; i++) {
delegate.getColumnReaders()[i] = this.readers[i].delegate();
}

this.rowStartPosInBatch =
pageStore
.getRowIndexOffset()
.orElseThrow(
() ->
new IllegalArgumentException(
"PageReadStore does not contain row index offset"));
}

public void setDeleteFilter(DeleteFilter<InternalRow> deleteFilter) {
this.deletes = deleteFilter;
}

@Override
public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) {
ColumnarBatch columnarBatch = new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch();
rowStartPosInBatch += numRowsToRead;
return columnarBatch;
return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch();
}

@Override
Expand Down Expand Up @@ -139,39 +117,12 @@ private class ColumnBatchLoader {

ColumnarBatch loadDataToColumnBatch() {
ColumnVector[] vectors = readDataToColumnVectors();
int numLiveRows = batchSize;

if (hasIsDeletedColumn) {
boolean[] isDeleted = buildIsDeleted(vectors);
readDeletedColumn(vectors, isDeleted);
} else {
Pair<int[], Integer> pair = buildRowIdMapping(vectors);
if (pair != null) {
int[] rowIdMapping = pair.first();
numLiveRows = pair.second();
for (int i = 0; i < vectors.length; i++) {
vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping);
}
}
}

if (deletes != null && deletes.hasEqDeletes()) {
vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors);
}

ColumnarBatch batch = new ColumnarBatch(vectors);
batch.setNumRows(numLiveRows);
batch.setNumRows(batchSize);
return batch;
}

private boolean[] buildIsDeleted(ColumnVector[] vectors) {
return ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, batchSize);
}

private Pair<int[], Integer> buildRowIdMapping(ColumnVector[] vectors) {
return ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, batchSize);
}

ColumnVector[] readDataToColumnVectors() {
ColumnVector[] columnVectors = new ColumnVector[readers.length];
// Fetch rows for all readers in the delegate
Expand All @@ -182,16 +133,5 @@ ColumnVector[] readDataToColumnVectors() {

return columnVectors;
}

void readDeletedColumn(ColumnVector[] columnVectors, boolean[] isDeleted) {
for (int i = 0; i < readers.length; i++) {
if (readers[i] instanceof CometDeleteColumnReader) {
CometDeleteColumnReader deleteColumnReader = new CometDeleteColumnReader<>(isDeleted);
deleteColumnReader.setBatchSize(batchSize);
deleteColumnReader.delegate().readBatch(batchSize);
columnVectors[i] = deleteColumnReader.delegate().currentBatch();
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.comet.parquet.MetadataColumnReader;
import org.apache.comet.parquet.Native;
import org.apache.comet.parquet.TypeUtil;
import org.apache.comet.vector.CometVector;
import org.apache.iceberg.MetadataColumns;
import org.apache.iceberg.types.Types;
import org.apache.spark.sql.types.DataTypes;
Expand All @@ -46,30 +47,34 @@ public void setBatchSize(int batchSize) {
}

private static class DeleteColumnReader extends MetadataColumnReader {
private boolean[] isDeleted;
private CometDeletedColumnVector deletedVector;

DeleteColumnReader() {
this(new boolean[0]);
}

DeleteColumnReader(boolean[] isDeleted) {
super(
DataTypes.BooleanType,
TypeUtil.convertToParquet(
new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())),
false /* useDecimal128 = false */,
false /* isConstant = false */);
this.isDeleted = new boolean[0];
}

DeleteColumnReader(boolean[] isDeleted) {
this();
this.isDeleted = isDeleted;
this.deletedVector = new CometDeletedColumnVector(isDeleted);
}

@Override
public void readBatch(int total) {
Native.resetBatch(nativeHandle);
// set isDeleted on the native side to be consumed by native execution
Native.setIsDeleted(nativeHandle, isDeleted);
Native.setIsDeleted(nativeHandle, deletedVector.isDeleted());

super.readBatch(total);
}

@Override
public CometVector currentBatch() {
return deletedVector;
}
}
}
Loading