From 3fa76fd0a6069ec0b3ae6396b29f6d712f6e2453 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Fri, 9 Feb 2024 13:55:12 -0800 Subject: [PATCH 01/13] Improve S3, parquet, and SeekableChannelsProvider Fixes #5096 --- Util/channel/build.gradle | 3 + .../util/channel/CachedChannelProvider.java | 11 + .../util/channel/LocalFSChannelProvider.java | 7 +- .../util/channel/PositionInputStream.java | 47 ++ .../deephaven/util/channel/ProviderImpl.java | 26 ++ .../channel/ReadableByteChannelNoClose.java | 40 ++ .../util/channel/SeekableChannelContext.java | 21 +- .../channel/SeekableChannelContextNull.java | 8 + .../channel/SeekableChannelsProvider.java | 40 ++ .../channel/SeekableChannelsProviderBase.java | 20 + .../channel/CachedChannelProviderTest.java | 7 +- .../parquet/base/ColumnChunkReaderImpl.java | 107 +++-- .../parquet/base/ColumnPageReaderImpl.java | 246 +++++----- .../parquet/base/ParquetFileReader.java | 66 +-- .../parquet/base/RowGroupReaderImpl.java | 38 +- .../table/location/ParquetColumnLocation.java | 5 +- .../table/location/ParquetTableLocation.java | 2 +- .../table/pagestore/ColumnChunkPageStore.java | 5 + .../OffsetIndexBasedColumnChunkPageStore.java | 23 +- .../VariablePageSizeColumnChunkPageStore.java | 26 +- extensions/s3/build.gradle | 3 +- .../deephaven/extensions/s3/BufferPool.java | 40 +- .../ByteBufferAsyncResponseTransformer.java | 115 ----- .../extensions/s3/S3ChannelContext.java | 430 ++++++++++++++++++ .../extensions/s3/S3Instructions.java | 12 +- .../extensions/s3/S3SeekableByteChannel.java | 345 ++------------ .../s3/S3SeekableChannelProvider.java | 49 +- .../extensions/s3/SegmentedBufferPool.java | 61 --- .../TrackedSeekableChannelsProvider.java | 10 +- .../jetty-app/src/main/resources/logback.xml | 4 + 30 files changed, 1013 insertions(+), 804 deletions(-) create mode 100644 Util/channel/src/main/java/io/deephaven/util/channel/PositionInputStream.java create mode 100644 Util/channel/src/main/java/io/deephaven/util/channel/ProviderImpl.java create mode 100644 Util/channel/src/main/java/io/deephaven/util/channel/ReadableByteChannelNoClose.java create mode 100644 Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContextNull.java create mode 100644 Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderBase.java delete mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java create mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java delete mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/SegmentedBufferPool.java diff --git a/Util/channel/build.gradle b/Util/channel/build.gradle index d6a7482f331..dffa3d52b91 100644 --- a/Util/channel/build.gradle +++ b/Util/channel/build.gradle @@ -9,6 +9,9 @@ dependencies { // Needed for SafeCloseable implementation project(':Util') + // For CountingInputStream + Classpaths.inheritGuava(project) + compileOnly depAnnotations Classpaths.inheritJUnitPlatform(project) diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java index 2abc89c1357..472e952dbba 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java @@ -13,6 +13,7 @@ import org.jetbrains.annotations.Nullable; import java.io.IOException; +import java.io.InputStream; import java.net.URI; import java.nio.ByteBuffer; import java.nio.channels.SeekableByteChannel; @@ -67,6 +68,11 @@ public SeekableChannelContext makeContext() { return wrappedProvider.makeContext(); } + @Override + public SeekableChannelContext makeSingleUseContext() { + return wrappedProvider.makeSingleUseContext(); + } + @Override public boolean isCompatibleWith(@NotNull final SeekableChannelContext channelContext) { return wrappedProvider.isCompatibleWith(channelContext); @@ -86,6 +92,11 @@ public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext return channel; } + @Override + public InputStream getInputStream(SeekableByteChannel channel) throws IOException { + return wrappedProvider.getInputStream(channel); + } + @Override public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boolean append) throws IOException { final String pathKey = path.toAbsolutePath().toString(); diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java index 59ddcbd3d4a..c144b6cb312 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java @@ -13,7 +13,12 @@ import java.nio.file.Path; import java.nio.file.StandardOpenOption; -public class LocalFSChannelProvider implements SeekableChannelsProvider { +public class LocalFSChannelProvider extends SeekableChannelsProviderBase { + + @Override + protected boolean readChannelIsBuffered() { + return false; + } @Override public SeekableChannelContext makeContext() { diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/PositionInputStream.java b/Util/channel/src/main/java/io/deephaven/util/channel/PositionInputStream.java new file mode 100644 index 00000000000..bc2cb7048ba --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/PositionInputStream.java @@ -0,0 +1,47 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +import com.google.common.io.CountingInputStream; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.channels.SeekableByteChannel; +import java.util.Objects; + +public final class PositionInputStream extends FilterInputStream { + + /** + * Wraps a channel-backed input stream {@code in}, ensuring upon {@link #close()} that {@code channel's} + * {@link SeekableByteChannel#position()} has been advanced the exact amount of bytes that have been consumed from + * the resulting input stream. {@code in} is closed during {@link #close()}; as such, the caller must ensure + * that closing {@code in} does _not_ close {@code channel}. To remain valid, the caller must ensure that the + * resulting input stream isn't re-wrapped by any downstream code in a way that would adversely effect the position + * (such as wrapping the resulting input stream with buffering). + * + * @param channel the channel + * @param in the input stream based on the channel + * @return a positional input stream + * @throws IOException if an IO exception occurs + */ + public static InputStream of(SeekableByteChannel channel, InputStream in) throws IOException { + return new PositionInputStream(channel, in); + } + + private final SeekableByteChannel ch; + private final long position; + + private PositionInputStream(SeekableByteChannel ch, InputStream in) throws IOException { + super(new CountingInputStream(in)); + this.ch = Objects.requireNonNull(ch); + this.position = ch.position(); + } + + @Override + public void close() throws IOException { + super.close(); + ch.position(position + ((CountingInputStream) in).getCount()); + } +} diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/ProviderImpl.java b/Util/channel/src/main/java/io/deephaven/util/channel/ProviderImpl.java new file mode 100644 index 00000000000..80780e02b82 --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/ProviderImpl.java @@ -0,0 +1,26 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +import io.deephaven.util.channel.SeekableChannelContext.Provider; + +import java.util.Objects; + +final class ProviderImpl implements Provider { + private final SeekableChannelContext context; + + public ProviderImpl(SeekableChannelContext context) { + this.context = Objects.requireNonNull(context); + } + + @Override + public SeekableChannelContext get() { + return context; + } + + @Override + public void close() { + context.close(); + } +} diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/ReadableByteChannelNoClose.java b/Util/channel/src/main/java/io/deephaven/util/channel/ReadableByteChannelNoClose.java new file mode 100644 index 00000000000..95b45c618a8 --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/ReadableByteChannelNoClose.java @@ -0,0 +1,40 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.util.Objects; + +final class ReadableByteChannelNoClose implements ReadableByteChannel { + + public static ReadableByteChannel of(ReadableByteChannel ch) { + if (ch instanceof ReadableByteChannelNoClose) { + return ch; + } + return new ReadableByteChannelNoClose(ch); + } + + private final ReadableByteChannel ch; + + private ReadableByteChannelNoClose(ReadableByteChannel ch) { + this.ch = Objects.requireNonNull(ch); + } + + @Override + public int read(ByteBuffer dst) throws IOException { + return ch.read(dst); + } + + @Override + public boolean isOpen() { + return ch.isOpen(); + } + + @Override + public void close() { + // skip + } +} diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java index c5e7e155068..4a8cfbfcf6b 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java @@ -2,15 +2,34 @@ import io.deephaven.util.SafeCloseable; +import java.io.Closeable; +import java.util.function.Supplier; + /** * Context object for reading and writing to channels created by {@link SeekableChannelsProvider}. */ public interface SeekableChannelContext extends SafeCloseable { - SeekableChannelContext NULL = new SeekableChannelContext() {}; + SeekableChannelContext NULL = SeekableChannelContextNull.NULL; + + static Provider upgrade(SeekableChannelsProvider provider, SeekableChannelContext context) { + if (context != NULL) { + return () -> context; + } + return new ProviderImpl(provider.makeSingleUseContext()); + } /** * Release any resources associated with this context. The context should not be used afterward. */ default void close() {} + + interface Provider extends Closeable, Supplier { + + @Override + SeekableChannelContext get(); + + @Override + default void close() {} + } } diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContextNull.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContextNull.java new file mode 100644 index 00000000000..3fa38a88e56 --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContextNull.java @@ -0,0 +1,8 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +enum SeekableChannelContextNull implements SeekableChannelContext { + NULL +} diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java index 75be19f4592..58481fecc2d 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java @@ -8,6 +8,7 @@ import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.net.URI; import java.net.URISyntaxException; import java.nio.channels.SeekableByteChannel; @@ -37,11 +38,34 @@ static URI convertToURI(final String source) { return uri; } + /** + * Wraps {@link SeekableChannelsProvider#getInputStream(SeekableByteChannel)} in a position-safe manner. To remain + * valid, the caller must ensure that the resulting input stream isn't re-wrapped by any downstream code in a way + * that would adversely effect the position (such as re-wrapping the resulting input stream with buffering). + * + *

+ * Equivalent to {@code PositionInputStream.of(ch, provider.getInputStream(ch))}. + * + * @param provider the provider + * @param ch the seekable channel + * @return the position-safe input stream + * @throws IOException if an IO exception occurs + * @see PositionInputStream#of(SeekableByteChannel, InputStream) + */ + static InputStream positionInputStream(SeekableChannelsProvider provider, SeekableByteChannel ch) + throws IOException { + return PositionInputStream.of(ch, provider.getInputStream(ch)); + } + /** * Create a new {@link SeekableChannelContext} object for creating read channels via this provider. */ SeekableChannelContext makeContext(); + default SeekableChannelContext makeSingleUseContext() { + return makeContext(); + } + /** * Check if the given context is compatible with this provider. Useful to test if we can use provided * {@code context} object for creating channels with this provider. @@ -56,6 +80,22 @@ default SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext chann SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull URI uri) throws IOException; + /** + * Creates an {@link InputStream} from the current position of {@code channel}; closing the resulting input stream + * does not close the {@code channel}. The {@link InputStream} will be buffered; either explicitly in the + * case where the implementation uses an unbuffered {@link #getReadChannel(SeekableChannelContext, URI)}, or + * implicitly when the implementation uses a buffered {@link #getReadChannel(SeekableChannelContext, URI)}. + * {@code channel} must have been created by {@code this} provider. The caller can't assume the position of + * {@code channel} after consuming the {@link InputStream}. For use-cases that require the channel's position to be + * incremented the exact amount the {@link InputStream} has been consumed, use + * {@link #positionInputStream(SeekableChannelsProvider, SeekableByteChannel)}. + * + * @param channel the channel + * @return the input stream + * @throws IOException if an IO exception occurs + */ + InputStream getInputStream(SeekableByteChannel channel) throws IOException; + default SeekableByteChannel getWriteChannel(@NotNull final String path, final boolean append) throws IOException { return getWriteChannel(Paths.get(path), append); } diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderBase.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderBase.java new file mode 100644 index 00000000000..bacee4ee022 --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderBase.java @@ -0,0 +1,20 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +import java.io.BufferedInputStream; +import java.io.InputStream; +import java.nio.channels.Channels; +import java.nio.channels.SeekableByteChannel; + +public abstract class SeekableChannelsProviderBase implements SeekableChannelsProvider { + + protected abstract boolean readChannelIsBuffered(); + + @Override + public final InputStream getInputStream(SeekableByteChannel channel) { + final InputStream in = Channels.newInputStream(ReadableByteChannelNoClose.of(channel)); + return readChannelIsBuffered() ? in : new BufferedInputStream(in, 8192); + } +} diff --git a/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java b/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java index 64ee9c667e9..f0182ab8ed5 100644 --- a/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java +++ b/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java @@ -170,13 +170,18 @@ public void testReuse10() throws IOException { } - private class TestChannelProvider implements SeekableChannelsProvider { + private class TestChannelProvider extends SeekableChannelsProviderBase { AtomicInteger count = new AtomicInteger(0); private final class TestChannelContext implements SeekableChannelContext { } + @Override + protected boolean readChannelIsBuffered() { + return true; // TestMockChannel always returns 0, might as well be "buffered" + } + @Override public SeekableChannelContext makeContext() { return new TestChannelContext(); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 88de4065b1e..bc3858d56c6 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java @@ -8,6 +8,7 @@ import io.deephaven.util.channel.SeekableChannelsProvider; import io.deephaven.parquet.compress.CompressorAdapter; import io.deephaven.parquet.compress.DeephavenCompressorAdapterFactory; +import io.deephaven.util.channel.SeekableChannelContext.Provider; import io.deephaven.util.datastructures.LazyCachingFunction; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.ColumnDescriptor; @@ -21,12 +22,10 @@ import org.apache.parquet.schema.Type; import org.jetbrains.annotations.NotNull; -import java.io.BufferedInputStream; import java.io.IOException; +import java.io.InputStream; import java.io.UncheckedIOException; import java.net.URI; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; import java.util.List; @@ -177,23 +176,13 @@ private Dictionary getDictionary(final SeekableChannelContext channelContext) { } else { return NULL_DICTIONARY; } - if (channelContext == SeekableChannelContext.NULL) { - // Create a new context object and use that for reading the dictionary - try (final SeekableChannelContext newChannelContext = channelsProvider.makeContext()) { - return getDictionaryHelper(newChannelContext, dictionaryPageOffset); - } - } else { - // Use the context object provided by the caller - return getDictionaryHelper(channelContext, dictionaryPageOffset); - } - } - - private Dictionary getDictionaryHelper(final SeekableChannelContext channelContext, - final long dictionaryPageOffset) { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, getURI())) { - readChannel.position(dictionaryPageOffset); - return readDictionary(readChannel); - } catch (final IOException e) { + // Use the context object provided by the caller, or create (and close) a new one + try ( + final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), getURI()); + final InputStream in = channelsProvider.getInputStream(ch.position(dictionaryPageOffset))) { + return readDictionary(in); + } catch (IOException e) { throw new UncheckedIOException(e); } } @@ -214,38 +203,34 @@ public SeekableChannelsProvider getChannelsProvider() { } @NotNull - private Dictionary readDictionary(ReadableByteChannel file) throws IOException { + private Dictionary readDictionary(InputStream in) throws IOException { // explicitly not closing this, caller is responsible - final BufferedInputStream inputStream = new BufferedInputStream(Channels.newInputStream(file)); - final PageHeader pageHeader = Util.readPageHeader(inputStream); + final PageHeader pageHeader = Util.readPageHeader(in); if (pageHeader.getType() != PageType.DICTIONARY_PAGE) { // In case our fallback in getDictionary was too optimistic... return NULL_DICTIONARY; } final DictionaryPageHeader dictHeader = pageHeader.getDictionary_page_header(); - - final BytesInput payload; final int compressedPageSize = pageHeader.getCompressed_page_size(); + final BytesInput payload; if (compressedPageSize == 0) { // Sometimes the size is explicitly empty, just use an empty payload payload = BytesInput.empty(); } else { - payload = decompressor.decompress(inputStream, compressedPageSize, pageHeader.getUncompressed_page_size()); + payload = decompressor.decompress(in, compressedPageSize, pageHeader.getUncompressed_page_size()); } - final DictionaryPage dictionaryPage = new DictionaryPage(payload, dictHeader.getNum_values(), Encoding.valueOf(dictHeader.getEncoding().name())); - return dictionaryPage.getEncoding().initDictionary(path, dictionaryPage); } private final class ColumnPageReaderIteratorImpl implements ColumnPageReaderIterator { - private long currentOffset; + private long nextHeaderOffset; private long remainingValues; ColumnPageReaderIteratorImpl(final long startOffset, final long numValues) { this.remainingValues = numValues; - this.currentOffset = startOffset; + this.nextHeaderOffset = startOffset; } @Override @@ -259,50 +244,60 @@ public ColumnPageReader next(@NotNull final SeekableChannelContext channelContex throw new NoSuchElementException("No next element"); } // NB: The channels provider typically caches channels; this avoids maintaining a handle per column chunk - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, getURI())) { - final long headerOffset = currentOffset; - readChannel.position(currentOffset); - // deliberately not closing this stream - final PageHeader pageHeader = Util.readPageHeader(Channels.newInputStream(readChannel)); - currentOffset = readChannel.position() + pageHeader.getCompressed_page_size(); + final long headerOffset = nextHeaderOffset; + try ( + final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), getURI())) { + ch.position(headerOffset); + final PageHeader pageHeader; + try (final InputStream in = SeekableChannelsProvider.positionInputStream(channelsProvider, ch)) { + pageHeader = Util.readPageHeader(in); + } + // relying on exact position of ch + final long dataOffset = ch.position(); + nextHeaderOffset = dataOffset + pageHeader.getCompressed_page_size(); if (pageHeader.isSetDictionary_page_header()) { // Dictionary page; skip it - return next(channelContext); + return next(upgrade.get()); } if (!pageHeader.isSetData_page_header() && !pageHeader.isSetData_page_header_v2()) { throw new IllegalStateException( "Expected data page, but neither v1 nor v2 data page header is set in file " - + readChannel + " at offset " + headerOffset); - } - remainingValues -= pageHeader.isSetData_page_header() - ? pageHeader.getData_page_header().getNum_values() - : pageHeader.getData_page_header_v2().getNum_values(); - final org.apache.parquet.format.Encoding encoding; - switch (pageHeader.type) { - case DATA_PAGE: - encoding = pageHeader.getData_page_header().getEncoding(); - break; - case DATA_PAGE_V2: - encoding = pageHeader.getData_page_header_v2().getEncoding(); - break; - default: - throw new UncheckedDeephavenException( - "Unknown parquet data page header type " + pageHeader.type); + + ch + " at offset " + headerOffset); } + remainingValues -= getNumValues(pageHeader); + final org.apache.parquet.format.Encoding encoding = getEncoding(pageHeader); final Function pageDictionarySupplier = (encoding == PLAIN_DICTIONARY || encoding == RLE_DICTIONARY) ? dictionarySupplier : (SeekableChannelContext context) -> NULL_DICTIONARY; - final ColumnPageReader nextReader = new ColumnPageReaderImpl(channelsProvider, decompressor, + return new ColumnPageReaderImpl(channelsProvider, decompressor, pageDictionarySupplier, nullMaterializerFactory, path, getURI(), fieldTypes, - readChannel.position(), pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES); - return nextReader; + dataOffset, pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES); } catch (IOException e) { throw new UncheckedDeephavenException("Error reading page header", e); } } } + private static org.apache.parquet.format.Encoding getEncoding(PageHeader pageHeader) { + switch (pageHeader.type) { + case DATA_PAGE: + return pageHeader.getData_page_header().getEncoding(); + case DATA_PAGE_V2: + return pageHeader.getData_page_header_v2().getEncoding(); + default: + throw new UncheckedDeephavenException( + "Unknown parquet data page header type " + pageHeader.type); + } + } + + private static int getNumValues(PageHeader pageHeader) { + return pageHeader.isSetData_page_header() + ? pageHeader.getData_page_header().getNum_values() + : pageHeader.getData_page_header_v2().getNum_values(); + } + private final class ColumnPageReaderIteratorIndexImpl implements ColumnPageReaderIterator { private int pos; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java index f84458e2d3a..2e147508890 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java @@ -5,11 +5,11 @@ import io.deephaven.base.Pair; import io.deephaven.base.verify.Assert; -import io.deephaven.parquet.base.util.Helpers; import io.deephaven.parquet.base.util.RunLengthBitPackingHybridBufferDecoder; +import io.deephaven.parquet.compress.CompressorAdapter; import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.parquet.compress.CompressorAdapter; +import io.deephaven.util.channel.SeekableChannelContext.Provider; import org.apache.parquet.bytes.ByteBufferInputStream; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.ColumnDescriptor; @@ -22,18 +22,18 @@ import org.apache.parquet.format.DataPageHeader; import org.apache.parquet.format.DataPageHeaderV2; import org.apache.parquet.format.PageHeader; +import org.apache.parquet.format.PageType; import org.apache.parquet.format.Util; import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.schema.Type; import org.jetbrains.annotations.NotNull; import java.io.IOException; +import java.io.InputStream; import java.net.URI; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.IntBuffer; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; import java.util.ArrayList; import java.util.List; @@ -42,9 +42,6 @@ import static org.apache.parquet.column.ValuesType.VALUES; public class ColumnPageReaderImpl implements ColumnPageReader { - - private static final int MAX_HEADER = 8192; - private static final int START_HEADER = 128; public static final int NULL_OFFSET = -1; static final int NULL_NUM_VALUES = -1; @@ -107,61 +104,48 @@ public class ColumnPageReaderImpl implements ColumnPageReader { @Override public Object materialize(@NotNull final Object nullValue, @NotNull final SeekableChannelContext channelContext) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { - ensurePageHeader(readChannel); - return readDataPage(nullValue, readChannel, channelContext); + try ( + final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), uri)) { + ensurePageHeader(channelsProvider, ch); + return readDataPage(nullValue, ch, upgrade.get()); } } public int readRowCount(@NotNull final SeekableChannelContext channelContext) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { - ensurePageHeader(readChannel); - return readRowCountFromDataPage(readChannel); + try ( + final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), uri)) { + ensurePageHeader(channelsProvider, ch); + return readRowCountFromDataPage(ch); } } - @Override public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder, @NotNull final SeekableChannelContext channelContext) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { - ensurePageHeader(readChannel); - return readKeyFromDataPage(keyDest, nullPlaceholder, readChannel, channelContext); + try ( + final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), uri)) { + ensurePageHeader(channelsProvider, ch); + return readKeyFromDataPage(keyDest, nullPlaceholder, ch, upgrade.get()); } } /** - * If {@link #pageHeader} is {@code null}, read it from the file, and increment the {@link #offset} by the length of - * page header. Channel position would be set to the end of page header or beginning of data before returning. + * If {@link #pageHeader} is {@code null}, read it from the channel, and increment the {@link #offset} by the length + * of page header. Channel position would be set to the end of page header or beginning of data before returning. */ - private void ensurePageHeader(final SeekableByteChannel file) throws IOException { + private void ensurePageHeader(SeekableChannelsProvider provider, SeekableByteChannel ch) throws IOException { // Set this channel's position to appropriate offset for reading. If pageHeader is null, this offset would be // the offset of page header, else it would be the offset of data. - file.position(offset); + ch.position(offset); synchronized (this) { if (pageHeader == null) { - int maxHeader = START_HEADER; - boolean success; - do { - final ByteBuffer headerBuffer = ByteBuffer.allocate(maxHeader); - Helpers.readExact(file, headerBuffer); - headerBuffer.flip(); - - final ByteBufferInputStream bufferedIS = ByteBufferInputStream.wrap(headerBuffer); - try { - pageHeader = Util.readPageHeader(bufferedIS); - offset += bufferedIS.position(); - success = true; - } catch (IOException e) { - success = false; - if (maxHeader > MAX_HEADER) { - throw e; - } - maxHeader <<= 1; - file.position(offset); - } - } while (!success); - file.position(offset); + try (final InputStream in = SeekableChannelsProvider.positionInputStream(provider, ch)) { + pageHeader = Util.readPageHeader(in); + } + offset = ch.position(); if (numValues >= 0) { final int numValuesFromHeader = readNumValuesFromPageHeader(pageHeader); if (numValues != numValuesFromHeader) { @@ -188,25 +172,69 @@ private static int readNumValuesFromPageHeader(@NotNull final PageHeader header) } } - private int readRowCountFromDataPage(ReadableByteChannel file) throws IOException { - int uncompressedPageSize = pageHeader.getUncompressed_page_size(); - int compressedPageSize = pageHeader.getCompressed_page_size(); + private DataPageV1 readV1(SeekableByteChannel ch) throws IOException { + try (final InputStream in = channelsProvider.getInputStream(ch)) { + return readV1Impl(in); + } + } + + private DataPageV1 readV1Impl(InputStream in) throws IOException { + if (pageHeader.type != PageType.DATA_PAGE) { + throw new IllegalArgumentException(); + } + final int uncompressedPageSize = pageHeader.getUncompressed_page_size(); + final int compressedPageSize = pageHeader.getCompressed_page_size(); + final BytesInput decompressedInput = compressorAdapter.decompress(in, compressedPageSize, uncompressedPageSize); + final DataPageHeader header = pageHeader.getData_page_header(); + return new DataPageV1( + decompressedInput, + header.getNum_values(), + uncompressedPageSize, + null, // TODO in the future might want to pull in statistics + getEncoding(header.getRepetition_level_encoding()), + getEncoding(header.getDefinition_level_encoding()), + getEncoding(header.getEncoding())); + } + + private DataPageV2 readV2(SeekableByteChannel ch) throws IOException { + try (final InputStream in = channelsProvider.getInputStream(ch)) { + return readV2Impl(in); + } + } + + private DataPageV2 readV2Impl(InputStream in) throws IOException { + if (pageHeader.type != PageType.DATA_PAGE_V2) { + throw new IllegalArgumentException(); + } + final int uncompressedPageSize = pageHeader.getUncompressed_page_size(); + final int compressedPageSize = pageHeader.getCompressed_page_size(); + final DataPageHeaderV2 header = pageHeader.getData_page_header_v2(); + final int compressedSize = compressedPageSize - header.getRepetition_levels_byte_length() + - header.getDefinition_levels_byte_length(); + final int uncompressedSize = uncompressedPageSize - header.getRepetition_levels_byte_length() + - header.getDefinition_levels_byte_length(); + final BytesInput repetitionLevels = + BytesInput.copy(BytesInput.from(in, header.getRepetition_levels_byte_length())); + final BytesInput definitionLevels = + BytesInput.copy(BytesInput.from(in, header.getDefinition_levels_byte_length())); + final BytesInput data = compressorAdapter.decompress(in, compressedSize, uncompressedSize); + return new DataPageV2( + header.getNum_rows(), + header.getNum_nulls(), + header.getNum_values(), + repetitionLevels, + definitionLevels, + getEncoding(header.getEncoding()), + data, + uncompressedPageSize, + null, // TODO in the future might want to pull in statistics, + false); + } + private int readRowCountFromDataPage(SeekableByteChannel ch) throws IOException { switch (pageHeader.type) { case DATA_PAGE: - final BytesInput decompressedInput = - compressorAdapter.decompress(Channels.newInputStream(file), compressedPageSize, - uncompressedPageSize); - - DataPageHeader dataHeaderV1 = pageHeader.getData_page_header(); - return readRowCountFromPageV1(new DataPageV1( - decompressedInput, - dataHeaderV1.getNum_values(), - uncompressedPageSize, - null, // TODO in the future might want to pull in statistics - getEncoding(dataHeaderV1.getRepetition_level_encoding()), - getEncoding(dataHeaderV1.getDefinition_level_encoding()), - getEncoding(dataHeaderV1.getEncoding()))); + return readRowCountFromPageV1(readV1(ch)); case DATA_PAGE_V2: DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); return dataHeaderV2.getNum_rows(); @@ -216,102 +244,30 @@ private int readRowCountFromDataPage(ReadableByteChannel file) throws IOExceptio } private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, - ReadableByteChannel file, @NotNull final SeekableChannelContext channelContext) + SeekableByteChannel ch, @NotNull final SeekableChannelContext channelContext) throws IOException { - int uncompressedPageSize = pageHeader.getUncompressed_page_size(); - int compressedPageSize = pageHeader.getCompressed_page_size(); - switch (pageHeader.type) { case DATA_PAGE: - BytesInput decompressedInput = - compressorAdapter.decompress(Channels.newInputStream(file), compressedPageSize, - uncompressedPageSize); - - DataPageHeader dataHeaderV1 = pageHeader.getData_page_header(); - return readKeysFromPageV1(new DataPageV1( - decompressedInput, - dataHeaderV1.getNum_values(), - uncompressedPageSize, - null, // TODO in the future might want to pull in statistics - getEncoding(dataHeaderV1.getRepetition_level_encoding()), - getEncoding(dataHeaderV1.getDefinition_level_encoding()), - getEncoding(dataHeaderV1.getEncoding())), keyDest, nullPlaceholder, channelContext); - + return readKeysFromPageV1(readV1(ch), keyDest, nullPlaceholder, channelContext); case DATA_PAGE_V2: - DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); - int dataSize = compressedPageSize - dataHeaderV2.getRepetition_levels_byte_length() - - dataHeaderV2.getDefinition_levels_byte_length(); - BytesInput repetitionLevels = - Helpers.readBytes(file, dataHeaderV2.getRepetition_levels_byte_length()); - BytesInput definitionLevels = - Helpers.readBytes(file, dataHeaderV2.getDefinition_levels_byte_length()); - BytesInput data = compressorAdapter.decompress(Channels.newInputStream(file), dataSize, - uncompressedPageSize - - dataHeaderV2.getRepetition_levels_byte_length() - - dataHeaderV2.getDefinition_levels_byte_length()); - - readKeysFromPageV2(new DataPageV2( - dataHeaderV2.getNum_rows(), - dataHeaderV2.getNum_nulls(), - dataHeaderV2.getNum_values(), - repetitionLevels, - definitionLevels, - getEncoding(dataHeaderV2.getEncoding()), - data, - uncompressedPageSize, - null, // TODO in the future might want to pull in statistics, - false), keyDest, nullPlaceholder, channelContext); + readKeysFromPageV2(readV2(ch), keyDest, nullPlaceholder, channelContext); return null; default: throw new IOException(String.format("Unexpected page of type %s of size %d", pageHeader.getType(), - compressedPageSize)); + pageHeader.getCompressed_page_size())); } } - private Object readDataPage(Object nullValue, SeekableByteChannel file, + private Object readDataPage(Object nullValue, SeekableByteChannel ch, @NotNull SeekableChannelContext channelContext) throws IOException { - final int uncompressedPageSize = pageHeader.getUncompressed_page_size(); - final int compressedPageSize = pageHeader.getCompressed_page_size(); switch (pageHeader.type) { case DATA_PAGE: - BytesInput decompressedInput = - compressorAdapter.decompress(Channels.newInputStream(file), compressedPageSize, - uncompressedPageSize); - - DataPageHeader dataHeaderV1 = pageHeader.getData_page_header(); - return readPageV1(new DataPageV1( - decompressedInput, - dataHeaderV1.getNum_values(), - uncompressedPageSize, - null, // TODO in the future might want to pull in statistics - getEncoding(dataHeaderV1.getRepetition_level_encoding()), - getEncoding(dataHeaderV1.getDefinition_level_encoding()), - getEncoding(dataHeaderV1.getEncoding())), nullValue, channelContext); + return readPageV1(readV1(ch), nullValue, channelContext); case DATA_PAGE_V2: - DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); - int dataSize = compressedPageSize - dataHeaderV2.getRepetition_levels_byte_length() - - dataHeaderV2.getDefinition_levels_byte_length(); - BytesInput repetitionLevels = Helpers.readBytes(file, dataHeaderV2.getRepetition_levels_byte_length()); - BytesInput definitionLevels = Helpers.readBytes(file, dataHeaderV2.getDefinition_levels_byte_length()); - BytesInput data = compressorAdapter.decompress(Channels.newInputStream(file), dataSize, - pageHeader.getUncompressed_page_size() - - dataHeaderV2.getRepetition_levels_byte_length() - - dataHeaderV2.getDefinition_levels_byte_length()); - - return readPageV2(new DataPageV2( - dataHeaderV2.getNum_rows(), - dataHeaderV2.getNum_nulls(), - dataHeaderV2.getNum_values(), - repetitionLevels, - definitionLevels, - getEncoding(dataHeaderV2.getEncoding()), - data, - uncompressedPageSize, - null, // TODO in the future might want to pull in statistics, - false), nullValue); + return readPageV2(readV2(ch), nullValue); default: throw new IOException(String.format("Unexpected page of type %s of size %d", pageHeader.getType(), - compressedPageSize)); + pageHeader.getCompressed_page_size())); } } @@ -321,10 +277,10 @@ private Encoding getEncoding(org.apache.parquet.format.Encoding encoding) { private int readRowCountFromPageV1(DataPageV1 page) { try { - ByteBuffer bytes = page.getBytes().toByteBuffer(); // TODO - move away from page and use - // ByteBuffers directly - bytes.order(ByteOrder.LITTLE_ENDIAN); if (path.getMaxRepetitionLevel() != 0) { + ByteBuffer bytes = page.getBytes().toByteBuffer(); // TODO - move away from page and use + // ByteBuffers directly + bytes.order(ByteOrder.LITTLE_ENDIAN); int length = bytes.getInt(); return readRepetitionLevels(bytes.slice().limit(length)); } else { @@ -629,8 +585,10 @@ public int numValues(@NotNull final SeekableChannelContext channelContext) throw if (numValues >= 0) { return numValues; } - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { - ensurePageHeader(readChannel); + try ( + final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), uri)) { + ensurePageHeader(channelsProvider, ch); // Above will block till it populates numValues Assert.geqZero(numValues, "numValues"); return numValues; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java index 5421d133a9b..a6f2e8fce69 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java @@ -11,8 +11,8 @@ import org.apache.parquet.format.Type; import org.apache.parquet.schema.*; -import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStream; import java.net.URI; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -57,41 +57,43 @@ public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvide // TODO(deephaven-core#5066): Add support for reading metadata files from non-file URIs rootURI = parquetFileURI; } - final byte[] footer; - try (final SeekableChannelContext channelContext = channelsProvider.makeContext(); - final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelContext, parquetFileURI)) { - final long fileLen = readChannel.size(); - if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + - // footerIndex + MAGIC - throw new InvalidParquetFileException( - parquetFileURI + " is not a Parquet file (too small length: " + fileLen + ")"); + try ( + final SeekableChannelContext context = channelsProvider.makeSingleUseContext(); + final SeekableByteChannel ch = channelsProvider.getReadChannel(context, parquetFileURI)) { + final int footerLength = prepareFooter(parquetFileURI, ch); + try (final InputStream in = channelsProvider.getInputStream(ch)) { + fileMetaData = Util.readFileMetaData(in); } - - final long footerLengthIndex = fileLen - FOOTER_LENGTH_SIZE - MAGIC.length; - readChannel.position(footerLengthIndex); - - final int footerLength = readIntLittleEndian(readChannel); - final byte[] magic = new byte[MAGIC.length]; - Helpers.readBytes(readChannel, magic); - if (!Arrays.equals(MAGIC, magic)) { - throw new InvalidParquetFileException( - parquetFileURI + " is not a Parquet file. expected magic number at tail " - + Arrays.toString(MAGIC) + " but found " + Arrays.toString(magic)); - } - final long footerIndex = footerLengthIndex - footerLength; - if (footerIndex < MAGIC.length || footerIndex >= footerLengthIndex) { - throw new InvalidParquetFileException( - "corrupted file: the footer index is not within the file: " + footerIndex); - } - readChannel.position(footerIndex); - footer = new byte[footerLength]; - Helpers.readBytes(readChannel, footer); } - fileMetaData = Util.readFileMetaData(new ByteArrayInputStream(footer)); type = fromParquetSchema(fileMetaData.schema, fileMetaData.column_orders); } + private static int prepareFooter(URI parquetFileURI, SeekableByteChannel readChannel) throws IOException { + final long fileLen = readChannel.size(); + if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + + // footerIndex + MAGIC + throw new InvalidParquetFileException( + parquetFileURI + " is not a Parquet file (too small length: " + fileLen + ")"); + } + final byte[] last8 = new byte[8]; + final long footerLengthIndex = fileLen - FOOTER_LENGTH_SIZE - MAGIC.length; + readChannel.position(footerLengthIndex); + Helpers.readBytes(readChannel, last8); + if (!Arrays.equals(MAGIC, 0, MAGIC.length, last8, 4, 8)) { + throw new InvalidParquetFileException( + parquetFileURI + " is not a Parquet file. expected magic number at tail " + Arrays.toString(MAGIC) + + " but found " + Arrays.toString(Arrays.copyOfRange(last8, 4, 8))); + } + final int footerLength = ByteBuffer.wrap(last8, 0, 4).order(ByteOrder.LITTLE_ENDIAN).getInt(); + final long footerIndex = footerLengthIndex - footerLength; + if (footerIndex < MAGIC.length || footerIndex >= footerLengthIndex) { + throw new InvalidParquetFileException( + "corrupted file: the footer index is not within the file: " + footerIndex); + } + readChannel.position(footerIndex); + return footerLength; + } + /** * @return The {@link SeekableChannelsProvider} used for this reader, appropriate to use for related file access */ @@ -180,7 +182,7 @@ private Set calculateColumnsWithDictionaryUsedOnEveryDataPage() { return result; } - private int readIntLittleEndian(SeekableByteChannel f) throws IOException { + private static int readIntLittleEndian(SeekableByteChannel f) throws IOException { ByteBuffer tempBuf = ByteBuffer.allocate(Integer.BYTES); tempBuf.order(ByteOrder.LITTLE_ENDIAN); Helpers.readExact(f, tempBuf); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java index 7ee0dbbdd3d..7d694a6371f 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java @@ -3,8 +3,9 @@ */ package io.deephaven.parquet.base; -import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelContext.Provider; +import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.format.ColumnChunk; import org.apache.parquet.format.RowGroup; import org.apache.parquet.format.Util; @@ -15,11 +16,10 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.io.BufferedInputStream; import java.io.IOException; +import java.io.InputStream; import java.io.UncheckedIOException; import java.net.URI; -import java.nio.channels.Channels; import java.nio.channels.SeekableByteChannel; import java.util.ArrayList; import java.util.HashMap; @@ -78,21 +78,31 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path, if (columnChunk == null) { return null; } - - OffsetIndex offsetIndex = null; - if (columnChunk.isSetOffset_index_offset()) { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, rootURI)) { - readChannel.position(columnChunk.getOffset_index_offset()); - offsetIndex = ParquetMetadataConverter.fromParquetOffsetIndex(Util.readOffsetIndex( - new BufferedInputStream(Channels.newInputStream(readChannel), BUFFER_SIZE))); - } catch (final IOException e) { - throw new UncheckedIOException(e); - } - } + final OffsetIndex offsetIndex = offsetIndex(columnChunk, channelContext); return new ColumnChunkReaderImpl(columnChunk, channelsProvider, rootURI, type, offsetIndex, fieldTypes, numRows(), version); } + private OffsetIndex offsetIndex(ColumnChunk chunk, @NotNull SeekableChannelContext context) { + if (!chunk.isSetOffset_index_offset()) { + return null; + } + return ParquetMetadataConverter.fromParquetOffsetIndex(readOffsetIndex(chunk, context)); + } + + private org.apache.parquet.format.OffsetIndex readOffsetIndex(ColumnChunk chunk, + @NotNull SeekableChannelContext channelContext) { + try ( + final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); + final SeekableByteChannel readChannel = channelsProvider.getReadChannel(upgrade.get(), rootURI); + final InputStream in = + channelsProvider.getInputStream(readChannel.position(chunk.getOffset_index_offset()))) { + return Util.readOffsetIndex(in); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + @Override public long numRows() { return rowGroup.num_rows; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java index 79caf1cda64..d5182938617 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java @@ -214,7 +214,7 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition final RowGroupReader rowGroupReader = parquetFileReader.getRowGroup(0, version); final ColumnChunkReader groupingKeyReader, beginPosReader, endPosReader; - try (final SeekableChannelContext channelContext = channelsProvider.makeContext()) { + try (final SeekableChannelContext channelContext = channelsProvider.makeSingleUseContext()) { groupingKeyReader = rowGroupReader.getColumnChunk(Collections.singletonList(GROUPING_KEY), channelContext); beginPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(BEGIN_POS), channelContext); @@ -223,11 +223,10 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition if (groupingKeyReader == null || beginPosReader == null || endPosReader == null) { log.warn().append("Index file ").append(indexFilePath) .append(" is missing one or more expected columns for table location ") - .append(tl()).append(", column ").append(getName()); + .append(tl()).append(", column ").append(getName()).endl(); return null; } - final PageCache localPageCache = ensurePageCache(); // noinspection unchecked diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java index 912cdffb002..d3652947bc3 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java @@ -155,7 +155,7 @@ protected ParquetColumnLocation makeColumnLocation(@NotNull final String final List nameList = columnPath == null ? Collections.singletonList(parquetColumnName) : Arrays.asList(columnPath); final ColumnChunkReader[] columnChunkReaders; - try (final SeekableChannelContext channelContext = getChannelProvider().makeContext()) { + try (final SeekableChannelContext channelContext = getChannelProvider().makeSingleUseContext()) { columnChunkReaders = Arrays.stream(getRowGroupReaders()) .map(rgr -> rgr.getColumnChunk(nameList, channelContext)).toArray(ColumnChunkReader[]::new); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java index 05caefa1e13..921a9c309dd 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java @@ -22,6 +22,7 @@ import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; import io.deephaven.util.SafeCloseable; +import io.deephaven.util.channel.SeekableChannelContext.Provider; import io.deephaven.vector.Vector; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -218,6 +219,10 @@ final SeekableChannelContext innerFillContext(@Nullable final FillContext contex return SeekableChannelContext.NULL; } + final Provider upgrade(@Nullable final FillContext context) { + return SeekableChannelContext.upgrade(columnChunkReader.getChannelsProvider(), innerFillContext(context)); + } + private T fillContextUpdater( int chunkCapacity, @Nullable final SharedContext sharedContext, diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java index 23ad57f5e63..dbe494bfdb3 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java @@ -7,10 +7,11 @@ import io.deephaven.base.verify.Require; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.ChunkPage; -import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.parquet.table.pagestore.PageCache.IntrusivePage; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; +import io.deephaven.util.channel.SeekableChannelContext.Provider; import org.apache.parquet.internal.column.columnindex.OffsetIndex; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -116,15 +117,7 @@ private ChunkPage getPage(@Nullable final FillContext fillContext, final i synchronized (pageState) { // Make sure no one materialized this page as we waited for the lock if ((localRef = pageState.pageRef) == null || (page = localRef.get()) == null) { - // Use the latest context while reading the page - final SeekableChannelContext channelContext = innerFillContext(fillContext); - final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(pageNum); - try { - page = new PageCache.IntrusivePage<>( - toPage(offsetIndex.getFirstRowIndex(pageNum), reader, channelContext)); - } catch (final IOException except) { - throw new UncheckedIOException(except); - } + page = new IntrusivePage<>(getPageImpl(fillContext, pageNum)); pageState.pageRef = new WeakReference<>(page); } } @@ -133,6 +126,16 @@ private ChunkPage getPage(@Nullable final FillContext fillContext, final i return page.getPage(); } + private ChunkPage getPageImpl(@Nullable FillContext fillContext, int pageNum) { + final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(pageNum); + // Use the latest context while reading the page, or create (and close) new one + try (final Provider upgrade = upgrade(fillContext)) { + return toPage(offsetIndex.getFirstRowIndex(pageNum), reader, upgrade.get()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + @Override @NotNull public ChunkPage getPageContaining(@Nullable final FillContext fillContext, long rowKey) { diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java index 7d24aba9ec5..1f6b102277a 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java @@ -11,6 +11,7 @@ import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; +import io.deephaven.util.channel.SeekableChannelContext.Provider; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -146,27 +147,22 @@ private ChunkPage getPage(@NotNull final SeekableChannelContext channelCon public ChunkPage getPageContaining(@Nullable final FillContext fillContext, long rowKey) { rowKey &= mask(); Require.inRange(rowKey - pageRowOffsets[0], "rowKey", numRows(), "numRows"); - int localNumPages = numPages; int pageNum = Arrays.binarySearch(pageRowOffsets, 1, localNumPages + 1, rowKey); - if (pageNum < 0) { pageNum = -2 - pageNum; } - - // Use the latest channel context while reading page headers - final SeekableChannelContext channelContext = innerFillContext(fillContext); - - if (pageNum >= localNumPages) { - final int minPageNum = fillToRow(channelContext, localNumPages, rowKey); - localNumPages = numPages; - pageNum = Arrays.binarySearch(pageRowOffsets, minPageNum + 1, localNumPages + 1, rowKey); - - if (pageNum < 0) { - pageNum = -2 - pageNum; + // Use the latest channel context while reading page headers, or create (and close) a new one + try (final Provider upgrade = upgrade(fillContext)) { + if (pageNum >= localNumPages) { + final int minPageNum = fillToRow(upgrade.get(), localNumPages, rowKey); + localNumPages = numPages; + pageNum = Arrays.binarySearch(pageRowOffsets, minPageNum + 1, localNumPages + 1, rowKey); + if (pageNum < 0) { + pageNum = -2 - pageNum; + } } + return getPage(upgrade.get(), pageNum); } - - return getPage(channelContext, pageNum); } } diff --git a/extensions/s3/build.gradle b/extensions/s3/build.gradle index a3d643c3a6c..22915b85379 100644 --- a/extensions/s3/build.gradle +++ b/extensions/s3/build.gradle @@ -12,9 +12,10 @@ dependencies { implementation project(':Util') implementation project(':Configuration') - implementation platform('software.amazon.awssdk:bom:2.21.43') + implementation platform('software.amazon.awssdk:bom:2.23.19') implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:aws-crt-client' + //implementation 'software.amazon.awssdk:netty-nio-client' compileOnly depAnnotations diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java index 8bcc56995ee..d6e97ec9700 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java @@ -1,28 +1,34 @@ package io.deephaven.extensions.s3; -import io.deephaven.util.SafeCloseable; -import org.jetbrains.annotations.Nullable; +import io.deephaven.util.datastructures.SegmentedSoftPool; import java.nio.ByteBuffer; -interface BufferPool { +final class BufferPool { - interface BufferHolder extends SafeCloseable { + private static final int POOL_SEGMENT_CAPACITY = 10; + private final SegmentedSoftPool pool; + private final int bufferSize; - /** - * @return The buffer if available, else {@code null} - */ - @Nullable - ByteBuffer get(); + /** + * @param bufferSize Upper limit on size of buffers to be pooled + */ + BufferPool(final int bufferSize) { + this.bufferSize = bufferSize; + this.pool = new SegmentedSoftPool<>( + POOL_SEGMENT_CAPACITY, + () -> ByteBuffer.allocate(bufferSize), + ByteBuffer::clear); + } - /** - * Return the held buffer to its pool, and cause subsequent calls to {@link #get()} to return {@code null} - */ - void close(); + public ByteBuffer take(final int size) { + if (size > bufferSize) { + throw new IllegalArgumentException("Buffer size " + size + " is larger than pool size " + bufferSize); + } + return pool.take(); } - /** - * Returns a {@link BufferHolder} that will hold a buffer of at least the requested size. - */ - BufferHolder take(int size); + public void give(ByteBuffer buffer) { + pool.give(buffer); + } } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java deleted file mode 100644 index 6aaff7f6ea2..00000000000 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java +++ /dev/null @@ -1,115 +0,0 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ -package io.deephaven.extensions.s3; - -import io.deephaven.base.verify.Assert; -import io.deephaven.util.SafeCloseable; -import org.jetbrains.annotations.NotNull; -import org.reactivestreams.Subscriber; -import org.reactivestreams.Subscription; -import software.amazon.awssdk.core.async.AsyncResponseTransformer; -import software.amazon.awssdk.core.async.SdkPublisher; - -import java.nio.ByteBuffer; -import java.util.concurrent.CompletableFuture; - -final class ByteBufferAsyncResponseTransformer - implements AsyncResponseTransformer, SafeCloseable { - - private final ByteBuffer byteBuffer; - - private volatile boolean released; - private volatile CompletableFuture currentFuture; - - /** - * @param byteBuffer A {@link ByteBuffer} to store the response bytes. - */ - ByteBufferAsyncResponseTransformer(@NotNull final ByteBuffer byteBuffer) { - this.byteBuffer = byteBuffer; - } - - @Override - public CompletableFuture prepare() { - return currentFuture = new CompletableFuture<>(); - } - - @Override - public void onResponse(final ResponseT response) { - // No need to store the response object as we are only interested in the byte buffer - } - - @Override - public void onStream(final SdkPublisher publisher) { - publisher.subscribe(new ByteBufferSubscriber(currentFuture)); - } - - @Override - public void exceptionOccurred(final Throwable throwable) { - currentFuture.completeExceptionally(throwable); - } - - /** - * Prevent further mutation of the underlying buffer by this ByteBufferAsyncResponseTransformer and any of its - * Subscribers. - */ - @Override - public synchronized void close() { - released = true; - } - - private final class ByteBufferSubscriber implements Subscriber { - - private final CompletableFuture resultFuture; - /** - * A duplicate of the underlying buffer used to store the response bytes without modifying the original reusable - * buffer's position, limit, or mark. - */ - private final ByteBuffer duplicate; - - private Subscription subscription; - - ByteBufferSubscriber(CompletableFuture resultFuture) { - this.resultFuture = resultFuture; - this.duplicate = byteBuffer.duplicate(); - } - - @Override - public void onSubscribe(final Subscription s) { - if (subscription != null) { - // Only maintain the first successful subscription - s.cancel(); - return; - } - subscription = s; - subscription.request(Long.MAX_VALUE); - } - - @Override - public void onNext(final ByteBuffer responseBytes) { - // Assuming responseBytes will fit in the buffer - Assert.assertion(responseBytes.remaining() <= duplicate.remaining(), - "responseBytes.remaining() <= duplicate.remaining()"); - if (released) { - return; - } - synchronized (ByteBufferAsyncResponseTransformer.this) { - if (released) { - return; - } - duplicate.put(responseBytes); - } - subscription.request(1); - } - - @Override - public void onError(final Throwable throwable) { - resultFuture.completeExceptionally(throwable); - } - - @Override - public void onComplete() { - resultFuture.complete(byteBuffer.asReadOnlyBuffer().limit(duplicate.position())); - } - } -} diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java new file mode 100644 index 00000000000..eecdab3474f --- /dev/null +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -0,0 +1,430 @@ +package io.deephaven.extensions.s3; + +import io.deephaven.util.channel.SeekableChannelContext; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.async.AsyncResponseTransformer; +import software.amazon.awssdk.core.async.SdkPublisher; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Uri; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.time.Instant; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.BiConsumer; + +/** + * Context object used to store read-ahead buffers for efficiently reading from S3. + */ +final class S3ChannelContext implements SeekableChannelContext { + private static final Logger log = LoggerFactory.getLogger(S3ChannelContext.class); + private static final long UNINITIALIZED_SIZE = -1; + + private final S3AsyncClient client; + final S3Instructions instructions; + private final BufferPool bufferPool; + + private S3Uri uri; + + /** + * Used to cache recently fetched fragments for faster lookup + */ + private final Request[] requests; + + /** + * The size of the object in bytes, stored in context to avoid fetching multiple times + */ + private long size; + + private long numFragmentsInObject; + + S3ChannelContext(S3AsyncClient client, S3Instructions instructions, BufferPool bufferPool) { + this.client = Objects.requireNonNull(client); + this.instructions = Objects.requireNonNull(instructions); + this.bufferPool = Objects.requireNonNull(bufferPool); + requests = new Request[instructions.maxCacheSize()]; + size = UNINITIALIZED_SIZE; + if (log.isDebugEnabled()) { + log.debug("creating context: {}", ctxStr()); + } + } + + public long size(S3Uri uri) throws IOException { + assume(uri); + populateSize(); + return size; + } + + public int fill(S3Uri uri, final long position, ByteBuffer dest) throws IOException { + assume(uri); + populateSize(); + final int destRemaining = dest.remaining(); + if (destRemaining == 0) { + return 0; + } + // Send async read requests for current fragment as well as read ahead fragments + final long firstFragmentIx = fragmentIndex(position); + final long readAhead; + { + final long lastFragmentIx = fragmentIndex(position + destRemaining - 1); + final int impliedReadAhead = (int) (lastFragmentIx - firstFragmentIx); + final int desiredReadAhead = instructions.readAheadCount(); + final long totalRemainingFragments = numFragmentsInObject - firstFragmentIx - 1; + final int maxReadAhead = requests.length - 1; + readAhead = Math.min( + Math.max(impliedReadAhead, desiredReadAhead), + (int) Math.min(maxReadAhead, totalRemainingFragments)); + } + final Request firstRequest = getOrCreateRequest(firstFragmentIx); + for (int i = 0; i < readAhead; ++i) { + getOrCreateRequest(firstFragmentIx + i + 1); + } + // blocking + int filled = firstRequest.fill(position, dest); + for (int i = 0; dest.hasRemaining(); ++i) { + final Request request = getRequest(firstFragmentIx + i + 1).orElse(null); + if (request == null || !request.isDone()) { + break; + } + // non-blocking since we know isDone + filled += request.fill(position + filled, dest); + } + return filled; + } + + @Override + public void close() { + if (log.isDebugEnabled()) { + log.debug("closing context: {}", ctxStr()); + } + // Cancel all outstanding requests + for (int i = 0; i < requests.length; i++) { + if (requests[i] != null) { + requests[i].release(); + requests[i] = null; + } + } + } + + // -------------------------------------------------------------------------------------------------- + + void assume(S3Uri uri) { + if (this.uri == null) { + this.uri = Objects.requireNonNull(uri); + } else { + if (!this.uri.equals(uri)) { + throw new IllegalStateException( + String.format("Inconsistent URIs. expected=%s, actual=%s", this.uri, uri)); + } + } + } + + private Optional getRequest(final long fragmentIndex) { + final int cacheIdx = cacheIndex(fragmentIndex); + final Request request = requests[cacheIdx]; + return request == null || !request.isFragment(fragmentIndex) + ? Optional.empty() + : Optional.of(request); + } + + private Request getOrCreateRequest(final long fragmentIndex) { + final int cacheIdx = cacheIndex(fragmentIndex); + Request request = requests[cacheIdx]; + if (request != null) { + if (!request.isFragment(fragmentIndex)) { + request.release(); + requests[cacheIdx] = (request = new Request(fragmentIndex)); + request.init(); + } + } else { + requests[cacheIdx] = (request = new Request(fragmentIndex)); + request.init(); + } + return request; + } + + private int cacheIndex(final long fragmentIndex) { + // TODO(deephaven-core#5061): Experiment with LRU caching + return (int) (fragmentIndex % requests.length); + } + + private long fragmentIndex(final long pos) { + return pos / instructions.fragmentSize(); + } + + private String ctxStr() { + if (uri != null) { + return String.format("ctx=%d %s/%s", System.identityHashCode(S3ChannelContext.this), + uri.bucket().orElseThrow(), uri.key().orElseThrow()); + } else { + return String.format("ctx=%d", System.identityHashCode(S3ChannelContext.this)); + } + } + + final class Request + implements AsyncResponseTransformer, BiConsumer { + + // implicitly + URI + private final long fragmentIndex; + private final long from; + private final long to; + private final Instant createdAt; + private Instant completedAt; + private final CompletableFuture released; + private CompletableFuture consumerFuture; + private volatile CompletableFuture producerFuture; + private GetObjectResponse response; + private int fillCount; + private long fillBytes; + + private Request(long fragmentIndex) { + createdAt = Instant.now(); + this.fragmentIndex = fragmentIndex; + from = fragmentIndex * instructions.fragmentSize(); + to = Math.min(from + instructions.fragmentSize(), size) - 1; + released = new CompletableFuture<>(); + } + + void init() { + if (log.isDebugEnabled()) { + log.debug("send: {}", requestStr()); + } + consumerFuture = client.getObject(getObjectRequest(), this); + consumerFuture.whenComplete(this); + } + + public boolean isDone() { + return consumerFuture.isDone(); + } + + public int fill(long localPosition, ByteBuffer dest) throws IOException { + final int outOffset = (int) (localPosition - from); + final int outLength = Math.min((int) (to - localPosition + 1), dest.remaining()); + final ByteBuffer fullFragment; + try { + fullFragment = get(); + } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { + throw handleS3Exception(e, String.format("fetching fragment %s", requestStr())); + } + dest.put(fullFragment.duplicate().position(outOffset).limit(outOffset + outLength)); + ++fillCount; + fillBytes += outLength; + return outLength; + } + + public void release() { + final boolean didCancel = consumerFuture.cancel(true); + if (log.isDebugEnabled()) { + final String cancelType = didCancel ? "fast" : (fillCount == 0 ? "unused" : "normal"); + log.debug("cancel {}: {} fillCount={}, fillBytes={}", cancelType, requestStr(), fillCount, fillBytes); + } + // Finishing via exception to ensure downstream subscribers can cleanup if SDK didn't complete them + released.cancel(true); + } + + // -------------------------------------------------------------------------------------------------- + + @Override + public void accept(ByteBuffer byteBuffer, Throwable throwable) { + completedAt = Instant.now(); + if (log.isDebugEnabled()) { + if (byteBuffer != null) { + log.debug("send complete: {} {}", requestStr(), Duration.between(createdAt, completedAt)); + } else { + log.debug("send error: {} {}", requestStr(), Duration.between(createdAt, completedAt)); + } + } + } + + // -------------------------------------------------------------------------------------------------- + + @Override + public CompletableFuture prepare() { + final CompletableFuture future = new CompletableFuture<>(); + producerFuture = future; + return future; + } + + @Override + public void onResponse(GetObjectResponse response) { + this.response = response; + } + + @Override + public void onStream(SdkPublisher publisher) { + publisher.subscribe(new Request.Sub()); + } + + @Override + public void exceptionOccurred(Throwable error) { + producerFuture.completeExceptionally(error); + } + + // -------------------------------------------------------------------------------------------------- + + private ByteBuffer get() throws ExecutionException, InterruptedException, TimeoutException { + if (released.isDone()) { + throw new IllegalStateException(String.format("Trying to get data after release, %s", requestStr())); + } + // Giving our own get() a bit of overhead - the clients should already be constructed with appropriate + // apiCallTimeout. + final long readNanos = instructions.readTimeout().plusMillis(100).toNanos(); + return consumerFuture.get(readNanos, TimeUnit.NANOSECONDS); + } + + private boolean isFragment(long fragmentIndex) { + return this.fragmentIndex == fragmentIndex; + } + + private int requestLength() { + return (int) (to - from + 1); + } + + private GetObjectRequest getObjectRequest() { + return GetObjectRequest.builder() + .bucket(uri.bucket().orElseThrow()) + .key(uri.key().orElseThrow()) + .range("bytes=" + from + "-" + to) + .build(); + } + + private String requestStr() { + if (uri != null) { + return String.format("ctx=%d ix=%d [%d, %d]/%d %s/%s", System.identityHashCode(S3ChannelContext.this), + fragmentIndex, from, to, requestLength(), uri.bucket().orElseThrow(), uri.key().orElseThrow()); + } else { + return String.format("ctx=%d ix=%d [%d, %d]/%d", System.identityHashCode(S3ChannelContext.this), + fragmentIndex, from, to, requestLength()); + } + } + + // -------------------------------------------------------------------------------------------------- + + final class Sub implements Subscriber, BiConsumer { + private final CompletableFuture localProducer; + private ByteBuffer buffer; + private Subscription subscription; + + Sub() { + localProducer = producerFuture; + buffer = bufferPool.take(requestLength()); + // 1. localProducer succeeds: whenComplete will executed when released + // 2. localProducer fails: whenComplete will executed asap + // 3. localProducer limbo (not expected): whenComplete will executed when released + CompletableFuture.allOf(localProducer, released).whenComplete(this); + } + + // ----------------------------------------------------------------------------- + + @Override + public synchronized void accept(Void unused, Throwable throwable) { + bufferPool.give(buffer); + buffer = null; + } + + // ----------------------------------------------------------------------------- + + @Override + public synchronized void onSubscribe(Subscription s) { + if (subscription != null) { + s.cancel(); + return; + } + subscription = s; + subscription.request(Long.MAX_VALUE); + } + + @Override + public synchronized void onNext(ByteBuffer byteBuffer) { + buffer.put(byteBuffer); + subscription.request(1); + } + + @Override + public synchronized void onError(Throwable t) { + localProducer.completeExceptionally(t); + } + + @Override + public synchronized void onComplete() { + buffer.flip(); + if (buffer.remaining() != requestLength()) { + localProducer.completeExceptionally(new IllegalStateException(String.format( + "Expected %d bytes, received %d, %s", requestLength(), buffer.remaining(), requestStr()))); + } else { + localProducer.complete(buffer.asReadOnlyBuffer()); + } + } + } + } + + private IOException handleS3Exception(final Exception e, final String operationDescription) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + return new IOException(String.format("Thread interrupted while %s", operationDescription), e); + } + if (e instanceof ExecutionException) { + return new IOException(String.format("Execution exception occurred while %s", operationDescription), e); + } + if (e instanceof TimeoutException) { + return new IOException(String.format( + "Operation timeout while %s after waiting for duration %s", operationDescription, + instructions.readTimeout()), e); + } + if (e instanceof CancellationException) { + return new IOException(String.format("Cancelled an operation while %s", operationDescription), e); + } + return new IOException(String.format("Exception caught while %s", operationDescription), e); + } + + private void populateSize() throws IOException { + if (size != UNINITIALIZED_SIZE) { + return; + } + if (log.isDebugEnabled()) { + log.debug("head: {}", ctxStr()); + } + // Fetch the size of the file on the first read using a blocking HEAD request, and store it in the context + // for future use + final HeadObjectResponse headObjectResponse; + try { + headObjectResponse = client + .headObject(HeadObjectRequest.builder() + .bucket(uri.bucket().orElseThrow()) + .key(uri.key().orElseThrow()) + .build()) + .get(instructions.readTimeout().toNanos(), TimeUnit.NANOSECONDS); + } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { + throw handleS3Exception(e, String.format("fetching HEAD for file %s, %s", uri, ctxStr())); + } + setSize(headObjectResponse.contentLength()); + } + + void hackSize(long size) { + if (this.size == UNINITIALIZED_SIZE) { + setSize(size); + } else if (this.size != size) { + throw new IllegalStateException(); + } + } + + private void setSize(long size) { + this.size = size; + // ceil(size / fragmentSize) + this.numFragmentsInObject = (size + instructions.fragmentSize() - 1) / instructions.fragmentSize(); + } +} diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index a36263ba80d..308f147f480 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -3,8 +3,8 @@ */ package io.deephaven.extensions.s3; -import io.deephaven.annotations.BuildableStyle; import io.deephaven.configuration.Configuration; +import org.immutables.value.Value; import org.immutables.value.Value.Check; import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; @@ -19,7 +19,13 @@ * documented in this class may change in the future. As such, callers may wish to explicitly set the values. */ @Immutable -@BuildableStyle +// Almost the same as BuildableStyle, but has copy-ability to support withReadAheadCount +@Value.Style(visibility = Value.Style.ImplementationVisibility.PACKAGE, + defaults = @Value.Immutable(copy = true), + strictBuilder = true, + weakInterning = true, + jdkOnly = true, + includeHashCode = "getClass().hashCode()") public abstract class S3Instructions { private final static int DEFAULT_MAX_CONCURRENT_REQUESTS = 50; @@ -117,6 +123,8 @@ public Credentials credentials() { */ public abstract Optional endpointOverride(); + abstract S3Instructions withReadAheadCount(int readAheadCount); + public interface Builder { Builder regionName(String regionName); diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index 308f44f1672..7cdf79eb708 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -4,33 +4,18 @@ package io.deephaven.extensions.s3; import io.deephaven.base.verify.Assert; - -import java.util.Objects; - import io.deephaven.util.channel.CachedChannelProvider; import io.deephaven.util.channel.SeekableChannelContext; -import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3Uri; -import software.amazon.awssdk.services.s3.model.GetObjectRequest; -import software.amazon.awssdk.services.s3.model.GetObjectResponse; -import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import java.io.IOException; -import java.net.URI; import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; import java.nio.channels.NonWritableChannelException; import java.nio.channels.SeekableByteChannel; -import java.util.concurrent.CancellationException; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; +import java.util.Objects; /** @@ -39,295 +24,63 @@ */ final class S3SeekableByteChannel implements SeekableByteChannel, CachedChannelProvider.ContextHolder { - private static final long CLOSED_SENTINEL = -1; - private static final long UNINITIALIZED_SIZE = -1; + private static final long CLOSED_SENTINEL = -1; - private static final long UNINITIALIZED_FRAGMENT_INDEX = -1; - - /** - * Context object used to store read-ahead buffers for efficiently reading from S3. - */ - static final class S3ChannelContext implements SeekableChannelContext { - - /** - * Used to store information related to a single fragment - */ - private static final class FragmentState { - - /** - * The index of the fragment in the object. - */ - private long fragmentIndex = UNINITIALIZED_FRAGMENT_INDEX; - - /** - * The future that will be completed with the fragment's bytes. - */ - private Future future; - - /** - * The {@link SafeCloseable} that will be used to release outstanding resources post-cancellation. - */ - private SafeCloseable bufferRelease; - - private boolean matches(final long fragmentIndex) { - return this.fragmentIndex == fragmentIndex; - } - - private void cancelAndRelease() { - try ( - final SafeCloseable ignored1 = cancelOnClose(future, true); - final SafeCloseable ignored2 = bufferRelease) { - fragmentIndex = UNINITIALIZED_FRAGMENT_INDEX; - future = null; - bufferRelease = null; - } - } - - // do not inline, needs to capture future at time of method call - private static SafeCloseable cancelOnClose(Future future, boolean mayInterruptIfRunning) { - return future == null ? null : () -> future.cancel(mayInterruptIfRunning); - } - - private void set( - final long fragmentIndex, - @NotNull final Future future, - @NotNull final SafeCloseable bufferRelease) { - this.fragmentIndex = fragmentIndex; - this.future = future; - this.bufferRelease = bufferRelease; - } - } - - /** - * Used to cache recently fetched fragments for faster lookup - */ - private final FragmentState[] bufferCache; - - /** - * The size of the object in bytes, stored in context to avoid fetching multiple times - */ - private long size; - - S3ChannelContext(final int maxCacheSize) { - bufferCache = new FragmentState[maxCacheSize]; - size = UNINITIALIZED_SIZE; - } - - private int getIndex(final long fragmentIndex) { - // TODO(deephaven-core#5061): Experiment with LRU caching - return (int) (fragmentIndex % bufferCache.length); - } - - private FragmentState getFragmentState(final long fragmentIndex) { - final int cacheIdx = getIndex(fragmentIndex); - FragmentState cachedEntry = bufferCache[cacheIdx]; - if (cachedEntry == null) { - bufferCache[cacheIdx] = cachedEntry = new FragmentState(); - } - return cachedEntry; - } - - /** - * Will return the {@link CompletableFuture} corresponding to provided fragment index if present in the cache, - * else will return {@code null} - */ - @Nullable - private Future getCachedFuture(final long fragmentIndex) { - final FragmentState cachedFragment = bufferCache[getIndex(fragmentIndex)]; - if (cachedFragment != null && cachedFragment.matches(fragmentIndex)) { - return cachedFragment.future; - } - return null; - } - - private long getSize() { - return size; - } - - private void setSize(final long size) { - this.size = size; - } - - @Override - public void close() { - // Cancel all outstanding requests - for (final FragmentState fragmentState : bufferCache) { - if (fragmentState != null) { - fragmentState.cancelAndRelease(); - } - } - } - } - - private final S3AsyncClient s3AsyncClient; - private final String bucket; - private final String key; - private final S3Instructions s3Instructions; - private final BufferPool bufferPool; - - /** - * The size of the object in bytes, fetched at the time of first read - */ - private long size; - private long numFragmentsInObject; + private final S3Uri uri; /** * The {@link SeekableChannelContext} object used to cache read-ahead buffers for efficiently reading from S3. This * is set before the read and cleared when closing the channel. */ - private SeekableChannelContext channelContext; + private S3ChannelContext context; private long position; + private long size; - S3SeekableByteChannel(@NotNull final URI uri, @NotNull final S3AsyncClient s3AsyncClient, - @NotNull final S3Instructions s3Instructions, @NotNull final BufferPool bufferPool) { - final S3Uri s3Uri = s3AsyncClient.utilities().parseUri(uri); - this.bucket = s3Uri.bucket().orElse(null); - this.key = s3Uri.key().orElse(null); - this.s3AsyncClient = s3AsyncClient; - this.s3Instructions = s3Instructions; - this.bufferPool = bufferPool; + S3SeekableByteChannel(S3Uri uri) { + this.uri = Objects.requireNonNull(uri); this.size = UNINITIALIZED_SIZE; - this.position = 0; } /** * @param channelContext The {@link SeekableChannelContext} object used to cache read-ahead buffers for efficiently * reading from S3. An appropriate channel context should be set before the read and should be cleared after - * the read is complete. A {@code null} parameter value is equivalent to clearing the context. This parameter - * will be {@link SeekableChannelContext#NULL} if no caching and read ahead is desired. + * the read is complete via {@link io.deephaven.util.channel.SeekableChannelsProvider#makeContext()}. A + * {@code null} parameter value is equivalent to clearing the context. */ @Override public void setContext(@Nullable final SeekableChannelContext channelContext) { - this.channelContext = channelContext; - } - - @Override - public int read(@NotNull final ByteBuffer destination) throws IOException { - Assert.neqNull(channelContext, "channelContext"); - if (!destination.hasRemaining()) { - return 0; - } - final long localPosition = position; - checkClosed(localPosition); - - final int numBytesCopied; - final S3ChannelContext s3ChannelContext = getS3ChannelContextFrom(channelContext); - try (final SafeCloseable ignored = s3ChannelContext == channelContext ? null : s3ChannelContext) { - // Fetch the file size if this is the first read - populateSize(s3ChannelContext); - if (localPosition >= size) { - // We are finished reading - return -1; - } - - // Send async read requests for current fragment as well as read ahead fragments - final long currFragmentIndex = fragmentIndexForByteNumber(localPosition); - final int numReadAheadFragments = channelContext != s3ChannelContext - ? 0 // We have a local S3ChannelContext, we don't want to do any read-ahead caching - : (int) Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); - for (long idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { - sendAsyncRequest(idx, s3ChannelContext); + if (channelContext != null && !(channelContext instanceof S3ChannelContext)) { + throw new IllegalArgumentException("Unsupported channel context " + channelContext); + } + this.context = (S3ChannelContext) channelContext; + if (this.context != null) { + this.context.assume(uri); + if (size != UNINITIALIZED_SIZE) { + this.context.hackSize(size); } - - // Wait till the current fragment is fetched - final Future currFragmentFuture = s3ChannelContext.getCachedFuture(currFragmentIndex); - final ByteBuffer currentFragment; - try { - currentFragment = currFragmentFuture.get(s3Instructions.readTimeout().toNanos(), TimeUnit.NANOSECONDS); - } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { - throw handleS3Exception(e, - String.format("fetching fragment %d for file %s in S3 bucket %s", currFragmentIndex, key, - bucket)); - } - - // Copy the bytes from fragment from the offset up to the min of remaining fragment and destination bytes. - // Therefore, the number of bytes read by this method can be less than the number of bytes remaining in the - // destination buffer. - final int fragmentOffset = (int) (localPosition - (currFragmentIndex * s3Instructions.fragmentSize())); - currentFragment.position(fragmentOffset); - numBytesCopied = Math.min(currentFragment.remaining(), destination.remaining()); - final int originalBufferLimit = currentFragment.limit(); - currentFragment.limit(currentFragment.position() + numBytesCopied); - destination.put(currentFragment); - // Need to reset buffer limit, so we can read from the same buffer again in future - currentFragment.limit(originalBufferLimit); - } - position = localPosition + numBytesCopied; - return numBytesCopied; - } - - /** - * If the provided {@link SeekableChannelContext} is {@link SeekableChannelContext#NULL}, this method creates and - * returns a new {@link S3ChannelContext} with a cache size of 1 to support a single read with no read ahead, and - * the caller is responsible to close the context after the read is complete. Else returns the provided - * {@link SeekableChannelContext} cast to {@link S3ChannelContext}. - */ - private static S3ChannelContext getS3ChannelContextFrom(@NotNull final SeekableChannelContext channelContext) { - final S3ChannelContext s3ChannelContext; - if (channelContext == SeekableChannelContext.NULL) { - s3ChannelContext = new S3ChannelContext(1); - } else { - Assert.instanceOf(channelContext, "channelContext", S3ChannelContext.class); - s3ChannelContext = (S3ChannelContext) channelContext; } - return s3ChannelContext; } - private long fragmentIndexForByteNumber(final long byteNumber) { - return byteNumber / s3Instructions.fragmentSize(); - } - - /** - * If not already cached in the context, sends an async request to fetch the fragment at the provided index and - * caches it in the context. - */ - private void sendAsyncRequest(final long fragmentIndex, @NotNull final S3ChannelContext s3ChannelContext) { - final S3ChannelContext.FragmentState fragmentState = s3ChannelContext.getFragmentState(fragmentIndex); - if (fragmentState.matches(fragmentIndex)) { - // We already have the fragment cached - return; + private long hackSize() throws IOException { + if (size != UNINITIALIZED_SIZE) { + return size; } - // Cancel any outstanding requests for the fragment in cached slot - fragmentState.cancelAndRelease(); - - final int fragmentSize = s3Instructions.fragmentSize(); - final long readFrom = fragmentIndex * fragmentSize; - final long readTo = Math.min(readFrom + fragmentSize, size) - 1; - final String range = "bytes=" + readFrom + "-" + readTo; - - final int numBytes = (int) (readTo - readFrom + 1); - final BufferPool.BufferHolder bufferHolder = bufferPool.take(numBytes); - final ByteBufferAsyncResponseTransformer asyncResponseTransformer = - new ByteBufferAsyncResponseTransformer<>(Objects.requireNonNull(bufferHolder.get())); - final CompletableFuture future = s3AsyncClient - .getObject(GetObjectRequest.builder() - .bucket(bucket) - .key(key) - .range(range) - .build(), asyncResponseTransformer) - .whenComplete((response, throwable) -> asyncResponseTransformer.close());; - fragmentState.set(fragmentIndex, future, bufferHolder); + return (size = context.size(uri)); } - private IOException handleS3Exception(final Exception e, final String operationDescription) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - return new IOException(String.format("Thread interrupted while %s", operationDescription), e); - } - if (e instanceof ExecutionException) { - return new IOException(String.format("Execution exception occurred while %s", operationDescription), e); - } - if (e instanceof TimeoutException) { - return new IOException(String.format( - "Operation timeout while %s after waiting for duration %s", operationDescription, - s3Instructions.readTimeout()), e); - } - if (e instanceof CancellationException) { - return new IOException(String.format("Cancelled an operation while %s", operationDescription), e); + @Override + public int read(@NotNull final ByteBuffer destination) throws IOException { + Assert.neqNull(context, "channelContext"); + checkClosed(position); + if (position >= hackSize()) { + // We are finished reading + return -1; } - return new IOException(String.format("Exception caught while %s", operationDescription), e); + final int filled = context.fill(uri, position, destination); + position += filled; + return filled; } @Override @@ -355,40 +108,8 @@ public SeekableByteChannel position(final long newPosition) throws ClosedChannel @Override public long size() throws IOException { checkClosed(position); - final S3ChannelContext s3ChannelContext = getS3ChannelContextFrom(channelContext); - try (final SafeCloseable ignored = s3ChannelContext == channelContext ? null : s3ChannelContext) { - populateSize(s3ChannelContext); - } - return size; - } - - private void populateSize(final S3ChannelContext s3ChannelContext) throws IOException { - if (size != UNINITIALIZED_SIZE) { - // Store the size in the context if it is uninitialized - if (s3ChannelContext.getSize() == UNINITIALIZED_SIZE) { - s3ChannelContext.setSize(size); - } - return; - } - if (s3ChannelContext.getSize() == UNINITIALIZED_SIZE) { - // Fetch the size of the file on the first read using a blocking HEAD request, and store it in the context - // for future use - final HeadObjectResponse headObjectResponse; - try { - headObjectResponse = s3AsyncClient - .headObject(HeadObjectRequest.builder() - .bucket(bucket) - .key(key) - .build()) - .get(s3Instructions.readTimeout().toNanos(), TimeUnit.NANOSECONDS); - } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { - throw handleS3Exception(e, String.format("fetching HEAD for file %s in S3 bucket %s", key, bucket)); - } - s3ChannelContext.setSize(headObjectResponse.contentLength()); - } - this.size = s3ChannelContext.getSize(); - final int fragmentSize = s3Instructions.fragmentSize(); - this.numFragmentsInObject = (size + fragmentSize - 1) / fragmentSize; // = ceil(size / fragmentSize) + Assert.neqNull(context, "channelContext"); + return context.size(uri); } @Override diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index ed23cebbc93..8e6227c5b8f 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -5,67 +5,81 @@ import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProviderBase; import org.jetbrains.annotations.NotNull; -import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.core.retry.RetryMode; import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3AsyncClientBuilder; +import software.amazon.awssdk.services.s3.S3Uri; import java.net.URI; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; -import static io.deephaven.extensions.s3.S3Instructions.MAX_FRAGMENT_SIZE; - /** - * {@link SeekableChannelsProvider} implementation that is used to fetch objects from S3 instances. + * {@link SeekableChannelsProvider} implementation that is used to fetch objects from AWS S3 instances. */ -final class S3SeekableChannelProvider implements SeekableChannelsProvider { +final class S3SeekableChannelProvider extends SeekableChannelsProviderBase { /** * We always allocate buffers of maximum allowed size for re-usability across reads with different fragment sizes. * There can be a performance penalty though if the fragment size is much smaller than the maximum size. */ - private static final int POOLED_BUFFER_SIZE = MAX_FRAGMENT_SIZE; - private static final BufferPool bufferPool = new SegmentedBufferPool(POOLED_BUFFER_SIZE); + private static final BufferPool BUFFER_POOL = new BufferPool(S3Instructions.MAX_FRAGMENT_SIZE); private final S3AsyncClient s3AsyncClient; private final S3Instructions s3Instructions; S3SeekableChannelProvider(@NotNull final S3Instructions s3Instructions) { - final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() - .maxConcurrency(s3Instructions.maxConcurrentRequests()) - .connectionTimeout(s3Instructions.connectionTimeout()) - .build(); // TODO(deephaven-core#5062): Add support for async client recovery and auto-close // TODO(deephaven-core#5063): Add support for caching clients for re-use final S3AsyncClientBuilder builder = S3AsyncClient.builder() + .httpClient(AwsCrtAsyncHttpClient.builder() + .maxConcurrency(s3Instructions.maxConcurrentRequests()) + .connectionTimeout(s3Instructions.connectionTimeout()) + .build()) + .overrideConfiguration(b -> b + // .addMetricPublisher(LoggingMetricPublisher.create(Level.INFO, Format.PRETTY)) + // .retryPolicy(RetryPolicy.builder(RetryMode.ADAPTIVE).fastFailRateLimiting(true).build()) + .retryPolicy(RetryMode.STANDARD) + .apiCallAttemptTimeout(s3Instructions.readTimeout().dividedBy(3)) + .apiCallTimeout(s3Instructions.readTimeout())) .region(Region.of(s3Instructions.regionName())) - .httpClient(asyncHttpClient) .credentialsProvider(s3Instructions.awsV2CredentialsProvider()); s3Instructions.endpointOverride().ifPresent(builder::endpointOverride); this.s3AsyncClient = builder.build(); this.s3Instructions = s3Instructions; } + @Override + protected boolean readChannelIsBuffered() { + // io.deephaven.extensions.s3.S3SeekableByteChannel is buffered based on context / options + return true; + } + @Override public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext, @NotNull final URI uri) { + final S3Uri s3Uri = s3AsyncClient.utilities().parseUri(uri); // context is unused here, will be set before reading from the channel - return new S3SeekableByteChannel(uri, s3AsyncClient, s3Instructions, bufferPool); + return new S3SeekableByteChannel(s3Uri); } @Override public SeekableChannelContext makeContext() { - return new S3SeekableByteChannel.S3ChannelContext(s3Instructions.maxCacheSize()); + return new S3ChannelContext(s3AsyncClient, s3Instructions, BUFFER_POOL); + } + + @Override + public SeekableChannelContext makeSingleUseContext() { + return new S3ChannelContext(s3AsyncClient, s3Instructions.withReadAheadCount(0), BUFFER_POOL); } @Override public boolean isCompatibleWith(@NotNull final SeekableChannelContext channelContext) { - // A null context implies no caching or read ahead - return channelContext == SeekableChannelContext.NULL - || channelContext instanceof S3SeekableByteChannel.S3ChannelContext; + return channelContext instanceof S3ChannelContext; } @Override @@ -73,6 +87,7 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boole throw new UnsupportedOperationException("Writing to S3 is currently unsupported"); } + @Override public void close() { s3AsyncClient.close(); } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/SegmentedBufferPool.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/SegmentedBufferPool.java deleted file mode 100644 index 8b30820810c..00000000000 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/SegmentedBufferPool.java +++ /dev/null @@ -1,61 +0,0 @@ -package io.deephaven.extensions.s3; - -import io.deephaven.util.datastructures.SegmentedSoftPool; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; - -final class SegmentedBufferPool implements BufferPool { - - private static final int POOL_SEGMENT_CAPACITY = 10; - private final SegmentedSoftPool pool; - private final int bufferSize; - - /** - * @param bufferSize Upper limit on size of buffers to be pooled - */ - SegmentedBufferPool(final int bufferSize) { - this.bufferSize = bufferSize; - this.pool = new SegmentedSoftPool<>( - POOL_SEGMENT_CAPACITY, - () -> ByteBuffer.allocate(bufferSize), - ByteBuffer::clear); - } - - @Override - public BufferHolder take(final int size) { - if (size > bufferSize) { - throw new IllegalArgumentException("Buffer size " + size + " is larger than pool size " + bufferSize); - } - return new SegmentedBufferHolder(pool); - } - - private static final class SegmentedBufferHolder implements BufferHolder { - - private static final AtomicReferenceFieldUpdater BUFFER_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(SegmentedBufferHolder.class, ByteBuffer.class, "buffer"); - - private final SegmentedSoftPool pool; - private volatile ByteBuffer buffer; - - private SegmentedBufferHolder(@NotNull final SegmentedSoftPool pool) { - this.pool = pool; - this.buffer = pool.take(); - } - - @Override - public @Nullable ByteBuffer get() { - return buffer; - } - - @Override - public void close() { - final ByteBuffer localBuffer = buffer; - if (localBuffer != null && BUFFER_UPDATER.compareAndSet(this, localBuffer, null)) { - pool.give(localBuffer); - } - } - } -} diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index 7b6f0c55fbe..215dcc3817d 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -10,6 +10,7 @@ import io.deephaven.engine.util.file.TrackedSeekableByteChannel; import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProviderBase; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,7 +26,7 @@ /** * {@link SeekableChannelsProvider} implementation that is constrained by a Deephaven {@link TrackedFileHandleFactory}. */ -class TrackedSeekableChannelsProvider implements SeekableChannelsProvider { +final class TrackedSeekableChannelsProvider extends SeekableChannelsProviderBase { private final TrackedFileHandleFactory fileHandleFactory; @@ -33,6 +34,13 @@ class TrackedSeekableChannelsProvider implements SeekableChannelsProvider { this.fileHandleFactory = fileHandleFactory; } + @Override + protected boolean readChannelIsBuffered() { + // io.deephaven.engine.util.file.TrackedSeekableByteChannel / io.deephaven.engine.util.file.FileHandle is not + // buffered + return false; + } + @Override public SeekableChannelContext makeContext() { // No additional context required for local FS diff --git a/server/jetty-app/src/main/resources/logback.xml b/server/jetty-app/src/main/resources/logback.xml index 14391026e53..870d62c0867 100644 --- a/server/jetty-app/src/main/resources/logback.xml +++ b/server/jetty-app/src/main/resources/logback.xml @@ -25,6 +25,10 @@ + + + + From 11123a7e95f07b0e6a90f923f4d24b08eddc92f1 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Mon, 12 Feb 2024 08:13:23 -0800 Subject: [PATCH 02/13] Anonymous access --- .../extensions/s3/AnonymousCredentials.java | 16 ++++++++++ .../deephaven/extensions/s3/Credentials.java | 9 +++++- .../extensions/s3/CredentialsTest.java | 31 +++++++++++++++++++ .../extensions/s3/S3InstructionsTest.java | 4 +-- .../extensions/s3/SingletonContainers.java | 6 ++-- py/server/deephaven/experimental/s3.py | 2 +- 6 files changed, 60 insertions(+), 8 deletions(-) create mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/AnonymousCredentials.java create mode 100644 extensions/s3/src/test/java/io/deephaven/extensions/s3/CredentialsTest.java diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/AnonymousCredentials.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AnonymousCredentials.java new file mode 100644 index 00000000000..a52a4598b76 --- /dev/null +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AnonymousCredentials.java @@ -0,0 +1,16 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.extensions.s3; + +import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; + +enum AnonymousCredentials implements AwsSdkV2Credentials { + ANONYMOUS_CREDENTIALS; + + @Override + public AwsCredentialsProvider awsV2CredentialsProvider() { + return AnonymousCredentialsProvider.create(); + } +} diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/Credentials.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/Credentials.java index 12cadeaaaaf..429067fbdf1 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/Credentials.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/Credentials.java @@ -21,7 +21,14 @@ static Credentials defaultCredentials() { * @param accessKeyId the access key id, used to identify the user * @param secretAccessKey the secret access key, used to authenticate the user */ - static Credentials basicCredentials(final String accessKeyId, final String secretAccessKey) { + static Credentials basic(final String accessKeyId, final String secretAccessKey) { return BasicCredentials.of(accessKeyId, secretAccessKey); } + + /** + * Anonymous credentials. This is useful when the S3 policy has been set to allow anonymous access. + */ + static Credentials anonymous() { + return AnonymousCredentials.ANONYMOUS_CREDENTIALS; + } } diff --git a/extensions/s3/src/test/java/io/deephaven/extensions/s3/CredentialsTest.java b/extensions/s3/src/test/java/io/deephaven/extensions/s3/CredentialsTest.java new file mode 100644 index 00000000000..313bf185f71 --- /dev/null +++ b/extensions/s3/src/test/java/io/deephaven/extensions/s3/CredentialsTest.java @@ -0,0 +1,31 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.extensions.s3; + + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +public class CredentialsTest { + + @Test + void defaultCredentials() { + isCredentials(Credentials.defaultCredentials()); + } + + @Test + void basic() { + isCredentials(Credentials.basic("accessKeyId", "secretAccessKey")); + } + + @Test + void anonymous() { + isCredentials(Credentials.anonymous()); + } + + private void isCredentials(Credentials c) { + assertThat(c).isInstanceOf(AwsSdkV2Credentials.class); + } +} diff --git a/extensions/s3/src/test/java/io/deephaven/extensions/s3/S3InstructionsTest.java b/extensions/s3/src/test/java/io/deephaven/extensions/s3/S3InstructionsTest.java index 14704234848..943cf02bf24 100644 --- a/extensions/s3/src/test/java/io/deephaven/extensions/s3/S3InstructionsTest.java +++ b/extensions/s3/src/test/java/io/deephaven/extensions/s3/S3InstructionsTest.java @@ -150,10 +150,10 @@ void tooSmallCacheSize() { void basicCredentials() { assertThat(S3Instructions.builder() .regionName("some-region") - .credentials(Credentials.basicCredentials("foo", "bar")) + .credentials(Credentials.basic("foo", "bar")) .build() .credentials()) - .isEqualTo(Credentials.basicCredentials("foo", "bar")); + .isEqualTo(Credentials.basic("foo", "bar")); } @Test diff --git a/extensions/s3/src/test/java/io/deephaven/extensions/s3/SingletonContainers.java b/extensions/s3/src/test/java/io/deephaven/extensions/s3/SingletonContainers.java index b3b0241923d..a00086de78b 100644 --- a/extensions/s3/src/test/java/io/deephaven/extensions/s3/SingletonContainers.java +++ b/extensions/s3/src/test/java/io/deephaven/extensions/s3/SingletonContainers.java @@ -38,9 +38,7 @@ static S3Instructions.Builder s3Instructions(S3Instructions.Builder builder) { return builder .endpointOverride(LOCALSTACK_S3.getEndpoint()) .regionName(LOCALSTACK_S3.getRegion()) - .credentials( - Credentials.basicCredentials(LOCALSTACK_S3.getAccessKey(), - LOCALSTACK_S3.getSecretKey())); + .credentials(Credentials.basic(LOCALSTACK_S3.getAccessKey(), LOCALSTACK_S3.getSecretKey())); } static S3Client s3Client() { @@ -73,7 +71,7 @@ static S3Instructions.Builder s3Instructions(S3Instructions.Builder builder) { return builder .endpointOverride(URI.create(MINIO.getS3URL())) .regionName(Region.AWS_GLOBAL.id()) - .credentials(Credentials.basicCredentials(MINIO.getUserName(), MINIO.getPassword())); + .credentials(Credentials.basic(MINIO.getUserName(), MINIO.getPassword())); } static S3Client s3Client() { diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index 62a3f9db6bd..0d0e4033cf3 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -109,7 +109,7 @@ def __init__(self, raise DHError("Either both access_key_id and secret_access_key must be provided or neither") if access_key_id is not None: - builder.credentials(_JCredentials.basicCredentials(access_key_id, secret_access_key)) + builder.credentials(_JCredentials.basic(access_key_id, secret_access_key)) if endpoint_override is not None: builder.endpointOverride(endpoint_override) From 55fef7c7b1ca33c4cfd3c522fff0e7aeb008d068 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Mon, 12 Feb 2024 10:58:13 -0800 Subject: [PATCH 03/13] Fix decompress call sites --- .../io/deephaven/parquet/base/ColumnChunkReaderImpl.java | 3 ++- .../java/io/deephaven/parquet/base/ColumnPageReaderImpl.java | 5 +++-- .../io/deephaven/parquet/compress/CompressorAdapter.java | 3 +++ 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 43e3b35a955..39cba8e7908 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java @@ -217,7 +217,8 @@ private Dictionary readDictionary(InputStream in) throws IOException { // Sometimes the size is explicitly empty, just use an empty payload payload = BytesInput.empty(); } else { - payload = decompressor.decompress(in, compressedPageSize, pageHeader.getUncompressed_page_size()); + payload = BytesInput + .copy(decompressor.decompress(in, compressedPageSize, pageHeader.getUncompressed_page_size())); } final DictionaryPage dictionaryPage = new DictionaryPage(payload, dictHeader.getNum_values(), Encoding.valueOf(dictHeader.getEncoding().name())); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java index 84bb798a379..f829c3b5d73 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java @@ -183,7 +183,8 @@ private DataPageV1 readV1Impl(InputStream in) throws IOException { } final int uncompressedPageSize = pageHeader.getUncompressed_page_size(); final int compressedPageSize = pageHeader.getCompressed_page_size(); - final BytesInput decompressedInput = compressorAdapter.decompress(in, compressedPageSize, uncompressedPageSize); + final BytesInput decompressedInput = + BytesInput.copy(compressorAdapter.decompress(in, compressedPageSize, uncompressedPageSize)); final DataPageHeader header = pageHeader.getData_page_header(); return new DataPageV1( decompressedInput, @@ -216,7 +217,7 @@ private DataPageV2 readV2Impl(InputStream in) throws IOException { BytesInput.copy(BytesInput.from(in, header.getRepetition_levels_byte_length())); final BytesInput definitionLevels = BytesInput.copy(BytesInput.from(in, header.getDefinition_levels_byte_length())); - final BytesInput data = compressorAdapter.decompress(in, compressedSize, uncompressedSize); + final BytesInput data = BytesInput.copy(compressorAdapter.decompress(in, compressedSize, uncompressedSize)); return new DataPageV2( header.getNum_rows(), header.getNum_nulls(), diff --git a/extensions/parquet/compression/src/main/java/io/deephaven/parquet/compress/CompressorAdapter.java b/extensions/parquet/compression/src/main/java/io/deephaven/parquet/compress/CompressorAdapter.java index cc4633ae96e..585a0ecadef 100644 --- a/extensions/parquet/compression/src/main/java/io/deephaven/parquet/compress/CompressorAdapter.java +++ b/extensions/parquet/compression/src/main/java/io/deephaven/parquet/compress/CompressorAdapter.java @@ -55,6 +55,9 @@ public void close() { /** * Returns an in-memory instance of BytesInput containing the fully decompressed results of the input stream. + * Callers should process the results before {@code inputStream} is closed; if the {@link BytesInput} interface + * needs to persist longer than {@code inputStream}, callers should use {@link BytesInput#copy(BytesInput)} on the + * results. * * @param inputStream an input stream containing compressed data * @param compressedSize the number of bytes in the compressed data From beb4aca3e1bd8b35d1df4c03310e0da151c4ead3 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Tue, 13 Feb 2024 16:09:22 -0800 Subject: [PATCH 04/13] Review stuff --- ...m.java => ChannelPositionInputStream.java} | 14 +-- .../io/deephaven/util/channel/Channels.java | 23 ++++ ...oviderImpl.java => ContextHolderImpl.java} | 6 +- .../util/channel/LocalFSChannelProvider.java | 17 +-- .../util/channel/SeekableChannelContext.java | 25 ++-- .../channel/SeekableChannelContextNull.java | 2 +- .../channel/SeekableChannelsProvider.java | 21 ++-- .../channel/SeekableChannelsProviderBase.java | 20 --- .../channel/CachedChannelProviderTest.java | 14 ++- .../parquet/base/ColumnChunkReaderImpl.java | 25 ++-- .../parquet/base/ColumnPageReaderImpl.java | 86 +++++++------ .../parquet/base/ParquetFileReader.java | 33 +++-- .../parquet/base/RowGroupReaderImpl.java | 6 +- .../table/pagestore/ColumnChunkPageStore.java | 6 +- .../OffsetIndexBasedColumnChunkPageStore.java | 6 +- .../VariablePageSizeColumnChunkPageStore.java | 8 +- .../extensions/s3/S3ChannelContext.java | 116 +++++++++--------- .../extensions/s3/S3Instructions.java | 22 +++- .../extensions/s3/S3SeekableByteChannel.java | 32 +++-- .../s3/S3SeekableChannelProvider.java | 40 +++--- .../TrackedSeekableChannelsProvider.java | 23 ++-- 21 files changed, 300 insertions(+), 245 deletions(-) rename Util/channel/src/main/java/io/deephaven/util/channel/{PositionInputStream.java => ChannelPositionInputStream.java} (72%) create mode 100644 Util/channel/src/main/java/io/deephaven/util/channel/Channels.java rename Util/channel/src/main/java/io/deephaven/util/channel/{ProviderImpl.java => ContextHolderImpl.java} (68%) delete mode 100644 Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderBase.java diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/PositionInputStream.java b/Util/channel/src/main/java/io/deephaven/util/channel/ChannelPositionInputStream.java similarity index 72% rename from Util/channel/src/main/java/io/deephaven/util/channel/PositionInputStream.java rename to Util/channel/src/main/java/io/deephaven/util/channel/ChannelPositionInputStream.java index bc2cb7048ba..7907caf961a 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/PositionInputStream.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/ChannelPositionInputStream.java @@ -11,13 +11,13 @@ import java.nio.channels.SeekableByteChannel; import java.util.Objects; -public final class PositionInputStream extends FilterInputStream { +public final class ChannelPositionInputStream extends FilterInputStream { /** * Wraps a channel-backed input stream {@code in}, ensuring upon {@link #close()} that {@code channel's} * {@link SeekableByteChannel#position()} has been advanced the exact amount of bytes that have been consumed from * the resulting input stream. {@code in} is closed during {@link #close()}; as such, the caller must ensure - * that closing {@code in} does _not_ close {@code channel}. To remain valid, the caller must ensure that the + * that closing {@code in} does not close {@code channel}. To remain valid, the caller must ensure that the * resulting input stream isn't re-wrapped by any downstream code in a way that would adversely effect the position * (such as wrapping the resulting input stream with buffering). * @@ -27,21 +27,21 @@ public final class PositionInputStream extends FilterInputStream { * @throws IOException if an IO exception occurs */ public static InputStream of(SeekableByteChannel channel, InputStream in) throws IOException { - return new PositionInputStream(channel, in); + return new ChannelPositionInputStream(channel, in); } private final SeekableByteChannel ch; - private final long position; + private final long startingPosition; - private PositionInputStream(SeekableByteChannel ch, InputStream in) throws IOException { + private ChannelPositionInputStream(SeekableByteChannel ch, InputStream in) throws IOException { super(new CountingInputStream(in)); this.ch = Objects.requireNonNull(ch); - this.position = ch.position(); + this.startingPosition = ch.position(); } @Override public void close() throws IOException { super.close(); - ch.position(position + ((CountingInputStream) in).getCount()); + ch.position(startingPosition + ((CountingInputStream) in).getCount()); } } diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/Channels.java b/Util/channel/src/main/java/io/deephaven/util/channel/Channels.java new file mode 100644 index 00000000000..22fb225f708 --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/Channels.java @@ -0,0 +1,23 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +import java.io.InputStream; +import java.nio.channels.ReadableByteChannel; +import java.time.Instant; +import java.time.temporal.ChronoUnit; + +public final class Channels { + + /** + * Constructs a stream that reads bytes from the given {@code channel}. Closing the resulting input stream does + * not close the {@code channel}. + * + * @param channel the channel from which bytes will be read + * @return the new input stream + */ + public static InputStream newInputStreamNoClose(ReadableByteChannel channel) { + return java.nio.channels.Channels.newInputStream(ReadableByteChannelNoClose.of(channel)); + } +} diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/ProviderImpl.java b/Util/channel/src/main/java/io/deephaven/util/channel/ContextHolderImpl.java similarity index 68% rename from Util/channel/src/main/java/io/deephaven/util/channel/ProviderImpl.java rename to Util/channel/src/main/java/io/deephaven/util/channel/ContextHolderImpl.java index 80780e02b82..e4dd62f4291 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/ProviderImpl.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/ContextHolderImpl.java @@ -3,14 +3,14 @@ */ package io.deephaven.util.channel; -import io.deephaven.util.channel.SeekableChannelContext.Provider; +import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import java.util.Objects; -final class ProviderImpl implements Provider { +final class ContextHolderImpl implements ContextHolder { private final SeekableChannelContext context; - public ProviderImpl(SeekableChannelContext context) { + public ContextHolderImpl(SeekableChannelContext context) { this.context = Objects.requireNonNull(context); } diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java index c144b6cb312..7f5c5d85461 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java @@ -6,20 +6,17 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.io.BufferedInputStream; import java.io.IOException; +import java.io.InputStream; import java.net.URI; +import java.nio.channels.Channels; import java.nio.channels.FileChannel; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; import java.nio.file.StandardOpenOption; -public class LocalFSChannelProvider extends SeekableChannelsProviderBase { - - @Override - protected boolean readChannelIsBuffered() { - return false; - } - +public class LocalFSChannelProvider implements SeekableChannelsProvider { @Override public SeekableChannelContext makeContext() { // No additional context required for local FS @@ -40,6 +37,12 @@ public SeekableByteChannel getReadChannel(@Nullable final SeekableChannelContext return FileChannel.open(Path.of(uri), StandardOpenOption.READ); } + @Override + public InputStream getInputStream(SeekableByteChannel channel) { + // FileChannel is not buffered, need to buffer + return new BufferedInputStream(Channels.newInputStream(ReadableByteChannelNoClose.of(channel))); + } + @Override public SeekableByteChannel getWriteChannel(@NotNull final Path filePath, final boolean append) throws IOException { final FileChannel result = FileChannel.open(filePath, diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java index 4a8cfbfcf6b..f0cfe73ac7c 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java @@ -2,7 +2,6 @@ import io.deephaven.util.SafeCloseable; -import java.io.Closeable; import java.util.function.Supplier; /** @@ -10,13 +9,25 @@ */ public interface SeekableChannelContext extends SafeCloseable { - SeekableChannelContext NULL = SeekableChannelContextNull.NULL; + SeekableChannelContext NULL = SeekableChannelContextNull.NULL_CONTEXT_INSTANCE; - static Provider upgrade(SeekableChannelsProvider provider, SeekableChannelContext context) { - if (context != NULL) { - return () -> context; + /** + * A pattern that allows callers to ensure a valid context has been created for {@code provider}. In the case where + * the given {@code context} {@link SeekableChannelsProvider#isCompatibleWith(SeekableChannelContext) is compatible + * with} {@code provider}, a no-op holder around that {@code context} will be returned. Otherwise, a holder with a + * new {@link SeekableChannelsProvider#makeSingleUseContext()} will be returned. The returned holder should ideally + * be used in a try-with-resources construction. + * + * @param provider the provider + * @param context the context + * @return the context holder + */ + static ContextHolder ensureContext(SeekableChannelsProvider provider, SeekableChannelContext context) { + if (!provider.isCompatibleWith(context)) { + return new ContextHolderImpl(provider.makeSingleUseContext()); } - return new ProviderImpl(provider.makeSingleUseContext()); + // An impl that does not close the context + return () -> context; } /** @@ -24,7 +35,7 @@ static Provider upgrade(SeekableChannelsProvider provider, SeekableChannelContex */ default void close() {} - interface Provider extends Closeable, Supplier { + interface ContextHolder extends AutoCloseable, Supplier { @Override SeekableChannelContext get(); diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContextNull.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContextNull.java index 3fa38a88e56..5c209f477b6 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContextNull.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContextNull.java @@ -4,5 +4,5 @@ package io.deephaven.util.channel; enum SeekableChannelContextNull implements SeekableChannelContext { - NULL + NULL_CONTEXT_INSTANCE } diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java index 58481fecc2d..17858b3be46 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java @@ -39,22 +39,23 @@ static URI convertToURI(final String source) { } /** - * Wraps {@link SeekableChannelsProvider#getInputStream(SeekableByteChannel)} in a position-safe manner. To remain - * valid, the caller must ensure that the resulting input stream isn't re-wrapped by any downstream code in a way - * that would adversely effect the position (such as re-wrapping the resulting input stream with buffering). + * Wraps {@link SeekableChannelsProvider#getInputStream(SeekableByteChannel)} to ensure the channel's position is + * incremented the exact amount that has been consumed from the resulting input stream. To remain valid, the caller + * must ensure that the resulting input stream isn't re-wrapped by any downstream code in a way that would adversely + * effect the position (such as re-wrapping the resulting input stream with buffering). * *

- * Equivalent to {@code PositionInputStream.of(ch, provider.getInputStream(ch))}. + * Equivalent to {@code ChannelPositionInputStream.of(ch, provider.getInputStream(ch))}. * * @param provider the provider * @param ch the seekable channel * @return the position-safe input stream * @throws IOException if an IO exception occurs - * @see PositionInputStream#of(SeekableByteChannel, InputStream) + * @see ChannelPositionInputStream#of(SeekableByteChannel, InputStream) */ - static InputStream positionInputStream(SeekableChannelsProvider provider, SeekableByteChannel ch) + static InputStream channelPositionInputStream(SeekableChannelsProvider provider, SeekableByteChannel ch) throws IOException { - return PositionInputStream.of(ch, provider.getInputStream(ch)); + return ChannelPositionInputStream.of(ch, provider.getInputStream(ch)); } /** @@ -62,6 +63,10 @@ static InputStream positionInputStream(SeekableChannelsProvider provider, Seekab */ SeekableChannelContext makeContext(); + /** + * Create a new "single-use" {@link SeekableChannelContext} object for creating read channels via this provider. + * This is meant for contexts that have a short lifecycle and expect to read a small amount from a read channel. + */ default SeekableChannelContext makeSingleUseContext() { return makeContext(); } @@ -88,7 +93,7 @@ SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContex * {@code channel} must have been created by {@code this} provider. The caller can't assume the position of * {@code channel} after consuming the {@link InputStream}. For use-cases that require the channel's position to be * incremented the exact amount the {@link InputStream} has been consumed, use - * {@link #positionInputStream(SeekableChannelsProvider, SeekableByteChannel)}. + * {@link #channelPositionInputStream(SeekableChannelsProvider, SeekableByteChannel)}. * * @param channel the channel * @return the input stream diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderBase.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderBase.java deleted file mode 100644 index bacee4ee022..00000000000 --- a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderBase.java +++ /dev/null @@ -1,20 +0,0 @@ -/** - * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending - */ -package io.deephaven.util.channel; - -import java.io.BufferedInputStream; -import java.io.InputStream; -import java.nio.channels.Channels; -import java.nio.channels.SeekableByteChannel; - -public abstract class SeekableChannelsProviderBase implements SeekableChannelsProvider { - - protected abstract boolean readChannelIsBuffered(); - - @Override - public final InputStream getInputStream(SeekableByteChannel channel) { - final InputStream in = Channels.newInputStream(ReadableByteChannelNoClose.of(channel)); - return readChannelIsBuffered() ? in : new BufferedInputStream(in, 8192); - } -} diff --git a/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java b/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java index f0182ab8ed5..e7aa5a0283b 100644 --- a/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java +++ b/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java @@ -8,6 +8,7 @@ import org.junit.jupiter.api.Test; import java.io.IOException; +import java.io.InputStream; import java.net.URI; import java.nio.ByteBuffer; import java.nio.channels.SeekableByteChannel; @@ -170,18 +171,13 @@ public void testReuse10() throws IOException { } - private class TestChannelProvider extends SeekableChannelsProviderBase { + private class TestChannelProvider implements SeekableChannelsProvider { AtomicInteger count = new AtomicInteger(0); private final class TestChannelContext implements SeekableChannelContext { } - @Override - protected boolean readChannelIsBuffered() { - return true; // TestMockChannel always returns 0, might as well be "buffered" - } - @Override public SeekableChannelContext makeContext() { return new TestChannelContext(); @@ -198,6 +194,12 @@ public SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channe return new TestMockChannel(count.getAndIncrement(), path, channelContext); } + @Override + public InputStream getInputStream(SeekableByteChannel channel) { + // TestMockChannel is always empty, so no need to buffer + return Channels.newInputStreamNoClose(channel); + } + @Override public SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull URI uri) { return new TestMockChannel(count.getAndIncrement(), uri.toString(), channelContext); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 39cba8e7908..96f7b9b6e94 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java @@ -8,7 +8,7 @@ import io.deephaven.util.channel.SeekableChannelsProvider; import io.deephaven.parquet.compress.CompressorAdapter; import io.deephaven.parquet.compress.DeephavenCompressorAdapterFactory; -import io.deephaven.util.channel.SeekableChannelContext.Provider; +import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import io.deephaven.util.datastructures.LazyCachingFunction; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.ColumnDescriptor; @@ -178,8 +178,8 @@ private Dictionary getDictionary(final SeekableChannelContext channelContext) { } // Use the context object provided by the caller, or create (and close) a new one try ( - final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); - final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), getURI()); + final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(holder.get(), getURI()); final InputStream in = channelsProvider.getInputStream(ch.position(dictionaryPageOffset))) { return readDictionary(in); } catch (IOException e) { @@ -217,12 +217,13 @@ private Dictionary readDictionary(InputStream in) throws IOException { // Sometimes the size is explicitly empty, just use an empty payload payload = BytesInput.empty(); } else { - payload = BytesInput - .copy(decompressor.decompress(in, compressedPageSize, pageHeader.getUncompressed_page_size())); + payload = decompressor.decompress(in, compressedPageSize, pageHeader.getUncompressed_page_size()); } - final DictionaryPage dictionaryPage = new DictionaryPage(payload, dictHeader.getNum_values(), - Encoding.valueOf(dictHeader.getEncoding().name())); - return dictionaryPage.getEncoding().initDictionary(path, dictionaryPage); + final Encoding encoding = Encoding.valueOf(dictHeader.getEncoding().name()); + final DictionaryPage dictionaryPage = new DictionaryPage(payload, dictHeader.getNum_values(), encoding); + // We are safe here because the Dictionary doesn't hold a reference to payload (and thus implicitly, doesn't + // hold a reference to the input stream). + return encoding.initDictionary(path, dictionaryPage); } private final class ColumnPageReaderIteratorImpl implements ColumnPageReaderIterator { @@ -247,11 +248,11 @@ public ColumnPageReader next(@NotNull final SeekableChannelContext channelContex // NB: The channels provider typically caches channels; this avoids maintaining a handle per column chunk final long headerOffset = nextHeaderOffset; try ( - final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); - final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), getURI())) { + final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(holder.get(), getURI())) { ch.position(headerOffset); final PageHeader pageHeader; - try (final InputStream in = SeekableChannelsProvider.positionInputStream(channelsProvider, ch)) { + try (final InputStream in = SeekableChannelsProvider.channelPositionInputStream(channelsProvider, ch)) { pageHeader = Util.readPageHeader(in); } // relying on exact position of ch @@ -259,7 +260,7 @@ public ColumnPageReader next(@NotNull final SeekableChannelContext channelContex nextHeaderOffset = dataOffset + pageHeader.getCompressed_page_size(); if (pageHeader.isSetDictionary_page_header()) { // Dictionary page; skip it - return next(upgrade.get()); + return next(holder.get()); } if (!pageHeader.isSetData_page_header() && !pageHeader.isSetData_page_header_v2()) { throw new IllegalStateException( diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java index f829c3b5d73..04d265974be 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java @@ -8,7 +8,7 @@ import io.deephaven.parquet.compress.CompressorAdapter; import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.util.channel.SeekableChannelContext.Provider; +import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import org.apache.parquet.bytes.ByteBufferInputStream; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.ColumnDescriptor; @@ -104,17 +104,17 @@ final class ColumnPageReaderImpl implements ColumnPageReader { public Object materialize(@NotNull final Object nullValue, @NotNull final SeekableChannelContext channelContext) throws IOException { try ( - final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); - final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), uri)) { + final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(holder.get(), uri)) { ensurePageHeader(channelsProvider, ch); - return readDataPage(nullValue, ch, upgrade.get()); + return readDataPage(nullValue, ch, holder.get()); } } public int readRowCount(@NotNull final SeekableChannelContext channelContext) throws IOException { try ( - final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); - final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), uri)) { + final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(holder.get(), uri)) { ensurePageHeader(channelsProvider, ch); return readRowCountFromDataPage(ch); } @@ -124,10 +124,10 @@ public int readRowCount(@NotNull final SeekableChannelContext channelContext) th public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder, @NotNull final SeekableChannelContext channelContext) throws IOException { try ( - final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); - final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), uri)) { + final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(holder.get(), uri)) { ensurePageHeader(channelsProvider, ch); - return readKeyFromDataPage(keyDest, nullPlaceholder, ch, upgrade.get()); + return readKeyFromDataPage(keyDest, nullPlaceholder, ch, holder.get()); } } @@ -141,7 +141,7 @@ private void ensurePageHeader(SeekableChannelsProvider provider, SeekableByteCha ch.position(offset); synchronized (this) { if (pageHeader == null) { - try (final InputStream in = SeekableChannelsProvider.positionInputStream(provider, ch)) { + try (final InputStream in = SeekableChannelsProvider.channelPositionInputStream(provider, ch)) { pageHeader = Util.readPageHeader(in); } offset = ch.position(); @@ -171,20 +171,16 @@ private static int readNumValuesFromPageHeader(@NotNull final PageHeader header) } } - private DataPageV1 readV1(SeekableByteChannel ch) throws IOException { - try (final InputStream in = channelsProvider.getInputStream(ch)) { - return readV1Impl(in); - } - } - - private DataPageV1 readV1Impl(InputStream in) throws IOException { + /** + * Callers must ensure resulting data page does not outlive the input stream. + */ + private DataPageV1 readV1Unsafe(InputStream in) throws IOException { if (pageHeader.type != PageType.DATA_PAGE) { throw new IllegalArgumentException(); } final int uncompressedPageSize = pageHeader.getUncompressed_page_size(); final int compressedPageSize = pageHeader.getCompressed_page_size(); - final BytesInput decompressedInput = - BytesInput.copy(compressorAdapter.decompress(in, compressedPageSize, uncompressedPageSize)); + final BytesInput decompressedInput = compressorAdapter.decompress(in, compressedPageSize, uncompressedPageSize); final DataPageHeader header = pageHeader.getData_page_header(); return new DataPageV1( decompressedInput, @@ -196,13 +192,10 @@ private DataPageV1 readV1Impl(InputStream in) throws IOException { getEncoding(header.getEncoding())); } - private DataPageV2 readV2(SeekableByteChannel ch) throws IOException { - try (final InputStream in = channelsProvider.getInputStream(ch)) { - return readV2Impl(in); - } - } - - private DataPageV2 readV2Impl(InputStream in) throws IOException { + /** + * Callers must ensure resulting data page does not outlive the input stream. + */ + private DataPageV2 readV2Unsafe(InputStream in) throws IOException { if (pageHeader.type != PageType.DATA_PAGE_V2) { throw new IllegalArgumentException(); } @@ -213,11 +206,16 @@ private DataPageV2 readV2Impl(InputStream in) throws IOException { - header.getDefinition_levels_byte_length(); final int uncompressedSize = uncompressedPageSize - header.getRepetition_levels_byte_length() - header.getDefinition_levels_byte_length(); + // With our current single input stream `in` construction, we must copy out the bytes for repetition and + // definition levels to proceed to data. We could theoretically restructure this to be fully lazy, either by + // creating a BytesInput impl for SeekableByteChannel, or by migrating this logic one layer up and ensuring we + // construct input streams separately for repetitionLevelsIn, definitionLevelsIn, and dataIn. Both of these + // solutions would potentially suffer from a disconnect in cache-ability that a single input stream provides. final BytesInput repetitionLevels = BytesInput.copy(BytesInput.from(in, header.getRepetition_levels_byte_length())); final BytesInput definitionLevels = BytesInput.copy(BytesInput.from(in, header.getDefinition_levels_byte_length())); - final BytesInput data = BytesInput.copy(compressorAdapter.decompress(in, compressedSize, uncompressedSize)); + final BytesInput data = compressorAdapter.decompress(in, compressedSize, uncompressedSize); return new DataPageV2( header.getNum_rows(), header.getNum_nulls(), @@ -234,7 +232,9 @@ private DataPageV2 readV2Impl(InputStream in) throws IOException { private int readRowCountFromDataPage(SeekableByteChannel ch) throws IOException { switch (pageHeader.type) { case DATA_PAGE: - return readRowCountFromPageV1(readV1(ch)); + try (final InputStream in = channelsProvider.getInputStream(ch)) { + return readRowCountFromPageV1(readV1Unsafe(in)); + } case DATA_PAGE_V2: DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); return dataHeaderV2.getNum_rows(); @@ -248,10 +248,14 @@ private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, throws IOException { switch (pageHeader.type) { case DATA_PAGE: - return readKeysFromPageV1(readV1(ch), keyDest, nullPlaceholder, channelContext); + try (final InputStream in = channelsProvider.getInputStream(ch)) { + return readKeysFromPageV1(readV1Unsafe(in), keyDest, nullPlaceholder, channelContext); + } case DATA_PAGE_V2: - readKeysFromPageV2(readV2(ch), keyDest, nullPlaceholder, channelContext); - return null; + try (final InputStream in = channelsProvider.getInputStream(ch)) { + readKeysFromPageV2(readV2Unsafe(in), keyDest, nullPlaceholder, channelContext); + return null; + } default: throw new IOException(String.format("Unexpected page of type %s of size %d", pageHeader.getType(), pageHeader.getCompressed_page_size())); @@ -262,9 +266,13 @@ private Object readDataPage(Object nullValue, SeekableByteChannel ch, @NotNull SeekableChannelContext channelContext) throws IOException { switch (pageHeader.type) { case DATA_PAGE: - return readPageV1(readV1(ch), nullValue, channelContext); + try (final InputStream in = channelsProvider.getInputStream(ch)) { + return readPageV1(readV1Unsafe(in), nullValue, channelContext); + } case DATA_PAGE_V2: - return readPageV2(readV2(ch), nullValue); + try (final InputStream in = channelsProvider.getInputStream(ch)) { + return readPageV2(readV2Unsafe(in), nullValue); + } default: throw new IOException(String.format("Unexpected page of type %s of size %d", pageHeader.getType(), pageHeader.getCompressed_page_size())); @@ -278,8 +286,8 @@ private Encoding getEncoding(org.apache.parquet.format.Encoding encoding) { private int readRowCountFromPageV1(DataPageV1 page) { try { if (path.getMaxRepetitionLevel() != 0) { - ByteBuffer bytes = page.getBytes().toByteBuffer(); // TODO - move away from page and use - // ByteBuffers directly + // TODO - move away from page and use ByteBuffers directly + ByteBuffer bytes = page.getBytes().toByteBuffer(); bytes.order(ByteOrder.LITTLE_ENDIAN); int length = bytes.getInt(); return readRepetitionLevels(bytes.slice().limit(length)); @@ -362,8 +370,8 @@ private Object readPageV1(DataPageV1 page, Object nullValue, @NotNull final SeekableChannelContext channelContext) { RunLengthBitPackingHybridBufferDecoder dlDecoder = null; try { - ByteBuffer bytes = page.getBytes().toByteBuffer(); // TODO - move away from page and use - // ByteBuffers directly + // TODO - move away from page and use ByteBuffers directly + ByteBuffer bytes = page.getBytes().toByteBuffer(); bytes.order(ByteOrder.LITTLE_ENDIAN); RunLengthBitPackingHybridBufferDecoder rlDecoder = null; if (path.getMaxRepetitionLevel() != 0) { @@ -586,8 +594,8 @@ public int numValues(@NotNull final SeekableChannelContext channelContext) throw return numValues; } try ( - final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); - final SeekableByteChannel ch = channelsProvider.getReadChannel(upgrade.get(), uri)) { + final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); + final SeekableByteChannel ch = channelsProvider.getReadChannel(holder.get(), uri)) { ensurePageHeader(channelsProvider, ch); // Above will block till it populates numValues Assert.geqZero(numValues, "numValues"); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java index 7a13adf7f32..e057047bf23 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java @@ -13,8 +13,6 @@ import java.io.IOException; import java.io.InputStream; import java.net.URI; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; import java.nio.channels.SeekableByteChannel; import java.nio.charset.StandardCharsets; import java.nio.file.Path; @@ -57,9 +55,9 @@ public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvide rootURI = parquetFileURI; } try ( - final SeekableChannelContext context = channelsProvider.makeSingleUseContext(); + final SeekableChannelContext context = channelsProvider.makeContext(); final SeekableByteChannel ch = channelsProvider.getReadChannel(context, parquetFileURI)) { - final int footerLength = prepareFooter(parquetFileURI, ch); + positionToFileMetadata(parquetFileURI, ch); try (final InputStream in = channelsProvider.getInputStream(ch)) { fileMetaData = Util.readFileMetaData(in); } @@ -67,30 +65,35 @@ public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvide type = fromParquetSchema(fileMetaData.schema, fileMetaData.column_orders); } - private static int prepareFooter(URI parquetFileURI, SeekableByteChannel readChannel) throws IOException { + private static void positionToFileMetadata(URI parquetFileURI, SeekableByteChannel readChannel) throws IOException { final long fileLen = readChannel.size(); if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + // footerIndex + MAGIC throw new InvalidParquetFileException( parquetFileURI + " is not a Parquet file (too small length: " + fileLen + ")"); } - final byte[] last8 = new byte[8]; + final byte[] trailer = new byte[Integer.BYTES + MAGIC.length]; final long footerLengthIndex = fileLen - FOOTER_LENGTH_SIZE - MAGIC.length; readChannel.position(footerLengthIndex); - Helpers.readBytes(readChannel, last8); - if (!Arrays.equals(MAGIC, 0, MAGIC.length, last8, 4, 8)) { + Helpers.readBytes(readChannel, trailer); + if (!Arrays.equals(MAGIC, 0, MAGIC.length, trailer, Integer.BYTES, trailer.length)) { throw new InvalidParquetFileException( parquetFileURI + " is not a Parquet file. expected magic number at tail " + Arrays.toString(MAGIC) - + " but found " + Arrays.toString(Arrays.copyOfRange(last8, 4, 8))); + + " but found " + Arrays.toString(Arrays.copyOfRange(trailer, 4, 8))); } - final int footerLength = ByteBuffer.wrap(last8, 0, 4).order(ByteOrder.LITTLE_ENDIAN).getInt(); + // final int footerLength = ByteBuffer.wrap(trailer, 0, Integer.BYTES).order(ByteOrder.LITTLE_ENDIAN).getInt(); + final int footerLength = makeLittleEndianInt(trailer[0], trailer[1], trailer[2], trailer[3]); final long footerIndex = footerLengthIndex - footerLength; if (footerIndex < MAGIC.length || footerIndex >= footerLengthIndex) { throw new InvalidParquetFileException( "corrupted file: the footer index is not within the file: " + footerIndex); } readChannel.position(footerIndex); - return footerLength; + // return footerLength; + } + + private static int makeLittleEndianInt(byte b0, byte b1, byte b2, byte b3) { + return (b0 & 0xff) | ((b1 & 0xff) << 8) | ((b2 & 0xff) << 16) | ((b3 & 0xff) << 24); } /** @@ -181,14 +184,6 @@ private Set calculateColumnsWithDictionaryUsedOnEveryDataPage() { return result; } - private static int readIntLittleEndian(SeekableByteChannel f) throws IOException { - ByteBuffer tempBuf = ByteBuffer.allocate(Integer.BYTES); - tempBuf.order(ByteOrder.LITTLE_ENDIAN); - Helpers.readExact(f, tempBuf); - tempBuf.flip(); - return tempBuf.getInt(); - } - /** * Create a {@link RowGroupReader} object for provided row group number * diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java index b8a2c5ef8fb..3bbba1aa586 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java @@ -4,7 +4,7 @@ package io.deephaven.parquet.base; import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.util.channel.SeekableChannelContext.Provider; +import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.format.ColumnChunk; import org.apache.parquet.format.RowGroup; @@ -93,8 +93,8 @@ private OffsetIndex offsetIndex(ColumnChunk chunk, @NotNull SeekableChannelConte private org.apache.parquet.format.OffsetIndex readOffsetIndex(ColumnChunk chunk, @NotNull SeekableChannelContext channelContext) { try ( - final Provider upgrade = SeekableChannelContext.upgrade(channelsProvider, channelContext); - final SeekableByteChannel readChannel = channelsProvider.getReadChannel(upgrade.get(), rootURI); + final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); + final SeekableByteChannel readChannel = channelsProvider.getReadChannel(holder.get(), rootURI); final InputStream in = channelsProvider.getInputStream(readChannel.position(chunk.getOffset_index_offset()))) { return Util.readOffsetIndex(in); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java index 921a9c309dd..f8ad63fe011 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java @@ -22,7 +22,7 @@ import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; import io.deephaven.util.SafeCloseable; -import io.deephaven.util.channel.SeekableChannelContext.Provider; +import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import io.deephaven.vector.Vector; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -219,8 +219,8 @@ final SeekableChannelContext innerFillContext(@Nullable final FillContext contex return SeekableChannelContext.NULL; } - final Provider upgrade(@Nullable final FillContext context) { - return SeekableChannelContext.upgrade(columnChunkReader.getChannelsProvider(), innerFillContext(context)); + final ContextHolder ensureContext(@Nullable final FillContext context) { + return SeekableChannelContext.ensureContext(columnChunkReader.getChannelsProvider(), innerFillContext(context)); } private T fillContextUpdater( diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java index dbe494bfdb3..2634c65b0b9 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java @@ -11,7 +11,7 @@ import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; -import io.deephaven.util.channel.SeekableChannelContext.Provider; +import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import org.apache.parquet.internal.column.columnindex.OffsetIndex; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -129,8 +129,8 @@ private ChunkPage getPage(@Nullable final FillContext fillContext, final i private ChunkPage getPageImpl(@Nullable FillContext fillContext, int pageNum) { final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(pageNum); // Use the latest context while reading the page, or create (and close) new one - try (final Provider upgrade = upgrade(fillContext)) { - return toPage(offsetIndex.getFirstRowIndex(pageNum), reader, upgrade.get()); + try (final ContextHolder holder = ensureContext(fillContext)) { + return toPage(offsetIndex.getFirstRowIndex(pageNum), reader, holder.get()); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java index 1f6b102277a..1a2e0f1ff62 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java @@ -11,7 +11,7 @@ import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; -import io.deephaven.util.channel.SeekableChannelContext.Provider; +import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -153,16 +153,16 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext pageNum = -2 - pageNum; } // Use the latest channel context while reading page headers, or create (and close) a new one - try (final Provider upgrade = upgrade(fillContext)) { + try (final ContextHolder holder = ensureContext(fillContext)) { if (pageNum >= localNumPages) { - final int minPageNum = fillToRow(upgrade.get(), localNumPages, rowKey); + final int minPageNum = fillToRow(holder.get(), localNumPages, rowKey); localNumPages = numPages; pageNum = Arrays.binarySearch(pageRowOffsets, minPageNum + 1, localNumPages + 1, rowKey); if (pageNum < 0) { pageNum = -2 - pageNum; } } - return getPage(upgrade.get(), pageNum); + return getPage(holder.get(), pageNum); } } } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java index eecdab3474f..d3e34aa68b1 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -50,7 +50,10 @@ final class S3ChannelContext implements SeekableChannelContext { */ private long size; - private long numFragmentsInObject; + /** + * The number of fragments in the object, ceil(size / fragmentSize) + */ + private long numFragments; S3ChannelContext(S3AsyncClient client, S3Instructions instructions, BufferPool bufferPool) { this.client = Objects.requireNonNull(client); @@ -63,19 +66,35 @@ final class S3ChannelContext implements SeekableChannelContext { } } - public long size(S3Uri uri) throws IOException { - assume(uri); - populateSize(); + void verifyOrSetUri(S3Uri uri) { + if (this.uri == null) { + this.uri = Objects.requireNonNull(uri); + } else if (!this.uri.equals(uri)) { + throw new IllegalStateException( + String.format("Inconsistent URIs. expected=%s, actual=%s", this.uri, uri)); + } + } + + void verifyOrSetSize(long size) { + if (this.size == UNINITIALIZED_SIZE) { + setSize(size); + } else if (this.size != size) { + throw new IllegalStateException( + String.format("Inconsistent size. expected=%d, actual=%d", size, this.size)); + } + } + + public long size() throws IOException { + ensureSize(); return size; } - public int fill(S3Uri uri, final long position, ByteBuffer dest) throws IOException { - assume(uri); - populateSize(); + public int fill(final long position, ByteBuffer dest) throws IOException { final int destRemaining = dest.remaining(); if (destRemaining == 0) { return 0; } + ensureSize(); // Send async read requests for current fragment as well as read ahead fragments final long firstFragmentIx = fragmentIndex(position); final long readAhead; @@ -83,7 +102,7 @@ public int fill(S3Uri uri, final long position, ByteBuffer dest) throws IOExcept final long lastFragmentIx = fragmentIndex(position + destRemaining - 1); final int impliedReadAhead = (int) (lastFragmentIx - firstFragmentIx); final int desiredReadAhead = instructions.readAheadCount(); - final long totalRemainingFragments = numFragmentsInObject - firstFragmentIx - 1; + final long totalRemainingFragments = numFragments - firstFragmentIx - 1; final int maxReadAhead = requests.length - 1; readAhead = Math.min( Math.max(impliedReadAhead, desiredReadAhead), @@ -122,17 +141,6 @@ public void close() { // -------------------------------------------------------------------------------------------------- - void assume(S3Uri uri) { - if (this.uri == null) { - this.uri = Objects.requireNonNull(uri); - } else { - if (!this.uri.equals(uri)) { - throw new IllegalStateException( - String.format("Inconsistent URIs. expected=%s, actual=%s", this.uri, uri)); - } - } - } - private Optional getRequest(final long fragmentIndex) { final int cacheIdx = cacheIndex(fragmentIndex); final Request request = requests[cacheIdx]; @@ -183,11 +191,9 @@ final class Request private final long from; private final long to; private final Instant createdAt; - private Instant completedAt; private final CompletableFuture released; private CompletableFuture consumerFuture; private volatile CompletableFuture producerFuture; - private GetObjectResponse response; private int fillCount; private long fillBytes; @@ -212,18 +218,25 @@ public boolean isDone() { } public int fill(long localPosition, ByteBuffer dest) throws IOException { - final int outOffset = (int) (localPosition - from); - final int outLength = Math.min((int) (to - localPosition + 1), dest.remaining()); + final int resultOffset = (int) (localPosition - from); + final int resultLength = Math.min((int) (to - localPosition + 1), dest.remaining()); final ByteBuffer fullFragment; try { - fullFragment = get(); + fullFragment = getFullFragment(); } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { throw handleS3Exception(e, String.format("fetching fragment %s", requestStr())); } - dest.put(fullFragment.duplicate().position(outOffset).limit(outOffset + outLength)); + // fullFragment is a slice. Lets us have safety around math and ability to simply clear to reset. + fullFragment.limit(resultOffset + resultLength); + fullFragment.position(resultOffset); + try { + dest.put(fullFragment); + } finally { + fullFragment.clear(); + } ++fillCount; - fillBytes += outLength; - return outLength; + fillBytes += resultLength; + return resultLength; } public void release() { @@ -240,8 +253,8 @@ public void release() { @Override public void accept(ByteBuffer byteBuffer, Throwable throwable) { - completedAt = Instant.now(); if (log.isDebugEnabled()) { + final Instant completedAt = Instant.now(); if (byteBuffer != null) { log.debug("send complete: {} {}", requestStr(), Duration.between(createdAt, completedAt)); } else { @@ -261,12 +274,12 @@ public CompletableFuture prepare() { @Override public void onResponse(GetObjectResponse response) { - this.response = response; + } @Override public void onStream(SdkPublisher publisher) { - publisher.subscribe(new Request.Sub()); + publisher.subscribe(new Sub()); } @Override @@ -276,14 +289,20 @@ public void exceptionOccurred(Throwable error) { // -------------------------------------------------------------------------------------------------- - private ByteBuffer get() throws ExecutionException, InterruptedException, TimeoutException { + private ByteBuffer getFullFragment() throws ExecutionException, InterruptedException, TimeoutException { if (released.isDone()) { throw new IllegalStateException(String.format("Trying to get data after release, %s", requestStr())); } // Giving our own get() a bit of overhead - the clients should already be constructed with appropriate // apiCallTimeout. final long readNanos = instructions.readTimeout().plusMillis(100).toNanos(); - return consumerFuture.get(readNanos, TimeUnit.NANOSECONDS); + final ByteBuffer result = consumerFuture.get(readNanos, TimeUnit.NANOSECONDS); + if (result.position() != 0 || result.limit() != result.capacity() || result.limit() != requestLength()) { + throw new IllegalStateException(String.format( + "Expected: pos=0, limit=%d, capacity=%d. Actual: pos=%d, limit=%d, capacity=%d", + requestLength(), requestLength(), result.position(), result.limit(), result.capacity())); + } + return result; } private boolean isFragment(long fragmentIndex) { @@ -303,13 +322,8 @@ private GetObjectRequest getObjectRequest() { } private String requestStr() { - if (uri != null) { - return String.format("ctx=%d ix=%d [%d, %d]/%d %s/%s", System.identityHashCode(S3ChannelContext.this), - fragmentIndex, from, to, requestLength(), uri.bucket().orElseThrow(), uri.key().orElseThrow()); - } else { - return String.format("ctx=%d ix=%d [%d, %d]/%d", System.identityHashCode(S3ChannelContext.this), - fragmentIndex, from, to, requestLength()); - } + return String.format("ctx=%d ix=%d [%d, %d]/%d %s/%s", System.identityHashCode(S3ChannelContext.this), + fragmentIndex, from, to, requestLength(), uri.bucket().orElseThrow(), uri.key().orElseThrow()); } // -------------------------------------------------------------------------------------------------- @@ -322,9 +336,9 @@ final class Sub implements Subscriber, BiConsumer { Sub() { localProducer = producerFuture; buffer = bufferPool.take(requestLength()); - // 1. localProducer succeeds: whenComplete will executed when released - // 2. localProducer fails: whenComplete will executed asap - // 3. localProducer limbo (not expected): whenComplete will executed when released + // 1. localProducer succeeds: whenComplete will be executed when released + // 2. localProducer fails: whenComplete will be executed asap + // 3. localProducer limbo (not expected): whenComplete will be executed when released CompletableFuture.allOf(localProducer, released).whenComplete(this); } @@ -362,11 +376,11 @@ public synchronized void onError(Throwable t) { @Override public synchronized void onComplete() { buffer.flip(); - if (buffer.remaining() != requestLength()) { + if (buffer.limit() != requestLength()) { localProducer.completeExceptionally(new IllegalStateException(String.format( - "Expected %d bytes, received %d, %s", requestLength(), buffer.remaining(), requestStr()))); + "Expected %d bytes, received %d, %s", requestLength(), buffer.limit(), requestStr()))); } else { - localProducer.complete(buffer.asReadOnlyBuffer()); + localProducer.complete(buffer.slice().asReadOnlyBuffer()); } } } @@ -391,7 +405,7 @@ private IOException handleS3Exception(final Exception e, final String operationD return new IOException(String.format("Exception caught while %s", operationDescription), e); } - private void populateSize() throws IOException { + private void ensureSize() throws IOException { if (size != UNINITIALIZED_SIZE) { return; } @@ -414,17 +428,9 @@ private void populateSize() throws IOException { setSize(headObjectResponse.contentLength()); } - void hackSize(long size) { - if (this.size == UNINITIALIZED_SIZE) { - setSize(size); - } else if (this.size != size) { - throw new IllegalStateException(); - } - } - private void setSize(long size) { this.size = size; // ceil(size / fragmentSize) - this.numFragmentsInObject = (size + instructions.fragmentSize() - 1) / instructions.fragmentSize(); + this.numFragments = (size + instructions.fragmentSize() - 1) / instructions.fragmentSize(); } } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index 308f147f480..e566afccf3c 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -8,6 +8,7 @@ import org.immutables.value.Value.Check; import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Lazy; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import java.net.URI; @@ -24,8 +25,7 @@ defaults = @Value.Immutable(copy = true), strictBuilder = true, weakInterning = true, - jdkOnly = true, - includeHashCode = "getClass().hashCode()") + jdkOnly = true) public abstract class S3Instructions { private final static int DEFAULT_MAX_CONCURRENT_REQUESTS = 50; @@ -35,7 +35,7 @@ public abstract class S3Instructions { final static int MAX_FRAGMENT_SIZE = Configuration.getInstance().getIntegerWithDefault(MAX_FRAGMENT_SIZE_CONFIG_PARAM, 5 << 20); // 5 MiB private final static int DEFAULT_FRAGMENT_SIZE = MAX_FRAGMENT_SIZE; - + private final static int SINGLE_USE_FRAGMENT_SIZE_DEFAULT = Math.min(65536, MAX_FRAGMENT_SIZE); // 64 KiB private final static int MIN_FRAGMENT_SIZE = 8 << 10; // 8 KiB private final static int DEFAULT_MAX_CACHE_SIZE = 32; private final static Duration DEFAULT_CONNECTION_TIMEOUT = Duration.ofSeconds(2); @@ -123,8 +123,6 @@ public Credentials credentials() { */ public abstract Optional endpointOverride(); - abstract S3Instructions withReadAheadCount(int readAheadCount); - public interface Builder { Builder regionName(String regionName); @@ -151,6 +149,20 @@ default Builder endpointOverride(String endpointOverride) { S3Instructions build(); } + abstract S3Instructions withReadAheadCount(int readAheadCount); + + abstract S3Instructions withFragmentSize(int fragmentSize); + + abstract S3Instructions withMaxCacheSize(int maxCacheSize); + + @Lazy + S3Instructions singleUse() { + final int readAheadCount = Math.min(DEFAULT_READ_AHEAD_COUNT, readAheadCount()); + return withReadAheadCount(readAheadCount) + .withFragmentSize(Math.min(SINGLE_USE_FRAGMENT_SIZE_DEFAULT, fragmentSize())) + .withMaxCacheSize(readAheadCount + 1); + } + @Check final void boundsCheckMaxConcurrentRequests() { if (maxConcurrentRequests() < 1) { diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index 7cdf79eb708..d4c743dc4ce 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -56,29 +56,29 @@ public void setContext(@Nullable final SeekableChannelContext channelContext) { } this.context = (S3ChannelContext) channelContext; if (this.context != null) { - this.context.assume(uri); + this.context.verifyOrSetUri(uri); if (size != UNINITIALIZED_SIZE) { - this.context.hackSize(size); + context.verifyOrSetSize(size); } } } - private long hackSize() throws IOException { - if (size != UNINITIALIZED_SIZE) { - return size; + private void prepareRead() throws IOException { + checkClosed(position); + Assert.neqNull(context, "channelContext"); + if (size == UNINITIALIZED_SIZE) { + size = context.size(); } - return (size = context.size(uri)); } @Override public int read(@NotNull final ByteBuffer destination) throws IOException { - Assert.neqNull(context, "channelContext"); - checkClosed(position); - if (position >= hackSize()) { + prepareRead(); + if (position >= size) { // We are finished reading return -1; } - final int filled = context.fill(uri, position, destination); + final int filled = context.fill(position, destination); position += filled; return filled; } @@ -90,26 +90,24 @@ public int write(final ByteBuffer src) { @Override public long position() throws ClosedChannelException { - final long localPosition = position; - checkClosed(localPosition); - return localPosition; + checkClosed(position); + return position; } @Override public SeekableByteChannel position(final long newPosition) throws ClosedChannelException { + checkClosed(position); if (newPosition < 0) { throw new IllegalArgumentException("newPosition cannot be < 0, provided newPosition=" + newPosition); } - checkClosed(position); position = newPosition; return this; } @Override public long size() throws IOException { - checkClosed(position); - Assert.neqNull(context, "channelContext"); - return context.size(uri); + prepareRead(); + return size; } @Override diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index 8e6227c5b8f..7739633bc3d 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -3,10 +3,11 @@ */ package io.deephaven.extensions.s3; +import io.deephaven.util.channel.Channels; import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.util.channel.SeekableChannelsProviderBase; import org.jetbrains.annotations.NotNull; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.core.retry.RetryMode; import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient; import software.amazon.awssdk.regions.Region; @@ -14,14 +15,15 @@ import software.amazon.awssdk.services.s3.S3AsyncClientBuilder; import software.amazon.awssdk.services.s3.S3Uri; +import java.io.InputStream; import java.net.URI; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; /** - * {@link SeekableChannelsProvider} implementation that is used to fetch objects from AWS S3 instances. + * {@link SeekableChannelsProvider} implementation that is used to fetch objects from an S3-compatible API. */ -final class S3SeekableChannelProvider extends SeekableChannelsProviderBase { +final class S3SeekableChannelProvider implements SeekableChannelsProvider { /** * We always allocate buffers of maximum allowed size for re-usability across reads with different fragment sizes. @@ -35,28 +37,30 @@ final class S3SeekableChannelProvider extends SeekableChannelsProviderBase { S3SeekableChannelProvider(@NotNull final S3Instructions s3Instructions) { // TODO(deephaven-core#5062): Add support for async client recovery and auto-close // TODO(deephaven-core#5063): Add support for caching clients for re-use + this.s3AsyncClient = buildClient(s3Instructions); + this.s3Instructions = s3Instructions; + } + + private static S3AsyncClient buildClient(@NotNull S3Instructions s3Instructions) { final S3AsyncClientBuilder builder = S3AsyncClient.builder() .httpClient(AwsCrtAsyncHttpClient.builder() .maxConcurrency(s3Instructions.maxConcurrentRequests()) .connectionTimeout(s3Instructions.connectionTimeout()) .build()) - .overrideConfiguration(b -> b - // .addMetricPublisher(LoggingMetricPublisher.create(Level.INFO, Format.PRETTY)) + .overrideConfiguration(ClientOverrideConfiguration.builder() + // If we find that the STANDARD retry policy does not work well in all situations, we might + // try experimenting with ADAPTIVE retry policy, potentially with fast fail. // .retryPolicy(RetryPolicy.builder(RetryMode.ADAPTIVE).fastFailRateLimiting(true).build()) .retryPolicy(RetryMode.STANDARD) .apiCallAttemptTimeout(s3Instructions.readTimeout().dividedBy(3)) - .apiCallTimeout(s3Instructions.readTimeout())) + .apiCallTimeout(s3Instructions.readTimeout()) + // Adding a metrics publisher may be useful for debugging, but it's very verbose. + // .addMetricPublisher(LoggingMetricPublisher.create(Level.INFO, Format.PRETTY)) + .build()) .region(Region.of(s3Instructions.regionName())) .credentialsProvider(s3Instructions.awsV2CredentialsProvider()); s3Instructions.endpointOverride().ifPresent(builder::endpointOverride); - this.s3AsyncClient = builder.build(); - this.s3Instructions = s3Instructions; - } - - @Override - protected boolean readChannelIsBuffered() { - // io.deephaven.extensions.s3.S3SeekableByteChannel is buffered based on context / options - return true; + return builder.build(); } @Override @@ -67,6 +71,12 @@ public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext return new S3SeekableByteChannel(s3Uri); } + @Override + public InputStream getInputStream(SeekableByteChannel channel) { + // S3SeekableByteChannel is internally buffered, no need to re-buffer + return Channels.newInputStreamNoClose(channel); + } + @Override public SeekableChannelContext makeContext() { return new S3ChannelContext(s3AsyncClient, s3Instructions, BUFFER_POOL); @@ -74,7 +84,7 @@ public SeekableChannelContext makeContext() { @Override public SeekableChannelContext makeSingleUseContext() { - return new S3ChannelContext(s3AsyncClient, s3Instructions.withReadAheadCount(0), BUFFER_POOL); + return new S3ChannelContext(s3AsyncClient, s3Instructions.singleUse(), BUFFER_POOL); } @Override diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index 215dcc3817d..fece237fa89 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -8,14 +8,16 @@ import io.deephaven.engine.util.file.FileHandleFactory; import io.deephaven.engine.util.file.TrackedFileHandleFactory; import io.deephaven.engine.util.file.TrackedSeekableByteChannel; +import io.deephaven.util.channel.Channels; import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.util.channel.SeekableChannelsProviderBase; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.io.BufferedInputStream; import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.net.URI; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -26,7 +28,7 @@ /** * {@link SeekableChannelsProvider} implementation that is constrained by a Deephaven {@link TrackedFileHandleFactory}. */ -final class TrackedSeekableChannelsProvider extends SeekableChannelsProviderBase { +final class TrackedSeekableChannelsProvider implements SeekableChannelsProvider { private final TrackedFileHandleFactory fileHandleFactory; @@ -34,13 +36,6 @@ final class TrackedSeekableChannelsProvider extends SeekableChannelsProviderBase this.fileHandleFactory = fileHandleFactory; } - @Override - protected boolean readChannelIsBuffered() { - // io.deephaven.engine.util.file.TrackedSeekableByteChannel / io.deephaven.engine.util.file.FileHandle is not - // buffered - return false; - } - @Override public SeekableChannelContext makeContext() { // No additional context required for local FS @@ -54,7 +49,7 @@ public boolean isCompatibleWith(@Nullable SeekableChannelContext channelContext) } @Override - public final SeekableByteChannel getReadChannel(@Nullable final SeekableChannelContext channelContext, + public SeekableByteChannel getReadChannel(@Nullable final SeekableChannelContext channelContext, @NotNull final URI uri) throws IOException { // context is unused here @@ -63,7 +58,13 @@ public final SeekableByteChannel getReadChannel(@Nullable final SeekableChannelC } @Override - public final SeekableByteChannel getWriteChannel(@NotNull final Path filePath, final boolean append) + public InputStream getInputStream(SeekableByteChannel channel) { + // TrackedSeekableByteChannel is not buffered, need to buffer + return new BufferedInputStream(Channels.newInputStreamNoClose(channel)); + } + + @Override + public SeekableByteChannel getWriteChannel(@NotNull final Path filePath, final boolean append) throws IOException { // NB: I'm not sure this is actually the intended behavior; the "truncate-once" is per-handle, not per file. return new TrackedSeekableByteChannel(append ? fileHandleFactory.writeAppendCreateHandleCreator From 3a35a56a64e9aec119f8fb55509c70c4136c8c80 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Tue, 13 Feb 2024 17:22:18 -0800 Subject: [PATCH 05/13] read timeout docs --- .../main/java/io/deephaven/extensions/s3/S3Instructions.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index e566afccf3c..59ee16c504c 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -100,7 +100,9 @@ public Duration connectionTimeout() { } /** - * The amount of time to wait when reading a fragment before giving up and timing out, defaults to 2 seconds + * The amount of time to wait when reading a fragment before giving up and timing out, defaults to 2 seconds. The + * implementation may choose to internally retry the request multiple times, so long as the total time does not + * exceed this timeout. */ @Default public Duration readTimeout() { From f71021f13a49dc9243862cff9534efbae4778b10 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Tue, 13 Feb 2024 20:10:59 -0800 Subject: [PATCH 06/13] small cleanup --- .../io/deephaven/util/channel/LocalFSChannelProvider.java | 3 +-- .../io/deephaven/parquet/base/ColumnChunkReaderImpl.java | 4 ++-- .../io/deephaven/parquet/base/ColumnPageReaderImpl.java | 7 +++---- .../java/io/deephaven/parquet/base/ParquetFileReader.java | 5 ++--- .../java/io/deephaven/extensions/s3/S3ChannelContext.java | 4 ++-- 5 files changed, 10 insertions(+), 13 deletions(-) diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java index 7f5c5d85461..cd5705b9ffc 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java @@ -10,7 +10,6 @@ import java.io.IOException; import java.io.InputStream; import java.net.URI; -import java.nio.channels.Channels; import java.nio.channels.FileChannel; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -40,7 +39,7 @@ public SeekableByteChannel getReadChannel(@Nullable final SeekableChannelContext @Override public InputStream getInputStream(SeekableByteChannel channel) { // FileChannel is not buffered, need to buffer - return new BufferedInputStream(Channels.newInputStream(ReadableByteChannelNoClose.of(channel))); + return new BufferedInputStream(Channels.newInputStreamNoClose(channel)); } @Override diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 96f7b9b6e94..ee77e129db1 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java @@ -221,8 +221,8 @@ private Dictionary readDictionary(InputStream in) throws IOException { } final Encoding encoding = Encoding.valueOf(dictHeader.getEncoding().name()); final DictionaryPage dictionaryPage = new DictionaryPage(payload, dictHeader.getNum_values(), encoding); - // We are safe here because the Dictionary doesn't hold a reference to payload (and thus implicitly, doesn't - // hold a reference to the input stream). + // We are safe to not copy the payload because the Dictionary doesn't hold a reference to dictionaryPage or + // payload and thus doesn't hold a reference to the input stream. return encoding.initDictionary(path, dictionaryPage); } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java index 04d265974be..743926ea86c 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java @@ -127,7 +127,7 @@ public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder, final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); final SeekableByteChannel ch = channelsProvider.getReadChannel(holder.get(), uri)) { ensurePageHeader(channelsProvider, ch); - return readKeyFromDataPage(keyDest, nullPlaceholder, ch, holder.get()); + return readKeysFromDataPage(keyDest, nullPlaceholder, ch, holder.get()); } } @@ -243,9 +243,8 @@ private int readRowCountFromDataPage(SeekableByteChannel ch) throws IOException } } - private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, - SeekableByteChannel ch, @NotNull final SeekableChannelContext channelContext) - throws IOException { + private IntBuffer readKeysFromDataPage(IntBuffer keyDest, int nullPlaceholder, SeekableByteChannel ch, + @NotNull final SeekableChannelContext channelContext) throws IOException { switch (pageHeader.type) { case DATA_PAGE: try (final InputStream in = channelsProvider.getInputStream(ch)) { diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java index e057047bf23..8c8ed1112fe 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java @@ -79,9 +79,9 @@ private static void positionToFileMetadata(URI parquetFileURI, SeekableByteChann if (!Arrays.equals(MAGIC, 0, MAGIC.length, trailer, Integer.BYTES, trailer.length)) { throw new InvalidParquetFileException( parquetFileURI + " is not a Parquet file. expected magic number at tail " + Arrays.toString(MAGIC) - + " but found " + Arrays.toString(Arrays.copyOfRange(trailer, 4, 8))); + + " but found " + + Arrays.toString(Arrays.copyOfRange(trailer, Integer.BYTES, trailer.length))); } - // final int footerLength = ByteBuffer.wrap(trailer, 0, Integer.BYTES).order(ByteOrder.LITTLE_ENDIAN).getInt(); final int footerLength = makeLittleEndianInt(trailer[0], trailer[1], trailer[2], trailer[3]); final long footerIndex = footerLengthIndex - footerLength; if (footerIndex < MAGIC.length || footerIndex >= footerLengthIndex) { @@ -89,7 +89,6 @@ private static void positionToFileMetadata(URI parquetFileURI, SeekableByteChann "corrupted file: the footer index is not within the file: " + footerIndex); } readChannel.position(footerIndex); - // return footerLength; } private static int makeLittleEndianInt(byte b0, byte b1, byte b2, byte b3) { diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java index d3e34aa68b1..b29ed8b606f 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -71,7 +71,7 @@ void verifyOrSetUri(S3Uri uri) { this.uri = Objects.requireNonNull(uri); } else if (!this.uri.equals(uri)) { throw new IllegalStateException( - String.format("Inconsistent URIs. expected=%s, actual=%s", this.uri, uri)); + String.format("Inconsistent URIs. expected=%s, actual=%s, ctx=%s", this.uri, uri, ctxStr())); } } @@ -80,7 +80,7 @@ void verifyOrSetSize(long size) { setSize(size); } else if (this.size != size) { throw new IllegalStateException( - String.format("Inconsistent size. expected=%d, actual=%d", size, this.size)); + String.format("Inconsistent size. expected=%d, actual=%d, ctx=%s", size, this.size, ctxStr())); } } From 3f32065632330498686747235d7cf9890892b5b1 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Wed, 14 Feb 2024 11:08:58 -0800 Subject: [PATCH 07/13] Slightly better flipping --- .../deephaven/extensions/s3/S3ChannelContext.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java index b29ed8b606f..b39d8ab1e78 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -375,13 +375,17 @@ public synchronized void onError(Throwable t) { @Override public synchronized void onComplete() { - buffer.flip(); - if (buffer.limit() != requestLength()) { + if (buffer.position() != requestLength()) { localProducer.completeExceptionally(new IllegalStateException(String.format( - "Expected %d bytes, received %d, %s", requestLength(), buffer.limit(), requestStr()))); - } else { - localProducer.complete(buffer.slice().asReadOnlyBuffer()); + "Expected %d bytes, received %d, %s", requestLength(), buffer.position(), requestStr()))); + return; + } + ByteBuffer toComplete = buffer.asReadOnlyBuffer(); + toComplete.flip(); + if (toComplete.capacity() != toComplete.limit()) { + toComplete = toComplete.slice(); } + localProducer.complete(toComplete); } } } From 5f0e8779a0041b44862452163f1c1d9972097b89 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Wed, 14 Feb 2024 11:17:55 -0800 Subject: [PATCH 08/13] Remove a little bit of Optional alloc --- .../java/io/deephaven/extensions/s3/S3ChannelContext.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java index b39d8ab1e78..c80d8a80329 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -115,7 +115,7 @@ public int fill(final long position, ByteBuffer dest) throws IOException { // blocking int filled = firstRequest.fill(position, dest); for (int i = 0; dest.hasRemaining(); ++i) { - final Request request = getRequest(firstFragmentIx + i + 1).orElse(null); + final Request request = getRequest(firstFragmentIx + i + 1); if (request == null || !request.isDone()) { break; } @@ -141,12 +141,10 @@ public void close() { // -------------------------------------------------------------------------------------------------- - private Optional getRequest(final long fragmentIndex) { + private Request getRequest(final long fragmentIndex) { final int cacheIdx = cacheIndex(fragmentIndex); final Request request = requests[cacheIdx]; - return request == null || !request.isFragment(fragmentIndex) - ? Optional.empty() - : Optional.of(request); + return request == null || !request.isFragment(fragmentIndex) ? null : request; } private Request getOrCreateRequest(final long fragmentIndex) { From 897064d4c4dd35de065cc9247e337070667e285e Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 14 Feb 2024 00:03:25 -0500 Subject: [PATCH 09/13] Port PooledObjectReference, and experimentally integrate it with S3ChannelContext --- .../base/reference/PooledObjectReference.java | 201 ++++++++++++++++++ .../referencecounting/ReferenceCounted.java | 5 +- .../deephaven/extensions/s3/BufferPool.java | 24 ++- .../extensions/s3/S3ChannelContext.java | 112 +++++----- 4 files changed, 285 insertions(+), 57 deletions(-) create mode 100644 Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java diff --git a/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java b/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java new file mode 100644 index 00000000000..bf754bc2dc3 --- /dev/null +++ b/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java @@ -0,0 +1,201 @@ +package io.deephaven.base.reference; + +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Objects; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; + +/** + * {@link SimpleReference} implementation with built-in reference-counting and pooling support. + */ +public abstract class PooledObjectReference implements SimpleReference, AutoCloseable { + + /** + * Field updater for {@code state}. + */ + @SuppressWarnings("rawtypes") + private static final AtomicIntegerFieldUpdater STATE_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PooledObjectReference.class, "state"); + + /** + * An available reference with zero outstanding permits will have {@code state == 1}. + */ + private static final int AVAILABLE_ZERO_PERMITS = 1; + + /** + * A cleared reference with zero outstanding permits will have {@code state == 0}. + */ + private static final int CLEARED_ZERO_PERMITS = 0; + + /** + * The bit we use to denote availability. + */ + private static final int STATE_AVAILABLE_BIT = 1; + + /** + * The quantity to add to state when incrementing the number of outstanding permits. + */ + private static final int STATE_PERMIT_ACQUIRE_QUANTITY = 2; + + /** + * The quantity to add to state when decrementing the number of outstanding permits. + */ + private static final int STATE_PERMIT_RELEASE_QUANTITY = -STATE_PERMIT_ACQUIRE_QUANTITY; + + private static boolean stateAllowsAcquire(final int currentState) { + return currentState != CLEARED_ZERO_PERMITS; + } + + private static boolean stateIsAvailable(final int currentState) { + return (currentState & STATE_AVAILABLE_BIT) != 0; + } + + private static boolean stateIsCleared(final int currentState) { + return (currentState & STATE_AVAILABLE_BIT) == 0; + } + + private static int calculateNewStateForClear(final int currentState) { + return currentState ^ STATE_AVAILABLE_BIT; + } + + private static int calculateNewStateForAcquire(final int currentState) { + return currentState + STATE_PERMIT_ACQUIRE_QUANTITY; + } + + /** + * Try to atomically update {@code state}. + * + * @param currentState The expected value + * @param newState The desired result value + * @return Whether {@code state} was successfully updated + */ + private boolean tryUpdateState(final int currentState, final int newState) { + return STATE_UPDATER.compareAndSet(this, currentState, newState); + } + + /** + * Atomically decrement the number of outstanding permits and get the new value of {@code state}. + * + * @return The new value of {@code state} + */ + private int decrementOutstandingPermits() { + return STATE_UPDATER.addAndGet(this, STATE_PERMIT_RELEASE_QUANTITY); + } + + /** + * The actual referent. Set to null after {@code state == CLEARED_ZERO_PERMITS} by the responsible thread, which + * returns it to the pool. + */ + private volatile REFERENT_TYPE referent; + + /** + * The state of this reference. The lowest bit is used to denote whether this reference is available (1) or cleared + * (0). The higher bits represent an integer count of the number of outstanding permits. + */ + private volatile int state = AVAILABLE_ZERO_PERMITS; + + /** + * Construct a new PooledObjectReference to the supplied referent. + * + * @param referent The referent of this reference + */ + protected PooledObjectReference(@NotNull final REFERENT_TYPE referent) { + this.referent = Objects.requireNonNull(referent, "referent"); + } + + /** + * Get the referent. It is an error to call this method if the caller does not have any outstanding permits. + * + * @return The referent if this reference has not been cleared, null otherwise (which implies an error by the + * caller) + */ + @Override + public final REFERENT_TYPE get() { + onReferentAccessed(); + return referent; + } + + /** + * Callback for accounting purposes, e.g. in support of a LRU policy for pooled item reclamation. + */ + protected void onReferentAccessed() {} + + /** + * Acquire an active use permit. + * + * @return Whether a permit was acquired + */ + public final boolean acquire() { + int currentState; + while (stateAllowsAcquire(currentState = state)) { + final int newState = calculateNewStateForAcquire(currentState); + if (tryUpdateState(currentState, newState)) { + return true; + } + } + return false; + } + + /** + * Acquire an active use permit and return the referent, if possible. + * + * @return The referent, or null if no permit could be acquired + */ + @Nullable + public final REFERENT_TYPE acquireAndGet() { + if (acquire()) { + return get(); + } + return null; + } + + /** + * Release an active use permit. It is a serious error to release more permits than acquired. + */ + public final void release() { + final int newState = decrementOutstandingPermits(); + if (newState < 0) { + throw new IllegalStateException(this + " released more than acquired"); + } + if (!stateAllowsAcquire(newState)) { + returnReferentToPool(referent); + referent = null; + } + } + + /** + * Clear this reference (and return its referent to the pool) when it no longer has any outstanding permits, which + * may mean immediately if the number of outstanding permits is already zero. All invocations after the first will + * have no effect. + */ + @Override + public final void clear() { + int currentState; + while (stateIsAvailable(currentState = state)) { + final int newState = calculateNewStateForClear(currentState); + if (tryUpdateState(currentState, newState)) { + if (!stateAllowsAcquire(newState)) { + returnReferentToPool(referent); + referent = null; + } + return; + } + } + } + + /** + * Return the referent to the pool. + * + * @param referent The referent to return + */ + protected abstract void returnReferentToPool(@NotNull REFERENT_TYPE referent); + + /** + * Synonym for {@link #release()}, intended for use as an {@link AutoCloseable} in a try-with-resources block. + */ + @Override + public final void close() { + release(); + } +} diff --git a/Util/src/main/java/io/deephaven/util/referencecounting/ReferenceCounted.java b/Util/src/main/java/io/deephaven/util/referencecounting/ReferenceCounted.java index 4969609d802..c33a09ead11 100644 --- a/Util/src/main/java/io/deephaven/util/referencecounting/ReferenceCounted.java +++ b/Util/src/main/java/io/deephaven/util/referencecounting/ReferenceCounted.java @@ -10,16 +10,13 @@ import org.jetbrains.annotations.NotNull; import java.io.ObjectInputStream; -import java.io.Serializable; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; /** * Implements a recurring reference counting pattern - a concurrent reference count that should refuse to go below zero, * and invokes {@link #onReferenceCountAtZero()} exactly once when the count returns to zero. */ -public abstract class ReferenceCounted implements LogOutputAppendable, Serializable { - - private static final long serialVersionUID = 1L; +public abstract class ReferenceCounted implements LogOutputAppendable { /** * Field updater for referenceCount, so we can avoid creating an {@link java.util.concurrent.atomic.AtomicInteger} diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java index d6e97ec9700..bd8a0f8de61 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java @@ -1,6 +1,10 @@ package io.deephaven.extensions.s3; +import io.deephaven.base.reference.PooledObjectReference; +import io.deephaven.base.reference.SimpleReference; import io.deephaven.util.datastructures.SegmentedSoftPool; +import io.deephaven.util.referencecounting.ReferenceCounted; +import org.jetbrains.annotations.NotNull; import java.nio.ByteBuffer; @@ -21,14 +25,28 @@ final class BufferPool { ByteBuffer::clear); } - public ByteBuffer take(final int size) { + public PooledObjectReference take(final int size) { if (size > bufferSize) { throw new IllegalArgumentException("Buffer size " + size + " is larger than pool size " + bufferSize); } - return pool.take(); + return new BufferReference(pool.take()); } - public void give(ByteBuffer buffer) { + private void give(ByteBuffer buffer) { pool.give(buffer); } + + final class BufferReference extends PooledObjectReference { + + private volatile ByteBuffer buffer; + + BufferReference(@NotNull final ByteBuffer buffer) { + super(buffer); + } + + @Override + protected void returnReferentToPool(@NotNull ByteBuffer referent) { + give(referent); + } + } } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java index c80d8a80329..7c694718749 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -1,5 +1,6 @@ package io.deephaven.extensions.s3; +import io.deephaven.base.reference.PooledObjectReference; import io.deephaven.util.channel.SeekableChannelContext; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; @@ -189,7 +190,7 @@ final class Request private final long from; private final long to; private final Instant createdAt; - private final CompletableFuture released; + private final PooledObjectReference bufferReference; private CompletableFuture consumerFuture; private volatile CompletableFuture producerFuture; private int fillCount; @@ -200,7 +201,7 @@ private Request(long fragmentIndex) { this.fragmentIndex = fragmentIndex; from = fragmentIndex * instructions.fragmentSize(); to = Math.min(from + instructions.fragmentSize(), size) - 1; - released = new CompletableFuture<>(); + bufferReference = bufferPool.take(requestLength()); } void init() { @@ -216,35 +217,40 @@ public boolean isDone() { } public int fill(long localPosition, ByteBuffer dest) throws IOException { + if (!bufferReference.acquire()) { + throw new IllegalStateException(String.format("Trying to get data after release, %s", requestStr())); + } final int resultOffset = (int) (localPosition - from); final int resultLength = Math.min((int) (to - localPosition + 1), dest.remaining()); - final ByteBuffer fullFragment; - try { - fullFragment = getFullFragment(); - } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { - throw handleS3Exception(e, String.format("fetching fragment %s", requestStr())); - } - // fullFragment is a slice. Lets us have safety around math and ability to simply clear to reset. - fullFragment.limit(resultOffset + resultLength); - fullFragment.position(resultOffset); - try { - dest.put(fullFragment); - } finally { - fullFragment.clear(); + try (final PooledObjectReference ignored = bufferReference) { + final ByteBuffer fullFragment; + try { + fullFragment = getFullFragment(); + } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { + throw handleS3Exception(e, String.format("fetching fragment %s", requestStr())); + } + // fullFragment has limit == capacity. This lets us have safety around math and ability to simply + // clear to reset. + fullFragment.limit(resultOffset + resultLength); + fullFragment.position(resultOffset); + try { + dest.put(fullFragment); + } finally { + fullFragment.clear(); + } + ++fillCount; + fillBytes += resultLength; } - ++fillCount; - fillBytes += resultLength; return resultLength; } public void release() { + bufferReference.clear(); final boolean didCancel = consumerFuture.cancel(true); if (log.isDebugEnabled()) { final String cancelType = didCancel ? "fast" : (fillCount == 0 ? "unused" : "normal"); log.debug("cancel {}: {} fillCount={}, fillBytes={}", cancelType, requestStr(), fillCount, fillBytes); } - // Finishing via exception to ensure downstream subscribers can cleanup if SDK didn't complete them - released.cancel(true); } // -------------------------------------------------------------------------------------------------- @@ -288,9 +294,6 @@ public void exceptionOccurred(Throwable error) { // -------------------------------------------------------------------------------------------------- private ByteBuffer getFullFragment() throws ExecutionException, InterruptedException, TimeoutException { - if (released.isDone()) { - throw new IllegalStateException(String.format("Trying to get data after release, %s", requestStr())); - } // Giving our own get() a bit of overhead - the clients should already be constructed with appropriate // apiCallTimeout. final long readNanos = instructions.readTimeout().plusMillis(100).toNanos(); @@ -326,32 +329,26 @@ private String requestStr() { // -------------------------------------------------------------------------------------------------- - final class Sub implements Subscriber, BiConsumer { + final class Sub implements Subscriber { private final CompletableFuture localProducer; - private ByteBuffer buffer; + private ByteBuffer bufferView; private Subscription subscription; Sub() { localProducer = producerFuture; - buffer = bufferPool.take(requestLength()); - // 1. localProducer succeeds: whenComplete will be executed when released - // 2. localProducer fails: whenComplete will be executed asap - // 3. localProducer limbo (not expected): whenComplete will be executed when released - CompletableFuture.allOf(localProducer, released).whenComplete(this); - } - - // ----------------------------------------------------------------------------- - - @Override - public synchronized void accept(Void unused, Throwable throwable) { - bufferPool.give(buffer); - buffer = null; + if (!bufferReference.acquire()) { + localProducer.completeExceptionally(new IllegalStateException( + String.format("Failed to acquire buffer for new subscriber, %s", requestStr()))); + } + try (final PooledObjectReference ignored = bufferReference) { + bufferView = bufferReference.get().duplicate(); + } } // ----------------------------------------------------------------------------- @Override - public synchronized void onSubscribe(Subscription s) { + public void onSubscribe(Subscription s) { if (subscription != null) { s.cancel(); return; @@ -361,29 +358,44 @@ public synchronized void onSubscribe(Subscription s) { } @Override - public synchronized void onNext(ByteBuffer byteBuffer) { - buffer.put(byteBuffer); + public void onNext(ByteBuffer byteBuffer) { + if (!bufferReference.acquire()) { + localProducer.completeExceptionally(new IllegalStateException( + String.format("Failed to acquire buffer for data, %s", requestStr()))); + } + try (final PooledObjectReference ignored = bufferReference) { + bufferView.put(byteBuffer); + } subscription.request(1); } @Override - public synchronized void onError(Throwable t) { + public void onError(Throwable t) { localProducer.completeExceptionally(t); } @Override - public synchronized void onComplete() { - if (buffer.position() != requestLength()) { - localProducer.completeExceptionally(new IllegalStateException(String.format( - "Expected %d bytes, received %d, %s", requestLength(), buffer.position(), requestStr()))); + public void onComplete() { + if (!bufferReference.acquire()) { + localProducer.completeExceptionally(new IllegalStateException( + String.format("Failed to acquire buffer for completion, %s", requestStr()))); return; } - ByteBuffer toComplete = buffer.asReadOnlyBuffer(); - toComplete.flip(); - if (toComplete.capacity() != toComplete.limit()) { - toComplete = toComplete.slice(); + try (final PooledObjectReference ignored = bufferReference) { + if (bufferView.position() != requestLength()) { + localProducer.completeExceptionally(new IllegalStateException(String.format( + "Expected %d bytes, received %d, %s", requestLength(), bufferView.position(), + requestStr()))); + return; + } + ByteBuffer toComplete = bufferView.asReadOnlyBuffer(); + toComplete.flip(); + if (toComplete.capacity() != toComplete.limit()) { + toComplete = toComplete.slice(); + } + localProducer.complete(toComplete); } - localProducer.complete(toComplete); + } } } From edda9a87597ddaa59da17752899c03d46e27e27b Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Wed, 14 Feb 2024 09:13:55 -0800 Subject: [PATCH 10/13] Some updates --- .../base/reference/PooledObjectReference.java | 130 +++++++++--- .../reference/PooledObjectReferenceTest.java | 197 ++++++++++++++++++ .../deephaven/extensions/s3/BufferPool.java | 2 - .../extensions/s3/S3ChannelContext.java | 46 ++-- 4 files changed, 330 insertions(+), 45 deletions(-) create mode 100644 Base/src/test/java/io/deephaven/base/reference/PooledObjectReferenceTest.java diff --git a/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java b/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java index bf754bc2dc3..50b28d9a145 100644 --- a/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java +++ b/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java @@ -3,13 +3,14 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.io.Closeable; import java.util.Objects; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; /** * {@link SimpleReference} implementation with built-in reference-counting and pooling support. */ -public abstract class PooledObjectReference implements SimpleReference, AutoCloseable { +public abstract class PooledObjectReference implements SimpleReference { /** * Field updater for {@code state}. @@ -44,7 +45,7 @@ public abstract class PooledObjectReference implements SimpleRefe private static final int STATE_PERMIT_RELEASE_QUANTITY = -STATE_PERMIT_ACQUIRE_QUANTITY; private static boolean stateAllowsAcquire(final int currentState) { - return currentState != CLEARED_ZERO_PERMITS; + return currentState > CLEARED_ZERO_PERMITS; } private static boolean stateIsAvailable(final int currentState) { @@ -105,30 +106,78 @@ protected PooledObjectReference(@NotNull final REFERENT_TYPE referent) { } /** - * Get the referent. It is an error to call this method if the caller does not have any outstanding permits. + * Get the referent. It is an error to call this method if the caller does not have any outstanding permits. Callers + * are encouraged to use this in the try block of a try-finally pattern: + * + *

+     * if (!ref.acquire()) {
+     *     return;
+     * }
+     * try {
+     *     doSomethingWith(ref.get());
+     * } finally {
+     *     ref.release();
+     * }
+     * 
* * @return The referent if this reference has not been cleared, null otherwise (which implies an error by the * caller) */ @Override + @Nullable public final REFERENT_TYPE get() { - onReferentAccessed(); return referent; } /** - * Callback for accounting purposes, e.g. in support of a LRU policy for pooled item reclamation. + * Acquire an active use permit. Callers should pair this with a corresponding {@link #release()}, ideally with a + * try-finally pattern: + * + *
+     * if (!ref.acquire()) {
+     *     return;
+     * }
+     * try {
+     *     doSomethingWith(ref.get());
+     * } finally {
+     *     ref.release();
+     * }
+     * 
+ * + * @return Whether a permit was acquired */ - protected void onReferentAccessed() {} + public final boolean acquire() { + int currentState; + while (stateAllowsAcquire(currentState = state)) { + final int newState = calculateNewStateForAcquire(currentState); + if (tryUpdateState(currentState, newState)) { + return true; + } + } + return false; + } /** - * Acquire an active use permit. + * Acquire an active use permit if this has not been {@link #clear() cleared}. This is useful in situations where + * callers want to fail-fast and don't need to guarantee re-entrancy. Callers should pair this with a corresponding + * {@link #release()}, ideally with a try-finally pattern: + * + *
+     * if (!ref.acquireIfAvailable()) {
+     *     return;
+     * }
+     * try {
+     *     doSomethingWith(ref.get());
+     * } finally {
+     *     ref.release();
+     * }
+     * 
* * @return Whether a permit was acquired */ - public final boolean acquire() { + public final boolean acquireIfAvailable() { int currentState; - while (stateAllowsAcquire(currentState = state)) { + while (stateAllowsAcquire(currentState = state) && stateIsAvailable(currentState)) { final int newState = calculateNewStateForAcquire(currentState); if (tryUpdateState(currentState, newState)) { return true; @@ -138,30 +187,57 @@ public final boolean acquire() { } /** - * Acquire an active use permit and return the referent, if possible. + * Acquire an active use permit and return the referent, if possible. Callers should pair this with a corresponding + * {@link #release()}, ideally with a try-finally pattern: + * + *
+     * final Object obj;
+     * if ((obj = ref.acquireAndGet()) == null) {
+     *     return;
+     * }
+     * try {
+     *     doSomethingWith(obj);
+     * } finally {
+     *     ref.release();
+     * }
+     * 
* * @return The referent, or null if no permit could be acquired */ @Nullable public final REFERENT_TYPE acquireAndGet() { - if (acquire()) { - return get(); + if (!acquire()) { + return null; + } + final REFERENT_TYPE localReferent = referent; + if (localReferent == null) { + release(); + throw new IllegalStateException(this + " acquired, but referent is null"); } - return null; + return localReferent; } /** - * Release an active use permit. It is a serious error to release more permits than acquired. + * Release a single active use permit. It is a serious error to release more permits than acquired. Callers are + * encouraged to use this in the finally block of a try-finally pattern: + * + *
+     * if (!ref.acquire()) {
+     *     return;
+     * }
+     * try {
+     *     doSomethingWith(ref.get());
+     * } finally {
+     *     ref.release();
+     * }
+     * 
*/ public final void release() { final int newState = decrementOutstandingPermits(); if (newState < 0) { throw new IllegalStateException(this + " released more than acquired"); } - if (!stateAllowsAcquire(newState)) { - returnReferentToPool(referent); - referent = null; - } + maybeReturnReferentToPool(newState); } /** @@ -175,10 +251,7 @@ public final void clear() { while (stateIsAvailable(currentState = state)) { final int newState = calculateNewStateForClear(currentState); if (tryUpdateState(currentState, newState)) { - if (!stateAllowsAcquire(newState)) { - returnReferentToPool(referent); - referent = null; - } + maybeReturnReferentToPool(newState); return; } } @@ -191,11 +264,12 @@ public final void clear() { */ protected abstract void returnReferentToPool(@NotNull REFERENT_TYPE referent); - /** - * Synonym for {@link #release()}, intended for use as an {@link AutoCloseable} in a try-with-resources block. - */ - @Override - public final void close() { - release(); + private void maybeReturnReferentToPool(int newState) { + if (stateAllowsAcquire(newState)) { + return; + } + final REFERENT_TYPE localReferent = referent; + referent = null; + returnReferentToPool(localReferent); } } diff --git a/Base/src/test/java/io/deephaven/base/reference/PooledObjectReferenceTest.java b/Base/src/test/java/io/deephaven/base/reference/PooledObjectReferenceTest.java new file mode 100644 index 00000000000..6c3bab79cc1 --- /dev/null +++ b/Base/src/test/java/io/deephaven/base/reference/PooledObjectReferenceTest.java @@ -0,0 +1,197 @@ +/** + * Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.base.reference; + +import org.jetbrains.annotations.NotNull; +import org.junit.Test; + +import java.util.concurrent.Callable; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class PooledObjectReferenceTest { + private final Object obj = new Object(); + private final MyReference ref = new MyReference(obj); + + @Test + public void clear() { + assertNotReturned(false); + ref.clear(); + assertReturned(); + } + + @Test + public void clearClear() { + assertNotReturned(false); + ref.clear(); + ref.clear(); + assertReturned(); + } + + @Test + public void acquireClear() throws Exception { + withAcquireIfAvailable(() -> { + assertNotReturned(false); + ref.clear(); + assertNotReturned(true); + return null; + }); + assertReturned(); + } + + @Test + public void acquireAcquireClear() throws Exception { + withAcquireIfAvailable(() -> { + assertNotReturned(false); + withAcquireIfAvailable(() -> { + assertNotReturned(false); + ref.clear(); + assertNotReturned(true); + return null; + }); + assertNotReturned(true); + return null; + }); + assertReturned(); + } + + @Test + public void acquireClearAcquire() throws Exception { + withAcquireIfAvailable(() -> { + assertNotReturned(false); + ref.clear(); + withAcquire(() -> { + assertNotReturned(true); + return null; + }); + assertNotReturned(true); + return null; + }); + assertReturned(); + } + + @Test + public void illegalRelease() { + try { + ref.release(); + fail("Expected IllegalStateException"); + } catch (IllegalStateException e) { + // expected + } + assertNoAcquire(); + ref.clear(); + assertReturned(); + } + + @Test + public void illegalGetClose() { + // Even though illegal, no errors and eq obj + assertEquals(obj, ref.get()); + ref.clear(); + assertReturned(); + } + + @Test + public void closeIllegalGet() { + ref.clear(); + assertReturned(); + // Even though illegal, no errors and eq null + assertNull(ref.get()); + } + + @Test + public void closeWithErrorIllegalGet() { + ref.throwOnReturnToPool(); + try { + ref.clear(); + fail("Expected ThrowOnReturnToPool"); + } catch (ThrowOnReturnToPool e) { + // expected + } + // Even though pool had error, we are no longer able to acquire, and get is null + assertNoAcquire(); + // Even though illegal, no errors and eq null + assertNull(ref.get()); + } + + void withAcquire(Callable callable) throws Exception { + assertTrue(ref.acquire()); + try { + assertEquals(obj, ref.get()); + callable.call(); + } finally { + ref.release(); + } + } + + void withAcquireIfAvailable(Callable callable) throws Exception { + // acquireIfAvailable is stricter than acquire, so acquire should always succeed here + assertTrue(ref.acquire()); + ref.release(); + + assertTrue(ref.acquireIfAvailable()); + try { + assertEquals(obj, ref.get()); + callable.call(); + } finally { + ref.release(); + } + } + + void assertReturned() { + assertTrue("ref.returned", ref.returned); + assertFalse(ref.acquire()); + assertFalse(ref.acquireIfAvailable()); + assertNull(ref.acquireAndGet()); + // Even though illegal, no errors and eq null + assertNull(ref.get()); + } + + void assertNotReturned(boolean isCleared) { + assertFalse("ref.returned", ref.returned); + // We know we can acquire + assertTrue(ref.acquire()); + ref.release(); + if (isCleared) { + assertFalse(ref.acquireIfAvailable()); + } else { + assertTrue(ref.acquireIfAvailable()); + ref.release(); + } + } + + void assertNoAcquire() { + assertFalse(ref.acquire()); + assertFalse(ref.acquireIfAvailable()); + } + + static final class MyReference extends PooledObjectReference { + private boolean returned; + private boolean throwOnReturnToPool; + + MyReference(@NotNull final Object buffer) { + super(buffer); + } + + public void throwOnReturnToPool() { + throwOnReturnToPool = true; + } + + @Override + protected void returnReferentToPool(@NotNull Object referent) { + if (throwOnReturnToPool) { + throw new ThrowOnReturnToPool(); + } + returned = true; + } + } + + static final class ThrowOnReturnToPool extends RuntimeException { + + } +} diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java index bd8a0f8de61..b528fbf8ef5 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java @@ -38,8 +38,6 @@ private void give(ByteBuffer buffer) { final class BufferReference extends PooledObjectReference { - private volatile ByteBuffer buffer; - BufferReference(@NotNull final ByteBuffer buffer) { super(buffer); } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java index 7c694718749..285b77e1b48 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -217,12 +217,12 @@ public boolean isDone() { } public int fill(long localPosition, ByteBuffer dest) throws IOException { - if (!bufferReference.acquire()) { - throw new IllegalStateException(String.format("Trying to get data after release, %s", requestStr())); - } final int resultOffset = (int) (localPosition - from); final int resultLength = Math.min((int) (to - localPosition + 1), dest.remaining()); - try (final PooledObjectReference ignored = bufferReference) { + if (!bufferReference.acquireIfAvailable()) { + throw new IllegalStateException(String.format("Trying to get data after release, %s", requestStr())); + } + try { final ByteBuffer fullFragment; try { fullFragment = getFullFragment(); @@ -240,13 +240,15 @@ public int fill(long localPosition, ByteBuffer dest) throws IOException { } ++fillCount; fillBytes += resultLength; + } finally { + bufferReference.release(); } return resultLength; } public void release() { - bufferReference.clear(); final boolean didCancel = consumerFuture.cancel(true); + bufferReference.clear(); if (log.isDebugEnabled()) { final String cancelType = didCancel ? "fast" : (fillCount == 0 ? "unused" : "normal"); log.debug("cancel {}: {} fillCount={}, fillBytes={}", cancelType, requestStr(), fillCount, fillBytes); @@ -331,24 +333,34 @@ private String requestStr() { final class Sub implements Subscriber { private final CompletableFuture localProducer; - private ByteBuffer bufferView; + // Access to this view must be guarded by bufferReference.acquire + private final ByteBuffer bufferView; private Subscription subscription; Sub() { localProducer = producerFuture; - if (!bufferReference.acquire()) { + final ByteBuffer get; + if ((get = bufferReference.acquireAndGet()) == null) { + bufferView = null; localProducer.completeExceptionally(new IllegalStateException( String.format("Failed to acquire buffer for new subscriber, %s", requestStr()))); + return; } - try (final PooledObjectReference ignored = bufferReference) { - bufferView = bufferReference.get().duplicate(); + try { + bufferView = get.duplicate(); + } finally { + bufferReference.release(); } } - // ----------------------------------------------------------------------------- + // ---------------------------------------------------- ------------------------- @Override public void onSubscribe(Subscription s) { + if (bufferView == null) { + s.cancel(); + return; + } if (subscription != null) { s.cancel(); return; @@ -359,12 +371,15 @@ public void onSubscribe(Subscription s) { @Override public void onNext(ByteBuffer byteBuffer) { - if (!bufferReference.acquire()) { + if (!bufferReference.acquireIfAvailable()) { localProducer.completeExceptionally(new IllegalStateException( String.format("Failed to acquire buffer for data, %s", requestStr()))); + return; } - try (final PooledObjectReference ignored = bufferReference) { + try { bufferView.put(byteBuffer); + } finally { + bufferReference.release(); } subscription.request(1); } @@ -376,12 +391,12 @@ public void onError(Throwable t) { @Override public void onComplete() { - if (!bufferReference.acquire()) { + if (!bufferReference.acquireIfAvailable()) { localProducer.completeExceptionally(new IllegalStateException( String.format("Failed to acquire buffer for completion, %s", requestStr()))); return; } - try (final PooledObjectReference ignored = bufferReference) { + try { if (bufferView.position() != requestLength()) { localProducer.completeExceptionally(new IllegalStateException(String.format( "Expected %d bytes, received %d, %s", requestLength(), bufferView.position(), @@ -394,8 +409,9 @@ public void onComplete() { toComplete = toComplete.slice(); } localProducer.complete(toComplete); + } finally { + bufferReference.release(); } - } } } From f037b3f0be660fc9116f0fe2e7946e34147bf03e Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Wed, 14 Feb 2024 13:20:49 -0800 Subject: [PATCH 11/13] Use acquireIfAvailable --- .../java/io/deephaven/extensions/s3/S3ChannelContext.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java index 285b77e1b48..7b4c457c5a8 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -339,15 +339,15 @@ final class Sub implements Subscriber { Sub() { localProducer = producerFuture; - final ByteBuffer get; - if ((get = bufferReference.acquireAndGet()) == null) { + if (!bufferReference.acquireIfAvailable()) { bufferView = null; localProducer.completeExceptionally(new IllegalStateException( String.format("Failed to acquire buffer for new subscriber, %s", requestStr()))); return; } try { - bufferView = get.duplicate(); + // noinspection DataFlowIssue + bufferView = bufferReference.get().duplicate(); } finally { bufferReference.release(); } From 458d29d5abb28c3c9e533daeaf8cc6c3a0984488 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Wed, 14 Feb 2024 13:51:12 -0800 Subject: [PATCH 12/13] responses --- .../base/reference/PooledObjectReference.java | 14 ++------------ .../base/reference/PooledObjectReferenceTest.java | 6 +++--- .../deephaven/extensions/s3/S3ChannelContext.java | 7 +++++-- 3 files changed, 10 insertions(+), 17 deletions(-) diff --git a/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java b/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java index 50b28d9a145..0587afbfcc9 100644 --- a/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java +++ b/Base/src/main/java/io/deephaven/base/reference/PooledObjectReference.java @@ -3,7 +3,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.io.Closeable; import java.util.Objects; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -124,7 +123,6 @@ protected PooledObjectReference(@NotNull final REFERENT_TYPE referent) { * caller) */ @Override - @Nullable public final REFERENT_TYPE get() { return referent; } @@ -159,7 +157,7 @@ public final boolean acquire() { /** * Acquire an active use permit if this has not been {@link #clear() cleared}. This is useful in situations where - * callers want to fail-fast and don't need to guarantee re-entrancy. Callers should pair this with a corresponding + * callers want to fail-fast and don't need to guarantee reentrancy. Callers should pair this with a corresponding * {@link #release()}, ideally with a try-finally pattern: * *
@@ -206,15 +204,7 @@ public final boolean acquireIfAvailable() {
      */
     @Nullable
     public final REFERENT_TYPE acquireAndGet() {
-        if (!acquire()) {
-            return null;
-        }
-        final REFERENT_TYPE localReferent = referent;
-        if (localReferent == null) {
-            release();
-            throw new IllegalStateException(this + " acquired, but referent is null");
-        }
-        return localReferent;
+        return acquire() ? referent : null;
     }
 
     /**
diff --git a/Base/src/test/java/io/deephaven/base/reference/PooledObjectReferenceTest.java b/Base/src/test/java/io/deephaven/base/reference/PooledObjectReferenceTest.java
index 6c3bab79cc1..0c6871af4f6 100644
--- a/Base/src/test/java/io/deephaven/base/reference/PooledObjectReferenceTest.java
+++ b/Base/src/test/java/io/deephaven/base/reference/PooledObjectReferenceTest.java
@@ -89,7 +89,7 @@ public void illegalRelease() {
     }
 
     @Test
-    public void illegalGetClose() {
+    public void illegalGetClear() {
         // Even though illegal, no errors and eq obj
         assertEquals(obj, ref.get());
         ref.clear();
@@ -97,7 +97,7 @@ public void illegalGetClose() {
     }
 
     @Test
-    public void closeIllegalGet() {
+    public void clearIllegalGet() {
         ref.clear();
         assertReturned();
         // Even though illegal, no errors and eq null
@@ -105,7 +105,7 @@ public void closeIllegalGet() {
     }
 
     @Test
-    public void closeWithErrorIllegalGet() {
+    public void clearWithErrorIllegalGet() {
         ref.throwOnReturnToPool();
         try {
             ref.clear();
diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java
index 7b4c457c5a8..1d7eec74ec9 100644
--- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java
@@ -334,7 +334,7 @@ private String requestStr() {
         final class Sub implements Subscriber {
             private final CompletableFuture localProducer;
             // Access to this view must be guarded by bufferReference.acquire
-            private final ByteBuffer bufferView;
+            private ByteBuffer bufferView;
             private Subscription subscription;
 
             Sub() {
@@ -346,7 +346,6 @@ final class Sub implements Subscriber {
                     return;
                 }
                 try {
-                    // noinspection DataFlowIssue
                     bufferView = bufferReference.get().duplicate();
                 } finally {
                     bufferReference.release();
@@ -372,6 +371,7 @@ public void onSubscribe(Subscription s) {
             @Override
             public void onNext(ByteBuffer byteBuffer) {
                 if (!bufferReference.acquireIfAvailable()) {
+                    bufferView = null;
                     localProducer.completeExceptionally(new IllegalStateException(
                             String.format("Failed to acquire buffer for data, %s", requestStr())));
                     return;
@@ -386,12 +386,14 @@ public void onNext(ByteBuffer byteBuffer) {
 
             @Override
             public void onError(Throwable t) {
+                bufferView = null;
                 localProducer.completeExceptionally(t);
             }
 
             @Override
             public void onComplete() {
                 if (!bufferReference.acquireIfAvailable()) {
+                    bufferView = null;
                     localProducer.completeExceptionally(new IllegalStateException(
                             String.format("Failed to acquire buffer for completion, %s", requestStr())));
                     return;
@@ -409,6 +411,7 @@ public void onComplete() {
                         toComplete = toComplete.slice();
                     }
                     localProducer.complete(toComplete);
+                    bufferView = null;
                 } finally {
                     bufferReference.release();
                 }

From 49dd2c93ddb4e1fee108ff7fbfe9faecfc04840d Mon Sep 17 00:00:00 2001
From: Devin Smith 
Date: Wed, 14 Feb 2024 14:03:28 -0800
Subject: [PATCH 13/13] plumb anonymous to python

---
 py/server/deephaven/experimental/s3.py | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py
index 0d0e4033cf3..6feb6de5d65 100644
--- a/py/server/deephaven/experimental/s3.py
+++ b/py/server/deephaven/experimental/s3.py
@@ -45,6 +45,7 @@ def __init__(self,
                      Duration, int, str, datetime.timedelta, np.timedelta64, pd.Timedelta, None] = None,
                  access_key_id: Optional[str] = None,
                  secret_access_key: Optional[str] = None,
+                 anonymous_access: bool = False,
                  endpoint_override: Optional[str] = None):
 
         """
@@ -71,6 +72,8 @@ def __init__(self,
                 to use static credentials, else default credentials will be used.
             secret_access_key (str): the secret access key for reading files. Both access key and secret key must be
                 provided to use static credentials, else default credentials will be used.
+            anonymous_access (bool): use anonymous credentials, this is useful when the S3 policy has been set to allow
+                anonymous access. Can't be combined with other credentials. By default, is False.
             endpoint_override (str): the endpoint to connect to. Callers connecting to AWS do not typically need to set
                 this; it is most useful when connecting to non-AWS, S3-compatible APIs.
 
@@ -109,7 +112,11 @@ def __init__(self,
                 raise DHError("Either both access_key_id and secret_access_key must be provided or neither")
 
             if access_key_id is not None:
+                if anonymous_access:
+                    raise DHError("Only one set of credentials may be used, requested both key and anonymous")
                 builder.credentials(_JCredentials.basic(access_key_id, secret_access_key))
+            elif anonymous_access:
+                builder.credentials(_JCredentials.anonymous())
 
             if endpoint_override is not None:
                 builder.endpointOverride(endpoint_override)