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
32 changes: 26 additions & 6 deletions .palantir/revapi.yml
Original file line number Diff line number Diff line change
Expand Up @@ -1030,11 +1030,14 @@ acceptedBreaks:
new: "class org.apache.iceberg.types.Types.NestedField"
justification: "new Constructor added"
org.apache.iceberg:iceberg-core:
- code: "java.method.visibilityReduced"
old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()"
new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()"
justification: "Deprecations for 1.6.0 release"
- code: "java.element.noLongerDeprecated"
old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()"
new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()"
justification: "Constructor became private as part of deprecations cleanup for\
\ 1.6.0 release"
justification: "Constructor became private as part of deprecations cleanup for 1.6.0 release"
- code: "java.element.noLongerDeprecated"
old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::<init>(java.util.Map<java.lang.String,\
\ java.lang.String>, java.lang.String, java.lang.String, java.lang.String,\
Expand All @@ -1058,10 +1061,6 @@ acceptedBreaks:
- code: "java.method.removed"
old: "method org.apache.iceberg.DataFiles.Builder org.apache.iceberg.DataFiles.Builder::withEqualityFieldIds(java.util.List<java.lang.Integer>)"
justification: "Deprecations for 1.6.0 release"
- code: "java.method.visibilityReduced"
old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()"
new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()"
justification: "Deprecations for 1.6.0 release"
"1.6.0":
org.apache.iceberg:iceberg-common:
- code: "java.method.removed"
Expand Down Expand Up @@ -1145,6 +1144,27 @@ acceptedBreaks:
new: "method org.apache.iceberg.BaseMetastoreOperations.CommitStatus org.apache.iceberg.BaseMetastoreTableOperations::checkCommitStatus(java.lang.String,\
\ org.apache.iceberg.TableMetadata)"
justification: "Removing deprecated code"
org.apache.iceberg:iceberg-parquet:
- code: "java.method.numberOfParametersChanged"
old: "method void org.apache.iceberg.parquet.ParquetReader<T>::<init>(org.apache.iceberg.io.InputFile,\
\ org.apache.iceberg.Schema, org.apache.parquet.ParquetReadOptions, java.util.function.Function<org.apache.parquet.schema.MessageType,\
\ org.apache.iceberg.parquet.ParquetValueReader<?>>, org.apache.iceberg.mapping.NameMapping,\
\ org.apache.iceberg.expressions.Expression, boolean, boolean)"
new: "method void org.apache.iceberg.parquet.ParquetReader<T>::<init>(org.apache.iceberg.io.InputFile,\
\ org.apache.iceberg.Schema, org.apache.parquet.ParquetReadOptions, java.util.function.Function<org.apache.parquet.schema.MessageType,\
\ org.apache.iceberg.parquet.ParquetValueReader<?>>, org.apache.iceberg.mapping.NameMapping,\
\ org.apache.iceberg.expressions.Expression, boolean, boolean, java.lang.Integer)"
justification: "{limit push down}"
- code: "java.method.numberOfParametersChanged"
old: "method void org.apache.iceberg.parquet.VectorizedParquetReader<T>::<init>(org.apache.iceberg.io.InputFile,\
\ org.apache.iceberg.Schema, org.apache.parquet.ParquetReadOptions, java.util.function.Function<org.apache.parquet.schema.MessageType,\
\ org.apache.iceberg.parquet.VectorizedReader<?>>, org.apache.iceberg.mapping.NameMapping,\
\ org.apache.iceberg.expressions.Expression, boolean, boolean, int)"
new: "method void org.apache.iceberg.parquet.VectorizedParquetReader<T>::<init>(org.apache.iceberg.io.InputFile,\
\ org.apache.iceberg.Schema, org.apache.parquet.ParquetReadOptions, java.util.function.Function<org.apache.parquet.schema.MessageType,\
\ org.apache.iceberg.parquet.VectorizedReader<?>>, org.apache.iceberg.mapping.NameMapping,\
\ org.apache.iceberg.expressions.Expression, boolean, boolean, int, java.lang.Integer)"
justification: "{limit push down}"
apache-iceberg-0.14.0:
org.apache.iceberg:iceberg-api:
- code: "java.class.defaultSerializationChanged"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -147,54 +147,60 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) {
}
if (vectorizedColumnIterator.hasNext()) {
if (dictEncoded) {
vectorizedColumnIterator.dictionaryBatchReader().nextBatch(vec, -1, nullabilityHolder);
vectorizedColumnIterator
.dictionaryBatchReader()
.nextBatch(numValsToRead, vec, -1, nullabilityHolder);
} else {
switch (readType) {
case VARBINARY:
case VARCHAR:
vectorizedColumnIterator
.varWidthTypeBatchReader()
.nextBatch(vec, -1, nullabilityHolder);
.nextBatch(numValsToRead, vec, -1, nullabilityHolder);
break;
case BOOLEAN:
vectorizedColumnIterator.booleanBatchReader().nextBatch(vec, -1, nullabilityHolder);
vectorizedColumnIterator
.booleanBatchReader()
.nextBatch(numValsToRead, vec, -1, nullabilityHolder);
break;
case INT:
case INT_BACKED_DECIMAL:
vectorizedColumnIterator
.integerBatchReader()
.nextBatch(vec, typeWidth, nullabilityHolder);
.nextBatch(numValsToRead, vec, typeWidth, nullabilityHolder);
break;
case LONG:
case LONG_BACKED_DECIMAL:
vectorizedColumnIterator.longBatchReader().nextBatch(vec, typeWidth, nullabilityHolder);
vectorizedColumnIterator
.longBatchReader()
.nextBatch(numValsToRead, vec, typeWidth, nullabilityHolder);
break;
case FLOAT:
vectorizedColumnIterator
.floatBatchReader()
.nextBatch(vec, typeWidth, nullabilityHolder);
.nextBatch(numValsToRead, vec, typeWidth, nullabilityHolder);
break;
case DOUBLE:
vectorizedColumnIterator
.doubleBatchReader()
.nextBatch(vec, typeWidth, nullabilityHolder);
.nextBatch(numValsToRead, vec, typeWidth, nullabilityHolder);
break;
case TIMESTAMP_MILLIS:
vectorizedColumnIterator
.timestampMillisBatchReader()
.nextBatch(vec, typeWidth, nullabilityHolder);
.nextBatch(numValsToRead, vec, typeWidth, nullabilityHolder);
break;
case TIMESTAMP_INT96:
vectorizedColumnIterator
.timestampInt96BatchReader()
.nextBatch(vec, typeWidth, nullabilityHolder);
.nextBatch(numValsToRead, vec, typeWidth, nullabilityHolder);
break;
case UUID:
case FIXED_WIDTH_BINARY:
case FIXED_LENGTH_DECIMAL:
vectorizedColumnIterator
.fixedSizeBinaryBatchReader()
.nextBatch(vec, typeWidth, nullabilityHolder);
.nextBatch(numValsToRead, vec, typeWidth, nullabilityHolder);
break;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.iceberg.arrow.vectorized.parquet;

import java.util.Locale;
import org.apache.arrow.vector.FieldVector;
import org.apache.arrow.vector.IntVector;
import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
Expand Down Expand Up @@ -69,12 +70,24 @@ public boolean producesDictionaryEncodedVector() {
}

public abstract class BatchReader {
public void nextBatch(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) {
public void nextBatch(
int numValsToRead, FieldVector fieldVector, int typeWidth, NullabilityHolder holder) {
int rowsReadSoFar = 0;
while (rowsReadSoFar < batchSize && hasNext()) {
while (rowsReadSoFar < batchSize && hasNext() && rowsReadSoFar < numValsToRead) {
advance();
int expectedBatchSize;
if (numValsToRead < 0) {
throw new IllegalStateException(
String.format(
Locale.ROOT,
"Cannot read a negative number of values. numValsToRead = %d",
numValsToRead));
} else {
expectedBatchSize = Math.min(batchSize - rowsReadSoFar, numValsToRead - rowsReadSoFar);
}

int rowsInThisBatch =
nextBatchOf(fieldVector, batchSize - rowsReadSoFar, rowsReadSoFar, typeWidth, holder);
nextBatchOf(fieldVector, expectedBatchSize, rowsReadSoFar, typeWidth, holder);
rowsReadSoFar += rowsInThisBatch;
triplesRead += rowsInThisBatch;
fieldVector.setValueCount(rowsReadSoFar);
Expand Down
20 changes: 18 additions & 2 deletions parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java
Original file line number Diff line number Diff line change
Expand Up @@ -1048,6 +1048,7 @@ public static class ReadBuilder {
private NameMapping nameMapping = null;
private ByteBuffer fileEncryptionKey = null;
private ByteBuffer fileAADPrefix = null;
private Integer pushedLimit;

private ReadBuilder(InputFile file) {
this.file = file;
Expand Down Expand Up @@ -1155,6 +1156,12 @@ public ReadBuilder withAADPrefix(ByteBuffer aadPrefix) {
return this;
}

public ReadBuilder pushedlimit(Integer limit) {
Preconditions.checkArgument(limit >= 0);
this.pushedLimit = limit;
return this;
}

@SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"})
public <D> CloseableIterable<D> build() {
FileDecryptionProperties fileDecryptionProperties = null;
Expand Down Expand Up @@ -1216,10 +1223,19 @@ public <D> CloseableIterable<D> build() {
filter,
reuseContainers,
caseSensitive,
maxRecordsPerBatch);
maxRecordsPerBatch,
pushedLimit);
} else {
return new org.apache.iceberg.parquet.ParquetReader<>(
file, schema, options, readerFunc, mapping, filter, reuseContainers, caseSensitive);
file,
schema,
options,
readerFunc,
mapping,
filter,
reuseContainers,
caseSensitive,
pushedLimit);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ public class ParquetReader<T> extends CloseableGroup implements CloseableIterabl
private final boolean reuseContainers;
private final boolean caseSensitive;
private final NameMapping nameMapping;
private Integer pushedLimit;

public ParquetReader(
InputFile input,
Expand All @@ -52,7 +53,8 @@ public ParquetReader(
NameMapping nameMapping,
Expression filter,
boolean reuseContainers,
boolean caseSensitive) {
boolean caseSensitive,
Integer pushedLimit) {
this.input = input;
this.expectedSchema = expectedSchema;
this.options = options;
Expand All @@ -62,6 +64,7 @@ public ParquetReader(
this.reuseContainers = reuseContainers;
this.caseSensitive = caseSensitive;
this.nameMapping = nameMapping;
this.pushedLimit = pushedLimit;
}

private ReadConf<T> conf = null;
Expand Down Expand Up @@ -89,6 +92,7 @@ private ReadConf<T> init() {
@Override
public CloseableIterator<T> iterator() {
FileIterator<T> iter = new FileIterator<>(init());
iter.pushedLimit = pushedLimit;
addCloseable(iter);
return iter;
}
Expand All @@ -105,6 +109,7 @@ private static class FileIterator<T> implements CloseableIterator<T> {
private long nextRowGroupStart = 0;
private long valuesRead = 0;
private T last = null;
private Integer pushedLimit;

FileIterator(ReadConf<T> conf) {
this.reader = conf.reader();
Expand All @@ -117,7 +122,11 @@ private static class FileIterator<T> implements CloseableIterator<T> {

@Override
public boolean hasNext() {
return valuesRead < totalValues;
if (pushedLimit != null && pushedLimit > 0) {
return valuesRead < Math.min(totalValues, pushedLimit);
} else {
return valuesRead < totalValues;
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ public class VectorizedParquetReader<T> extends CloseableGroup implements Closea
private final boolean caseSensitive;
private final int batchSize;
private final NameMapping nameMapping;
private Integer pushedLimit;

public VectorizedParquetReader(
InputFile input,
Expand All @@ -59,7 +60,8 @@ public VectorizedParquetReader(
Expression filter,
boolean reuseContainers,
boolean caseSensitive,
int maxRecordsPerBatch) {
int maxRecordsPerBatch,
Integer pushedLimit) {
this.input = input;
this.expectedSchema = expectedSchema;
this.options = options;
Expand All @@ -70,6 +72,7 @@ public VectorizedParquetReader(
this.caseSensitive = caseSensitive;
this.batchSize = maxRecordsPerBatch;
this.nameMapping = nameMapping;
this.pushedLimit = pushedLimit;
}

private ReadConf conf = null;
Expand Down Expand Up @@ -97,6 +100,7 @@ private ReadConf init() {
@Override
public CloseableIterator<T> iterator() {
FileIterator<T> iter = new FileIterator<>(init());
iter.pushedLimit = pushedLimit;
addCloseable(iter);
return iter;
}
Expand All @@ -114,6 +118,7 @@ private static class FileIterator<T> implements CloseableIterator<T> {
private long valuesRead = 0;
private T last = null;
private final long[] rowGroupsStartRowPos;
private Integer pushedLimit;

FileIterator(ReadConf conf) {
this.reader = conf.reader();
Expand All @@ -129,7 +134,11 @@ private static class FileIterator<T> implements CloseableIterator<T> {

@Override
public boolean hasNext() {
return valuesRead < totalValues;
if (pushedLimit != null && pushedLimit > 0) {
return valuesRead < Math.min(totalValues, pushedLimit);
} else {
return valuesRead < totalValues;
}
}

@Override
Expand All @@ -141,8 +150,18 @@ public T next() {
advance();
}

// batchSize is an integer, so casting to integer is safe
int numValuesToRead = (int) Math.min(nextRowGroupStart - valuesRead, batchSize);
long remainingValues = nextRowGroupStart - valuesRead;

int numValuesToRead;
if (pushedLimit != null && pushedLimit - valuesRead > 0) {
// batchSize is an integer, so casting to integer is safe
numValuesToRead =
(int) Math.min(remainingValues, Math.min(batchSize, pushedLimit - valuesRead));
} else {
// batchSize is an integer, so casting to integer is safe
numValuesToRead = (int) Math.min(remainingValues, batchSize);
}

if (reuseContainers) {
this.last = model.read(last, numValuesToRead);
} else {
Expand Down
2 changes: 2 additions & 0 deletions spark/v3.5/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,8 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") {
dependencies {
implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow')
api project(':iceberg-api')
annotationProcessor libs.immutables.value
compileOnly libs.immutables.value
implementation project(':iceberg-common')
implementation project(':iceberg-core')
implementation project(':iceberg-data')
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.spark;

import java.io.Serializable;
import org.immutables.value.Value;

@Value.Immutable
public interface OrcBatchReadConf extends Serializable {
int batchSize();
}
Loading