-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-2149: Async IO implementation for ParquetFileReader #968
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
base: master
Are you sure you want to change the base?
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,166 @@ | ||
| /* | ||
| * 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.parquet.bytes; | ||
|
|
||
| import java.io.IOException; | ||
| import java.io.UncheckedIOException; | ||
| import java.nio.ByteBuffer; | ||
| import java.util.List; | ||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.TimeoutException; | ||
| import java.util.concurrent.atomic.LongAccumulator; | ||
| import java.util.concurrent.atomic.LongAdder; | ||
| import org.apache.parquet.io.SeekableInputStream; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| class AsyncMultiBufferInputStream extends MultiBufferInputStream { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(AsyncMultiBufferInputStream.class); | ||
|
|
||
| private int fetchIndex = 0; | ||
| private final SeekableInputStream fileInputStream; | ||
| private int readIndex = 0; | ||
| private ExecutorService threadPool; | ||
| private LinkedBlockingQueue<Future<Void>> readFutures; | ||
| private boolean closed = false; | ||
|
|
||
| private LongAdder totalTimeBlocked = new LongAdder(); | ||
| private LongAdder totalCountBlocked = new LongAdder(); | ||
| private LongAccumulator maxTimeBlocked = new LongAccumulator(Long::max, 0L); | ||
|
|
||
| AsyncMultiBufferInputStream( | ||
| ExecutorService threadPool, SeekableInputStream fileInputStream, List<ByteBuffer> buffers) { | ||
| super(buffers); | ||
| this.fileInputStream = fileInputStream; | ||
| this.threadPool = threadPool; | ||
| readFutures = new LinkedBlockingQueue<>(buffers.size()); | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("ASYNC: Begin read into buffers "); | ||
| for (ByteBuffer buf : buffers) { | ||
| LOG.debug("ASYNC: buffer {} ", buf); | ||
| } | ||
| } | ||
| fetchAll(); | ||
| } | ||
|
|
||
| private void checkState() { | ||
| if (closed) { | ||
| throw new RuntimeException("Stream is closed"); | ||
| } | ||
| } | ||
|
|
||
| private void fetchAll() { | ||
| checkState(); | ||
| submitReadTask(0); | ||
| } | ||
|
|
||
| private void submitReadTask(int bufferNo) { | ||
| ByteBuffer buffer = buffers.get(bufferNo); | ||
| try { | ||
| readFutures.put(threadPool.submit(() -> { | ||
| readOneBuffer(buffer); | ||
| if (bufferNo < buffers.size() - 1) { | ||
| submitReadTask(bufferNo + 1); | ||
| } | ||
| return null; | ||
| })); | ||
| } catch (InterruptedException e) { | ||
| Thread.currentThread().interrupt(); | ||
| throw new RuntimeException(e); | ||
|
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. be nice if it was always UncheckedIOException being raised here and elsewhere |
||
| } | ||
| } | ||
|
|
||
| private void readOneBuffer(ByteBuffer buffer) { | ||
| long startTime = System.nanoTime(); | ||
| try { | ||
| fileInputStream.readFully(buffer); | ||
| buffer.flip(); | ||
| long readCompleted = System.nanoTime(); | ||
| long timeSpent = readCompleted - startTime; | ||
| LOG.debug("ASYNC Stream: READ - {}", timeSpent / 1000.0); | ||
| fetchIndex++; | ||
| } catch (IOException e) { | ||
| throw new UncheckedIOException(e); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public boolean nextBuffer() { | ||
| checkState(); | ||
| // hack: parent constructor can call this method before this class is fully initialized. | ||
|
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 see the comment with 'hack'. What is the proper 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. The fix would be to change behavior of the parent class. Basically the parent class implicitly assumes that there will be no derived classes and called a private method |
||
| // Just return without doing anything. | ||
| if (readFutures == null) { | ||
| return false; | ||
| } | ||
|
Comment on lines
+111
to
+116
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. Because
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. Glad you made me look. The synchronization was needed because I was saving any exception that might have occurred in the reader thread. However, that exception will be captured by the Future object and thrown as an ExecutionException when |
||
| if (readIndex < buffers.size()) { | ||
| long start = System.nanoTime(); | ||
| try { | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("ASYNC (next): Getting next buffer"); | ||
| } | ||
| Future<Void> future = readFutures.take(); | ||
| future.get(); | ||
| long timeSpent = System.nanoTime() - start; | ||
| totalCountBlocked.add(1); | ||
| totalTimeBlocked.add(timeSpent); | ||
| maxTimeBlocked.accumulate(timeSpent); | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("ASYNC (next): {}: Time blocked for read {} ns", this, timeSpent); | ||
| } | ||
| } catch (InterruptedException | ExecutionException e) { | ||
| if (e instanceof InterruptedException) { | ||
| Thread.currentThread().interrupt(); | ||
| } | ||
| LOG.error("Async (next): exception while getting next buffer: ", e); | ||
| throw new RuntimeException(e); | ||
| } | ||
| readIndex++; | ||
| } | ||
| return super.nextBuffer(); | ||
| } | ||
|
|
||
| public void close() { | ||
| LOG.debug( | ||
| "ASYNC Stream: Blocked: {} {} {}", | ||
| totalTimeBlocked.longValue() / 1000.0, | ||
| totalCountBlocked.longValue(), | ||
| maxTimeBlocked.longValue() / 1000.0); | ||
| Future<Void> readResult; | ||
| while (!readFutures.isEmpty()) { | ||
| try { | ||
| readResult = readFutures.poll(); | ||
| readResult.get(); | ||
| if (!readResult.isDone() && !readResult.isCancelled()) { | ||
| readResult.cancel(true); | ||
| } else { | ||
| readResult.get(1, TimeUnit.MILLISECONDS); | ||
| } | ||
| } catch (ExecutionException | InterruptedException | TimeoutException e) { | ||
| // Do nothing | ||
|
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. Do you think adding a log make sense?
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. It's not really useful to log anything here because there is nothing to be done. Some of the calls in the try block throw exceptions and we catch them to prevent them from propagating. We are shutting down and cleaning up the futures so it doesn't matter if there was an exception or not. |
||
| } | ||
| } | ||
| closed = true; | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we add
Closeableas an implemented interface?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Cool. I know a lot of times I check if instances of closeable and then call close.