-
Notifications
You must be signed in to change notification settings - Fork 3k
core: Adding read vector to range readable interface and adding mappe… #13997
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,55 @@ | ||
| /* | ||
| * 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.io; | ||
|
|
||
| import java.io.EOFException; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
|
|
||
| public class FileRange { | ||
| private final CompletableFuture<ByteBuffer> byteBuffer; | ||
| private final long offset; | ||
| private final int length; | ||
|
|
||
| public FileRange(CompletableFuture<ByteBuffer> byteBuffer, long offset, int length) | ||
| throws EOFException { | ||
| Preconditions.checkNotNull(byteBuffer, "byteBuffer can't be null"); | ||
| Preconditions.checkArgument( | ||
| length() >= 0, "Invalid length: %s in range (must be >= 0)", length); | ||
| Preconditions.checkArgument( | ||
| offset() >= 0, "Invalid offset: %s in range (must be >= 0)", offset); | ||
|
|
||
| this.byteBuffer = byteBuffer; | ||
| this.offset = offset; | ||
| this.length = length; | ||
| } | ||
|
|
||
| public CompletableFuture<ByteBuffer> byteBuffer() { | ||
| return byteBuffer; | ||
| } | ||
|
|
||
| public long offset() { | ||
| return offset; | ||
| } | ||
|
|
||
| public int length() { | ||
| return length; | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1340,6 +1340,7 @@ public <D> CloseableIterable<D> build() { | |
| optionsBuilder.withDecryption(fileDecryptionProperties); | ||
| } | ||
|
|
||
| optionsBuilder.withUseHadoopVectoredIo(true); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There were some efforts to allow Iceberg working without Hadoop on the classpath. Could you please help me understand the consequences of always using Thanks,
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For part 1 about the effort to reduce the dependencies on Hadoop I don't think that was ever completed I do see a TODO comment about wanting to do it. I am probably making the effort more complicated as I am adding 2 new imports from Hadoop but I don't think that is a big risk. for 2) withUseHadoopVectoredIo is used in the file reader in conjunction with readVectoredAvailable() so moving to always using readVector doesn't change anything unless the stream also supports readVectored.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think the naming of this option is a little misleading. The |
||
| ParquetReadOptions options = optionsBuilder.build(); | ||
|
|
||
| NameMapping mapping; | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,9 +18,15 @@ | |
| */ | ||
| package org.apache.iceberg.parquet; | ||
|
|
||
| import java.io.EOFException; | ||
| import java.io.IOException; | ||
| import java.io.InputStream; | ||
| import java.io.OutputStream; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.List; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.function.IntFunction; | ||
| import java.util.stream.Collectors; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.FSDataInputStream; | ||
| import org.apache.hadoop.fs.FSDataOutputStream; | ||
|
|
@@ -29,11 +35,16 @@ | |
| import org.apache.iceberg.hadoop.HadoopOutputFile; | ||
| import org.apache.iceberg.io.DelegatingInputStream; | ||
| import org.apache.iceberg.io.DelegatingOutputStream; | ||
| import org.apache.iceberg.io.FileRange; | ||
| import org.apache.iceberg.io.RangeReadable; | ||
| import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; | ||
| import org.apache.parquet.bytes.ByteBufferAllocator; | ||
| import org.apache.parquet.hadoop.util.HadoopStreams; | ||
| import org.apache.parquet.io.DelegatingPositionOutputStream; | ||
| import org.apache.parquet.io.DelegatingSeekableInputStream; | ||
| import org.apache.parquet.io.InputFile; | ||
| import org.apache.parquet.io.OutputFile; | ||
| import org.apache.parquet.io.ParquetFileRange; | ||
| import org.apache.parquet.io.PositionOutputStream; | ||
| import org.apache.parquet.io.SeekableInputStream; | ||
|
|
||
|
|
@@ -91,6 +102,11 @@ static SeekableInputStream stream(org.apache.iceberg.io.SeekableInputStream stre | |
| return HadoopStreams.wrap((FSDataInputStream) wrapped); | ||
| } | ||
| } | ||
|
|
||
| if (stream instanceof RangeReadable) { | ||
stubz151 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return new ParquetRangeReadableInputStreamAdapter(stream); | ||
| } | ||
stubz151 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| return new ParquetInputStreamAdapter(stream); | ||
| } | ||
|
|
||
|
|
@@ -123,6 +139,73 @@ public void seek(long newPos) throws IOException { | |
| } | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| static class ParquetRangeReadableInputStreamAdapter< | ||
| T extends org.apache.iceberg.io.SeekableInputStream & RangeReadable> | ||
| extends DelegatingSeekableInputStream implements RangeReadable { | ||
| private final T delegate; | ||
|
|
||
| ParquetRangeReadableInputStreamAdapter(T delegate) { | ||
| super(delegate); | ||
| this.delegate = delegate; | ||
| } | ||
|
|
||
| @Override | ||
| public long getPos() throws IOException { | ||
| return delegate.getPos(); | ||
| } | ||
|
|
||
| @Override | ||
| public void seek(long newPos) throws IOException { | ||
| delegate.seek(newPos); | ||
| } | ||
|
|
||
| @Override | ||
| public void readFully(long position, byte[] buffer, int offset, int length) throws IOException { | ||
| delegate.readFully(position, buffer, offset, length); | ||
| } | ||
|
|
||
| @Override | ||
| public int readTail(byte[] buffer, int offset, int length) throws IOException { | ||
| return delegate.readTail(buffer, offset, length); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean readVectoredAvailable(ByteBufferAllocator allocate) { | ||
| return true; | ||
| } | ||
|
|
||
| @Override | ||
| public void readVectored(List<ParquetFileRange> ranges, ByteBufferAllocator allocate) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we add some tests at the ParquetIO level to validate this? I know we're adding some in S3FileIO, but it would be good to have this interface tested (even if there's a mock implementation)
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I added in testRangeReadableAdapterReadVectored which does something similar to the tests in S3FileIO, but focused a bit more on checking that the buffers/ranges are being used correctly, I skipped the other operations but can add them in if we want. Let me know |
||
| throws IOException { | ||
| IntFunction<ByteBuffer> delegateAllocate = (allocate::allocate); | ||
| List<FileRange> delegateRange = convertRanges(ranges); | ||
danielcweeks marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| delegate.readVectored(delegateRange, delegateAllocate); | ||
| } | ||
|
|
||
| private static List<FileRange> convertRanges(List<ParquetFileRange> ranges) { | ||
| return ranges.stream() | ||
| .map( | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this just maps between the internal parquet hadoop range and the new iceberg one. |
||
| parquetFileRange -> { | ||
| CompletableFuture<ByteBuffer> future = new CompletableFuture<>(); | ||
| parquetFileRange.setDataReadFuture(future); | ||
| try { | ||
| return new FileRange( | ||
| parquetFileRange.getDataReadFuture(), | ||
| parquetFileRange.getOffset(), | ||
| parquetFileRange.getLength()); | ||
| } catch (EOFException e) { | ||
| throw new RuntimeIOException( | ||
| e, | ||
| "Failed to create range file for offset: %s and length: %s", | ||
| parquetFileRange.getOffset(), | ||
| parquetFileRange.getLength()); | ||
| } | ||
| }) | ||
| .collect(Collectors.toList()); | ||
| } | ||
| } | ||
|
|
||
| private static class ParquetOutputStreamAdapter extends DelegatingPositionOutputStream { | ||
| private final org.apache.iceberg.io.PositionOutputStream delegate; | ||
|
|
||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.