diff --git a/Util/channel/build.gradle b/Util/channel/build.gradle new file mode 100644 index 00000000000..d6a7482f331 --- /dev/null +++ b/Util/channel/build.gradle @@ -0,0 +1,22 @@ +plugins { + id 'java-library' + id 'io.deephaven.project.register' +} + +dependencies { + implementation project(':Base') + + // Needed for SafeCloseable + implementation project(':Util') + + compileOnly depAnnotations + + Classpaths.inheritJUnitPlatform(project) + Classpaths.inheritAssertJ(project) + testImplementation 'org.junit.jupiter:junit-jupiter' + testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine' +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/Util/channel/gradle.properties b/Util/channel/gradle.properties new file mode 100644 index 00000000000..c186bbfdde1 --- /dev/null +++ b/Util/channel/gradle.properties @@ -0,0 +1 @@ +io.deephaven.project.ProjectType=JAVA_PUBLIC diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java similarity index 84% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java rename to Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java index b8e7ef70268..2abc89c1357 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java @@ -1,17 +1,19 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.base.util; +package io.deephaven.util.channel; import io.deephaven.base.RAPriQueue; import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.io.IOException; +import java.net.URI; import java.nio.ByteBuffer; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -22,6 +24,15 @@ */ public class CachedChannelProvider implements SeekableChannelsProvider { + public interface ContextHolder { + void setContext(SeekableChannelContext channelContext); + + @FinalDefault + default void clearContext() { + setContext(null); + } + } + private final SeekableChannelsProvider wrappedProvider; private final int maximumPooledCount; @@ -52,13 +63,27 @@ public CachedChannelProvider(@NotNull final SeekableChannelsProvider wrappedProv } @Override - public SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { - final String pathKey = path.toAbsolutePath().toString(); + public SeekableChannelContext makeContext() { + return wrappedProvider.makeContext(); + } + + @Override + public boolean isCompatibleWith(@NotNull final SeekableChannelContext channelContext) { + return wrappedProvider.isCompatibleWith(channelContext); + } + + @Override + public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext, + @NotNull final URI uri) + throws IOException { + final String uriString = uri.toString(); final KeyedObjectHashMap channelPool = channelPools.get(ChannelType.Read); - final CachedChannel result = tryGetPooledChannel(pathKey, channelPool); - return result == null - ? new CachedChannel(wrappedProvider.getReadChannel(path), ChannelType.Read, pathKey) + final CachedChannel result = tryGetPooledChannel(uriString, channelPool); + final CachedChannel channel = result == null + ? new CachedChannel(wrappedProvider.getReadChannel(channelContext, uri), ChannelType.Read, uriString) : result.position(0); + channel.setContext(channelContext); + return channel; } @Override @@ -125,10 +150,15 @@ private long advanceClock() { return logicalClock = 1; } + @Override + public void close() { + wrappedProvider.close(); + } + /** * {@link SeekableByteChannel Channel} wrapper for pooled usage. */ - private class CachedChannel implements SeekableByteChannel { + private class CachedChannel implements SeekableByteChannel, ContextHolder { private final SeekableByteChannel wrappedChannel; private final ChannelType channelType; @@ -163,7 +193,7 @@ public long position() throws IOException { } @Override - public SeekableByteChannel position(final long newPosition) throws IOException { + public CachedChannel position(final long newPosition) throws IOException { Require.eqTrue(isOpen, "isOpen"); wrappedChannel.position(newPosition); return this; @@ -196,12 +226,20 @@ public boolean isOpen() { public void close() throws IOException { Require.eqTrue(isOpen, "isOpen"); isOpen = false; + clearContext(); returnPoolableChannel(this); } private void dispose() throws IOException { wrappedChannel.close(); } + + @Override + public final void setContext(@Nullable final SeekableChannelContext channelContext) { + if (wrappedChannel instanceof ContextHolder) { + ((ContextHolder) wrappedChannel).setContext(channelContext); + } + } } /** diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java similarity index 54% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java rename to Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java index 29322e25489..59ddcbd3d4a 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java @@ -1,11 +1,13 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.base.util; +package io.deephaven.util.channel; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.IOException; +import java.net.URI; import java.nio.channels.FileChannel; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -14,8 +16,23 @@ public class LocalFSChannelProvider implements SeekableChannelsProvider { @Override - public SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { - return FileChannel.open(path, StandardOpenOption.READ); + public SeekableChannelContext makeContext() { + // No additional context required for local FS + return SeekableChannelContext.NULL; + } + + @Override + public boolean isCompatibleWith(@Nullable final SeekableChannelContext channelContext) { + // Context is not used, hence always compatible + return true; + } + + @Override + public SeekableByteChannel getReadChannel(@Nullable final SeekableChannelContext channelContext, + @NotNull final URI uri) + throws IOException { + // context is unused here + return FileChannel.open(Path.of(uri), StandardOpenOption.READ); } @Override @@ -31,4 +48,7 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path filePath, final b } return result; } + + @Override + public void close() {} } 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 new file mode 100644 index 00000000000..c5e7e155068 --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java @@ -0,0 +1,16 @@ +package io.deephaven.util.channel; + +import io.deephaven.util.SafeCloseable; + +/** + * Context object for reading and writing to channels created by {@link SeekableChannelsProvider}. + */ +public interface SeekableChannelContext extends SafeCloseable { + + SeekableChannelContext NULL = new SeekableChannelContext() {}; + + /** + * Release any resources associated with this context. The context should not be used afterward. + */ + default void close() {} +} 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 new file mode 100644 index 00000000000..75be19f4592 --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java @@ -0,0 +1,64 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.channels.SeekableByteChannel; +import java.nio.file.Path; +import java.nio.file.Paths; + +public interface SeekableChannelsProvider extends SafeCloseable { + + /** + * Take the file source path or URI and convert it to a URI object. + * + * @param source The file source path or URI + * @return The URI object + */ + static URI convertToURI(final String source) { + final URI uri; + try { + uri = new URI(source); + } catch (final URISyntaxException e) { + // If the URI is invalid, assume it's a file path + return new File(source).toURI(); + } + if (uri.getScheme() == null) { + // Need to convert to a "file" URI + return new File(source).toURI(); + } + return uri; + } + + /** + * Create a new {@link SeekableChannelContext} object for creating read channels via this provider. + */ + SeekableChannelContext 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. + */ + boolean isCompatibleWith(@NotNull SeekableChannelContext channelContext); + + default SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull String uriStr) + throws IOException { + return getReadChannel(channelContext, convertToURI(uriStr)); + } + + SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull URI uri) + throws IOException; + + default SeekableByteChannel getWriteChannel(@NotNull final String path, final boolean append) throws IOException { + return getWriteChannel(Paths.get(path), append); + } + + SeekableByteChannel getWriteChannel(@NotNull Path path, boolean append) throws IOException; +} diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java new file mode 100644 index 00000000000..47ba411d70d --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java @@ -0,0 +1,56 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.ServiceLoader; + +/** + * A service loader class for loading {@link SeekableChannelsProviderPlugin} implementations at runtime and provide + * {@link SeekableChannelsProvider} implementations for different URI schemes, e.g., S3. + */ +public final class SeekableChannelsProviderLoader { + + private static volatile SeekableChannelsProviderLoader instance; + + public static SeekableChannelsProviderLoader getInstance() { + if (instance == null) { + instance = new SeekableChannelsProviderLoader(); + } + return instance; + } + + private final List providers; + + private SeekableChannelsProviderLoader() { + providers = new ArrayList<>(); + // Load the plugins + for (final SeekableChannelsProviderPlugin plugin : ServiceLoader.load(SeekableChannelsProviderPlugin.class)) { + providers.add(plugin); + } + } + + /** + * Create a new {@link SeekableChannelsProvider} based on given URI and object using the plugins loaded by the + * {@link ServiceLoader}. For example, for a "S3" URI, we will create a {@link SeekableChannelsProvider} which can + * read files from S3. + * + * @param uri The URI + * @param object An optional object to pass to the {@link SeekableChannelsProviderPlugin} implementations. + * @return A {@link SeekableChannelsProvider} for the given URI. + */ + public SeekableChannelsProvider fromServiceLoader(@NotNull final URI uri, @Nullable final Object object) { + for (final SeekableChannelsProviderPlugin plugin : providers) { + if (plugin.isCompatible(uri, object)) { + return plugin.createProvider(uri, object); + } + } + throw new UnsupportedOperationException("No plugin found for uri: " + uri); + } +} diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java new file mode 100644 index 00000000000..efb2e4892c0 --- /dev/null +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java @@ -0,0 +1,25 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.util.channel; + +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.net.URI; + +/** + * A plugin interface for providing {@link SeekableChannelsProvider} implementations for different URI schemes, e.g. S3. + * Check out {@link SeekableChannelsProviderLoader} for more details. + */ +public interface SeekableChannelsProviderPlugin { + /** + * Check if this plugin is compatible with the given URI and config object. + */ + boolean isCompatible(@NotNull URI uri, @Nullable Object config); + + /** + * Create a {@link SeekableChannelsProvider} for the given URI and config object. + */ + SeekableChannelsProvider createProvider(@NotNull URI uri, @Nullable Object object); +} diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java similarity index 63% rename from extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java rename to Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java index 29cd046d782..64ee9c667e9 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java @@ -1,13 +1,14 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.base.util; +package io.deephaven.util.channel; import org.jetbrains.annotations.NotNull; -import org.junit.Assert; -import org.junit.Test; +import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.Test; import java.io.IOException; +import java.net.URI; import java.nio.ByteBuffer; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -15,15 +16,15 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + public class CachedChannelProviderTest { private final List closed = new ArrayList<>(); - @org.junit.After - public void tearDown() { - closed.clear(); - } - @Test public void testSimpleRead() throws IOException { final SeekableChannelsProvider wrappedProvider = new TestChannelProvider(); @@ -31,15 +32,19 @@ public void testSimpleRead() throws IOException { for (int ii = 0; ii < 100; ++ii) { final SeekableByteChannel[] sameFile = new SeekableByteChannel[10]; for (int jj = 0; jj < sameFile.length; ++jj) { - sameFile[jj] = cachedChannelProvider.getReadChannel("r" + ii); + sameFile[jj] = cachedChannelProvider.getReadChannel(wrappedProvider.makeContext(), "r" + ii); } + final ByteBuffer buffer = ByteBuffer.allocate(1); for (int jj = 0; jj < 10; ++jj) { + // Call read to hit the assertions inside the mock channel, which doesn't read anything + sameFile[jj].read(buffer); + assertEquals(buffer.remaining(), buffer.capacity()); sameFile[jj].close(); } } - Assert.assertEquals(closed.size(), 900); + assertEquals(900, closed.size()); for (int ii = 0; ii < 900; ++ii) { - Assert.assertTrue(closed.get(ii).endsWith("r" + ii / 10)); + assertTrue(closed.get(ii).endsWith("r" + ii / 10)); } } @@ -48,12 +53,13 @@ public void testSimpleReadWrite() throws IOException { SeekableChannelsProvider wrappedProvider = new TestChannelProvider(); CachedChannelProvider cachedChannelProvider = new CachedChannelProvider(wrappedProvider, 100); for (int i = 0; i < 1000; i++) { - SeekableByteChannel rc = ((i / 100) % 2 == 0 ? cachedChannelProvider.getReadChannel("r" + i) - : cachedChannelProvider.getWriteChannel("w" + i, false)); + SeekableByteChannel rc = + ((i / 100) % 2 == 0 ? cachedChannelProvider.getReadChannel(wrappedProvider.makeContext(), "r" + i) + : cachedChannelProvider.getWriteChannel("w" + i, false)); rc.close(); } - Assert.assertEquals(closed.size(), 900); - Assert.assertTrue(closed.get(0).endsWith("r0")); + assertEquals(900, closed.size()); + assertTrue(closed.get(0).endsWith("r0")); } @Test @@ -62,11 +68,14 @@ public void testSimpleWrite() throws IOException { CachedChannelProvider cachedChannelProvider = new CachedChannelProvider(wrappedProvider, 100); for (int i = 0; i < 1000; i++) { SeekableByteChannel rc = cachedChannelProvider.getWriteChannel("w" + i, false); + // Call write to hit the assertions inside the mock channel + final ByteBuffer buffer = ByteBuffer.allocate(1); + rc.write(buffer); rc.close(); } - Assert.assertEquals(closed.size(), 900); + assertEquals(900, closed.size()); for (int i = 0; i < 900; i++) { - Assert.assertTrue(closed.get(i).endsWith("w" + (i))); + assertTrue(closed.get(i).endsWith("w" + (i))); } } @@ -78,9 +87,9 @@ public void testSimpleAppend() throws IOException { SeekableByteChannel rc = cachedChannelProvider.getWriteChannel("a" + i, true); rc.close(); } - Assert.assertEquals(closed.size(), 900); + assertEquals(900, closed.size()); for (int i = 0; i < 900; i++) { - Assert.assertTrue(closed.get(i).endsWith("a" + (i))); + assertTrue(closed.get(i).endsWith("a" + (i))); } } @@ -91,16 +100,16 @@ public void testCloseOrder() throws IOException { for (int i = 0; i < 20; i++) { List channels = new ArrayList<>(); for (int j = 0; j < 50; j++) { - channels.add(cachedChannelProvider.getReadChannel("r" + (j + 50 * i))); + channels.add(cachedChannelProvider.getReadChannel(wrappedProvider.makeContext(), "r" + (j + 50 * i))); } for (int j = 0; j < 50; j++) { channels.get(49 - j).close(); } } - Assert.assertEquals(closed.size(), 900); + assertEquals(900, closed.size()); for (int i = 0; i < 1; i++) { for (int j = 0; j < 50; j++) { - Assert.assertTrue(closed.get(j + 50 * i).endsWith("r" + (50 * i + 49 - j))); + assertTrue(closed.get(j + 50 * i).endsWith("r" + (50 * i + 49 - j))); } } } @@ -110,21 +119,27 @@ public void testReuse() throws IOException { final SeekableChannelsProvider wrappedProvider = new TestChannelProvider(); final CachedChannelProvider cachedChannelProvider = new CachedChannelProvider(wrappedProvider, 50); final SeekableByteChannel[] someResult = new SeekableByteChannel[50]; + final ByteBuffer buffer = ByteBuffer.allocate(1); for (int ci = 0; ci < someResult.length; ++ci) { - someResult[ci] = cachedChannelProvider.getReadChannel("r" + ci); + someResult[ci] = cachedChannelProvider.getReadChannel(wrappedProvider.makeContext(), "r" + ci); + // Call read to hit the assertions inside the mock channel, which doesn't read anything + someResult[ci].read(buffer); } for (int ci = 0; ci < someResult.length; ++ci) { someResult[someResult.length - ci - 1].close(); } for (int step = 0; step < 10; ++step) { for (int ci = 0; ci < someResult.length; ++ci) { - Assert.assertSame(someResult[ci], cachedChannelProvider.getReadChannel("r" + ci)); + assertSame(someResult[ci], + cachedChannelProvider.getReadChannel(wrappedProvider.makeContext(), "r" + ci)); + // Call read to hit the assertions inside the mock channel, which doesn't read anything + someResult[ci].read(buffer); } for (int ci = 0; ci < someResult.length; ++ci) { someResult[someResult.length - ci - 1].close(); } } - Assert.assertEquals(closed.size(), 0); + assertEquals(0, closed.size()); } @Test @@ -144,14 +159,14 @@ public void testReuse10() throws IOException { final SeekableByteChannel[] reused = new SeekableByteChannel[100]; for (int ri = 0; ri < 100; ++ri) { SeekableByteChannel rc = cachedChannelProvider.getWriteChannel("w" + (ri / 10) % 10, false); - Assert.assertSame(rc, someResult[ri % 100]); + assertSame(rc, someResult[ri % 100]); reused[ri] = rc; } for (int ri = 0; ri < 100; ++ri) { reused[99 - ri].close(); } } - Assert.assertEquals(closed.size(), 0); + assertEquals(0, closed.size()); } @@ -159,14 +174,28 @@ private class TestChannelProvider implements SeekableChannelsProvider { AtomicInteger count = new AtomicInteger(0); + private final class TestChannelContext implements SeekableChannelContext { + } + @Override - public SeekableByteChannel getReadChannel(@NotNull String path) { - return new TestMockChannel(count.getAndIncrement(), path); + public SeekableChannelContext makeContext() { + return new TestChannelContext(); } @Override - public SeekableByteChannel getReadChannel(@NotNull Path path) { - return new TestMockChannel(count.getAndIncrement(), path.toString()); + public boolean isCompatibleWith(@NotNull SeekableChannelContext channelContext) { + return channelContext == SeekableChannelContext.NULL; + } + + @Override + public SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, + @NotNull String path) { + return new TestMockChannel(count.getAndIncrement(), path, channelContext); + } + + @Override + public SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull URI uri) { + return new TestMockChannel(count.getAndIncrement(), uri.toString(), channelContext); } @Override @@ -178,25 +207,37 @@ public SeekableByteChannel getWriteChannel(@NotNull String path, boolean append) public SeekableByteChannel getWriteChannel(@NotNull Path path, boolean append) { return new TestMockChannel(count.getAndIncrement(), path.toString()); } + + @Override + public void close() {} } - private class TestMockChannel implements SeekableByteChannel { + private final class TestMockChannel implements SeekableByteChannel, CachedChannelProvider.ContextHolder { private final int id; private final String path; + private SeekableChannelContext channelContext; - public TestMockChannel(int id, String path) { + private TestMockChannel(int id, String path, SeekableChannelContext channelContext) { + this(id, path); + this.channelContext = channelContext; + } + + private TestMockChannel(int id, String path) { this.id = id; this.path = path; + this.channelContext = null; } @Override public int read(ByteBuffer dst) { + assertTrue(channelContext instanceof TestChannelProvider.TestChannelContext); return 0; } @Override public int write(ByteBuffer src) { + assertNull(channelContext); return 0; } @@ -228,6 +269,12 @@ public boolean isOpen() { @Override public void close() { closing(id, path); + clearContext(); + } + + @Override + public void setContext(@Nullable SeekableChannelContext channelContext) { + this.channelContext = channelContext; } } diff --git a/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java new file mode 100644 index 00000000000..98e2d47fb79 --- /dev/null +++ b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java @@ -0,0 +1,52 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit LazyCachingSupplier and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.util.datastructures; + +import org.jetbrains.annotations.NotNull; + +import java.lang.ref.SoftReference; +import java.util.function.Function; + +/** + * {@link Function} wrapper that caches the result in a {@link SoftReference}. Only suitable to wrap functions that are + * safely repeatable and don't return {@code null}. + * + * @param the type of results supplied by this function + */ +public final class LazyCachingFunction implements Function { + + private final Function internalFunction; + + private volatile SoftReference cachedResultRef; + + /** + * Construct a {@link Function} wrapper. + * + * @param internalFunction The {@link Function} to wrap. Must be safely repeatable and must not return {@code null}. + */ + public LazyCachingFunction(@NotNull final Function internalFunction) { + this.internalFunction = internalFunction; + } + + @Override + public OUTPUT_TYPE apply(final INPUT_TYPE arg) { + SoftReference currentRef; + OUTPUT_TYPE current; + if ((currentRef = cachedResultRef) != null && (current = currentRef.get()) != null) { + return current; + } + synchronized (this) { + if ((currentRef = cachedResultRef) != null && (current = currentRef.get()) != null) { + return current; + } + cachedResultRef = new SoftReference<>(current = internalFunction.apply(arg)); + } + return current; + } +} diff --git a/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingSupplier.java b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingSupplier.java index 1ffb135a69a..cadc1105a7b 100644 --- a/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingSupplier.java +++ b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingSupplier.java @@ -11,26 +11,28 @@ /** * {@link Supplier} wrapper that caches the result in a {@link SoftReference}. Only suitable to wrap suppliers that are * safely repeatable and don't return {@code null}. + * + * @param the type of results supplied by this supplier */ -public final class LazyCachingSupplier implements Supplier { +public final class LazyCachingSupplier implements Supplier { - private final Supplier internalSupplier; + private final Supplier internalSupplier; - private volatile SoftReference cachedResultRef; + private volatile SoftReference cachedResultRef; /** * Construct a {@link Supplier} wrapper. * * @param internalSupplier The {@link Supplier} to wrap. Must be safely repeatable and must not return {@code null}. */ - public LazyCachingSupplier(@NotNull final Supplier internalSupplier) { + public LazyCachingSupplier(@NotNull final Supplier internalSupplier) { this.internalSupplier = internalSupplier; } @Override - public T get() { - SoftReference currentRef; - T current; + public OUTPUT_TYPE get() { + SoftReference currentRef; + OUTPUT_TYPE current; if ((currentRef = cachedResultRef) != null && (current = currentRef.get()) != null) { return current; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/FileTableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/FileTableLocationKey.java index da998585c20..b9cd4012126 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/FileTableLocationKey.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/FileTableLocationKey.java @@ -68,7 +68,7 @@ public String toString() { /** * Precedence-wise this implementation compares {@code order}, then applies a {@link PartitionsComparator} to * {@code partitions}, then compares {@code file}. - * + * * @inheritDoc */ @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URITableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URITableLocationKey.java new file mode 100644 index 00000000000..e5852c6dcf2 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URITableLocationKey.java @@ -0,0 +1,130 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +/* + * --------------------------------------------------------------------------------------------------------------------- + * AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY - for any changes edit FileTableLocationKey and regenerate + * --------------------------------------------------------------------------------------------------------------------- + */ +package io.deephaven.engine.table.impl.locations.local; + +import io.deephaven.base.log.LogOutput; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; +import io.deephaven.engine.table.impl.locations.impl.PartitionedTableLocationKey; +import io.deephaven.engine.table.impl.locations.TableLocationKey; +import io.deephaven.io.log.impl.LogOutputStringImpl; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.net.URI; +import java.util.Map; + +/** + * Base {@link ImmutableTableLocationKey} implementation for table locations that may be enclosed by partitions and + * described by a {@link URI}. Sub-classes should override {@link #compareTo(TableLocationKey)} and + * {@link #equals(Object)} only if they need to prevent equality with other {@link URITableLocationKey} implementations. + */ +public class URITableLocationKey extends PartitionedTableLocationKey { + + private static final String IMPLEMENTATION_NAME = URITableLocationKey.class.getSimpleName(); + + protected final URI uri; + private final int order; + + private int cachedHashCode; + + /** + * Construct a new URITableLocationKey for the supplied {@code uri} and {@code partitions}. + * + * @param uri The uri (or directory) that backs the keyed location. Will be adjusted to an absolute path. + * @param order Explicit ordering value for this location key. {@link Comparable#compareTo(Object)} will sort + * URITableLocationKeys with a lower {@code order} before other keys. Comparing this ordering value takes + * precedence over other fields. + * @param partitions The table partitions enclosing the table location keyed by {@code this}. Note that if this + * parameter is {@code null}, the location will be a member of no partitions. An ordered copy of the map will + * be made, so the calling code is free to mutate the map after this call completes, but the partition keys + * and values themselves must be effectively immutable. + */ + public URITableLocationKey(@NotNull final URI uri, final int order, + @Nullable final Map> partitions) { + super(partitions); + if (!uri.isAbsolute()) { + throw new IllegalArgumentException("URI must be absolute"); + } + this.uri = uri; + this.order = order; + } + + public final URI getURI() { + return uri; + } + + @Override + public LogOutput append(@NotNull final LogOutput logOutput) { + return logOutput.append(getImplementationName()) + .append(":[uri=").append(uri.getPath()) + .append(",partitions=").append(PartitionsFormatter.INSTANCE, partitions) + .append(']'); + } + + @Override + public String toString() { + return new LogOutputStringImpl().append(this).toString(); + } + + /** + * Precedence-wise this implementation compares {@code order}, then applies a {@link PartitionsComparator} to + * {@code partitions}, then compares {@code uri}. + * + * @inheritDoc + */ + @Override + public int compareTo(@NotNull final TableLocationKey other) { + if (other instanceof URITableLocationKey) { + final URITableLocationKey otherTyped = (URITableLocationKey) other; + final int orderingComparisonResult = Integer.compare(order, otherTyped.order); + if (orderingComparisonResult != 0) { + return orderingComparisonResult; + } + final int partitionComparisonResult = + PartitionsComparator.INSTANCE.compare(partitions, otherTyped.partitions); + if (partitionComparisonResult != 0) { + return partitionComparisonResult; + } + return uri.compareTo(otherTyped.uri); + } + throw new ClassCastException("Cannot compare " + getClass() + " to " + other.getClass()); + } + + @Override + public int hashCode() { + if (cachedHashCode == 0) { + final int computedHashCode = 31 * partitions.hashCode() + uri.hashCode(); + // Don't use 0; that's used by StandaloneTableLocationKey, and also our sentinel for the need to compute + if (computedHashCode == 0) { + final int fallbackHashCode = URITableLocationKey.class.hashCode(); + cachedHashCode = fallbackHashCode == 0 ? 1 : fallbackHashCode; + } else { + cachedHashCode = computedHashCode; + } + } + return cachedHashCode; + } + + @Override + public boolean equals(@Nullable final Object other) { + if (this == other) { + return true; + } + if (!(other instanceof URITableLocationKey)) { + return false; + } + final URITableLocationKey otherTyped = (URITableLocationKey) other; + return uri.equals(otherTyped.uri) && partitions.equals(otherTyped.partitions); + } + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.java index 069954b2f3f..7b70f141b5b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.java @@ -488,7 +488,7 @@ public FillContext makeFillContext(int chunkCapacity, SharedContext sharedContex } @Override - public FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { + public ChunkSource.FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { return new FillContext(this, chunkCapacity, sharedContext, false, ascendingMapping); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index ba050092fb9..6b4a2d9d278 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -3896,7 +3896,8 @@ public void testMultiPartitionSymbolTableBy() throws IOException { t4.updateView("Date=`2021-07-21`", "Num=400")).moveColumnsUp("Date", "Num"); final Table loaded = ParquetTools.readPartitionedTableInferSchema( - new ParquetKeyValuePartitionedLayout(testRootFile, 2), ParquetInstructions.EMPTY); + new ParquetKeyValuePartitionedLayout(testRootFile, 2, ParquetInstructions.EMPTY), + ParquetInstructions.EMPTY); // verify the sources are identical assertTableEquals(merged, loaded); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestGroupingProviders.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestGroupingProviders.java index 6f7cd3e22db..2b5fee65f09 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestGroupingProviders.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestGroupingProviders.java @@ -159,7 +159,8 @@ private void doTest(final boolean missingGroups) { // restore it here. final Table actual = ParquetTools.readPartitionedTable( - DeephavenNestedPartitionLayout.forParquet(dataDirectory, tableName, "Part", ipn -> ipn.equals("IP")), + DeephavenNestedPartitionLayout.forParquet(dataDirectory, tableName, "Part", ipn -> ipn.equals("IP"), + ParquetInstructions.EMPTY), ParquetInstructions.EMPTY, partitionedDataDefinition).coalesce(); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestChunkedRegionedOperations.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestChunkedRegionedOperations.java index b6e780300fe..6cd375e659e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestChunkedRegionedOperations.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestChunkedRegionedOperations.java @@ -266,7 +266,8 @@ public void setUp() throws Exception { "DT_R = epochNanos(DT)"); actual = ParquetTools.readPartitionedTable( - DeephavenNestedPartitionLayout.forParquet(dataDirectory, tableName, "PC", null), + DeephavenNestedPartitionLayout.forParquet(dataDirectory, tableName, "PC", null, + ParquetInstructions.EMPTY), ParquetInstructions.EMPTY, partitionedDataDefinition).updateView( List.of( diff --git a/extensions/parquet/base/build.gradle b/extensions/parquet/base/build.gradle index b6871736347..028731cd85a 100644 --- a/extensions/parquet/base/build.gradle +++ b/extensions/parquet/base/build.gradle @@ -6,6 +6,8 @@ plugins { description 'Parquet Base: Libraries for working with Parquet files' dependencies { + api project(':util-channel') + Classpaths.inheritParquetHadoop(project) implementation project(':extensions-parquet-compression') diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReader.java index b9290e96407..a3967dc24ea 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReader.java @@ -3,14 +3,15 @@ */ package io.deephaven.parquet.base; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; import org.apache.parquet.column.Dictionary; import org.apache.parquet.internal.column.columnindex.OffsetIndex; import org.apache.parquet.schema.PrimitiveType; import org.jetbrains.annotations.Nullable; import java.io.IOException; -import java.util.Iterator; -import java.util.function.Supplier; +import java.util.function.Function; public interface ColumnChunkReader { /** @@ -19,7 +20,7 @@ public interface ColumnChunkReader { long numRows(); /** - * @return The value stored under the corresponding ColumnMetaData.num_values field + * @return The value stored under the corresponding ColumnMetaData.num_values field. */ long numValues(); @@ -36,24 +37,41 @@ public interface ColumnChunkReader { OffsetIndex getOffsetIndex(); /** - * @return An iterator over individual parquet pages + * Used to iterate over column page readers for each page with the capability to set channel context to for reading + * the pages. */ - Iterator getPageIterator() throws IOException; + interface ColumnPageReaderIterator { + /** + * @return Whether there are more pages to iterate over. + */ + boolean hasNext(); + + /** + * @param channelContext The channel context to use for constructing the reader + * @return The next page reader. + */ + ColumnPageReader next(SeekableChannelContext channelContext); + } + + /** + * @return An iterator over individual parquet pages. + */ + ColumnPageReaderIterator getPageIterator() throws IOException; interface ColumnPageDirectAccessor { /** * Directly access a page reader for a given page number. */ - ColumnPageReader getPageReader(final int pageNum); + ColumnPageReader getPageReader(int pageNum); } /** - * @return An accessor for individual parquet pages + * @return An accessor for individual parquet pages. */ ColumnPageDirectAccessor getPageAccessor(); /** - * @return Whether this column chunk uses a dictionary-based encoding on every page + * @return Whether this column chunk uses a dictionary-based encoding on every page. */ boolean usesDictionaryOnEveryPage(); @@ -61,7 +79,7 @@ interface ColumnPageDirectAccessor { * @return Supplier for a Parquet dictionary for this column chunk * @apiNote The result will never return {@code null}. It will instead supply {@link #NULL_DICTIONARY}. */ - Supplier getDictionarySupplier(); + Function getDictionarySupplier(); Dictionary NULL_DICTIONARY = new NullDictionary(); @@ -85,4 +103,9 @@ public int getMaxId() { */ @Nullable String getVersion(); + + /** + * @return The channel provider for this column chunk reader. + */ + SeekableChannelsProvider getChannelsProvider(); } 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 8de3d8b9281..9a1570aac53 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 @@ -4,10 +4,11 @@ package io.deephaven.parquet.base; import io.deephaven.UncheckedDeephavenException; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; import io.deephaven.parquet.compress.CompressorAdapter; import io.deephaven.parquet.compress.DeephavenCompressorAdapterFactory; -import io.deephaven.util.datastructures.LazyCachingSupplier; +import io.deephaven.util.datastructures.LazyCachingFunction; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; @@ -15,24 +16,24 @@ import org.apache.parquet.column.page.DictionaryPage; import org.apache.parquet.format.*; import org.apache.parquet.internal.column.columnindex.OffsetIndex; -import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; 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.Iterator; import java.util.List; import java.util.NoSuchElementException; -import java.util.function.Supplier; +import java.util.function.Function; +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; import static org.apache.parquet.format.Encoding.PLAIN_DICTIONARY; import static org.apache.parquet.format.Encoding.RLE_DICTIONARY; @@ -40,15 +41,18 @@ public class ColumnChunkReaderImpl implements ColumnChunkReader { private final ColumnChunk columnChunk; private final SeekableChannelsProvider channelsProvider; - private final Path rootPath; + /** + * If reading a single parquet file, root URI is the URI of the file, else the parent directory for a metadata file + */ + private final URI rootURI; private final CompressorAdapter decompressor; private final ColumnDescriptor path; private final OffsetIndex offsetIndex; private final List fieldTypes; - private final Supplier dictionarySupplier; + private final Function dictionarySupplier; private final PageMaterializer.Factory nullMaterializerFactory; - private Path filePath; + private URI uri; /** * Number of rows in the row group of this column chunk. */ @@ -58,12 +62,12 @@ public class ColumnChunkReaderImpl implements ColumnChunkReader { */ private final String version; - ColumnChunkReaderImpl(ColumnChunk columnChunk, SeekableChannelsProvider channelsProvider, Path rootPath, + ColumnChunkReaderImpl(ColumnChunk columnChunk, SeekableChannelsProvider channelsProvider, URI rootURI, MessageType type, OffsetIndex offsetIndex, List fieldTypes, final long numRows, final String version) { this.channelsProvider = channelsProvider; this.columnChunk = columnChunk; - this.rootPath = rootPath; + this.rootURI = rootURI; this.path = type .getColumnDescription(columnChunk.meta_data.getPath_in_schema().toArray(new String[0])); if (columnChunk.getMeta_data().isSetCodec()) { @@ -74,7 +78,7 @@ public class ColumnChunkReaderImpl implements ColumnChunkReader { } this.offsetIndex = offsetIndex; this.fieldTypes = fieldTypes; - this.dictionarySupplier = new LazyCachingSupplier<>(this::getDictionary); + this.dictionarySupplier = new LazyCachingFunction<>(this::getDictionary); this.nullMaterializerFactory = PageMaterializer.factoryForType(path.getPrimitiveType().getPrimitiveTypeName()); this.numRows = numRows; this.version = version; @@ -100,7 +104,7 @@ public final OffsetIndex getOffsetIndex() { } @Override - public Iterator getPageIterator() { + public ColumnPageReaderIterator getPageIterator() { final long dataPageOffset = columnChunk.meta_data.getData_page_offset(); if (offsetIndex == null) { return new ColumnPageReaderIteratorImpl(dataPageOffset, columnChunk.getMeta_data().getNum_values()); @@ -117,14 +121,15 @@ public final ColumnPageDirectAccessor getPageAccessor() { return new ColumnPageDirectAccessorImpl(); } - private Path getFilePath() { - if (filePath != null) { - return filePath; + private URI getURI() { + if (uri != null) { + return uri; } - if (columnChunk.isSetFile_path()) { - return filePath = rootPath.resolve(columnChunk.getFile_path()); + if (columnChunk.isSetFile_path() && FILE_URI_SCHEME.equals(uri.getScheme())) { + return uri = Path.of(rootURI).resolve(columnChunk.getFile_path()).toUri(); } else { - return filePath = rootPath; + // TODO(deephaven-core#5066): Add support for reading metadata files from non-file URIs + return uri = rootURI; } } @@ -151,12 +156,12 @@ public boolean usesDictionaryOnEveryPage() { } @Override - public Supplier getDictionarySupplier() { + public Function getDictionarySupplier() { return dictionarySupplier; } @NotNull - private Dictionary getDictionary() { + private Dictionary getDictionary(final SeekableChannelContext channelContext) { final long dictionaryPageOffset; final ColumnMetaData chunkMeta = columnChunk.getMeta_data(); if (chunkMeta.isSetDictionary_page_offset()) { @@ -172,10 +177,23 @@ private Dictionary getDictionary() { } else { return NULL_DICTIONARY; } - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(getFilePath())) { + 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 (IOException e) { + } catch (final IOException e) { throw new UncheckedIOException(e); } } @@ -190,10 +208,15 @@ public String getVersion() { return version; } + @Override + public SeekableChannelsProvider getChannelsProvider() { + return channelsProvider; + } + @NotNull private Dictionary readDictionary(ReadableByteChannel file) throws IOException { // explicitly not closing this, caller is responsible - final InputStream inputStream = Channels.newInputStream(file); + final BufferedInputStream inputStream = new BufferedInputStream(Channels.newInputStream(file)); final PageHeader pageHeader = Util.readPageHeader(inputStream); if (pageHeader.getType() != PageType.DICTIONARY_PAGE) { // In case our fallback in getDictionary was too optimistic... @@ -202,7 +225,7 @@ private Dictionary readDictionary(ReadableByteChannel file) throws IOException { final DictionaryPageHeader dictHeader = pageHeader.getDictionary_page_header(); final BytesInput payload; - int compressedPageSize = pageHeader.getCompressed_page_size(); + final int compressedPageSize = pageHeader.getCompressed_page_size(); if (compressedPageSize == 0) { // Sometimes the size is explicitly empty, just use an empty payload payload = BytesInput.empty(); @@ -216,7 +239,7 @@ private Dictionary readDictionary(ReadableByteChannel file) throws IOException { return dictionaryPage.getEncoding().initDictionary(path, dictionaryPage); } - private final class ColumnPageReaderIteratorImpl implements Iterator { + private final class ColumnPageReaderIteratorImpl implements ColumnPageReaderIterator { private long currentOffset; private long remainingValues; @@ -231,12 +254,12 @@ public boolean hasNext() { } @Override - public ColumnPageReader next() { + public ColumnPageReader next(@NotNull final SeekableChannelContext channelContext) { if (!hasNext()) { 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(getFilePath())) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, getURI())) { final long headerOffset = currentOffset; readChannel.position(currentOffset); // deliberately not closing this stream @@ -244,7 +267,7 @@ public ColumnPageReader next() { currentOffset = readChannel.position() + pageHeader.getCompressed_page_size(); if (pageHeader.isSetDictionary_page_header()) { // Dictionary page; skip it - return next(); + return next(channelContext); } if (!pageHeader.isSetData_page_header() && !pageHeader.isSetData_page_header_v2()) { throw new IllegalStateException( @@ -266,20 +289,21 @@ public ColumnPageReader next() { throw new UncheckedDeephavenException( "Unknown parquet data page header type " + pageHeader.type); } - final Supplier pageDictionarySupplier = + final Function pageDictionarySupplier = (encoding == PLAIN_DICTIONARY || encoding == RLE_DICTIONARY) ? dictionarySupplier - : () -> NULL_DICTIONARY; - return new ColumnPageReaderImpl(channelsProvider, decompressor, pageDictionarySupplier, - nullMaterializerFactory, path, getFilePath(), fieldTypes, readChannel.position(), pageHeader, - ColumnPageReaderImpl.NULL_NUM_VALUES); + : (SeekableChannelContext context) -> NULL_DICTIONARY; + final ColumnPageReader nextReader = new ColumnPageReaderImpl(channelsProvider, decompressor, + pageDictionarySupplier, nullMaterializerFactory, path, getURI(), fieldTypes, + readChannel.position(), pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES); + return nextReader; } catch (IOException e) { throw new UncheckedDeephavenException("Error reading page header", e); } } } - private final class ColumnPageReaderIteratorIndexImpl implements Iterator { + private final class ColumnPageReaderIteratorIndexImpl implements ColumnPageReaderIterator { private int pos; ColumnPageReaderIteratorIndexImpl() { @@ -292,7 +316,7 @@ public boolean hasNext() { } @Override - public ColumnPageReader next() { + public ColumnPageReader next(@NotNull final SeekableChannelContext channelContext) { if (!hasNext()) { throw new NoSuchElementException("No next element"); } @@ -302,9 +326,9 @@ public ColumnPageReader next() { // parquet files written before deephaven-core/pull/4844. final int numValues = (int) (offsetIndex.getLastRowIndex(pos, columnChunk.getMeta_data().getNum_values()) - offsetIndex.getFirstRowIndex(pos) + 1); - ColumnPageReaderImpl columnPageReader = + final ColumnPageReader columnPageReader = new ColumnPageReaderImpl(channelsProvider, decompressor, dictionarySupplier, - nullMaterializerFactory, path, getFilePath(), fieldTypes, offsetIndex.getOffset(pos), null, + nullMaterializerFactory, path, getURI(), fieldTypes, offsetIndex.getOffset(pos), null, numValues); pos++; return columnPageReader; @@ -323,7 +347,7 @@ public ColumnPageReader getPageReader(final int pageNum) { } // Page header and number of values will be populated later when we read the page header from the file return new ColumnPageReaderImpl(channelsProvider, decompressor, dictionarySupplier, nullMaterializerFactory, - path, getFilePath(), fieldTypes, offsetIndex.getOffset(pageNum), null, + path, getURI(), fieldTypes, offsetIndex.getOffset(pageNum), null, ColumnPageReaderImpl.NULL_NUM_VALUES); } } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReader.java index b529bb08270..58b66e6abdd 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReader.java @@ -4,6 +4,7 @@ package io.deephaven.parquet.base; +import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.column.Dictionary; import org.jetbrains.annotations.NotNull; @@ -16,40 +17,46 @@ public interface ColumnPageReader extends AutoCloseable { /** + * @param channelContext The channel context to use for reading the parquet file * @return The number of rows in this ColumnChunk, or -1 if it's unknown. */ - default long numRows() throws IOException { - return numValues(); + default long numRows(final SeekableChannelContext channelContext) throws IOException { + return numValues(channelContext); } /** * Triggers the value decompression and decoding * * @param nullValue The value to be stored under the null entries + * @param channelContext The channel context to use for reading the parquet file * @return the data for that page in a format that makes sense for the given type - typically array of something * that makes sense */ - Object materialize(Object nullValue) throws IOException; + Object materialize(Object nullValue, SeekableChannelContext channelContext) throws IOException; /** * Directly read the key integral values when there's a dictionary. * * @param keyDest A properly sized buffer (at least numValues()) to hold the keys * @param nullPlaceholder The value to use for nulls. + * @param channelContext The channel context to use for reading the parquet file * * @return A buffer holding the end of each repeated row. If the column is not repeating, null. */ - IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder) throws IOException; + IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder, + SeekableChannelContext channelContext) throws IOException; /** + * @param channelContext The channel context to use for reading the parquet file * @return The value stored under number DataPageHeader.num_values */ - int numValues() throws IOException; + int numValues(SeekableChannelContext channelContext) throws IOException; /** + * @param channelContext The channel context to use for reading the parquet file * @return Parquet dictionary for this column chunk * @apiNote The result will never be {@code null}. It will instead be {@link ColumnChunkReader#NULL_DICTIONARY}. */ @NotNull - Dictionary getDictionary(); + Dictionary getDictionary(SeekableChannelContext channelContext); } 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 1e367b42758..89dd0b6ea3c 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 @@ -7,7 +7,8 @@ import io.deephaven.base.verify.Assert; import io.deephaven.parquet.base.util.Helpers; import io.deephaven.parquet.base.util.RunLengthBitPackingHybridBufferDecoder; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; import io.deephaven.parquet.compress.CompressorAdapter; import org.apache.parquet.bytes.ByteBufferInputStream; import org.apache.parquet.bytes.BytesInput; @@ -27,16 +28,16 @@ import org.jetbrains.annotations.NotNull; import java.io.IOException; +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.nio.file.Path; import java.util.ArrayList; import java.util.List; -import java.util.function.Supplier; +import java.util.function.Function; import static org.apache.parquet.column.ValuesType.VALUES; @@ -49,10 +50,10 @@ public class ColumnPageReaderImpl implements ColumnPageReader { private final SeekableChannelsProvider channelsProvider; private final CompressorAdapter compressorAdapter; - private final Supplier dictionarySupplier; + private final Function dictionarySupplier; private final PageMaterializer.Factory pageMaterializerFactory; private final ColumnDescriptor path; - private final Path filePath; + private final URI uri; private final List fieldTypes; /** @@ -73,7 +74,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { * page isn't dictionary encoded * @param materializerFactory The factory for creating {@link PageMaterializer}. * @param path The path of the column. - * @param filePath The path of the file. + * @param uri The uri of the parquet file. * @param fieldTypes The types of the fields in the column. * @param offset The offset for page header if supplied {@code pageHeader} is {@code null}. Else, the offset of data * following the header in the page. @@ -83,10 +84,10 @@ public class ColumnPageReaderImpl implements ColumnPageReader { */ ColumnPageReaderImpl(SeekableChannelsProvider channelsProvider, CompressorAdapter compressorAdapter, - Supplier dictionarySupplier, + Function dictionarySupplier, PageMaterializer.Factory materializerFactory, ColumnDescriptor path, - Path filePath, + URI uri, List fieldTypes, long offset, PageHeader pageHeader, @@ -96,7 +97,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { this.dictionarySupplier = dictionarySupplier; this.pageMaterializerFactory = materializerFactory; this.path = path; - this.filePath = filePath; + this.uri = uri; this.fieldTypes = fieldTypes; this.offset = offset; this.pageHeader = pageHeader; @@ -104,15 +105,16 @@ public class ColumnPageReaderImpl implements ColumnPageReader { } @Override - public Object materialize(Object nullValue) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + 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); + return readDataPage(nullValue, readChannel, channelContext); } } - public int readRowCount() throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + public int readRowCount(@NotNull final SeekableChannelContext channelContext) throws IOException { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { ensurePageHeader(readChannel); return readRowCountFromDataPage(readChannel); } @@ -120,10 +122,11 @@ public int readRowCount() throws IOException { @Override - public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + 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); + return readKeyFromDataPage(keyDest, nullPlaceholder, readChannel, channelContext); } } @@ -212,7 +215,8 @@ private int readRowCountFromDataPage(ReadableByteChannel file) throws IOExceptio } private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, - ReadableByteChannel file) throws IOException { + ReadableByteChannel file, @NotNull final SeekableChannelContext channelContext) + throws IOException { int uncompressedPageSize = pageHeader.getUncompressed_page_size(); int compressedPageSize = pageHeader.getCompressed_page_size(); @@ -230,7 +234,7 @@ private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, 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); + getEncoding(dataHeaderV1.getEncoding())), keyDest, nullPlaceholder, channelContext); case DATA_PAGE_V2: DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); @@ -255,7 +259,7 @@ private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, data, uncompressedPageSize, null, // TODO in the future might want to pull in statistics, - false), keyDest, nullPlaceholder); + false), keyDest, nullPlaceholder, channelContext); return null; default: throw new IOException(String.format("Unexpected page of type %s of size %d", pageHeader.getType(), @@ -263,7 +267,8 @@ private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, } } - private Object readDataPage(Object nullValue, SeekableByteChannel file) throws IOException { + private Object readDataPage(Object nullValue, SeekableByteChannel file, + @NotNull SeekableChannelContext channelContext) throws IOException { final int uncompressedPageSize = pageHeader.getUncompressed_page_size(); final int compressedPageSize = pageHeader.getCompressed_page_size(); switch (pageHeader.type) { @@ -280,7 +285,7 @@ private Object readDataPage(Object nullValue, SeekableByteChannel file) throws I 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); + getEncoding(dataHeaderV1.getEncoding())), nullValue, channelContext); case DATA_PAGE_V2: DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); int dataSize = compressedPageSize - dataHeaderV2.getRepetition_levels_byte_length() @@ -329,7 +334,8 @@ private int readRowCountFromPageV1(DataPageV1 page) { } } - private IntBuffer readKeysFromPageV1(DataPageV1 page, IntBuffer keyDest, int nullPlaceholder) { + private IntBuffer readKeysFromPageV1(DataPageV1 page, IntBuffer keyDest, int nullPlaceholder, + @NotNull SeekableChannelContext channelContext) { RunLengthBitPackingHybridBufferDecoder rlDecoder = null; RunLengthBitPackingHybridBufferDecoder dlDecoder = null; try { @@ -360,7 +366,7 @@ private IntBuffer readKeysFromPageV1(DataPageV1 page, IntBuffer keyDest, int nul } ValuesReader dataReader = new KeyIndexReader((DictionaryValuesReader) getDataReader(page.getValueEncoding(), - bytes, page.getValueCount())); + bytes, page.getValueCount(), channelContext)); Object result = materialize(PageMaterializer.IntFactory, dlDecoder, rlDecoder, dataReader, nullPlaceholder); if (result instanceof DataWithOffsets) { @@ -395,7 +401,8 @@ private int readRepetitionLevels(ByteBuffer byteBuffer) throws IOException { return rowsRead; } - private Object readPageV1(DataPageV1 page, Object nullValue) { + 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 @@ -415,7 +422,7 @@ private Object readPageV1(DataPageV1 page, Object nullValue) { bytes.position(bytes.position() + length); } ValuesReader dataReader = - getDataReader(page.getValueEncoding(), bytes, page.getValueCount()); + getDataReader(page.getValueEncoding(), bytes, page.getValueCount(), channelContext); return materialize(pageMaterializerFactory, dlDecoder, rlDecoder, dataReader, nullValue); } catch (IOException e) { @@ -435,7 +442,8 @@ private Object materialize(PageMaterializer.Factory factory, } } - private void readKeysFromPageV2(DataPageV2 page, IntBuffer keyDest, int nullPlaceholder) + private void readKeysFromPageV2(DataPageV2 page, IntBuffer keyDest, int nullPlaceholder, + @NotNull final SeekableChannelContext channelContext) throws IOException { if (path.getMaxRepetitionLevel() > 0) { throw new RuntimeException("Repeating levels not supported"); @@ -450,7 +458,7 @@ private void readKeysFromPageV2(DataPageV2 page, IntBuffer keyDest, int nullPlac // page.getValueCount()); try { ValuesReader dataReader = getDataReader(page.getDataEncoding(), - page.getData().toByteBuffer(), page.getValueCount()); + page.getData().toByteBuffer(), page.getValueCount(), channelContext); if (dlDecoder != null) { readKeysWithNulls(keyDest, nullPlaceholder, dlDecoder, dataReader); } else { @@ -590,13 +598,14 @@ private static Object materializeNonNull(PageMaterializer.Factory factory, int n return factory.makeMaterializerNonNull(dataReader, numberOfValues).fillAll(); } - private ValuesReader getDataReader(Encoding dataEncoding, ByteBuffer in, int valueCount) { + private ValuesReader getDataReader(Encoding dataEncoding, ByteBuffer in, int valueCount, + @NotNull final SeekableChannelContext channelContext) { if (dataEncoding == Encoding.DELTA_BYTE_ARRAY) { throw new RuntimeException("DELTA_BYTE_ARRAY encoding not supported"); } ValuesReader dataReader; if (dataEncoding.usesDictionary()) { - final Dictionary dictionary = dictionarySupplier.get(); + final Dictionary dictionary = dictionarySupplier.apply(channelContext); if (dictionary == ColumnChunkReader.NULL_DICTIONARY) { throw new ParquetDecodingException("Could not read page in col " + path + " as the dictionary was " + "missing for encoding " + dataEncoding); @@ -615,11 +624,11 @@ private ValuesReader getDataReader(Encoding dataEncoding, ByteBuffer in, int val } @Override - public int numValues() throws IOException { + public int numValues(@NotNull final SeekableChannelContext channelContext) throws IOException { if (numValues >= 0) { return numValues; } - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { ensurePageHeader(readChannel); // Above will block till it populates numValues Assert.geqZero(numValues, "numValues"); @@ -629,8 +638,8 @@ public int numValues() throws IOException { @NotNull @Override - public Dictionary getDictionary() { - return dictionarySupplier.get(); + public Dictionary getDictionary(@NotNull final SeekableChannelContext channelContext) { + return dictionarySupplier.apply(channelContext); } @Override @@ -639,12 +648,12 @@ public void close() throws Exception { } @Override - public long numRows() throws IOException { + public long numRows(@NotNull final SeekableChannelContext channelContext) throws IOException { if (rowCount == -1) { if (path.getMaxRepetitionLevel() == 0) { - rowCount = numValues(); + rowCount = numValues(channelContext); } else { - rowCount = readRowCount(); + rowCount = readRowCount(channelContext); } } return rowCount; 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 4545c52b855..2156bf2a299 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 @@ -4,7 +4,8 @@ package io.deephaven.parquet.base; import io.deephaven.parquet.base.util.Helpers; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; import org.apache.parquet.format.*; import org.apache.parquet.format.ColumnOrder; import org.apache.parquet.format.Type; @@ -12,42 +13,59 @@ import java.io.ByteArrayInputStream; import java.io.IOException; +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; -import java.nio.file.Paths; import java.util.*; +import static io.deephaven.util.channel.SeekableChannelsProvider.convertToURI; + /** - * Top level accessor for a parquet file + * Top level accessor for a parquet file which can read both from a file path string or a CLI style file URI, + * ex."s3://bucket/key". */ public class ParquetFileReader { private static final int FOOTER_LENGTH_SIZE = 4; private static final String MAGIC_STR = "PAR1"; static final byte[] MAGIC = MAGIC_STR.getBytes(StandardCharsets.US_ASCII); + public static final String FILE_URI_SCHEME = "file"; public final FileMetaData fileMetaData; private final SeekableChannelsProvider channelsProvider; - private final Path rootPath; + + /** + * If reading a single parquet file, root URI is the URI of the file, else the parent directory for a metadata file + */ + private final URI rootURI; private final MessageType type; - public ParquetFileReader(final String filePath, final SeekableChannelsProvider channelsProvider) + public ParquetFileReader(final String source, final SeekableChannelsProvider channelsProvider) throws IOException { - this.channelsProvider = channelsProvider; - // Root path should be this file if a single file, else the parent directory for a metadata - // file - rootPath = - filePath.endsWith(".parquet") ? Paths.get(filePath) : Paths.get(filePath).getParent(); + this(convertToURI(source), channelsProvider); + } + public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvider channelsProvider) + throws IOException { + this.channelsProvider = channelsProvider; + if (!parquetFileURI.getRawPath().endsWith(".parquet") && FILE_URI_SCHEME.equals(parquetFileURI.getScheme())) { + // Construct a new file URI for the parent directory + rootURI = Path.of(parquetFileURI).getParent().toUri(); + } else { + // TODO(deephaven-core#5066): Add support for reading metadata files from non-file URIs + rootURI = parquetFileURI; + } final byte[] footer; - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + 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( - filePath + " is not a Parquet file (too small length: " + fileLen + ")"); + parquetFileURI + " is not a Parquet file (too small length: " + fileLen + ")"); } final long footerLengthIndex = fileLen - FOOTER_LENGTH_SIZE - MAGIC.length; @@ -58,7 +76,7 @@ public ParquetFileReader(final String filePath, final SeekableChannelsProvider c Helpers.readBytes(readChannel, magic); if (!Arrays.equals(MAGIC, magic)) { throw new InvalidParquetFileException( - filePath + " is not a Parquet file. expected magic number at tail " + parquetFileURI + " is not a Parquet file. expected magic number at tail " + Arrays.toString(MAGIC) + " but found " + Arrays.toString(magic)); } final long footerIndex = footerLengthIndex - footerLength; @@ -182,7 +200,7 @@ public RowGroupReader getRowGroup(final int groupNumber, final String version) { return new RowGroupReaderImpl( fileMetaData.getRow_groups().get(groupNumber), channelsProvider, - rootPath, + rootURI, type, getSchema(), version); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java index ef461e539e1..db4515ed364 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java @@ -4,7 +4,7 @@ package io.deephaven.parquet.base; import org.apache.parquet.format.converter.ParquetMetadataConverter; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProvider; import io.deephaven.parquet.compress.CompressorAdapter; import io.deephaven.parquet.compress.DeephavenCompressorAdapterFactory; import org.apache.parquet.Version; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReader.java index 139418a83b0..71f02961ca3 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReader.java @@ -3,6 +3,7 @@ */ package io.deephaven.parquet.base; +import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.format.RowGroup; import org.jetbrains.annotations.NotNull; @@ -16,9 +17,10 @@ public interface RowGroupReader { * Returns the accessor to a given Column Chunk * * @param path the full column path + * @param channelContext the channel context to use while reading the parquet file * @return the accessor to a given Column Chunk */ - ColumnChunkReader getColumnChunk(@NotNull List path); + ColumnChunkReader getColumnChunk(@NotNull List path, @NotNull final SeekableChannelContext channelContext); long numRows(); 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 90d4e20add3..7ee0dbbdd3d 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,7 +3,8 @@ */ package io.deephaven.parquet.base; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; import org.apache.parquet.format.ColumnChunk; import org.apache.parquet.format.RowGroup; import org.apache.parquet.format.Util; @@ -17,9 +18,9 @@ import java.io.BufferedInputStream; import java.io.IOException; import java.io.UncheckedIOException; +import java.net.URI; import java.nio.channels.Channels; import java.nio.channels.SeekableByteChannel; -import java.nio.file.Path; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -34,19 +35,22 @@ public class RowGroupReaderImpl implements RowGroupReader { private final Map> schemaMap = new HashMap<>(); private final Map chunkMap = new HashMap<>(); - private final Path rootPath; + /** + * If reading a single parquet file, root URI is the URI of the file, else the parent directory for a metadata file + */ + private final URI rootURI; private final String version; RowGroupReaderImpl( @NotNull final RowGroup rowGroup, @NotNull final SeekableChannelsProvider channelsProvider, - @NotNull final Path rootPath, + @NotNull final URI rootURI, @NotNull final MessageType type, @NotNull final MessageType schema, @Nullable final String version) { this.channelsProvider = channelsProvider; this.rowGroup = rowGroup; - this.rootPath = rootPath; + this.rootURI = rootURI; this.type = type; for (ColumnChunk column : rowGroup.columns) { List path_in_schema = column.getMeta_data().path_in_schema; @@ -66,7 +70,8 @@ public class RowGroupReaderImpl implements RowGroupReader { } @Override - public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { + public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path, + @NotNull final SeekableChannelContext channelContext) { String key = path.toString(); ColumnChunk columnChunk = chunkMap.get(key); List fieldTypes = schemaMap.get(key); @@ -76,15 +81,15 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { OffsetIndex offsetIndex = null; if (columnChunk.isSetOffset_index_offset()) { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(rootPath)) { + 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 (IOException e) { + } catch (final IOException e) { throw new UncheckedIOException(e); } } - return new ColumnChunkReaderImpl(columnChunk, channelsProvider, rootPath, type, offsetIndex, fieldTypes, + return new ColumnChunkReaderImpl(columnChunk, channelsProvider, rootURI, type, offsetIndex, fieldTypes, numRows(), version); } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProvider.java deleted file mode 100644 index 2da3f131879..00000000000 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProvider.java +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending - */ -package io.deephaven.parquet.base.util; - -import org.jetbrains.annotations.NotNull; - -import java.io.IOException; -import java.nio.channels.SeekableByteChannel; -import java.nio.file.Path; -import java.nio.file.Paths; - -public interface SeekableChannelsProvider { - - default SeekableByteChannel getReadChannel(@NotNull final String path) throws IOException { - return getReadChannel(Paths.get(path)); - } - - SeekableByteChannel getReadChannel(@NotNull Path path) throws IOException; - - default SeekableByteChannel getWriteChannel(@NotNull final String path, final boolean append) throws IOException { - return getWriteChannel(Paths.get(path), append); - } - - SeekableByteChannel getWriteChannel(@NotNull Path path, boolean append) throws IOException; -} diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 232f9b69b73..d6cbff09f22 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -17,12 +17,12 @@ sourceSets { dependencies { api project(':engine-api') api project(':engine-stringset') + api project(':engine-table') implementation project(':extensions-parquet-base') Classpaths.inheritParquetHadoop(project) implementation project(':engine-base') - implementation project(':engine-table') implementation project(':extensions-csv') implementation project(':log-factory') implementation project(':Configuration') @@ -48,6 +48,9 @@ dependencies { project(path: ':test-configs') Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') + runtimeOnly project(':extensions-trackedfile') + testImplementation project(':extensions-s3') + brotliTestImplementation project(':extensions-parquet-table') brotliTestImplementation('com.github.rdblue:brotli-codec:0.1.1') brotliTestRuntimeOnly project(':log-to-slf4j'), diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java index 718670af146..b209af59c32 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetInstructions.java @@ -11,6 +11,7 @@ import io.deephaven.util.annotations.VisibleForTesting; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Collections; import java.util.Objects; @@ -137,6 +138,8 @@ public final String getColumnNameFromParquetColumnNameOrDefault(final String par */ public abstract boolean useDictionary(String columnName); + public abstract Object getSpecialInstructions(); + public abstract String getCompressionCodecName(); /** @@ -201,6 +204,11 @@ public boolean useDictionary(final String columnName) { return false; } + @Override + public @Nullable String getSpecialInstructions() { + return null; + } + @Override public String getCompressionCodecName() { return defaultCompressionCodecName; @@ -297,6 +305,7 @@ private static final class ReadOnly extends ParquetInstructions { private final boolean isLegacyParquet; private final int targetPageSize; private final boolean isRefreshing; + private final Object specialInstructions; private ReadOnly( final KeyedObjectHashMap columnNameToInstructions, @@ -306,7 +315,8 @@ private ReadOnly( final int maximumDictionarySize, final boolean isLegacyParquet, final int targetPageSize, - final boolean isRefreshing) { + final boolean isRefreshing, + final Object specialInstructions) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; this.compressionCodecName = compressionCodecName; @@ -315,6 +325,7 @@ private ReadOnly( this.isLegacyParquet = isLegacyParquet; this.targetPageSize = targetPageSize; this.isRefreshing = isRefreshing; + this.specialInstructions = specialInstructions; } private String getOrDefault(final String columnName, final String defaultValue, @@ -403,6 +414,12 @@ public boolean isRefreshing() { return isRefreshing; } + @Override + public @Nullable Object getSpecialInstructions() { + return specialInstructions; + } + + KeyedObjectHashMap copyColumnNameToInstructions() { // noinspection unchecked return (columnNameToInstructions == null) @@ -453,6 +470,7 @@ public static class Builder { private boolean isLegacyParquet; private int targetPageSize = defaultTargetPageSize; private boolean isRefreshing = DEFAULT_IS_REFRESHING; + private Object specialInstructions; public Builder() {} @@ -624,6 +642,11 @@ public Builder setIsRefreshing(final boolean isRefreshing) { return this; } + public Builder setSpecialInstructions(final Object specialInstructions) { + this.specialInstructions = specialInstructions; + return this; + } + public ParquetInstructions build() { final KeyedObjectHashMap columnNameToInstructionsOut = columnNameToInstructions; columnNameToInstructions = null; @@ -631,7 +654,8 @@ public ParquetInstructions build() { parquetColumnNameToInstructions; parquetColumnNameToInstructions = null; return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, - maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing); + maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, + specialInstructions); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java index b9648636066..d6bb66d7c10 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java @@ -98,7 +98,8 @@ public static ParquetInstructions readParquetSchema( @NotNull final ParquetInstructions readInstructions, @NotNull final ColumnDefinitionConsumer consumer, @NotNull final BiFunction, String> legalizeColumnNameFunc) throws IOException { - final ParquetFileReader parquetFileReader = ParquetTools.getParquetFileReaderChecked(new File(filePath)); + final ParquetFileReader parquetFileReader = + ParquetTools.getParquetFileReaderChecked(new File(filePath), readInstructions); final ParquetMetadata parquetMetadata = new ParquetMetadataConverter().fromParquetMetadata(parquetFileReader.fileMetaData); return readParquetSchema(parquetFileReader.getSchema(), parquetMetadata.getFileMetaData().getKeyValueMetaData(), diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java index 61ebaa14b91..ad733d6abbf 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java @@ -20,12 +20,12 @@ import io.deephaven.parquet.base.ColumnWriter; import io.deephaven.parquet.base.ParquetFileWriter; import io.deephaven.parquet.base.RowGroupWriter; +import io.deephaven.util.channel.SeekableChannelsProviderLoader; import io.deephaven.parquet.table.metadata.CodecInfo; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; import io.deephaven.parquet.table.metadata.GroupingColumnInfo; import io.deephaven.parquet.table.metadata.TableInfo; import io.deephaven.parquet.table.transfer.*; -import io.deephaven.parquet.table.util.TrackedSeekableChannelsProvider; import io.deephaven.stringset.StringSet; import io.deephaven.util.QueryConstants; import io.deephaven.util.SafeCloseable; @@ -45,6 +45,8 @@ import java.nio.file.Paths; import java.util.*; +import static io.deephaven.util.channel.SeekableChannelsProvider.convertToURI; + /** * API for writing DH tables in parquet format */ @@ -312,7 +314,8 @@ private static ParquetFileWriter getParquetFileWriter( final Map extraMetaData = new HashMap<>(tableMeta); extraMetaData.put(METADATA_KEY, tableInfoBuilder.build().serializeToJSON()); - return new ParquetFileWriter(path, TrackedSeekableChannelsProvider.getInstance(), + return new ParquetFileWriter(path, + SeekableChannelsProviderLoader.getInstance().fromServiceLoader(convertToURI(path), null), writeInstructions.getTargetPageSize(), new HeapByteBufferAllocator(), mappedSchema.getParquetSchema(), writeInstructions.getCompressionCodecName(), extraMetaData); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java index c477d65bf69..5ea3e1ca9ba 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java @@ -14,9 +14,11 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProviderLoader; +import io.deephaven.util.channel.SeekableChannelsProviderPlugin; import io.deephaven.vector.*; import io.deephaven.stringset.StringSet; -import io.deephaven.engine.util.file.TrackedFileHandleFactory; import io.deephaven.engine.table.impl.PartitionAwareSourceTable; import io.deephaven.engine.table.impl.SimpleSourceTable; import io.deephaven.engine.table.impl.locations.TableDataException; @@ -28,7 +30,6 @@ import io.deephaven.parquet.table.layout.ParquetSingleFileLayout; import io.deephaven.parquet.table.location.ParquetTableLocationFactory; import io.deephaven.parquet.table.location.ParquetTableLocationKey; -import io.deephaven.parquet.table.util.TrackedSeekableChannelsProvider; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; import io.deephaven.api.util.NameValidator; import io.deephaven.util.SimpleTypeMap; @@ -37,7 +38,7 @@ import io.deephaven.io.logger.Logger; import io.deephaven.parquet.base.ParquetFileReader; import org.apache.parquet.format.converter.ParquetMetadataConverter; -import io.deephaven.parquet.base.util.CachedChannelProvider; +import io.deephaven.util.channel.CachedChannelProvider; import io.deephaven.util.annotations.VisibleForTesting; import org.apache.commons.lang3.mutable.MutableObject; import org.apache.parquet.hadoop.metadata.ParquetMetadata; @@ -46,12 +47,15 @@ import java.io.File; import java.io.IOException; +import java.net.URI; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributes; import java.util.*; +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; +import static io.deephaven.util.channel.SeekableChannelsProvider.convertToURI; import static io.deephaven.parquet.table.ParquetTableWriter.PARQUET_FILE_EXTENSION; import static io.deephaven.util.type.TypeUtils.getUnboxedTypeIfBoxed; @@ -68,7 +72,9 @@ private ParquetTools() {} private static final Logger log = LoggerFactory.getLogger(ParquetTools.class); /** - * Reads in a table from a single parquet, metadata file, or directory with recognized layout. + * Reads in a table from a single parquet file, metadata file, or directory with recognized layout. The source + * provided can be a local file path or a URI to be resolved via the provided + * {@link SeekableChannelsProviderPlugin}. * *

* This method attempts to "do the right thing." It examines the source to determine if it's a single parquet file, @@ -77,24 +83,27 @@ private ParquetTools() {} * key} order) location found will be used to infer schema. * *

- * Delegates to one of {@link #readSingleFileTable(File, ParquetInstructions)}, + * Delegates to one of {@link #readSingleFileTable(String, ParquetInstructions)}, + * {@link #readSingleFileTable(File, ParquetInstructions)}, * {@link #readPartitionedTableWithMetadata(File, ParquetInstructions)}, * {@link #readFlatPartitionedTable(File, ParquetInstructions)}, or * {@link #readKeyValuePartitionedTable(File, ParquetInstructions)}. * - * @param sourceFilePath The file or directory to examine + * @param source The path or URI of file or directory to examine * @return table * @see ParquetSingleFileLayout * @see ParquetMetadataFileLayout * @see ParquetKeyValuePartitionedLayout * @see ParquetFlatPartitionedLayout */ - public static Table readTable(@NotNull final String sourceFilePath) { - return readTableInternal(new File(sourceFilePath), ParquetInstructions.EMPTY); + public static Table readTable(@NotNull final String source) { + return readTableInternal(convertToURI(source), ParquetInstructions.EMPTY); } /** - * Reads in a table from a single parquet, metadata file, or directory with recognized layout. + * Reads in a table from a single parquet file, metadata file, or directory with recognized layout. The source + * provided can be a local file path or a URI to be resolved via the provided + * {@link SeekableChannelsProviderPlugin}. * *

* This method attempts to "do the right thing." It examines the source to determine if it's a single parquet file, @@ -108,7 +117,7 @@ public static Table readTable(@NotNull final String sourceFilePath) { * {@link #readFlatPartitionedTable(File, ParquetInstructions)}, or * {@link #readKeyValuePartitionedTable(File, ParquetInstructions)}. * - * @param sourceFilePath The file or directory to examine + * @param source The path or URI of file or directory to examine * @param readInstructions Instructions for customizations while reading * @return table * @see ParquetSingleFileLayout @@ -117,13 +126,13 @@ public static Table readTable(@NotNull final String sourceFilePath) { * @see ParquetFlatPartitionedLayout */ public static Table readTable( - @NotNull final String sourceFilePath, + @NotNull final String source, @NotNull final ParquetInstructions readInstructions) { - return readTableInternal(new File(sourceFilePath), readInstructions); + return readTableInternal(convertToURI(source), readInstructions); } /** - * Reads in a table from a single parquet, metadata file, or directory with recognized layout. + * Reads in a table from a single parquet file, metadata file, or directory with recognized layout. * *

* This method attempts to "do the right thing." It examines the source to determine if it's a single parquet file, @@ -149,7 +158,7 @@ public static Table readTable(@NotNull final File sourceFile) { } /** - * Reads in a table from a single parquet, metadata file, or directory with recognized layout. + * Reads in a table from a single parquet file, metadata file, or directory with recognized layout. * *

* This method attempts to "do the right thing." It examines the source to determine if it's a single parquet file, @@ -620,23 +629,24 @@ private static Table readTableInternal( } final String sourceFileName = sourcePath.getFileName().toString(); final BasicFileAttributes sourceAttr = readAttributes(sourcePath); + final File sourceFile = sourcePath.toFile(); if (sourceAttr.isRegularFile()) { if (sourceFileName.endsWith(PARQUET_FILE_EXTENSION)) { return readSingleFileTable(source, instructions); } if (sourceFileName.equals(ParquetMetadataFileLayout.METADATA_FILE_NAME)) { - return readPartitionedTableWithMetadata(source.getParentFile(), instructions); + return readPartitionedTableWithMetadata(sourceFile.getParentFile(), instructions); } if (sourceFileName.equals(ParquetMetadataFileLayout.COMMON_METADATA_FILE_NAME)) { - return readPartitionedTableWithMetadata(source.getParentFile(), instructions); + return readPartitionedTableWithMetadata(sourceFile.getParentFile(), instructions); } throw new TableDataException( - "Source file " + source + " does not appear to be a parquet file or metadata file"); + "Source file " + sourceFile + " does not appear to be a parquet file or metadata file"); } if (sourceAttr.isDirectory()) { final Path metadataPath = sourcePath.resolve(ParquetMetadataFileLayout.METADATA_FILE_NAME); if (Files.exists(metadataPath)) { - return readPartitionedTableWithMetadata(source, instructions); + return readPartitionedTableWithMetadata(sourceFile, instructions); } final Path firstEntryPath; // Ignore dot files while looking for the first entry @@ -653,16 +663,32 @@ private static Table readTableInternal( final String firstEntryFileName = firstEntryPath.getFileName().toString(); final BasicFileAttributes firstEntryAttr = readAttributes(firstEntryPath); if (firstEntryAttr.isDirectory() && firstEntryFileName.contains("=")) { - return readKeyValuePartitionedTable(source, instructions); + return readKeyValuePartitionedTable(sourceFile, instructions); } if (firstEntryAttr.isRegularFile() && firstEntryFileName.endsWith(PARQUET_FILE_EXTENSION)) { - return readFlatPartitionedTable(source, instructions); + return readFlatPartitionedTable(sourceFile, instructions); } throw new TableDataException("No recognized Parquet table layout found in " + source); } throw new TableDataException("Source " + source + " is neither a directory nor a regular file"); } + /** + * Same as {@link #readTableInternal(File, ParquetInstructions)} but with a URI. + * + * @param source The source URI + * @param instructions Instructions for reading + * @return A {@link Table} + */ + private static Table readTableInternal( + @NotNull final URI source, + @NotNull final ParquetInstructions instructions) { + if (!FILE_URI_SCHEME.equals(source.getScheme())) { + return readSingleFileTable(source, instructions); + } + return readTableInternal(new File(source), instructions); + } + private static boolean ignoreDotFiles(Path path) { final String filename = path.getFileName().toString(); return !filename.isEmpty() && filename.charAt(0) != '.'; @@ -701,7 +727,7 @@ public static Table readSingleFileTable( new ParquetTableLocationFactory(readInstructions), null); return new SimpleSourceTable(tableDefinition.getWritable(), - "Read single parquet file from " + tableLocationKey.getFile(), + "Read single parquet file from " + tableLocationKey.getURI(), RegionedTableComponentFactoryImpl.INSTANCE, locationProvider, null); } @@ -846,13 +872,14 @@ public static Table readPartitionedTableWithMetadata( * @param directory the source of {@link ParquetTableLocationKey location keys} to include * @param readInstructions the instructions for customizations while reading * @return the table - * @see ParquetKeyValuePartitionedLayout#ParquetKeyValuePartitionedLayout(File, int) + * @see ParquetKeyValuePartitionedLayout#ParquetKeyValuePartitionedLayout(File, int, ParquetInstructions) * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions) */ public static Table readKeyValuePartitionedTable( @NotNull final File directory, @NotNull final ParquetInstructions readInstructions) { - return readPartitionedTable(new ParquetKeyValuePartitionedLayout(directory, MAX_PARTITIONING_LEVELS_INFERENCE), + return readPartitionedTable( + new ParquetKeyValuePartitionedLayout(directory, MAX_PARTITIONING_LEVELS_INFERENCE, readInstructions), readInstructions); } @@ -864,7 +891,8 @@ public static Table readKeyValuePartitionedTable( * @param readInstructions the instructions for customizations while reading * @param tableDefinition the table definition * @return the table - * @see ParquetKeyValuePartitionedLayout#ParquetKeyValuePartitionedLayout(File, TableDefinition) + * @see ParquetKeyValuePartitionedLayout#ParquetKeyValuePartitionedLayout(File, TableDefinition, + * ParquetInstructions) * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions, TableDefinition) */ public static Table readKeyValuePartitionedTable( @@ -874,8 +902,8 @@ public static Table readKeyValuePartitionedTable( if (tableDefinition.getColumnStream().noneMatch(ColumnDefinition::isPartitioning)) { throw new IllegalArgumentException("No partitioning columns"); } - return readPartitionedTable(new ParquetKeyValuePartitionedLayout(directory, tableDefinition), readInstructions, - tableDefinition); + return readPartitionedTable(new ParquetKeyValuePartitionedLayout(directory, tableDefinition, readInstructions), + readInstructions, tableDefinition); } /** @@ -890,12 +918,12 @@ public static Table readKeyValuePartitionedTable( * @param readInstructions the instructions for customizations while reading * @return the table * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions) - * @see ParquetFlatPartitionedLayout#ParquetFlatPartitionedLayout(File) + * @see ParquetFlatPartitionedLayout#ParquetFlatPartitionedLayout(File, ParquetInstructions) */ public static Table readFlatPartitionedTable( @NotNull final File directory, @NotNull final ParquetInstructions readInstructions) { - return readPartitionedTable(new ParquetFlatPartitionedLayout(directory), readInstructions); + return readPartitionedTable(new ParquetFlatPartitionedLayout(directory, readInstructions), readInstructions); } /** @@ -907,13 +935,14 @@ public static Table readFlatPartitionedTable( * @param tableDefinition the table definition * @return the table * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions, TableDefinition) - * @see ParquetFlatPartitionedLayout#ParquetFlatPartitionedLayout(File) + * @see ParquetFlatPartitionedLayout#ParquetFlatPartitionedLayout(File, ParquetInstructions) */ public static Table readFlatPartitionedTable( @NotNull final File directory, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { - return readPartitionedTable(new ParquetFlatPartitionedLayout(directory), readInstructions, tableDefinition); + return readPartitionedTable(new ParquetFlatPartitionedLayout(directory, readInstructions), readInstructions, + tableDefinition); } /** @@ -926,13 +955,38 @@ public static Table readFlatPartitionedTable( * @param file the parquet file * @param readInstructions the instructions for customizations while reading * @return the table - * @see ParquetTableLocationKey#ParquetTableLocationKey(File, int, Map) + * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) */ public static Table readSingleFileTable( @NotNull final File file, @NotNull final ParquetInstructions readInstructions) { - final ParquetSingleFileLayout keyFinder = new ParquetSingleFileLayout(file); + return readSingleFileTable(file.toURI(), readInstructions); + } + + /** + * Creates a single table via the parquet {@code source} using the provided {@code tableDefinition}. The source + * provided can be a local file path or a URI to be resolved via the provided {@link SeekableChannelsProviderPlugin} + *

+ * Callers wishing to be more explicit (for example, to skip some columns) may prefer to call + * {@link #readSingleFileTable(String, ParquetInstructions, TableDefinition)}. + * + * @param source the path or URI for the parquet file + * @param readInstructions the instructions for customizations while reading + * @return the table + * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) + * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) + */ + public static Table readSingleFileTable( + @NotNull final String source, + @NotNull final ParquetInstructions readInstructions) { + return readSingleFileTable(convertToURI(source), readInstructions); + } + + private static Table readSingleFileTable( + @NotNull final URI parquetFileURI, + @NotNull final ParquetInstructions readInstructions) { + final ParquetSingleFileLayout keyFinder = new ParquetSingleFileLayout(parquetFileURI, readInstructions); final KnownLocationKeyFinder inferenceKeys = toKnownKeys(keyFinder); final Pair inference = infer(inferenceKeys, readInstructions); return readSingleFileTable(inferenceKeys.getFirstKey().orElseThrow(), inference.getSecond(), @@ -946,14 +1000,40 @@ public static Table readSingleFileTable( * @param readInstructions the instructions for customizations while reading * @param tableDefinition the table definition * @return the table - * @see ParquetTableLocationKey#ParquetTableLocationKey(File, int, Map) + * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) */ public static Table readSingleFileTable( @NotNull final File file, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { - return readSingleFileTable(new ParquetTableLocationKey(file, 0, null), readInstructions, tableDefinition); + return readSingleFileTable(file.toURI(), readInstructions, tableDefinition); + } + + /** + * Creates a single table via the parquet {@code source} using the provided {@code tableDefinition}. The source + * provided can be a local file path or a URI to be resolved via the provided {@link SeekableChannelsProviderPlugin} + * + * @param source the path or URI for the parquet file + * @param readInstructions the instructions for customizations while reading + * @param tableDefinition the table definition + * @return the table + * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) + * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) + */ + public static Table readSingleFileTable( + @NotNull final String source, + @NotNull final ParquetInstructions readInstructions, + @NotNull final TableDefinition tableDefinition) { + return readSingleFileTable(convertToURI(source), readInstructions, tableDefinition); + } + + private static Table readSingleFileTable( + @NotNull final URI parquetFileURI, + @NotNull final ParquetInstructions readInstructions, + @NotNull final TableDefinition tableDefinition) { + return readSingleFileTable(new ParquetTableLocationKey(parquetFileURI, 0, null, readInstructions), + readInstructions, tableDefinition); } private static final SimpleTypeMap> VECTOR_TYPE_MAP = SimpleTypeMap.create( @@ -1026,16 +1106,35 @@ private static ParquetSchemaReader.ColumnDefinitionConsumer makeSchemaReaderCons * {@link TableDataException}. * * @param parquetFile The {@link File} to read + * @param readInstructions the instructions for customizations while reading * @return The new {@link ParquetFileReader} */ - public static ParquetFileReader getParquetFileReader(@NotNull final File parquetFile) { + public static ParquetFileReader getParquetFileReader(@NotNull final File parquetFile, + @NotNull final ParquetInstructions readInstructions) { try { - return getParquetFileReaderChecked(parquetFile); + return getParquetFileReaderChecked(parquetFile, readInstructions); } catch (IOException e) { throw new TableDataException("Failed to create Parquet file reader: " + parquetFile, e); } } + /** + * Make a {@link ParquetFileReader} for the supplied {@link URI}. Wraps {@link IOException} as + * {@link TableDataException}. + * + * @param parquetFileURI The {@link URI} to read + * @param readInstructions the instructions for customizations while reading + * @return The new {@link ParquetFileReader} + */ + public static ParquetFileReader getParquetFileReader(@NotNull final URI parquetFileURI, + @NotNull final ParquetInstructions readInstructions) { + try { + return getParquetFileReaderChecked(parquetFileURI, readInstructions); + } catch (IOException e) { + throw new TableDataException("Failed to create Parquet file reader: " + parquetFileURI, e); + } + } + /** * Make a {@link ParquetFileReader} for the supplied {@link File}. * @@ -1043,18 +1142,33 @@ public static ParquetFileReader getParquetFileReader(@NotNull final File parquet * @return The new {@link ParquetFileReader} * @throws IOException if an IO exception occurs */ - public static ParquetFileReader getParquetFileReaderChecked(@NotNull File parquetFile) throws IOException { - return new ParquetFileReader( - parquetFile.getAbsolutePath(), - new CachedChannelProvider( - new TrackedSeekableChannelsProvider(TrackedFileHandleFactory.getInstance()), 1 << 7)); + public static ParquetFileReader getParquetFileReaderChecked( + @NotNull final File parquetFile, + @NotNull final ParquetInstructions readInstructions) throws IOException { + return getParquetFileReaderChecked(parquetFile.toURI(), readInstructions); + } + + /** + * Make a {@link ParquetFileReader} for the supplied {@link URI}. + * + * @param parquetFileURI The {@link URI} to read + * @return The new {@link ParquetFileReader} + * @throws IOException if an IO exception occurs + */ + public static ParquetFileReader getParquetFileReaderChecked( + @NotNull final URI parquetFileURI, + @NotNull final ParquetInstructions readInstructions) throws IOException { + final SeekableChannelsProvider provider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader( + parquetFileURI, readInstructions.getSpecialInstructions()); + return new ParquetFileReader(parquetFileURI, new CachedChannelProvider(provider, 1 << 7)); } @VisibleForTesting public static Table readParquetSchemaAndTable( @NotNull final File source, @NotNull final ParquetInstructions readInstructionsIn, MutableObject instructionsOut) { - final ParquetTableLocationKey tableLocationKey = new ParquetTableLocationKey(source, 0, null); + final ParquetTableLocationKey tableLocationKey = + new ParquetTableLocationKey(source, 0, null, readInstructionsIn); final Pair>, ParquetInstructions> schemaInfo = convertSchema( tableLocationKey.getFileReader().getSchema(), tableLocationKey.getMetadata().getFileMetaData().getKeyValueMetaData(), diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/DeephavenNestedPartitionLayout.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/DeephavenNestedPartitionLayout.java index 081ef2262ef..79f8ee18a1e 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/DeephavenNestedPartitionLayout.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/DeephavenNestedPartitionLayout.java @@ -5,7 +5,8 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; -import io.deephaven.engine.table.impl.locations.local.FileTableLocationKey; +import io.deephaven.engine.table.impl.locations.local.URITableLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import io.deephaven.util.annotations.VisibleForTesting; import org.jetbrains.annotations.NotNull; @@ -31,10 +32,10 @@ * tableRootDirectory/internalPartitionValue/columnPartitionValue/tableName/... * * - * , producing {@link FileTableLocationKey}'s with two partitions, for keys {@value INTERNAL_PARTITION_KEY} and the + * , producing {@link URITableLocationKey}'s with two partitions, for keys {@value INTERNAL_PARTITION_KEY} and the * specified {@code columnPartitionKey}. */ -public abstract class DeephavenNestedPartitionLayout +public abstract class DeephavenNestedPartitionLayout implements TableLocationKeyFinder { @VisibleForTesting @@ -44,14 +45,15 @@ public static DeephavenNestedPartitionLayout forParquet @NotNull final File tableRootDirectory, @NotNull final String tableName, @NotNull final String columnPartitionKey, - @Nullable final Predicate internalPartitionValueFilter) { + @Nullable final Predicate internalPartitionValueFilter, + @NotNull final ParquetInstructions readInstructions) { return new DeephavenNestedPartitionLayout<>(tableRootDirectory, tableName, columnPartitionKey, internalPartitionValueFilter) { @Override protected ParquetTableLocationKey makeKey(@NotNull Path tableLeafDirectory, @NotNull Map> partitions) { return new ParquetTableLocationKey(tableLeafDirectory.resolve(PARQUET_FILE_NAME).toFile(), 0, - partitions); + partitions, readInstructions); } }; } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFlatPartitionedLayout.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFlatPartitionedLayout.java index d81979a7616..bbaefc5d971 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFlatPartitionedLayout.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFlatPartitionedLayout.java @@ -5,6 +5,7 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import org.jetbrains.annotations.NotNull; @@ -22,19 +23,23 @@ */ public final class ParquetFlatPartitionedLayout implements TableLocationKeyFinder { - private static ParquetTableLocationKey locationKey(Path path) { - return new ParquetTableLocationKey(path.toFile(), 0, null); + private static ParquetTableLocationKey locationKey(Path path, @NotNull final ParquetInstructions readInstructions) { + return new ParquetTableLocationKey(path.toFile(), 0, null, readInstructions); } private final File tableRootDirectory; private final Map cache; + private final ParquetInstructions readInstructions; /** * @param tableRootDirectory The directory to search for .parquet files. + * @param readInstructions the instructions for customizations while reading */ - public ParquetFlatPartitionedLayout(@NotNull final File tableRootDirectory) { + public ParquetFlatPartitionedLayout(@NotNull final File tableRootDirectory, + @NotNull final ParquetInstructions readInstructions) { this.tableRootDirectory = tableRootDirectory; - cache = new HashMap<>(); + this.cache = new HashMap<>(); + this.readInstructions = readInstructions; } public String toString() { @@ -48,7 +53,7 @@ public synchronized void findKeys(@NotNull final Consumer new LocationTableBuilderDefinition(tableDefinition), - (path, partitions) -> new ParquetTableLocationKey(path.toFile(), 0, partitions), + (path, partitions) -> new ParquetTableLocationKey(path.toFile(), 0, partitions, readInstructions), Math.toIntExact(tableDefinition.getColumnStream().filter(ColumnDefinition::isPartitioning).count())); } public ParquetKeyValuePartitionedLayout( @NotNull final File tableRootDirectory, - final int maxPartitioningLevels) { + final int maxPartitioningLevels, + @NotNull final ParquetInstructions readInstructions) { super(tableRootDirectory, ParquetFileHelper::fileNameMatches, () -> new LocationTableBuilderCsv(tableRootDirectory), - (path, partitions) -> new ParquetTableLocationKey(path.toFile(), 0, partitions), + (path, partitions) -> new ParquetTableLocationKey(path.toFile(), 0, partitions, readInstructions), maxPartitioningLevels); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetMetadataFileLayout.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetMetadataFileLayout.java index b1720cdbdce..a7ed9a754c3 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetMetadataFileLayout.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetMetadataFileLayout.java @@ -94,7 +94,7 @@ public ParquetMetadataFileLayout( if (!metadataFile.exists()) { throw new TableDataException(String.format("Parquet metadata file %s does not exist", metadataFile)); } - final ParquetFileReader metadataFileReader = ParquetTools.getParquetFileReader(metadataFile); + final ParquetFileReader metadataFileReader = ParquetTools.getParquetFileReader(metadataFile, inputInstructions); final ParquetMetadataConverter converter = new ParquetMetadataConverter(); final ParquetMetadata metadataFileMetadata = convertMetadata(metadataFile, metadataFileReader, converter); @@ -104,7 +104,8 @@ public ParquetMetadataFileLayout( inputInstructions); if (commonMetadataFile != null && commonMetadataFile.exists()) { - final ParquetFileReader commonMetadataFileReader = ParquetTools.getParquetFileReader(commonMetadataFile); + final ParquetFileReader commonMetadataFileReader = + ParquetTools.getParquetFileReader(commonMetadataFile, inputInstructions); final Pair>, ParquetInstructions> fullSchemaInfo = ParquetTools.convertSchema( commonMetadataFileReader.getSchema(), convertMetadata(commonMetadataFile, commonMetadataFileReader, converter).getFileMetaData() @@ -196,7 +197,7 @@ public ParquetMetadataFileLayout( } } final ParquetTableLocationKey tlk = new ParquetTableLocationKey(new File(directory, filePathString), - partitionOrder.getAndIncrement(), partitions); + partitionOrder.getAndIncrement(), partitions, inputInstructions); tlk.setFileReader(metadataFileReader); tlk.setMetadata(metadataFileMetadata); tlk.setRowGroupIndices(rowGroupIndices); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetSingleFileLayout.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetSingleFileLayout.java index 36dc561ee84..8cfd5466df0 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetSingleFileLayout.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetSingleFileLayout.java @@ -4,32 +4,36 @@ package io.deephaven.parquet.table.layout; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import org.jetbrains.annotations.NotNull; -import java.io.File; +import java.net.URI; import java.util.function.Consumer; /** * Parquet {@link TableLocationKeyFinder location finder} that will discover a single file. */ public final class ParquetSingleFileLayout implements TableLocationKeyFinder { - - private final File parquetFile; + private final URI parquetFileUri; + private final ParquetInstructions readInstructions; /** - * @param parquetFile The single parquet file to find + * @param parquetFileUri URI of single parquet file to find + * @param readInstructions the instructions for customizations while reading */ - public ParquetSingleFileLayout(@NotNull final File parquetFile) { - this.parquetFile = parquetFile; + public ParquetSingleFileLayout(@NotNull final URI parquetFileUri, + @NotNull final ParquetInstructions readInstructions) { + this.parquetFileUri = parquetFileUri; + this.readInstructions = readInstructions; } public String toString() { - return ParquetSingleFileLayout.class.getSimpleName() + '[' + parquetFile + ']'; + return ParquetSingleFileLayout.class.getSimpleName() + '[' + parquetFileUri + ']'; } @Override public void findKeys(@NotNull final Consumer locationKeyObserver) { - locationKeyObserver.accept(new ParquetTableLocationKey(parquetFile, 0, null)); + locationKeyObserver.accept(new ParquetTableLocationKey(parquetFileUri, 0, null, readInstructions)); } } 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 6c61c9276e5..79caf1cda64 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 @@ -26,6 +26,8 @@ import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.base.RowGroupReader; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; import org.apache.parquet.format.converter.ParquetMetadataConverter; import io.deephaven.parquet.table.*; import io.deephaven.parquet.table.metadata.CodecInfo; @@ -50,6 +52,7 @@ import java.io.UncheckedIOException; import java.math.BigDecimal; import java.math.BigInteger; +import java.net.URI; import java.util.*; import java.util.function.Function; import java.util.function.LongFunction; @@ -58,6 +61,7 @@ import java.util.stream.Stream; import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK; +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; import static io.deephaven.parquet.table.ParquetTableWriter.*; final class ParquetColumnLocation extends AbstractColumnLocation { @@ -163,16 +167,20 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition if (!hasGroupingTable) { return null; } - final File parquetFile = tl().getParquetFile(); + final URI parquetFileURI = tl().getParquetKey().getURI(); + Assert.assertion(FILE_URI_SCHEME.equals(parquetFileURI.getScheme()), + "Expected a file uri, got " + parquetFileURI); + final File parquetFile = new File(parquetFileURI); try { ParquetFileReader parquetFileReader; final String indexFilePath; final GroupingColumnInfo groupingColumnInfo = tl().getGroupingColumns().get(parquetColumnName); + final SeekableChannelsProvider channelsProvider = tl().getChannelProvider(); if (groupingColumnInfo != null) { final String indexFileRelativePath = groupingColumnInfo.groupingTablePath(); indexFilePath = parquetFile.toPath().getParent().resolve(indexFileRelativePath).toString(); try { - parquetFileReader = new ParquetFileReader(indexFilePath, tl().getChannelProvider()); + parquetFileReader = new ParquetFileReader(indexFilePath, channelsProvider); } catch (final RuntimeException e) { logWarnFailedToRead(indexFilePath); return null; @@ -182,7 +190,7 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition ParquetTools.getRelativeIndexFilePath(parquetFile, parquetColumnName); indexFilePath = parquetFile.toPath().getParent().resolve(relativeIndexFilePath).toString(); try { - parquetFileReader = new ParquetFileReader(indexFilePath, tl().getChannelProvider()); + parquetFileReader = new ParquetFileReader(indexFilePath, channelsProvider); } catch (final RuntimeException e1) { // Retry with legacy grouping file path final String legacyGroupingFileName = @@ -190,7 +198,7 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition final File legacyGroupingFile = new File(parquetFile.getParent(), legacyGroupingFileName); try { parquetFileReader = - new ParquetFileReader(legacyGroupingFile.getAbsolutePath(), tl().getChannelProvider()); + new ParquetFileReader(legacyGroupingFile.getAbsolutePath(), channelsProvider); } catch (final RuntimeException e2) { logWarnFailedToRead(indexFilePath); return null; @@ -205,12 +213,13 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition final String version = tableInfo.map(TableInfo::version).orElse(null); final RowGroupReader rowGroupReader = parquetFileReader.getRowGroup(0, version); - final ColumnChunkReader groupingKeyReader = - rowGroupReader.getColumnChunk(Collections.singletonList(GROUPING_KEY)); - final ColumnChunkReader beginPosReader = - rowGroupReader.getColumnChunk(Collections.singletonList(BEGIN_POS)); - final ColumnChunkReader endPosReader = - rowGroupReader.getColumnChunk(Collections.singletonList(END_POS)); + final ColumnChunkReader groupingKeyReader, beginPosReader, endPosReader; + try (final SeekableChannelContext channelContext = channelsProvider.makeContext()) { + groupingKeyReader = + rowGroupReader.getColumnChunk(Collections.singletonList(GROUPING_KEY), channelContext); + beginPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(BEGIN_POS), channelContext); + endPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(END_POS), channelContext); + } 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 ") 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 88d2ae87acc..912cdffb002 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 @@ -4,7 +4,9 @@ package io.deephaven.parquet.table.location; import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.engine.table.impl.locations.TableLocationState; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.ParquetSchemaReader; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; @@ -19,7 +21,7 @@ import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.base.RowGroupReader; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProvider; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.format.RowGroup; import org.apache.parquet.hadoop.metadata.ParquetMetadata; @@ -29,6 +31,8 @@ import java.util.*; import java.util.stream.IntStream; +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; + public class ParquetTableLocation extends AbstractTableLocation { private static final String IMPLEMENTATION_NAME = ParquetColumnLocation.class.getSimpleName(); @@ -87,7 +91,12 @@ public ParquetTableLocation(@NotNull final TableKey tableKey, columnTypes = tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); version = tableInfo.map(TableInfo::version).orElse(null); - handleUpdate(computeIndex(), tableLocationKey.getFile().lastModified()); + if (!FILE_URI_SCHEME.equals(tableLocationKey.getURI().getScheme())) { + // We do not have the last modified time for non-file URIs + handleUpdate(computeIndex(), TableLocationState.NULL_TIME); + } else { + handleUpdate(computeIndex(), new File(tableLocationKey.getURI()).lastModified()); + } } @Override @@ -98,8 +107,8 @@ public String getImplementationName() { @Override public void refresh() {} - File getParquetFile() { - return ((ParquetTableLocationKey) getKey()).getFile(); + ParquetTableLocationKey getParquetKey() { + return (ParquetTableLocationKey) getKey(); } ParquetInstructions getReadInstructions() { @@ -145,8 +154,11 @@ protected ParquetColumnLocation makeColumnLocation(@NotNull final String final String[] columnPath = parquetColumnNameToPath.get(parquetColumnName); final List nameList = columnPath == null ? Collections.singletonList(parquetColumnName) : Arrays.asList(columnPath); - final ColumnChunkReader[] columnChunkReaders = Arrays.stream(getRowGroupReaders()) - .map(rgr -> rgr.getColumnChunk(nameList)).toArray(ColumnChunkReader[]::new); + final ColumnChunkReader[] columnChunkReaders; + try (final SeekableChannelContext channelContext = getChannelProvider().makeContext()) { + columnChunkReaders = Arrays.stream(getRowGroupReaders()) + .map(rgr -> rgr.getColumnChunk(nameList, channelContext)).toArray(ColumnChunkReader[]::new); + } final boolean exists = Arrays.stream(columnChunkReaders).anyMatch(ccr -> ccr != null && ccr.numRows() > 0); return new ParquetColumnLocation<>(this, columnName, parquetColumnName, exists ? columnChunkReaders : null, diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationFactory.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationFactory.java index 4a047c16c2f..e06608a97e3 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationFactory.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationFactory.java @@ -13,6 +13,9 @@ import org.jetbrains.annotations.Nullable; import java.io.File; +import java.net.URI; + +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; /** * {@link TableLocationFactory} for {@link ParquetTableLocation}s. @@ -30,8 +33,8 @@ public ParquetTableLocationFactory(@NotNull final ParquetInstructions readInstru public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final ParquetTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { - final File parquetFile = locationKey.getFile(); - if (parquetFile.exists()) { + final URI parquetFileURI = locationKey.getURI(); + if (!FILE_URI_SCHEME.equals(parquetFileURI.getScheme()) || new File(parquetFileURI).exists()) { return new ParquetTableLocation(tableKey, locationKey, readInstructions); } else { return new NonexistentTableLocation(tableKey, locationKey); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java index 1925250eb8e..f8f4a24556e 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java @@ -3,10 +3,11 @@ */ package io.deephaven.parquet.table.location; +import io.deephaven.engine.table.impl.locations.local.URITableLocationKey; +import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.ParquetTools; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocationKey; -import io.deephaven.engine.table.impl.locations.local.FileTableLocationKey; import io.deephaven.parquet.table.ParquetTableWriter; import io.deephaven.parquet.base.ParquetFileReader; import org.apache.parquet.format.converter.ParquetMetadataConverter; @@ -17,6 +18,7 @@ import java.io.File; import java.io.IOException; +import java.net.URI; import java.util.List; import java.util.Map; import java.util.stream.IntStream; @@ -24,13 +26,14 @@ /** * {@link TableLocationKey} implementation for use with data stored in the parquet format. */ -public class ParquetTableLocationKey extends FileTableLocationKey { +public class ParquetTableLocationKey extends URITableLocationKey { private static final String IMPLEMENTATION_NAME = ParquetTableLocationKey.class.getSimpleName(); private ParquetFileReader fileReader; private ParquetMetadata metadata; private int[] rowGroupIndices; + private final ParquetInstructions readInstructions; /** * Construct a new ParquetTableLocationKey for the supplied {@code file} and {@code partitions}. @@ -40,17 +43,41 @@ public class ParquetTableLocationKey extends FileTableLocationKey { * @param partitions The table partitions enclosing the table location keyed by {@code this}. Note that if this * parameter is {@code null}, the location will be a member of no partitions. An ordered copy of the map will * be made, so the calling code is free to mutate the map after this call + * @param readInstructions the instructions for customizations while reading */ public ParquetTableLocationKey(@NotNull final File file, final int order, - @Nullable final Map> partitions) { + @Nullable final Map> partitions, + @NotNull final ParquetInstructions readInstructions) { super(validateParquetFile(file), order, partitions); + this.readInstructions = readInstructions; } - private static File validateParquetFile(@NotNull final File file) { - if (!file.getName().endsWith(ParquetTableWriter.PARQUET_FILE_EXTENSION)) { + /** + * Construct a new ParquetTableLocationKey for the supplied {@code parquetFileUri} and {@code partitions}. + * + * @param parquetFileUri The parquet file that backs the keyed location. Will be adjusted to an absolute path. + * @param order Explicit ordering index, taking precedence over other fields + * @param partitions The table partitions enclosing the table location keyed by {@code this}. Note that if this + * parameter is {@code null}, the location will be a member of no partitions. An ordered copy of the map will + * be made, so the calling code is free to mutate the map after this call + * @param readInstructions the instructions for customizations while reading + */ + public ParquetTableLocationKey(@NotNull final URI parquetFileUri, final int order, + @Nullable final Map> partitions, + @NotNull final ParquetInstructions readInstructions) { + super(validateParquetFile(parquetFileUri), order, partitions); + this.readInstructions = readInstructions; + } + + private static URI validateParquetFile(@NotNull final File file) { + return validateParquetFile(file.toURI()); + } + + private static URI validateParquetFile(@NotNull final URI parquetFileUri) { + if (!parquetFileUri.getRawPath().endsWith(ParquetTableWriter.PARQUET_FILE_EXTENSION)) { throw new IllegalArgumentException("Parquet file must end in " + ParquetTableWriter.PARQUET_FILE_EXTENSION); } - return file; + return parquetFileUri; } @Override @@ -72,7 +99,7 @@ public String getImplementationName() { * * * Callers wishing to handle these cases more explicit may call - * {@link ParquetTools#getParquetFileReaderChecked(File)}. + * {@link ParquetTools#getParquetFileReaderChecked(URI, ParquetInstructions)}. * * @return true if the file reader exists or was successfully created */ @@ -81,7 +108,7 @@ public synchronized boolean verifyFileReader() { return true; } try { - fileReader = ParquetTools.getParquetFileReaderChecked(file); + fileReader = ParquetTools.getParquetFileReaderChecked(uri, readInstructions); } catch (IOException e) { return false; } @@ -98,7 +125,7 @@ public synchronized ParquetFileReader getFileReader() { if (fileReader != null) { return fileReader; } - return fileReader = ParquetTools.getParquetFileReader(file); + return fileReader = ParquetTools.getParquetFileReader(uri, readInstructions); } /** @@ -127,7 +154,7 @@ public synchronized ParquetMetadata getMetadata() { try { return metadata = new ParquetMetadataConverter().fromParquetMetadata(getFileReader().fileMetaData); } catch (IOException e) { - throw new TableDataException("Failed to convert Parquet file metadata: " + getFile(), e); + throw new TableDataException("Failed to convert Parquet file metadata: " + getURI(), e); } } @@ -162,7 +189,7 @@ public synchronized int[] getRowGroupIndices() { // we're not expecting that in this code path. To support it, discovery tools should figure out // the row groups for a partition themselves and call setRowGroupReaders. final String filePath = rowGroups.get(rgi).getColumns().get(0).getFile_path(); - return filePath == null || new File(filePath).getAbsoluteFile().equals(file); + return filePath == null || new File(filePath).getAbsoluteFile().toURI().equals(uri); }).toArray(); } 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 c91911de204..05caefa1e13 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 @@ -8,6 +8,8 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Context; import io.deephaven.engine.table.SharedContext; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.engine.table.Releasable; import io.deephaven.chunk.attributes.Any; @@ -133,9 +135,10 @@ public static CreatorResult create( this.numRows = Require.inRange(columnChunkReader.numRows(), "numRows", mask, "mask"); } - ChunkPage toPage(final long offset, @NotNull final ColumnPageReader columnPageReader) + ChunkPage toPage(final long offset, @NotNull final ColumnPageReader columnPageReader, + @NotNull final SeekableChannelContext channelContext) throws IOException { - return toPage.toPage(offset, columnPageReader, mask); + return toPage.toPage(offset, columnPageReader, channelContext, mask); } @Override @@ -171,27 +174,66 @@ public boolean usesDictionaryOnEveryPage() { @Override public void close() {} - FillContext innerFillContext(@NotNull final FillContext context) { - // TODO(deephaven-core#4836): Call this method from the appropriate place in the implementation of - // getPageContaining to populate the context object - return ((PagingContextHolder) context) - .updateInnerContext(this::fillContextUpdater); + /** + * Wrapper class for holding a {@link SeekableChannelContext}. + */ + private static class ChannelContextWrapper extends PagingContextHolder { + @NotNull + private final SeekableChannelContext channelContext; + + private ChannelContextWrapper( + final int chunkCapacity, + @Nullable final SharedContext sharedContext, + @NotNull final SeekableChannelContext channelContext) { + super(chunkCapacity, sharedContext); + this.channelContext = channelContext; + } + + @NotNull + SeekableChannelContext getChannelContext() { + return channelContext; + } + + @Override + public void close() { + super.close(); + channelContext.close(); + } } - private boolean isFillContextCompatible(@Nullable final Context currentInnerContext) { - // TODO(deephaven-core#4836): Replace this with a test to see if the fill context comes from - // this.ColumnChunkReader - return currentInnerContext == DEFAULT_FILL_INSTANCE; + /** + * Take an object of {@link PagingContextHolder} and populate the inner context with values from + * {@link #columnChunkReader}, if required. + * + * @param context The context to populate. + * @return The {@link SeekableChannelContext} to use for reading pages via {@link #columnChunkReader}. + */ + final SeekableChannelContext innerFillContext(@Nullable final FillContext context) { + if (context != null) { + // Assuming PagingContextHolder is holding an object of ChannelContextWrapper + final ChannelContextWrapper innerContext = + ((PagingContextHolder) context).updateInnerContext(this::fillContextUpdater); + return innerContext.getChannelContext(); + } + return SeekableChannelContext.NULL; } private T fillContextUpdater( int chunkCapacity, @Nullable final SharedContext sharedContext, @Nullable final Context currentInnerContext) { + final SeekableChannelsProvider channelsProvider = columnChunkReader.getChannelsProvider(); + if (currentInnerContext != null) { + // Check if we can reuse the context object + final SeekableChannelContext channelContext = + ((ChannelContextWrapper) currentInnerContext).getChannelContext(); + if (channelsProvider.isCompatibleWith(channelContext)) { + // noinspection unchecked + return (T) currentInnerContext; + } + } + // Create a new context object // noinspection unchecked - return (T) (isFillContextCompatible(currentInnerContext) - ? currentInnerContext - // TODO(deephaven-core#4836): Replace this with getting a context from this.ColumnChunkReader - : makeFillContext(chunkCapacity, sharedContext)); + return (T) new ChannelContextWrapper(chunkCapacity, sharedContext, channelsProvider.makeContext()); } } 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 7518470d8fe..23ad57f5e63 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,6 +7,7 @@ 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.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; @@ -98,7 +99,7 @@ private static int findPageNumUsingOffsetIndex(final OffsetIndex offsetIndex, fi return (low - 1); // 'row' is somewhere in the middle of page } - private ChunkPage getPage(final int pageNum) { + private ChunkPage getPage(@Nullable final FillContext fillContext, final int pageNum) { if (pageNum < 0 || pageNum >= numPages) { throw new IllegalArgumentException("pageNum " + pageNum + " is out of range [0, " + numPages + ")"); } @@ -115,11 +116,12 @@ private ChunkPage getPage(final int pageNum) { synchronized (pageState) { // Make sure no one materialized this page as we waited for the lock if ((localRef = pageState.pageRef) == null || (page = localRef.get()) == null) { - // TODO(deephaven-core#4836): getPage() should accept the outer fill context, and get an inner fill - // context from this.ColumnChunkReader to pass into getPageReader. + // 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)); + page = new PageCache.IntrusivePage<>( + toPage(offsetIndex.getFirstRowIndex(pageNum), reader, channelContext)); } catch (final IOException except) { throw new UncheckedIOException(except); } @@ -135,7 +137,7 @@ private ChunkPage getPage(final int pageNum) { @NotNull public ChunkPage getPageContaining(@Nullable final FillContext fillContext, long rowKey) { rowKey &= mask(); - Require.inRange(rowKey, "row", numRows(), "numRows"); + Require.inRange(rowKey, "rowKey", numRows(), "numRows"); int pageNum; if (fixedPageSize == PAGE_SIZE_NOT_FIXED) { @@ -150,6 +152,7 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext pageNum = (numPages - 1); } } - return getPage(pageNum); + + return getPage(fillContext, pageNum); } } 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 df27c76ba38..7d24aba9ec5 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 @@ -7,6 +7,7 @@ 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.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; @@ -17,7 +18,6 @@ import java.io.UncheckedIOException; import java.lang.ref.WeakReference; import java.util.Arrays; -import java.util.Iterator; final class VariablePageSizeColumnChunkPageStore extends ColumnChunkPageStore { @@ -29,7 +29,7 @@ final class VariablePageSizeColumnChunkPageStore extends Colum private volatile int numPages = 0; private volatile long[] pageRowOffsets; private volatile ColumnPageReader[] columnPageReaders; - private final Iterator columnPageReaderIterator; + private final ColumnChunkReader.ColumnPageReaderIterator columnPageReaderIterator; private volatile WeakReference>[] pages; VariablePageSizeColumnChunkPageStore( @@ -51,11 +51,11 @@ final class VariablePageSizeColumnChunkPageStore extends Colum pages = (WeakReference>[]) new WeakReference[INIT_ARRAY_SIZE]; } - private void extendOnePage(final int prevNumPages) { + private void extendOnePage(@NotNull final SeekableChannelContext channelContext, final int prevNumPages) { PageCache.IntrusivePage page = null; synchronized (this) { - int localNumPages = numPages; + final int localNumPages = numPages; // Make sure that no one has already extended to this page yet. if (localNumPages == prevNumPages) { @@ -64,28 +64,27 @@ private void extendOnePage(final int prevNumPages) { "Parquet num rows and page iterator don't match, not enough pages."); if (columnPageReaders.length == localNumPages) { - int newSize = 2 * localNumPages; + final int newSize = 2 * localNumPages; pageRowOffsets = Arrays.copyOf(pageRowOffsets, newSize + 1); columnPageReaders = Arrays.copyOf(columnPageReaders, newSize); pages = Arrays.copyOf(pages, newSize); } - final ColumnPageReader columnPageReader = columnPageReaderIterator.next(); - + final ColumnPageReader columnPageReader = columnPageReaderIterator.next(channelContext); long numRows; WeakReference> pageRef = PageCache.getNullPage(); - long prevRowOffset = pageRowOffsets[localNumPages]; + final long prevRowOffset = pageRowOffsets[localNumPages]; try { - numRows = columnPageReader.numRows(); + numRows = columnPageReader.numRows(channelContext); if (numRows < 0) { - page = new PageCache.IntrusivePage<>(toPage(prevRowOffset, columnPageReader)); + page = new PageCache.IntrusivePage<>(toPage(prevRowOffset, columnPageReader, channelContext)); pageRef = new WeakReference<>(page); numRows = page.getPage().size(); } - } catch (IOException except) { + } catch (final IOException except) { throw new UncheckedIOException(except); } @@ -101,19 +100,21 @@ private void extendOnePage(final int prevNumPages) { } } - private int fillToRow(int minPageNum, long row) { + private int fillToRow(@NotNull final SeekableChannelContext channelContext, int minPageNum, + long row) { int localNumPages = numPages; while (row >= pageRowOffsets[localNumPages]) { minPageNum = localNumPages; - extendOnePage(localNumPages); + extendOnePage(channelContext, localNumPages); localNumPages = numPages; } return minPageNum; } - private ChunkPage getPage(final int pageNum) { + private ChunkPage getPage(@NotNull final SeekableChannelContext channelContext, + final int pageNum) { PageCache.IntrusivePage page = pages[pageNum].get(); if (page == null) { @@ -123,9 +124,9 @@ private ChunkPage getPage(final int pageNum) { if (page == null) { try { - page = new PageCache.IntrusivePage<>( - toPage(pageRowOffsets[pageNum], columnPageReaders[pageNum])); - } catch (IOException except) { + page = new PageCache.IntrusivePage<>(toPage(pageRowOffsets[pageNum], columnPageReaders[pageNum], + channelContext)); + } catch (final IOException except) { throw new UncheckedIOException(except); } @@ -144,7 +145,7 @@ private ChunkPage getPage(final int pageNum) { @NotNull public ChunkPage getPageContaining(@Nullable final FillContext fillContext, long rowKey) { rowKey &= mask(); - Require.inRange(rowKey - pageRowOffsets[0], "row", numRows(), "numRows"); + Require.inRange(rowKey - pageRowOffsets[0], "rowKey", numRows(), "numRows"); int localNumPages = numPages; int pageNum = Arrays.binarySearch(pageRowOffsets, 1, localNumPages + 1, rowKey); @@ -153,8 +154,11 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext pageNum = -2 - pageNum; } + // Use the latest channel context while reading page headers + final SeekableChannelContext channelContext = innerFillContext(fillContext); + if (pageNum >= localNumPages) { - int minPageNum = fillToRow(localNumPages, rowKey); + final int minPageNum = fillToRow(channelContext, localNumPages, rowKey); localNumPages = numPages; pageNum = Arrays.binarySearch(pageRowOffsets, minPageNum + 1, localNumPages + 1, rowKey); @@ -163,6 +167,6 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext } } - return getPage(pageNum); + return getPage(channelContext, pageNum); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ChunkDictionary.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ChunkDictionary.java index 92010372faa..ac57f7686e9 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ChunkDictionary.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ChunkDictionary.java @@ -6,19 +6,20 @@ import gnu.trove.map.TObjectIntMap; import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.chunk.attributes.Any; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.stringset.LongBitmapStringSet; import io.deephaven.chunk.ObjectChunk; import io.deephaven.util.datastructures.LazyCachingSupplier; import org.apache.parquet.column.Dictionary; import org.jetbrains.annotations.NotNull; +import java.util.function.Function; import java.util.function.Supplier; /** * Chunk-backed dictionary for use by {@link ToPage} implementations. */ -public class ChunkDictionary - implements LongBitmapStringSet.ReversibleLookup { +public class ChunkDictionary implements LongBitmapStringSet.ReversibleLookup { @FunctionalInterface public interface Lookup { @@ -34,9 +35,6 @@ public interface Lookup { T lookup(@NotNull final Dictionary dictionary, final int key); } - private final Lookup lookup; - private final Supplier dictionarySupplier; - private final Supplier> valuesSupplier; private final Supplier> reverseMapSupplier; @@ -48,11 +46,10 @@ public interface Lookup { */ ChunkDictionary( @NotNull final Lookup lookup, - @NotNull final Supplier dictionarySupplier) { - this.lookup = lookup; - this.dictionarySupplier = dictionarySupplier; + @NotNull final Function dictionarySupplier) { this.valuesSupplier = new LazyCachingSupplier<>(() -> { - final Dictionary dictionary = dictionarySupplier.get(); + // We use NULL channel context here and rely on materialization logic to provide the correct context + final Dictionary dictionary = dictionarySupplier.apply(SeekableChannelContext.NULL); final T[] values = ObjectChunk.makeArray(dictionary.getMaxId() + 1); for (int ki = 0; ki < values.length; ++ki) { values[ki] = lookup.lookup(dictionary, ki); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToObjectPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToObjectPage.java index 5b7dd26c1e9..c2b0e667d27 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToObjectPage.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToObjectPage.java @@ -5,13 +5,14 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.codec.ObjectCodec; import org.apache.parquet.column.Dictionary; import org.apache.parquet.io.api.Binary; import org.jetbrains.annotations.NotNull; import java.lang.reflect.Array; -import java.util.function.Supplier; +import java.util.function.Function; public class ToObjectPage implements ToPage { @@ -22,7 +23,7 @@ public class ToObjectPage implements ToPage { ToPage create( final Class nativeType, @NotNull final ObjectCodec codec, - final Supplier dictionarySupplier) { + final Function dictionarySupplier) { if (!nativeType.isPrimitive()) { return dictionarySupplier == null ? new ToObjectPage<>(nativeType, codec) : new ToPageWithDictionary<>( diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToPage.java index a395c1a09da..e3098678084 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToPage.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToPage.java @@ -6,6 +6,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.ChunkPageFactory; import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.vector.Vector; import io.deephaven.engine.page.ChunkPage; import io.deephaven.vector.VectorFactory; @@ -53,8 +54,9 @@ default Object nullValue() { /** * @return Gets the result from the columnPageReader. */ - default Object getResult(ColumnPageReader columnPageReader) throws IOException { - return columnPageReader.materialize(nullValue()); + default Object getResult(ColumnPageReader columnPageReader, + SeekableChannelContext channelContext) throws IOException { + return columnPageReader.materialize(nullValue(), channelContext); } /** @@ -78,10 +80,11 @@ default Vector makeVector(RESULT result) { */ @NotNull @FinalDefault - default ChunkPage toPage(long offset, ColumnPageReader columnPageReader, long mask) + default ChunkPage toPage(long offset, ColumnPageReader columnPageReader, + @NotNull final SeekableChannelContext channelContext, long mask) throws IOException { return ChunkPageFactory.forChunkType(getChunkType()) - .pageWrap(offset, convertResult(getResult(columnPageReader)), mask); + .pageWrap(offset, convertResult(getResult(columnPageReader, channelContext)), mask); } /** @@ -124,8 +127,9 @@ public Object nullValue() { @NotNull @Override - public Object getResult(ColumnPageReader columnPageReader) throws IOException { - return toPage.getResult(columnPageReader); + public Object getResult(ColumnPageReader columnPageReader, + SeekableChannelContext channelContext) throws IOException { + return toPage.getResult(columnPageReader, channelContext); } @Override diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToPageWithDictionary.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToPageWithDictionary.java index 35258d16cd7..d733e8a198f 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToPageWithDictionary.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToPageWithDictionary.java @@ -3,6 +3,7 @@ */ package io.deephaven.parquet.table.pagestore.topage; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.stringset.LongBitmapStringSet; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; @@ -51,14 +52,14 @@ public final ChunkType getChunkType() { @Override @NotNull - public final Object getResult(@NotNull final ColumnPageReader columnPageReader) - throws IOException { - if (columnPageReader.getDictionary() == ColumnChunkReader.NULL_DICTIONARY) { - return ToPage.super.getResult(columnPageReader); + public final Object getResult(@NotNull final ColumnPageReader columnPageReader, + @NotNull final SeekableChannelContext channelContext) throws IOException { + if (columnPageReader.getDictionary(channelContext) == ColumnChunkReader.NULL_DICTIONARY) { + return ToPage.super.getResult(columnPageReader, channelContext); } - final int[] keys = new int[columnPageReader.numValues()]; - final IntBuffer offsets = columnPageReader.readKeyValues(IntBuffer.wrap(keys), NULL_INT); + final int[] keys = new int[columnPageReader.numValues(channelContext)]; + final IntBuffer offsets = columnPageReader.readKeyValues(IntBuffer.wrap(keys), NULL_INT, channelContext); return offsets == null ? keys : new DataWithOffsets(offsets, keys); } @@ -115,9 +116,10 @@ public Object nullValue() { } @Override - public Object getResult(@NotNull final ColumnPageReader columnPageReader) + public Object getResult(@NotNull final ColumnPageReader columnPageReader, + @NotNull final SeekableChannelContext channelContext) throws IOException { - return ToPageWithDictionary.this.getResult(columnPageReader); + return ToPageWithDictionary.this.getResult(columnPageReader, channelContext); } @Override diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringPage.java index b9a43add297..2b3a61e97f6 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringPage.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringPage.java @@ -5,11 +5,12 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; +import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.column.Dictionary; import org.apache.parquet.io.api.Binary; import org.jetbrains.annotations.NotNull; -import java.util.function.Supplier; +import java.util.function.Function; public class ToStringPage implements ToPage { @@ -17,7 +18,7 @@ public class ToStringPage implements ToPage { public static ToPage create( final Class nativeType, - final Supplier dictionarySupplier) { + final Function dictionarySupplier) { if (nativeType == null || String.class.equals(nativeType)) { // noinspection unchecked return dictionarySupplier == null ? (ToPage) INSTANCE diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringSetPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringSetPage.java index 66166a6f9da..7a69cb2f2cc 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringSetPage.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringSetPage.java @@ -4,6 +4,7 @@ package io.deephaven.parquet.table.pagestore.topage; import io.deephaven.chunk.attributes.Any; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.stringset.HashStringSet; import io.deephaven.stringset.StringSet; import io.deephaven.stringset.LongBitmapStringSet; @@ -98,8 +99,9 @@ public final ChunkType getChunkType() { @Override @NotNull - public final Object getResult(ColumnPageReader columnPageReader) throws IOException { - return toPage.getDictionaryKeysToPage().getResult(columnPageReader); + public Object getResult(ColumnPageReader columnPageReader, + SeekableChannelContext channelContext) throws IOException { + return toPage.getDictionaryKeysToPage().getResult(columnPageReader, channelContext); } @Override diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java index 1ea62f13652..a59a259799f 100644 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java @@ -6,6 +6,7 @@ import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.Selectable; import io.deephaven.base.FileUtils; +import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.primitive.function.ByteConsumer; import io.deephaven.engine.primitive.function.CharConsumer; @@ -26,11 +27,13 @@ import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.engine.util.BigDecimalUtils; import io.deephaven.engine.util.file.TrackedFileHandleFactory; +import io.deephaven.extensions.s3.AwsCredentials; import io.deephaven.parquet.base.NullStatistics; import io.deephaven.parquet.base.InvalidParquetFileException; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import io.deephaven.parquet.table.pagestore.ColumnChunkPageStore; import io.deephaven.parquet.table.transfer.StringDictionary; +import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.stringset.ArrayStringSet; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; @@ -54,6 +57,7 @@ import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.PrimitiveType; import org.junit.After; +import org.junit.Assume; import org.junit.Before; import org.junit.Ignore; import org.junit.Rule; @@ -64,6 +68,8 @@ import java.io.Serializable; import java.math.BigDecimal; import java.math.BigInteger; +import java.net.URI; +import java.time.Duration; import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; @@ -112,6 +118,10 @@ public final class ParquetTableReadWriteTest { private static final ParquetInstructions EMPTY = ParquetInstructions.EMPTY; private static final ParquetInstructions REFRESHING = ParquetInstructions.builder().setIsRefreshing(true).build(); + // TODO(deephaven-core#5064): Add support for local S3 testing + private static final boolean ENABLE_S3_TESTING = + Configuration.getInstance().getBooleanWithDefault("ParquetTest.enableS3Testing", false); + private static File rootFile; @Rule @@ -394,7 +404,7 @@ public void test_lz4_compressed() { } catch (RuntimeException e) { TestCase.fail("Failed to read parquet file sample_lz4_compressed.parquet"); } - File randomDest = new File(rootFile, "random.parquet"); + final File randomDest = new File(rootFile, "random.parquet"); writeTable(fromDisk, randomDest, ParquetTools.LZ4_RAW); // Read the LZ4 compressed file again, to make sure we use a new adapter @@ -554,7 +564,7 @@ public void testArrayColumns() { writeReadTableTest(arrayTable, dest, writeInstructions); // Make sure the column didn't use dictionary encoding - ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null).getMetadata(); + ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null, ParquetInstructions.EMPTY).getMetadata(); String firstColumnMetadata = metadata.getBlocks().get(0).getColumns().get(0).toString(); assertTrue(firstColumnMetadata.contains("someStringArrayColumn") && !firstColumnMetadata.contains("RLE_DICTIONARY")); @@ -563,12 +573,90 @@ public void testArrayColumns() { writeReadTableTest(vectorTable, dest, writeInstructions); // Make sure the column didn't use dictionary encoding - metadata = new ParquetTableLocationKey(dest, 0, null).getMetadata(); + metadata = new ParquetTableLocationKey(dest, 0, null, ParquetInstructions.EMPTY).getMetadata(); firstColumnMetadata = metadata.getBlocks().get(0).getColumns().get(0).toString(); assertTrue(firstColumnMetadata.contains("someStringArrayColumn") && !firstColumnMetadata.contains("RLE_DICTIONARY")); } + @Test + public void readSampleParquetFilesFromS3Test1() { + Assume.assumeTrue("Skipping test because s3 testing disabled.", ENABLE_S3_TESTING); + final S3Instructions s3Instructions = S3Instructions.builder() + .awsRegionName("us-east-1") + .readAheadCount(1) + .fragmentSize(5 * 1024 * 1024) + .maxConcurrentRequests(50) + .maxCacheSize(32) + .readTimeout(Duration.ofSeconds(60)) + .credentials(AwsCredentials.defaultCredentials()) + .build(); + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setSpecialInstructions(s3Instructions) + .build(); + final Table fromAws1 = + ParquetTools.readTable("s3://dh-s3-parquet-test1/multiColFile.parquet", readInstructions).select(); + final Table dhTable1 = TableTools.emptyTable(1_000_000).update("A=(int)i", "B=(double)(i+1)"); + assertTableEquals(fromAws1, dhTable1); + + final Table fromAws2 = + ParquetTools.readTable("s3://dh-s3-parquet-test1/singleColFile.parquet", readInstructions).select(); + final Table dhTable2 = TableTools.emptyTable(5).update("A=(int)i"); + assertTableEquals(fromAws2, dhTable2); + + final Table fromAws3 = ParquetTools + .readTable("s3://dh-s3-parquet-test1/single%20col%20file%20with%20spaces%20in%20name.parquet", + readInstructions) + .select(); + assertTableEquals(fromAws3, dhTable2); + + final Table fromAws4 = + ParquetTools.readTable("s3://dh-s3-parquet-test1/singleColFile.parquet", readInstructions) + .select().sumBy(); + final Table dhTable4 = TableTools.emptyTable(5).update("A=(int)i").sumBy(); + assertTableEquals(fromAws4, dhTable4); + } + + @Test + public void readSampleParquetFilesFromS3Test2() { + Assume.assumeTrue("Skipping test because s3 testing disabled.", ENABLE_S3_TESTING); + final S3Instructions s3Instructions = S3Instructions.builder() + .awsRegionName("us-east-2") + .readAheadCount(1) + .fragmentSize(5 * 1024 * 1024) + .maxConcurrentRequests(50) + .maxCacheSize(32) + .connectionTimeout(Duration.ofSeconds(1)) + .readTimeout(Duration.ofSeconds(60)) + .build(); + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setSpecialInstructions(s3Instructions) + .build(); + final TableDefinition tableDefinition = TableDefinition.of( + ColumnDefinition.ofString("hash"), + ColumnDefinition.ofLong("version"), + ColumnDefinition.ofLong("size"), + ColumnDefinition.ofString("block_hash"), + ColumnDefinition.ofLong("block_number"), + ColumnDefinition.ofLong("index"), + ColumnDefinition.ofLong("virtual_size"), + ColumnDefinition.ofLong("lock_time"), + ColumnDefinition.ofLong("input_count"), + ColumnDefinition.ofLong("output_count"), + ColumnDefinition.ofBoolean("isCoinbase"), + ColumnDefinition.ofDouble("output_value"), + ColumnDefinition.ofTime("last_modified"), + ColumnDefinition.ofDouble("input_value")); + + ParquetTools.readSingleFileTable( + "s3://aws-public-blockchain/v1.0/btc/transactions/date=2009-01-03/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet", + readInstructions, tableDefinition).select(); + + ParquetTools.readSingleFileTable( + "s3://aws-public-blockchain/v1.0/btc/transactions/date=2023-11-13/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet", + readInstructions, tableDefinition).select(); + } + @Test public void stringDictionaryTest() { final int nullPos = -5; @@ -646,7 +734,7 @@ private static Table readParquetFileFromGitLFS(final File dest) { * Test if the current code can read the parquet data written by the old code. There is logic in * {@link ColumnChunkPageStore#create} that decides page store based on the version of the parquet file. The old * data is generated using following logic: - * + * *

      *  // Enforce a smaller page size to write multiple pages
      *  final ParquetInstructions writeInstructions = new ParquetInstructions.Builder()
@@ -670,7 +758,8 @@ private static Table readParquetFileFromGitLFS(final File dest) {
     public void testReadOldParquetData() {
         String path = ParquetTableReadWriteTest.class.getResource("/ReferenceParquetData.parquet").getFile();
         readParquetFileFromGitLFS(new File(path)).select();
-        final ParquetMetadata metadata = new ParquetTableLocationKey(new File(path), 0, null).getMetadata();
+        final ParquetMetadata metadata =
+                new ParquetTableLocationKey(new File(path), 0, null, ParquetInstructions.EMPTY).getMetadata();
         assertTrue(metadata.getFileMetaData().getKeyValueMetaData().get("deephaven").contains("\"version\":\"0.4.0\""));
 
         path = ParquetTableReadWriteTest.class.getResource("/ReferenceParquetVectorData.parquet").getFile();
@@ -831,6 +920,41 @@ private static void basicWriteTestsImpl(TestParquetTableWriter writer) {
         FileUtils.deleteRecursively(parentDir);
     }
 
+    @Test
+    public void basicWriteAndReadFromFileURITests() {
+        final Table tableToSave = TableTools.emptyTable(5).update("A=(int)i", "B=(long)i", "C=(double)i");
+        final String filename = "basicWriteTests.parquet";
+        final File destFile = new File(rootFile, filename);
+        final String absolutePath = destFile.getAbsolutePath();
+        final URI fileURI = destFile.toURI();
+        ParquetTools.writeTable(tableToSave, absolutePath);
+
+        // Read from file URI
+        final Table fromDisk = ParquetTools.readTable(fileURI.toString());
+        assertTableEquals(tableToSave, fromDisk);
+
+        // Read from "file://" + absolutePath
+        final Table fromDisk2 = ParquetTools.readTable("file://" + absolutePath);
+        assertTableEquals(tableToSave, fromDisk2);
+
+        // Read from absolutePath
+        final Table fromDisk3 = ParquetTools.readTable(absolutePath);
+        assertTableEquals(tableToSave, fromDisk3);
+
+        // Read from relative path
+        final String relativePath = rootFile.getName() + "/" + filename;
+        final Table fromDisk4 = ParquetTools.readTable(relativePath);
+        assertTableEquals(tableToSave, fromDisk4);
+
+        // Read from unsupported URI
+        try {
+            ParquetTools.readTable("https://" + absolutePath);
+            TestCase.fail("Exception expected for invalid scheme");
+        } catch (final RuntimeException e) {
+            assertTrue(e instanceof UnsupportedOperationException);
+        }
+    }
+
     /**
      * These are tests for writing multiple parquet tables in a single call.
      */
@@ -894,6 +1018,50 @@ public void writeMultiTableExceptionTest() {
         assertTrue(parentDir.list().length == 0);
     }
 
+    @Test
+    public void writingParquetFilesWithSpacesInName() {
+        final String parentDirName = "tempDir";
+        final String tableNameWithSpaces = "table name with spaces.parquet";
+        final Table table = TableTools.emptyTable(5)
+                .updateView("InputString = Long.toString(ii)", "A=InputString.charAt(0)");
+        writingParquetFilesWithSpacesInNameHelper(table, parentDirName, tableNameWithSpaces);
+
+        // Same test but for tables with grouping data
+        Integer data[] = new Integer[500 * 4];
+        for (int i = 0; i < data.length; i++) {
+            data[i] = i / 4;
+        }
+        final TableDefinition groupingTableDefinition =
+                TableDefinition.of(ColumnDefinition.ofInt("vvv").withGrouping());
+        final Table tableWithGroupingData = newTable(groupingTableDefinition, TableTools.col("vvv", data));
+        writingParquetFilesWithSpacesInNameHelper(tableWithGroupingData, parentDirName, tableNameWithSpaces);
+    }
+
+    private void writingParquetFilesWithSpacesInNameHelper(final Table table, final String parentDirName,
+            final String parquetFileName) {
+        final File parentDir = new File(rootFile, parentDirName);
+        parentDir.mkdir();
+        final File dest = new File(parentDir, parquetFileName);
+
+        ParquetTools.writeTable(table, dest);
+        Table fromDisk = readSingleFileTable(dest, ParquetInstructions.EMPTY);
+        assertTableEquals(table, fromDisk);
+        FileUtils.deleteRecursively(parentDir);
+
+        final String destAbsolutePathStr = dest.getAbsolutePath();
+        ParquetTools.writeTable(table, destAbsolutePathStr);
+        fromDisk = readSingleFileTable(destAbsolutePathStr, ParquetInstructions.EMPTY);
+        assertTableEquals(table, fromDisk);
+        FileUtils.deleteRecursively(parentDir);
+
+        final String destRelativePathStr = rootFile.getName() + "/" + parentDirName + "/" + parquetFileName;
+        ParquetTools.writeTable(table, destRelativePathStr);
+        fromDisk = readSingleFileTable(destRelativePathStr, ParquetInstructions.EMPTY);
+        assertTableEquals(table, fromDisk);
+        FileUtils.deleteRecursively(parentDir);
+    }
+
+
     /**
      * These are tests for writing to a table with grouping columns to a parquet file and making sure there are no
      * unnecessary files left in the directory after we finish writing.
@@ -926,7 +1094,8 @@ public void groupingColumnsBasicWriteTestsImpl(TestParquetTableWriter writer) {
         checkSingleTable(tableToSave, destFile);
 
         // Verify that the key-value metadata in the file has the correct name
-        ParquetTableLocationKey tableLocationKey = new ParquetTableLocationKey(destFile, 0, null);
+        ParquetTableLocationKey tableLocationKey =
+                new ParquetTableLocationKey(destFile, 0, null, ParquetInstructions.EMPTY);
         String metadataString = tableLocationKey.getMetadata().getFileMetaData().toString();
         assertTrue(metadataString.contains(vvvIndexFilePath));
 
@@ -959,7 +1128,8 @@ public void legacyGroupingFileReadTest() {
         assertTrue(fromDisk.getDefinition().getColumn(groupingColName).isGrouping());
 
         // Verify that the key-value metadata in the file has the correct legacy grouping file name
-        final ParquetTableLocationKey tableLocationKey = new ParquetTableLocationKey(destFile, 0, null);
+        final ParquetTableLocationKey tableLocationKey =
+                new ParquetTableLocationKey(destFile, 0, null, ParquetInstructions.EMPTY);
         final String metadataString = tableLocationKey.getMetadata().getFileMetaData().toString();
         String groupingFileName = ParquetTools.legacyGroupingFileName(destFile, groupingColName);
         assertTrue(metadataString.contains(groupingFileName));
@@ -1087,10 +1257,11 @@ public void writeMultiTableGroupingColumnTest() {
                 Map.of("vvv", new String[] {firstIndexFilePath, secondIndexFilePath}));
 
         // Verify that the key-value metadata in the file has the correct name
-        ParquetTableLocationKey tableLocationKey = new ParquetTableLocationKey(firstDestFile, 0, null);
+        ParquetTableLocationKey tableLocationKey =
+                new ParquetTableLocationKey(firstDestFile, 0, null, ParquetInstructions.EMPTY);
         String metadataString = tableLocationKey.getMetadata().getFileMetaData().toString();
         assertTrue(metadataString.contains(firstIndexFilePath));
-        tableLocationKey = new ParquetTableLocationKey(secondDestFile, 0, null);
+        tableLocationKey = new ParquetTableLocationKey(secondDestFile, 0, null, ParquetInstructions.EMPTY);
         metadataString = tableLocationKey.getMetadata().getFileMetaData().toString();
         assertTrue(metadataString.contains(secondIndexFilePath));
 
@@ -1137,7 +1308,8 @@ public void groupingColumnsOverwritingTestsImpl(TestParquetTableWriter writer) {
 
         checkSingleTable(anotherTableToSave, destFile);
 
-        ParquetTableLocationKey tableLocationKey = new ParquetTableLocationKey(destFile, 0, null);
+        ParquetTableLocationKey tableLocationKey =
+                new ParquetTableLocationKey(destFile, 0, null, ParquetInstructions.EMPTY);
         String metadataString = tableLocationKey.getMetadata().getFileMetaData().toString();
         assertTrue(metadataString.contains(xxxIndexFilePath) && !metadataString.contains(vvvIndexFilePath));
 
@@ -1153,7 +1325,7 @@ public void groupingColumnsOverwritingTestsImpl(TestParquetTableWriter writer) {
                 Map.of("vvv", new String[] {vvvIndexFilePath},
                         "xxx", new String[] {xxxIndexFilePath}));
 
-        tableLocationKey = new ParquetTableLocationKey(destFile, 0, null);
+        tableLocationKey = new ParquetTableLocationKey(destFile, 0, null, ParquetInstructions.EMPTY);
         metadataString = tableLocationKey.getMetadata().getFileMetaData().toString();
         assertTrue(metadataString.contains(xxxIndexFilePath) && !metadataString.contains(vvvIndexFilePath)
                 && !metadataString.contains(backupXXXIndexFileName));
@@ -1244,7 +1416,8 @@ public void dictionaryEncodingTest() {
         checkSingleTable(stringTable, dest);
 
         // Verify that string columns are properly dictionary encoded
-        final ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null).getMetadata();
+        final ParquetMetadata metadata =
+                new ParquetTableLocationKey(dest, 0, null, ParquetInstructions.EMPTY).getMetadata();
         final String firstColumnMetadata = metadata.getBlocks().get(0).getColumns().get(0).toString();
         assertTrue(firstColumnMetadata.contains("shortStringColumn") && firstColumnMetadata.contains("RLE_DICTIONARY"));
         final String secondColumnMetadata = metadata.getBlocks().get(0).getColumns().get(1).toString();
@@ -1298,7 +1471,7 @@ private static ColumnChunkMetaData overflowingStringsTestHelper(final Collection
         writeTable(stringTable, dest, writeInstructions);
         checkSingleTable(stringTable, dest);
 
-        ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null).getMetadata();
+        ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null, ParquetInstructions.EMPTY).getMetadata();
         ColumnChunkMetaData columnMetadata = metadata.getBlocks().get(0).getColumns().get(0);
         return columnMetadata;
     }
@@ -1322,7 +1495,8 @@ public void overflowingCodecsTest() {
         writeTable(table, dest, writeInstructions);
         checkSingleTable(table, dest);
 
-        final ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null).getMetadata();
+        final ParquetMetadata metadata =
+                new ParquetTableLocationKey(dest, 0, null, ParquetInstructions.EMPTY).getMetadata();
         final String metadataStr = metadata.getFileMetaData().getKeyValueMetaData().get("deephaven");
         assertTrue(
                 metadataStr.contains("VariableWidthByteArrayColumn") && metadataStr.contains("SimpleByteArrayCodec"));
@@ -1382,7 +1556,8 @@ public void readWriteDateTimeTest() {
         writeReadTableTest(table, dest);
 
         // Verify that the types are correct in the schema
-        final ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null).getMetadata();
+        final ParquetMetadata metadata =
+                new ParquetTableLocationKey(dest, 0, null, ParquetInstructions.EMPTY).getMetadata();
         final ColumnChunkMetaData dateColMetadata = metadata.getBlocks().get(0).getColumns().get(0);
         assertTrue(dateColMetadata.toString().contains("someDateColumn"));
         assertEquals(PrimitiveType.PrimitiveTypeName.INT32, dateColMetadata.getPrimitiveType().getPrimitiveTypeName());
@@ -1842,7 +2017,8 @@ public void readSingleColumn() {
 
     private void assertTableStatistics(Table inputTable, File dest) {
         // Verify that the columns have the correct statistics.
-        final ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null).getMetadata();
+        final ParquetMetadata metadata =
+                new ParquetTableLocationKey(dest, 0, null, ParquetInstructions.EMPTY).getMetadata();
 
         final String[] colNames = inputTable.getDefinition().getColumnNamesArray();
         for (int colIdx = 0; colIdx < inputTable.numColumns(); ++colIdx) {
diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetTools.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetTools.java
index ccaeebda9fc..38d45d69208 100644
--- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetTools.java
+++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetTools.java
@@ -348,7 +348,8 @@ public void testPartitionedRead() {
         final TableDefinition partitionedDefinition = TableDefinition.of(allColumns);
 
         final Table result = ParquetTools.readPartitionedTableInferSchema(
-                new ParquetKeyValuePartitionedLayout(testRootFile, 2), ParquetInstructions.EMPTY);
+                new ParquetKeyValuePartitionedLayout(testRootFile, 2, ParquetInstructions.EMPTY),
+                ParquetInstructions.EMPTY);
         TestCase.assertEquals(partitionedDefinition, result.getDefinition());
         final Table expected = TableTools.merge(
                 table1.updateView("Date=`2021-07-20`", "Num=100"),
diff --git a/extensions/s3/build.gradle b/extensions/s3/build.gradle
new file mode 100644
index 00000000000..45b356bb34c
--- /dev/null
+++ b/extensions/s3/build.gradle
@@ -0,0 +1,23 @@
+plugins {
+    id 'java-library'
+    id 'io.deephaven.project.register'
+}
+
+description 'Used to create a channel provider plugin for reading and writing files stored in S3.'
+
+dependencies {
+    api project(':util-channel')
+
+    implementation project(':Base')
+    implementation project(':Util')
+    implementation project(':Configuration')
+
+    implementation platform('software.amazon.awssdk:bom:2.21.43')
+    implementation 'software.amazon.awssdk:s3'
+    implementation 'software.amazon.awssdk:aws-crt-client'
+
+    compileOnly depAnnotations
+
+    Classpaths.inheritAutoService(project)
+    Classpaths.inheritImmutables(project)
+}
diff --git a/extensions/s3/gradle.properties b/extensions/s3/gradle.properties
new file mode 100644
index 00000000000..c186bbfdde1
--- /dev/null
+++ b/extensions/s3/gradle.properties
@@ -0,0 +1 @@
+io.deephaven.project.ProjectType=JAVA_PUBLIC
diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentials.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentials.java
new file mode 100644
index 00000000000..5318ec540af
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentials.java
@@ -0,0 +1,15 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.extensions.s3;
+
+public interface AwsCredentials {
+
+    static AwsCredentials defaultCredentials() {
+        return DefaultCredentials.DEFAULT_CREDENTIALS;
+    }
+
+    static AwsCredentials basicCredentials(final String awsAccessKeyId, final String awsSecretAccessKey) {
+        return BasicCredentials.of(awsAccessKeyId, awsSecretAccessKey);
+    }
+}
diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsSdkV2Credentials.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsSdkV2Credentials.java
new file mode 100644
index 00000000000..04f58eff1df
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsSdkV2Credentials.java
@@ -0,0 +1,11 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.extensions.s3;
+
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+
+interface AwsSdkV2Credentials extends AwsCredentials {
+
+    AwsCredentialsProvider awsCredentialsProvider();
+}
diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BasicCredentials.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BasicCredentials.java
new file mode 100644
index 00000000000..3459ef9ea6b
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BasicCredentials.java
@@ -0,0 +1,35 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.extensions.s3;
+
+import io.deephaven.annotations.SimpleStyle;
+import org.immutables.value.Value;
+import org.immutables.value.Value.Immutable;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+
+/**
+ * AWS credentials provider that uses access key and secret key provided at construction.
+ */
+@Immutable
+@SimpleStyle
+abstract class BasicCredentials implements AwsSdkV2Credentials {
+
+    static BasicCredentials of(final String awsAccessKeyId, final String awsSecretAccessKey) {
+        return ImmutableBasicCredentials.of(awsAccessKeyId, awsSecretAccessKey);
+    }
+
+    @Value.Parameter
+    abstract String awsAccessKeyId();
+
+    @Value.Redacted
+    @Value.Parameter
+    abstract String awsSecretAccessKey();
+
+    public AwsCredentialsProvider awsCredentialsProvider() {
+        final AwsBasicCredentials awsCreds = AwsBasicCredentials.create(awsAccessKeyId(), awsSecretAccessKey());
+        return StaticCredentialsProvider.create(awsCreds);
+    }
+}
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
new file mode 100644
index 00000000000..8bcc56995ee
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java
@@ -0,0 +1,28 @@
+package io.deephaven.extensions.s3;
+
+import io.deephaven.util.SafeCloseable;
+import org.jetbrains.annotations.Nullable;
+
+import java.nio.ByteBuffer;
+
+interface BufferPool {
+
+    interface BufferHolder extends SafeCloseable {
+
+        /**
+         * @return The buffer if available, else {@code null}
+         */
+        @Nullable
+        ByteBuffer get();
+
+        /**
+         * Return the held buffer to its pool, and cause subsequent calls to {@link #get()} to return {@code null}
+         */
+        void close();
+    }
+
+    /**
+     * Returns a {@link BufferHolder} that will hold a buffer of at least the requested size.
+     */
+    BufferHolder take(int size);
+}
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
new file mode 100644
index 00000000000..6aaff7f6ea2
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java
@@ -0,0 +1,115 @@
+/**
+ * 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/DefaultCredentials.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/DefaultCredentials.java
new file mode 100644
index 00000000000..5f69f50653f
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/DefaultCredentials.java
@@ -0,0 +1,20 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.extensions.s3;
+
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+
+/**
+ * Default AWS credentials provider that looks for credentials at a number of locations as described in
+ * {@link DefaultCredentialsProvider}
+ */
+enum DefaultCredentials implements AwsSdkV2Credentials {
+    DEFAULT_CREDENTIALS;
+
+    @Override
+    public AwsCredentialsProvider awsCredentialsProvider() {
+        return DefaultCredentialsProvider.create();
+    }
+}
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
new file mode 100644
index 00000000000..1c2239045d6
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java
@@ -0,0 +1,157 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.extensions.s3;
+
+import io.deephaven.annotations.BuildableStyle;
+import io.deephaven.configuration.Configuration;
+import org.immutables.value.Value;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+
+import java.time.Duration;
+
+/**
+ * This class provides instructions intended for reading and writing data to AWS S3 instances.
+ */
+@Value.Immutable
+@BuildableStyle
+public abstract class S3Instructions {
+
+    private final static int DEFAULT_MAX_CONCURRENT_REQUESTS = 50;
+    private final static int DEFAULT_READ_AHEAD_COUNT = 1;
+
+    private final static String MAX_FRAGMENT_SIZE_CONFIG_PARAM = "S3.maxFragmentSize";
+    final static int MAX_FRAGMENT_SIZE =
+            Configuration.getInstance().getIntegerWithDefault(MAX_FRAGMENT_SIZE_CONFIG_PARAM, 5 << 20); // 5 MB
+    private final static int DEFAULT_FRAGMENT_SIZE = MAX_FRAGMENT_SIZE;
+
+    private final static int MIN_FRAGMENT_SIZE = 8 << 10; // 8 KB
+    private final static int DEFAULT_MAX_CACHE_SIZE = 32;
+    private final static Duration DEFAULT_CONNECTION_TIMEOUT = Duration.ofSeconds(2);
+    private final static Duration DEFAULT_READ_TIMEOUT = Duration.ofSeconds(2);
+
+    public static Builder builder() {
+        return ImmutableS3Instructions.builder();
+    }
+
+    /**
+     * The AWS region name to use when reading or writing to S3.
+     */
+    public abstract String awsRegionName();
+
+    /**
+     * The maximum number of concurrent requests to make to S3, defaults to {@value #DEFAULT_MAX_CONCURRENT_REQUESTS}.
+     */
+    @Value.Default
+    public int maxConcurrentRequests() {
+        return DEFAULT_MAX_CONCURRENT_REQUESTS;
+    }
+
+    /**
+     * The number of fragments to send asynchronous read requests for while reading the current fragment. Defaults to
+     * {@value #DEFAULT_READ_AHEAD_COUNT}, which means by default, we will fetch {@value #DEFAULT_READ_AHEAD_COUNT}
+     * fragments in advance when reading current fragment.
+     */
+    @Value.Default
+    public int readAheadCount() {
+        return DEFAULT_READ_AHEAD_COUNT;
+    }
+
+    /**
+     * The maximum size of each fragment to read from S3, defaults to the value of config parameter
+     * {@value MAX_FRAGMENT_SIZE_CONFIG_PARAM}. If there are fewer bytes remaining in the file, the fetched fragment can
+     * be smaller.
+     */
+    @Value.Default
+    public int fragmentSize() {
+        return DEFAULT_FRAGMENT_SIZE;
+    }
+
+    /**
+     * The maximum number of fragments to cache in memory, defaults to {@value #DEFAULT_MAX_CACHE_SIZE}. This caching is
+     * done at the deephaven layer for faster access to recently read fragments.
+     */
+    @Value.Default
+    public int maxCacheSize() {
+        return DEFAULT_MAX_CACHE_SIZE;
+    }
+
+    /**
+     * The amount of time to wait when initially establishing a connection before giving up and timing out, defaults to
+     * 2 seconds.
+     */
+    @Value.Default
+    public Duration connectionTimeout() {
+        return DEFAULT_CONNECTION_TIMEOUT;
+    }
+
+    /**
+     * The amount of time to wait when reading a fragment before giving up and timing out, defaults to 2 seconds
+     */
+    @Value.Default
+    public Duration readTimeout() {
+        return DEFAULT_READ_TIMEOUT;
+    }
+
+    /**
+     * The credentials to use when reading or writing to S3. By default, uses {@link DefaultCredentialsProvider}.
+     */
+    @Value.Default
+    public AwsCredentials credentials() {
+        return AwsCredentials.defaultCredentials();
+    }
+
+    @Value.Check
+    final void boundsCheckMaxConcurrentRequests() {
+        if (maxConcurrentRequests() < 1) {
+            throw new IllegalArgumentException("maxConcurrentRequests(=" + maxConcurrentRequests() + ") must be >= 1");
+        }
+    }
+
+    @Value.Check
+    final void boundsCheckReadAheadCount() {
+        if (readAheadCount() < 0) {
+            throw new IllegalArgumentException("readAheadCount(=" + readAheadCount() + ") must be >= 0");
+        }
+    }
+
+    @Value.Check
+    final void boundsCheckMaxFragmentSize() {
+        if (fragmentSize() < MIN_FRAGMENT_SIZE) {
+            throw new IllegalArgumentException("fragmentSize(=" + fragmentSize() + ") must be >= " + MIN_FRAGMENT_SIZE +
+                    " bytes");
+        }
+        if (fragmentSize() > MAX_FRAGMENT_SIZE) {
+            throw new IllegalArgumentException("fragmentSize(=" + fragmentSize() + ") must be <= " + MAX_FRAGMENT_SIZE +
+                    " bytes");
+        }
+    }
+
+    @Value.Check
+    final void boundsCheckMaxCacheSize() {
+        if (maxCacheSize() < readAheadCount() + 1) {
+            throw new IllegalArgumentException("maxCacheSize(=" + maxCacheSize() + ") must be >= 1 + " +
+                    "readAheadCount(=" + readAheadCount() + ")");
+        }
+    }
+
+    public interface Builder {
+        Builder awsRegionName(String awsRegionName);
+
+        Builder maxConcurrentRequests(int maxConcurrentRequests);
+
+        Builder readAheadCount(int readAheadCount);
+
+        Builder fragmentSize(int fragmentSize);
+
+        Builder maxCacheSize(int maxCacheSize);
+
+        Builder connectionTimeout(Duration connectionTimeout);
+
+        Builder readTimeout(Duration connectionTimeout);
+
+        Builder credentials(AwsCredentials credentials);
+
+        S3Instructions build();
+    }
+}
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
new file mode 100644
index 00000000000..13cb4b457a3
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java
@@ -0,0 +1,408 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+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;
+
+
+/**
+ * {@link SeekableByteChannel} class used to fetch objects from AWS S3 buckets using an async client with the ability to
+ * read ahead and cache fragments of the object.
+ */
+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 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 ignored = () -> SafeCloseable.closeAll(
+                        future == null ? null : () -> future.cancel(true), bufferRelease)) {
+                    fragmentIndex = UNINITIALIZED_FRAGMENT_INDEX;
+                    future = null;
+                    bufferRelease = null;
+                }
+            }
+
+            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;
+
+    /**
+     * 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 long position;
+
+    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;
+        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.
+     */
+    @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);
+            }
+
+            // 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;
+        }
+        // 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);
+    }
+
+    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);
+        }
+        return new IOException(String.format("Exception caught while %s", operationDescription), e);
+    }
+
+    @Override
+    public int write(final ByteBuffer src) {
+        throw new NonWritableChannelException();
+    }
+
+    @Override
+    public long position() throws ClosedChannelException {
+        final long localPosition = position;
+        checkClosed(localPosition);
+        return localPosition;
+    }
+
+    @Override
+    public SeekableByteChannel position(final long newPosition) throws ClosedChannelException {
+        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);
+        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)
+    }
+
+    @Override
+    public SeekableByteChannel truncate(final long size) {
+        throw new NonWritableChannelException();
+    }
+
+    @Override
+    public boolean isOpen() {
+        return position != CLOSED_SENTINEL;
+    }
+
+    @Override
+    public void close() {
+        position = CLOSED_SENTINEL;
+    }
+
+    private static void checkClosed(final long position) throws ClosedChannelException {
+        if (position == CLOSED_SENTINEL) {
+            throw new ClosedChannelException();
+        }
+    }
+}
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
new file mode 100644
index 00000000000..5f146619e18
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java
@@ -0,0 +1,79 @@
+/**
+ * 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.channel.SeekableChannelContext;
+import io.deephaven.util.channel.SeekableChannelsProvider;
+import org.jetbrains.annotations.NotNull;
+import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3AsyncClient;
+import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient;
+
+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 AWS S3 instances.
+ */
+final class S3SeekableChannelProvider implements SeekableChannelsProvider {
+
+    /**
+     * 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 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
+        Assert.instanceOf(s3Instructions.credentials(), "credentials", AwsSdkV2Credentials.class);
+        this.s3AsyncClient = S3AsyncClient.builder()
+                .region(Region.of(s3Instructions.awsRegionName()))
+                .httpClient(asyncHttpClient)
+                .credentialsProvider(((AwsSdkV2Credentials) s3Instructions.credentials()).awsCredentialsProvider())
+                .build();
+        this.s3Instructions = s3Instructions;
+    }
+
+    @Override
+    public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext,
+            @NotNull final URI uri) {
+        // context is unused here, will be set before reading from the channel
+        return new S3SeekableByteChannel(uri, s3AsyncClient, s3Instructions, bufferPool);
+    }
+
+    @Override
+    public SeekableChannelContext makeContext() {
+        return new S3SeekableByteChannel.S3ChannelContext(s3Instructions.maxCacheSize());
+    }
+
+    @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;
+    }
+
+    @Override
+    public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boolean append) {
+        throw new UnsupportedOperationException("Writing to S3 is currently unsupported");
+    }
+
+    public void close() {
+        s3AsyncClient.close();
+    }
+}
diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProviderPlugin.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProviderPlugin.java
new file mode 100644
index 00000000000..69144f5062e
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProviderPlugin.java
@@ -0,0 +1,38 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.extensions.s3;
+
+import com.google.auto.service.AutoService;
+import io.deephaven.util.channel.SeekableChannelsProvider;
+import io.deephaven.util.channel.SeekableChannelsProviderPlugin;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import java.net.URI;
+
+/**
+ * {@link SeekableChannelsProviderPlugin} implementation used for reading files from S3.
+ */
+@AutoService(SeekableChannelsProviderPlugin.class)
+public final class S3SeekableChannelProviderPlugin implements SeekableChannelsProviderPlugin {
+
+    private static final String S3_URI_SCHEME = "s3";
+
+    @Override
+    public boolean isCompatible(@NotNull final URI uri, @Nullable final Object config) {
+        return S3_URI_SCHEME.equals(uri.getScheme());
+    }
+
+    @Override
+    public SeekableChannelsProvider createProvider(@NotNull final URI uri, @Nullable final Object config) {
+        if (!isCompatible(uri, config)) {
+            if (!(config instanceof S3Instructions)) {
+                throw new IllegalArgumentException("Must provide S3Instructions to read files from S3");
+            }
+            throw new IllegalArgumentException("Arguments not compatible, provided uri " + uri);
+        }
+        final S3Instructions s3Instructions = (S3Instructions) config;
+        return new S3SeekableChannelProvider(s3Instructions);
+    }
+}
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
new file mode 100644
index 00000000000..8b30820810c
--- /dev/null
+++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/SegmentedBufferPool.java
@@ -0,0 +1,61 @@
+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/build.gradle b/extensions/trackedfile/build.gradle
new file mode 100644
index 00000000000..e5df2b81c16
--- /dev/null
+++ b/extensions/trackedfile/build.gradle
@@ -0,0 +1,18 @@
+plugins {
+    id 'java-library'
+    id 'io.deephaven.project.register'
+}
+
+description 'Used to create a channel provider plugin for reading and writing local files.'
+
+dependencies {
+    api project(':util-channel')
+
+    implementation project(':Base')
+    implementation project(':Util')
+    implementation project(':engine-table')
+
+    compileOnly depAnnotations
+
+    Classpaths.inheritAutoService(project)
+}
diff --git a/extensions/trackedfile/gradle.properties b/extensions/trackedfile/gradle.properties
new file mode 100644
index 00000000000..c186bbfdde1
--- /dev/null
+++ b/extensions/trackedfile/gradle.properties
@@ -0,0 +1 @@
+io.deephaven.project.ProjectType=JAVA_PUBLIC
diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java
similarity index 65%
rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java
rename to extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java
index fcf15c1d971..7b6f0c55fbe 100644
--- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java
+++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java
@@ -1,48 +1,57 @@
 /**
  * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
  */
-package io.deephaven.parquet.table.util;
+package io.deephaven.extensions.trackedfile;
 
+import io.deephaven.base.verify.Assert;
 import io.deephaven.engine.util.file.FileHandle;
 import io.deephaven.engine.util.file.FileHandleFactory;
 import io.deephaven.engine.util.file.TrackedFileHandleFactory;
 import io.deephaven.engine.util.file.TrackedSeekableByteChannel;
-import io.deephaven.parquet.base.util.SeekableChannelsProvider;
+import io.deephaven.util.channel.SeekableChannelContext;
+import io.deephaven.util.channel.SeekableChannelsProvider;
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.nio.channels.SeekableByteChannel;
 import java.nio.file.Path;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
+import static io.deephaven.extensions.trackedfile.TrackedSeekableChannelsProviderPlugin.FILE_URI_SCHEME;
+
 /**
  * {@link SeekableChannelsProvider} implementation that is constrained by a Deephaven {@link TrackedFileHandleFactory}.
  */
-public class TrackedSeekableChannelsProvider implements SeekableChannelsProvider {
+class TrackedSeekableChannelsProvider implements SeekableChannelsProvider {
 
-    private static volatile SeekableChannelsProvider instance;
+    private final TrackedFileHandleFactory fileHandleFactory;
 
-    public static SeekableChannelsProvider getInstance() {
-        if (instance == null) {
-            synchronized (TrackedSeekableChannelsProvider.class) {
-                if (instance == null) {
-                    return instance = new TrackedSeekableChannelsProvider(TrackedFileHandleFactory.getInstance());
-                }
-            }
-        }
-        return instance;
+    TrackedSeekableChannelsProvider(@NotNull final TrackedFileHandleFactory fileHandleFactory) {
+        this.fileHandleFactory = fileHandleFactory;
     }
 
-    private final TrackedFileHandleFactory fileHandleFactory;
+    @Override
+    public SeekableChannelContext makeContext() {
+        // No additional context required for local FS
+        return SeekableChannelContext.NULL;
+    }
 
-    public TrackedSeekableChannelsProvider(@NotNull final TrackedFileHandleFactory fileHandleFactory) {
-        this.fileHandleFactory = fileHandleFactory;
+    @Override
+    public boolean isCompatibleWith(@Nullable SeekableChannelContext channelContext) {
+        // Context is not used, hence always compatible
+        return true;
     }
 
     @Override
-    public final SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException {
-        return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, path.toFile());
+    public final SeekableByteChannel getReadChannel(@Nullable final SeekableChannelContext channelContext,
+            @NotNull final URI uri)
+            throws IOException {
+        // context is unused here
+        Assert.assertion(FILE_URI_SCHEME.equals(uri.getScheme()), "Expected a file uri, got " + uri);
+        return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, new File(uri));
     }
 
     @Override
@@ -77,4 +86,7 @@ public final FileHandle invoke(@NotNull final File file) throws IOException {
             return fileHandleFactory.writeAppendCreateHandleCreator.invoke(file);
         }
     }
+
+    @Override
+    public void close() {}
 }
diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java
new file mode 100644
index 00000000000..e33529147f3
--- /dev/null
+++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java
@@ -0,0 +1,38 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.extensions.trackedfile;
+
+import com.google.auto.service.AutoService;
+import io.deephaven.engine.util.file.TrackedFileHandleFactory;
+import io.deephaven.util.channel.SeekableChannelsProvider;
+import io.deephaven.util.channel.SeekableChannelsProviderPlugin;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import java.net.URI;
+
+/**
+ * {@link SeekableChannelsProviderPlugin} implementation used for reading files from local disk.
+ */
+@AutoService(SeekableChannelsProviderPlugin.class)
+public final class TrackedSeekableChannelsProviderPlugin implements SeekableChannelsProviderPlugin {
+
+    static final String FILE_URI_SCHEME = "file";
+
+    @Override
+    public boolean isCompatible(@NotNull final URI uri, @Nullable final Object object) {
+        return FILE_URI_SCHEME.equals(uri.getScheme());
+    }
+
+    @Override
+    public SeekableChannelsProvider createProvider(@NotNull final URI uri, @Nullable final Object object) {
+        if (!isCompatible(uri, object)) {
+            if (object != null) {
+                throw new IllegalArgumentException("Arguments not compatible, provided non null object");
+            }
+            throw new IllegalArgumentException("Arguments not compatible, provided uri " + uri);
+        }
+        return new TrackedSeekableChannelsProvider(TrackedFileHandleFactory.getInstance());
+    }
+}
diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py
new file mode 100644
index 00000000000..c4c34b3d9ab
--- /dev/null
+++ b/py/server/deephaven/experimental/s3.py
@@ -0,0 +1,111 @@
+#
+# Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending
+#
+import datetime
+from typing import Optional, Union
+
+import jpy
+import numpy as np
+import pandas as pd
+
+from deephaven import time, DHError
+from deephaven._wrapper import JObjectWrapper
+from deephaven.dtypes import Duration
+
+_JAwsCredentials = jpy.get_type("io.deephaven.extensions.s3.AwsCredentials")
+_JS3Instructions = jpy.get_type("io.deephaven.extensions.s3.S3Instructions")
+
+"""
+    This module is useful for reading files stored in S3.
+    Importing this module requires the S3 specific deephaven extensions (artifact name deephaven-extensions-s3) to be
+    included in the package. This is an opt-out functionality included by default. If not included, importing this
+    module will fail to find the java types.
+"""
+class S3Instructions(JObjectWrapper):
+    """
+    S3Instructions provides specialized instructions for reading from AWS S3.
+    """
+
+    j_object_type = _JS3Instructions
+
+    def __init__(self,
+                 aws_region_name: str,
+                 max_concurrent_requests: Optional[int] = None,
+                 read_ahead_count: Optional[int] = None,
+                 fragment_size: Optional[int] = None,
+                 max_cache_size: Optional[int] = None,
+                 connection_timeout: Union[
+                     Duration, int, str, datetime.timedelta, np.timedelta64, pd.Timedelta, None] = None,
+                 read_timeout: Union[
+                     Duration, int, str, datetime.timedelta, np.timedelta64, pd.Timedelta, None] = None,
+                 aws_access_key_id: Optional[str] = None,
+                 aws_secret_access_key: Optional[str] = None):
+
+        """
+        Initializes the instructions.
+
+        Args:
+            aws_region_name (str): the AWS region name for reading parquet files stored in AWS S3, mandatory parameter.
+            max_concurrent_requests (int): the maximum number of concurrent requests for reading parquet files stored in S3.
+                default is 50.
+            read_ahead_count (int): the number of fragments to send asynchronous read requests for while reading the current
+                fragment. Default to 1, which means fetch the next fragment in advance when reading the current fragment.
+            fragment_size (int): the maximum size of each fragment to read from S3, defaults to 5 MB. If there are fewer
+                bytes remaining in the file, the fetched fragment can be smaller.
+            max_cache_size (int): the maximum number of fragments to cache in memory while reading, defaults to 32. This
+                caching is done at the deephaven layer for faster access to recently read fragments.
+            connection_timeout (Union[Duration, int, str, datetime.timedelta, np.timedelta64, pd.Timedelta]):
+                the amount of time to wait when initially establishing a connection before giving up and timing out, can
+                be expressed as an integer in nanoseconds, a time interval string, e.g. "PT00:00:00.001" or "PT1s", or
+                other time duration types. Default to 2 seconds.
+            read_timeout (Union[Duration, int, str, datetime.timedelta, np.timedelta64, pd.Timedelta]):
+                the amount of time to wait when reading a fragment before giving up and timing out, can be expressed as
+                an integer in nanoseconds, a time interval string, e.g. "PT00:00:00.001" or "PT1s", or other time
+                duration types. Default to 2 seconds.
+            aws_access_key_id (str): the AWS access key for reading parquet files stored in AWS S3. Both access key and
+                secret key must be provided to use static credentials, else default credentials will be used from
+                software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider.
+            aws_secret_access_key (str): the AWS secret access key for reading parquet files stored in AWS S3. Both
+                access key and secret key must be provided to use static credentials, else default credentials will be
+                used from software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider.
+
+        Raises:
+            DHError: If unable to build the instructions object.
+        """
+
+        try:
+            builder = self.j_object_type.builder()
+            builder.awsRegionName(aws_region_name)
+
+            if max_concurrent_requests is not None:
+                builder.maxConcurrentRequests(max_concurrent_requests)
+
+            if read_ahead_count is not None:
+                builder.readAheadCount(read_ahead_count)
+
+            if fragment_size is not None:
+                builder.fragmentSize(fragment_size)
+
+            if max_cache_size is not None:
+                builder.maxCacheSize(max_cache_size)
+
+            if connection_timeout is not None:
+                builder.connectionTimeout(time.to_j_duration(connection_timeout))
+
+            if read_timeout is not None:
+                builder.readTimeout(time.to_j_duration(read_timeout))
+
+            if ((aws_access_key_id is not None and aws_secret_access_key is None) or
+                    (aws_access_key_id is None and aws_secret_access_key is not None)):
+                raise DHError("Either both aws_access_key_id and aws_secret_access_key must be provided or neither")
+
+            if aws_access_key_id is not None:
+                builder.credentials(_JAwsCredentials.basicCredentials(aws_access_key_id, aws_secret_access_key))
+
+            self._j_object = builder.build()
+        except Exception as e:
+            raise DHError(e, "Failed to build S3 instructions") from e
+
+    @property
+    def j_object(self) -> jpy.JType:
+        return self._j_object
diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py
index 4ec51e7db26..053da2b1c76 100644
--- a/py/server/deephaven/parquet.py
+++ b/py/server/deephaven/parquet.py
@@ -14,6 +14,7 @@
 from deephaven.column import Column
 from deephaven.dtypes import DType
 from deephaven.table import Table
+from deephaven.experimental import s3
 
 _JParquetTools = jpy.get_type("io.deephaven.parquet.table.ParquetTools")
 _JFile = jpy.get_type("java.io.File")
@@ -25,23 +26,24 @@
 @dataclass
 class ColumnInstruction:
     """  This class specifies the instructions for reading/writing a Parquet column. """
-    column_name: str = None
-    parquet_column_name: str = None
-    codec_name: str = None
-    codec_args: str = None
+    column_name: Optional[str] = None
+    parquet_column_name: Optional[str] = None
+    codec_name: Optional[str] = None
+    codec_args: Optional[str] = None
     use_dictionary: bool = False
 
 
 def _build_parquet_instructions(
-    col_instructions: List[ColumnInstruction] = None,
-    compression_codec_name: str = None,
-    max_dictionary_keys: int = None,
-    max_dictionary_size: int = None,
+    col_instructions: Optional[List[ColumnInstruction]] = None,
+    compression_codec_name: Optional[str] = None,
+    max_dictionary_keys: Optional[int] = None,
+    max_dictionary_size: Optional[int] = None,
     is_legacy_parquet: bool = False,
-    target_page_size: int = None,
+    target_page_size: Optional[int] = None,
     is_refreshing: bool = False,
     for_read: bool = True,
     force_build: bool = False,
+    special_instructions: Optional[s3.S3Instructions] = None,
 ):
     if not any(
         [
@@ -53,6 +55,7 @@ def _build_parquet_instructions(
             is_legacy_parquet,
             target_page_size is not None,
             is_refreshing,
+            special_instructions is not None
         ]
     ):
         return None
@@ -89,6 +92,9 @@ def _build_parquet_instructions(
     if is_refreshing:
         builder.setIsRefreshing(is_refreshing)
 
+    if special_instructions is not None:
+        builder.setSpecialInstructions(special_instructions.j_object)
+
     return builder.build()
 
 def _j_table_definition(table_definition: Union[Dict[str, DType], List[Column], None]) -> Optional[jpy.JType]:
@@ -108,6 +114,7 @@ def _j_table_definition(table_definition: Union[Dict[str, DType], List[Column],
     else:
         raise DHError(f"Unexpected table_definition type: {type(table_definition)}")
 
+
 class ParquetFileLayout(Enum):
     """ The parquet file layout. """
 
@@ -131,6 +138,7 @@ def read(
     is_refreshing: bool = False,
     file_layout: Optional[ParquetFileLayout] = None,
     table_definition: Union[Dict[str, DType], List[Column], None] = None,
+    special_instructions: Optional[s3.S3Instructions] = None,
 ) -> Table:
     """ Reads in a table from a single parquet, metadata file, or directory with recognized layout.
 
@@ -144,9 +152,12 @@ def read(
             inferred.
         table_definition (Union[Dict[str, DType], List[Column], None]): the table definition, by default None. When None,
             the definition is inferred from the parquet file(s). Setting a definition guarantees the returned table will
-            have that definition. This is useful for bootstrapping purposes when the initial partitioned directory is
+            have that definition. This is useful for bootstrapping purposes when the initially partitioned directory is
             empty and is_refreshing=True. It is also useful for specifying a subset of the parquet definition. When set,
             file_layout must also be set.
+        special_instructions (Optional[s3.S3Instructions]): Special instructions for reading parquet files, useful when
+            reading files from a non-local file system, like S3. By default, None.
+
     Returns:
         a table
 
@@ -161,13 +172,14 @@ def read(
             is_refreshing=is_refreshing,
             for_read=True,
             force_build=True,
+            special_instructions=special_instructions,
         )
         j_table_definition = _j_table_definition(table_definition)
         if j_table_definition is not None:
             if not file_layout:
                 raise DHError("Must provide file_layout when table_definition is set")
             if file_layout == ParquetFileLayout.SINGLE_FILE:
-                j_table = _JParquetTools.readSingleFileTable(_JFile(path), read_instructions, j_table_definition)
+                j_table = _JParquetTools.readSingleFileTable(path, read_instructions, j_table_definition)
             elif file_layout == ParquetFileLayout.FLAT_PARTITIONED:
                 j_table = _JParquetTools.readFlatPartitionedTable(_JFile(path), read_instructions, j_table_definition)
             elif file_layout == ParquetFileLayout.KV_PARTITIONED:
@@ -180,7 +192,7 @@ def read(
             if not file_layout:
                 j_table = _JParquetTools.readTable(path, read_instructions)
             elif file_layout == ParquetFileLayout.SINGLE_FILE:
-                j_table = _JParquetTools.readSingleFileTable(_JFile(path), read_instructions)
+                j_table = _JParquetTools.readSingleFileTable(path, read_instructions)
             elif file_layout == ParquetFileLayout.FLAT_PARTITIONED:
                 j_table = _JParquetTools.readFlatPartitionedTable(_JFile(path), read_instructions)
             elif file_layout == ParquetFileLayout.KV_PARTITIONED:
@@ -216,12 +228,12 @@ def delete(path: str) -> None:
 def write(
     table: Table,
     path: str,
-    col_definitions: List[Column] = None,
-    col_instructions: List[ColumnInstruction] = None,
-    compression_codec_name: str = None,
-    max_dictionary_keys: int = None,
-    max_dictionary_size: int = None,
-    target_page_size: int = None,
+    col_definitions: Optional[List[Column]] = None,
+    col_instructions: Optional[List[ColumnInstruction]] = None,
+    compression_codec_name: Optional[str] = None,
+    max_dictionary_keys: Optional[int] = None,
+    max_dictionary_size: Optional[int] = None,
+    target_page_size: Optional[int] = None,
 ) -> None:
     """ Write a table to a Parquet file.
 
@@ -230,12 +242,12 @@ def write(
         path (str): the destination file path; the file name should end in a ".parquet" extension. If the path
             includes non-existing directories they are created. If there is an error, any intermediate directories
             previously created are removed; note this makes this method unsafe for concurrent use
-        col_definitions (List[Column]): the column definitions to use, default is None
-        col_instructions (List[ColumnInstruction]): instructions for customizations while writing, default is None
-        compression_codec_name (str): the default compression codec to use, if not specified, defaults to SNAPPY
-        max_dictionary_keys (int): the maximum dictionary keys allowed, if not specified, defaults to 2^20 (1,048,576)
-        max_dictionary_size (int): the maximum dictionary size (in bytes) allowed, defaults to 2^20 (1,048,576)
-        target_page_size (int): the target page size in bytes, if not specified, defaults to 2^20 bytes (1 MiB)
+        col_definitions (Optional[List[Column]]): the column definitions to use, default is None
+        col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while writing, default is None
+        compression_codec_name (Optional[str]): the default compression codec to use, if not specified, defaults to SNAPPY
+        max_dictionary_keys (Optional[int]): the maximum dictionary keys allowed, if not specified, defaults to 2^20 (1,048,576)
+        max_dictionary_size (Optional[int]): the maximum dictionary size (in bytes) allowed, defaults to 2^20 (1,048,576)
+        target_page_size (Optional[int]): the target page size in bytes, if not specified, defaults to 2^20 bytes (1 MiB)
 
     Raises:
         DHError
@@ -272,12 +284,12 @@ def batch_write(
     tables: List[Table],
     paths: List[str],
     col_definitions: List[Column],
-    col_instructions: List[ColumnInstruction] = None,
-    compression_codec_name: str = None,
-    max_dictionary_keys: int = None,
-    max_dictionary_size: int = None,
-    target_page_size: int = None,
-    grouping_cols: List[str] = None,
+    col_instructions: Optional[List[ColumnInstruction]] = None,
+    compression_codec_name: Optional[str] = None,
+    max_dictionary_keys: Optional[int] = None,
+    max_dictionary_size: Optional[int] = None,
+    target_page_size: Optional[int] = None,
+    grouping_cols: Optional[List[str]] = None,
 ):
     """ Writes tables to disk in parquet format to a supplied set of paths.
 
@@ -292,12 +304,12 @@ def batch_write(
             created. If there is an error, any intermediate directories previously created are removed; note this makes
             this method unsafe for concurrent use
         col_definitions (List[Column]): the column definitions to use
-        col_instructions (List[ColumnInstruction]): instructions for customizations while writing
-        compression_codec_name (str): the compression codec to use, if not specified, defaults to SNAPPY
-        max_dictionary_keys (int): the maximum dictionary keys allowed, if not specified, defaults to 2^20 (1,048,576)
-        max_dictionary_size (int): the maximum dictionary size (in bytes) allowed, defaults to 2^20 (1,048,576)
-        target_page_size (int): the target page size in bytes, if not specified, defaults to 2^20 bytes (1 MiB)
-        grouping_cols (List[str]): the group column names
+        col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while writing
+        compression_codec_name (Optional[str]): the compression codec to use, if not specified, defaults to SNAPPY
+        max_dictionary_keys (Optional[int]): the maximum dictionary keys allowed, if not specified, defaults to 2^20 (1,048,576)
+        max_dictionary_size (Optional[int]): the maximum dictionary size (in bytes) allowed, defaults to 2^20 (1,048,576)
+        target_page_size (Optional[int]): the target page size in bytes, if not specified, defaults to 2^20 bytes (1 MiB)
+        grouping_cols (Optional[List[str]]): the group column names
 
     Raises:
         DHError
diff --git a/py/server/tests/test_parquet.py b/py/server/tests/test_parquet.py
index 56cce45957a..688f0e237a9 100644
--- a/py/server/tests/test_parquet.py
+++ b/py/server/tests/test_parquet.py
@@ -16,6 +16,7 @@
 from deephaven.pandas import to_pandas, to_table
 from deephaven.parquet import write, batch_write, read, delete, ColumnInstruction, ParquetFileLayout
 from tests.testbase import BaseTestCase
+from deephaven.experimental import s3
 
 
 class ParquetTestCase(BaseTestCase):
@@ -552,6 +553,29 @@ def test_read_with_table_definition_no_type(self):
                 "Must provide file_layout when table_definition is set", str(cm.exception)
             )
 
+    def test_read_parquet_from_s3(self):
+        """ Test that we can read parquet files from s3 """
+
+        # Fails since we have a negative read_ahead_count
+        with self.assertRaises(DHError):
+            s3.S3Instructions(aws_region_name="us-east-1",
+                              read_ahead_count=-1,
+                              )
+
+        # Fails since we provide the key without the secret key
+        with self.assertRaises(DHError):
+            s3.S3Instructions(aws_region_name="us-east-1",
+                              aws_access_key_id="Some key without secret",
+                              )
+
+        s3_instructions = s3.S3Instructions(aws_region_name="us-east-1",
+                                            read_ahead_count=1,
+                                            )
+        # Fails because we don't have the right credentials
+        with self.assertRaises(Exception):
+            read("s3://dh-s3-parquet-test1/multiColFile.parquet", special_instructions=s3_instructions).select()
+        # TODO(deephaven-core#5064): Add support for local S3 testing
+
 
 if __name__ == '__main__':
     unittest.main()
diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java
new file mode 100644
index 00000000000..aeeae50e6d4
--- /dev/null
+++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java
@@ -0,0 +1,27 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.replicators;
+
+import java.io.IOException;
+
+import static io.deephaven.replication.ReplicatePrimitiveCode.replaceAll;
+
+public class ReplicateCachingSupplier {
+    private static final String LAZY_CACHING_SUPPLIER_DIR = "util/src/main/java/io/deephaven/util/datastructures/";
+    private static final String LAZY_CACHING_SUPPLIER_PATH = LAZY_CACHING_SUPPLIER_DIR + "LazyCachingSupplier.java";
+    private static final String LAZY_CACHING_FUNCTION_PATH = LAZY_CACHING_SUPPLIER_DIR + "LazyCachingFunction.java";
+
+    private static final String[] NO_EXCEPTIONS = new String[0];
+
+    public static void main(final String[] args) throws IOException {
+        final String[][] pairs = new String[][] {
+                {"Supplier", "Function"},
+                {"internalSupplier\\.get\\(\\)", "internalFunction\\.apply\\(arg\\)"},
+                {"OUTPUT_TYPE get\\(\\)", "OUTPUT_TYPE apply\\(final INPUT_TYPE arg\\)"},
+                {"Supplier", "Function"},
+                {"supplier", "function"},
+        };
+        replaceAll(LAZY_CACHING_SUPPLIER_PATH, LAZY_CACHING_FUNCTION_PATH, null, NO_EXCEPTIONS, pairs);
+    }
+}
diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateParquetTransferObjects.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateParquetTransferObjects.java
index 508386a3b0c..407a42778ee 100644
--- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateParquetTransferObjects.java
+++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateParquetTransferObjects.java
@@ -1,3 +1,6 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
 package io.deephaven.replicators;
 
 import java.io.IOException;
diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateTableLocationKey.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateTableLocationKey.java
new file mode 100644
index 00000000000..b4fd25d9b9e
--- /dev/null
+++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateTableLocationKey.java
@@ -0,0 +1,27 @@
+/**
+ * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending
+ */
+package io.deephaven.replicators;
+
+import java.io.IOException;
+
+import static io.deephaven.replication.ReplicatePrimitiveCode.replaceAll;
+
+public class ReplicateTableLocationKey {
+    private static final String TABLE_LOCATION_KEY_DIR =
+            "engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/";
+    private static final String FILE_TABLE_LOCATION_KEY_PATH = TABLE_LOCATION_KEY_DIR + "FileTableLocationKey.java";
+    private static final String URI_TABLE_LOCATION_KEY_PATH = TABLE_LOCATION_KEY_DIR + "URITableLocationKey.java";
+
+    private static final String[] NO_EXCEPTIONS = new String[0];
+
+    public static void main(final String[] args) throws IOException {
+        final String[][] pairs = new String[][] {
+                {"file\\.getAbsoluteFile\\(\\)", "uri"},
+                {"java.io.File", "java.net.URI"},
+                {"file", "uri"},
+                {"File", "URI"},
+        };
+        replaceAll(FILE_TABLE_LOCATION_KEY_PATH, URI_TABLE_LOCATION_KEY_PATH, null, NO_EXCEPTIONS, pairs);
+    }
+}
diff --git a/server/jetty-app/build.gradle b/server/jetty-app/build.gradle
index f795f21a6df..ee51f12d41e 100644
--- a/server/jetty-app/build.gradle
+++ b/server/jetty-app/build.gradle
@@ -51,6 +51,12 @@ if (!hasProperty('excludeSql')) {
     }
 }
 
+if (!hasProperty('excludeS3')) {
+    dependencies {
+        runtimeOnly project(':extensions-s3')
+    }
+}
+
 def authHandlers = []
 def authConfigs = ['AuthHandlers']
 if (hasProperty('anonymous')) {
diff --git a/settings.gradle b/settings.gradle
index 6a724c3de62..52a854f3027 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -260,6 +260,12 @@ project(':extensions-source-support').projectDir = file('extensions/source-suppo
 include(':extensions-protobuf')
 project(':extensions-protobuf').projectDir = file('extensions/protobuf')
 
+include(':extensions-trackedfile')
+project(':extensions-trackedfile').projectDir = file('extensions/trackedfile')
+
+include(':extensions-s3')
+project(':extensions-s3').projectDir = file('extensions/s3')
+
 include(':plugin')
 
 include(':plugin-dagger')
@@ -327,6 +333,9 @@ project(':util-immutables').projectDir = file('Util/immutables')
 include ':util-function'
 project(':util-function').projectDir = file('Util/function')
 
+include(':util-channel')
+project(':util-channel').projectDir = file('Util/channel')
+
 include(':deephaven-jpy-config')
 project(':deephaven-jpy-config').projectDir = file('py/jpy-config')