Skip to content
Closed
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
8 changes: 8 additions & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -836,6 +836,7 @@ project(':iceberg-orc') {
}

project(':iceberg-parquet') {

test {
useJUnitPlatform()
}
Expand All @@ -845,6 +846,13 @@ project(':iceberg-parquet') {
implementation project(':iceberg-core')
implementation project(':iceberg-common')

implementation("org.apache.datafusion:comet-spark-spark${ sparkVersionsString}_${scalaVersion}:${libs.versions.comet.get()}") {
exclude group: 'org.apache.arrow'
exclude group: 'org.apache.parquet'
exclude group: 'org.apache.spark'
exclude group: 'org.apache.iceberg'
}

implementation(libs.parquet.avro) {
exclude group: 'org.apache.avro', module: 'avro'
// already shaded by Parquet
Expand Down
2 changes: 1 addition & 1 deletion gradle.properties
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ jmhJsonOutputPath=build/reports/jmh/results.json
jmhIncludeRegex=.*
systemProp.defaultFlinkVersions=2.0
systemProp.knownFlinkVersions=1.19,1.20,2.0
systemProp.defaultSparkVersions=4.0
systemProp.defaultSparkVersions=3.4
systemProp.knownSparkVersions=3.4,3.5,4.0
systemProp.defaultKafkaVersions=3
systemProp.knownKafkaVersions=3
Expand Down
2 changes: 1 addition & 1 deletion gradle/libs.versions.toml
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ awssdk-s3accessgrants = "2.3.0"
bson-ver = "4.11.5"
caffeine = "2.9.3"
calcite = "1.39.0"
comet = "0.8.1"
comet = "0.9.0.1-SNAPSHOT"
datasketches = "6.2.0"
delta-standalone = "3.3.2"
delta-spark = "3.3.2"
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,315 @@
/*
* 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.iceberg.parquet;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.function.Function;
import org.apache.comet.parquet.FileReader;
import org.apache.comet.parquet.ParquetColumnSpec;
import org.apache.comet.parquet.ReadOptions;
import org.apache.comet.parquet.RowGroupReader;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.Schema;
import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.hadoop.HadoopInputFile;
import org.apache.iceberg.io.CloseableGroup;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.mapping.NameMapping;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.ByteBuffers;
import org.apache.parquet.ParquetReadOptions;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.hadoop.metadata.ColumnPath;
import org.apache.parquet.schema.LogicalTypeAnnotation;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.PrimitiveType;
import org.apache.parquet.schema.Type;

public class CometVectorizedParquetReader<T> extends CloseableGroup
implements CloseableIterable<T> {
private final InputFile input;
private final ParquetReadOptions options;
private final Schema expectedSchema;
private final Function<MessageType, VectorizedReader<?>> batchReaderFunc;
private final Expression filter;
private final boolean reuseContainers;
private final boolean caseSensitive;
private final int batchSize;
private final NameMapping nameMapping;
private final Map<String, String> properties;
private Long start = null;
private Long length = null;
private ByteBuffer fileEncryptionKey = null;
private ByteBuffer fileAADPrefix = null;

public CometVectorizedParquetReader(
InputFile input,
Schema expectedSchema,
ParquetReadOptions options,
Function<MessageType, VectorizedReader<?>> readerFunc,
NameMapping nameMapping,
Expression filter,
boolean reuseContainers,
boolean caseSensitive,
int maxRecordsPerBatch,
Map<String, String> properties,
Long start,
Long length,
ByteBuffer fileEncryptionKey,
ByteBuffer fileAADPrefix) {
this.input = input;
this.expectedSchema = expectedSchema;
this.options = options;
this.batchReaderFunc = readerFunc;
// replace alwaysTrue with null to avoid extra work evaluating a trivial filter
this.filter = filter == Expressions.alwaysTrue() ? null : filter;
this.reuseContainers = reuseContainers;
this.caseSensitive = caseSensitive;
this.batchSize = maxRecordsPerBatch;
this.nameMapping = nameMapping;
this.properties = properties;
this.start = start;
this.length = length;
this.fileEncryptionKey = fileEncryptionKey;
this.fileAADPrefix = fileAADPrefix;
}

private ReadConf conf = null;

private ReadConf init() {
if (conf == null) {
ReadConf readConf =
new ReadConf(
input,
options,
expectedSchema,
filter,
null,
batchReaderFunc,
nameMapping,
reuseContainers,
caseSensitive,
batchSize);
this.conf = readConf.copy();
return readConf;
}
return conf;
}

@Override
public CloseableIterator<T> iterator() {
FileIterator<T> iter =
new FileIterator<>(init(), properties, start, length, fileEncryptionKey, fileAADPrefix);
addCloseable(iter);
return iter;
}

private static class FileIterator<T> implements CloseableIterator<T> {
// private final ParquetFileReader reader;
private final boolean[] shouldSkip;
private final VectorizedReader<T> model;
private final long totalValues;
private final int batchSize;
private final List<Map<ColumnPath, ColumnChunkMetaData>> columnChunkMetadata;
private final boolean reuseContainers;
private int nextRowGroup = 0;
private long nextRowGroupStart = 0;
private long valuesRead = 0;
private T last = null;
private final FileReader cometReader;

FileIterator(
ReadConf conf,
Map<String, String> properties,
Long start,
Long length,
ByteBuffer fileEncryptionKey,
ByteBuffer fileAADPrefix) {
this.shouldSkip = conf.shouldSkip();
this.totalValues = conf.totalValues();
this.reuseContainers = conf.reuseContainers();
this.model = conf.vectorizedModel();
this.batchSize = conf.batchSize();
this.model.setBatchSize(this.batchSize);
this.columnChunkMetadata = conf.columnChunkMetadataForRowGroups();
this.cometReader =
newCometReader(
conf.file(),
conf.projection(),
properties,
start,
length,
fileEncryptionKey,
fileAADPrefix);
}

private FileReader newCometReader(
InputFile file,
MessageType projection,
Map<String, String> properties,
Long start,
Long length,
ByteBuffer fileEncryptionKey,
ByteBuffer fileAADPrefix) {
try {
ReadOptions cometOptions = ReadOptions.builder(new Configuration()).build();

FileReader fileReader =
new FileReader(
((HadoopInputFile) file).getPath(),
new Configuration(((HadoopInputFile) file).getConf()),
cometOptions,
properties,
start,
length,
ByteBuffers.toByteArray(fileEncryptionKey),
ByteBuffers.toByteArray(fileAADPrefix));

List<ColumnDescriptor> columnDescriptors = projection.getColumns();

List<ParquetColumnSpec> specs = Lists.newArrayList();

for (ColumnDescriptor descriptor : columnDescriptors) {
String[] path = descriptor.getPath();
PrimitiveType primitiveType = descriptor.getPrimitiveType();
String physicalType = primitiveType.getPrimitiveTypeName().name();

int typeLength =
primitiveType.getPrimitiveTypeName()
== PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY
? primitiveType.getTypeLength()
: 0;

boolean isRepeated = primitiveType.getRepetition() == Type.Repetition.REPEATED;

// ToDo: extract this into a Util method
String logicalTypeName = null;
Map<String, String> logicalTypeParams = Maps.newHashMap();
LogicalTypeAnnotation logicalType = primitiveType.getLogicalTypeAnnotation();

if (logicalType != null) {
logicalTypeName = logicalType.getClass().getSimpleName();

// Handle specific logical types
if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) {
LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal =
(LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType;
logicalTypeParams.put("precision", String.valueOf(decimal.getPrecision()));
logicalTypeParams.put("scale", String.valueOf(decimal.getScale()));
} else if (logicalType
instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) {
LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp =
(LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType;
logicalTypeParams.put("isAdjustedToUTC", String.valueOf(timestamp.isAdjustedToUTC()));
logicalTypeParams.put("unit", timestamp.getUnit().name());
} else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) {
LogicalTypeAnnotation.TimeLogicalTypeAnnotation time =
(LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType;
logicalTypeParams.put("isAdjustedToUTC", String.valueOf(time.isAdjustedToUTC()));
logicalTypeParams.put("unit", time.getUnit().name());
} else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) {
LogicalTypeAnnotation.IntLogicalTypeAnnotation intType =
(LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType;
logicalTypeParams.put("isSigned", String.valueOf(intType.isSigned()));
logicalTypeParams.put("bitWidth", String.valueOf(intType.getBitWidth()));
}
}

ParquetColumnSpec spec =
new ParquetColumnSpec(
1, // ToDo: pass in the correct id
path,
physicalType,
typeLength,
isRepeated,
descriptor.getMaxDefinitionLevel(),
descriptor.getMaxRepetitionLevel(),
logicalTypeName,
logicalTypeParams);
specs.add(spec);
}

fileReader.setRequestedSchemaFromSpecs(specs);
return fileReader;
} catch (IOException e) {
throw new UncheckedIOException("Failed to open Parquet file: " + file.location(), e);
}
}

@Override
public boolean hasNext() {
return valuesRead < totalValues;
}

@Override
public T next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
if (valuesRead >= nextRowGroupStart) {
advance();
}

// batchSize is an integer, so casting to integer is safe
int numValuesToRead = (int) Math.min(nextRowGroupStart - valuesRead, batchSize);
if (reuseContainers) {
this.last = model.read(last, numValuesToRead);
} else {
this.last = model.read(null, numValuesToRead);
}
valuesRead += numValuesToRead;

return last;
}

private void advance() {
while (shouldSkip[nextRowGroup]) {
nextRowGroup += 1;
cometReader.skipNextRowGroup();
}
RowGroupReader pages;
try {
pages = cometReader.readNextRowGroup();
} catch (IOException e) {
throw new RuntimeIOException(e);
}

model.setRowGroupInfo(pages, columnChunkMetadata.get(nextRowGroup));
nextRowGroupStart += pages.getRowCount();
nextRowGroup += 1;
}

@Override
public void close() throws IOException {
model.close();
cometReader.close();
}
}
}
Loading