From 97230944a97be73057547ae8f4661defd1416ab2 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 7 Dec 2023 00:45:11 +0530 Subject: [PATCH 01/39] Initial commit --- .../locations/impl/TestGroupingProviders.java | 3 +- .../TestChunkedRegionedOperations.java | 3 +- .../parquet/base/RowGroupReaderImpl.java | 4 + extensions/parquet/table/build.gradle | 3 + .../parquet/table/ParquetInstructions.java | 30 ++++- .../parquet/table/ParquetSchemaReader.java | 3 +- .../deephaven/parquet/table/ParquetTools.java | 59 ++++++--- .../DeephavenNestedPartitionLayout.java | 6 +- .../layout/ParquetFlatPartitionedLayout.java | 15 ++- .../ParquetKeyValuePartitionedLayout.java | 11 +- .../layout/ParquetMetadataFileLayout.java | 7 +- .../table/layout/ParquetSingleFileLayout.java | 9 +- .../location/ParquetTableLocationFactory.java | 3 +- .../location/ParquetTableLocationKey.java | 13 +- .../util/S3BackedSeekableByteChannel.java | 114 ++++++++++++++++++ .../util/S3BackedSeekableChannelProvider.java | 76 ++++++++++++ .../table/ParquetTableReadWriteTest.java | 53 +++++--- .../parquet/table/TestParquetTools.java | 3 +- py/server/deephaven/parquet.py | 8 ++ 19 files changed, 367 insertions(+), 56 deletions(-) create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java 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/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java index 90d4e20add3..9b582888ffa 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 @@ -78,6 +78,10 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { if (columnChunk.isSetOffset_index_offset()) { try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(rootPath)) { readChannel.position(columnChunk.getOffset_index_offset()); + // TODO Think if we need to reduce the buffer size. + // We read BUFFER_SIZE (=65536) number of bytes from the channel, which leads to a big read request to + // aws, even if the offset index is much smaller. Same thing happens for non aws parquet files too but + // reads are less expensive there. offsetIndex = ParquetMetadataConverter.fromParquetOffsetIndex(Util.readOffsetIndex( new BufferedInputStream(Channels.newInputStream(readChannel), BUFFER_SIZE))); } catch (IOException e) { diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 98a40a10546..bab0edd3591 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -27,6 +27,9 @@ dependencies { implementation project(':log-factory') implementation project(':Configuration') implementation depCommonsLang3 + implementation platform('com.amazonaws:aws-java-sdk-bom:1.12.529') + implementation 'com.amazonaws:aws-java-sdk-s3' + Classpaths.inheritCommonsText(project, 'implementation') 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..ca0e34bb6b3 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,11 @@ public final String getColumnNameFromParquetColumnNameOrDefault(final String par */ public abstract boolean useDictionary(String columnName); + /** + * @return The AWS region name to use for S3 operations; defaults to null + */ + public abstract String getAwsRegionName(); + public abstract String getCompressionCodecName(); /** @@ -201,6 +207,11 @@ public boolean useDictionary(final String columnName) { return false; } + @Override + public @Nullable String getAwsRegionName() { + return null; + } + @Override public String getCompressionCodecName() { return defaultCompressionCodecName; @@ -297,6 +308,7 @@ private static final class ReadOnly extends ParquetInstructions { private final boolean isLegacyParquet; private final int targetPageSize; private final boolean isRefreshing; + private final String awsRegionName; private ReadOnly( final KeyedObjectHashMap columnNameToInstructions, @@ -306,7 +318,8 @@ private ReadOnly( final int maximumDictionarySize, final boolean isLegacyParquet, final int targetPageSize, - final boolean isRefreshing) { + final boolean isRefreshing, + final String awsRegionName) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; this.compressionCodecName = compressionCodecName; @@ -315,6 +328,7 @@ private ReadOnly( this.isLegacyParquet = isLegacyParquet; this.targetPageSize = targetPageSize; this.isRefreshing = isRefreshing; + this.awsRegionName = awsRegionName; } private String getOrDefault(final String columnName, final String defaultValue, @@ -403,6 +417,11 @@ public boolean isRefreshing() { return isRefreshing; } + @Override + public String getAwsRegionName() { + return awsRegionName; + } + KeyedObjectHashMap copyColumnNameToInstructions() { // noinspection unchecked return (columnNameToInstructions == null) @@ -453,6 +472,7 @@ public static class Builder { private boolean isLegacyParquet; private int targetPageSize = defaultTargetPageSize; private boolean isRefreshing = DEFAULT_IS_REFRESHING; + private String awsRegionName; public Builder() {} @@ -624,6 +644,11 @@ public Builder setIsRefreshing(final boolean isRefreshing) { return this; } + public Builder setAwsRegionName(final String awsRegionName) { + this.awsRegionName = awsRegionName; + return this; + } + public ParquetInstructions build() { final KeyedObjectHashMap columnNameToInstructionsOut = columnNameToInstructions; columnNameToInstructions = null; @@ -631,7 +656,8 @@ public ParquetInstructions build() { parquetColumnNameToInstructions; parquetColumnNameToInstructions = null; return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, - maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing); + maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, + awsRegionName); } } 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/ParquetTools.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java index c477d65bf69..b6c9b3639b9 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,6 +14,7 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.parquet.table.util.S3BackedSeekableChannelProvider; import io.deephaven.vector.*; import io.deephaven.stringset.StringSet; import io.deephaven.engine.util.file.TrackedFileHandleFactory; @@ -90,6 +91,11 @@ private ParquetTools() {} * @see ParquetFlatPartitionedLayout */ public static Table readTable(@NotNull final String sourceFilePath) { + if (sourceFilePath.startsWith("s3:") && sourceFilePath.endsWith(PARQUET_FILE_EXTENSION)) { + // TODO This is hacky, because here URI is getting converted to a file path and // will change to / + // We need to keep this as a URI and internally check if its a file or S3 backed URI + return readSingleFileTable(new File(sourceFilePath), ParquetInstructions.EMPTY); + } return readTableInternal(new File(sourceFilePath), ParquetInstructions.EMPTY); } @@ -119,6 +125,11 @@ public static Table readTable(@NotNull final String sourceFilePath) { public static Table readTable( @NotNull final String sourceFilePath, @NotNull final ParquetInstructions readInstructions) { + if (sourceFilePath.startsWith("s3:") && sourceFilePath.endsWith(PARQUET_FILE_EXTENSION)) { + // TODO This is hacky, because here URI is getting converted to a file path and // will change to / + // We need to keep this as a URI and internally check if its a file or S3 backed URI + return readSingleFileTable(new File(sourceFilePath), readInstructions); + } return readTableInternal(new File(sourceFilePath), readInstructions); } @@ -846,13 +857,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 +876,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 +887,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 +903,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 +920,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); } /** @@ -932,7 +946,7 @@ public static Table readFlatPartitionedTable( public static Table readSingleFileTable( @NotNull final File file, @NotNull final ParquetInstructions readInstructions) { - final ParquetSingleFileLayout keyFinder = new ParquetSingleFileLayout(file); + final ParquetSingleFileLayout keyFinder = new ParquetSingleFileLayout(file, readInstructions); final KnownLocationKeyFinder inferenceKeys = toKnownKeys(keyFinder); final Pair inference = infer(inferenceKeys, readInstructions); return readSingleFileTable(inferenceKeys.getFirstKey().orElseThrow(), inference.getSecond(), @@ -953,7 +967,8 @@ 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(new ParquetTableLocationKey(file, 0, null, readInstructions), readInstructions, + tableDefinition); } private static final SimpleTypeMap> VECTOR_TYPE_MAP = SimpleTypeMap.create( @@ -1026,11 +1041,13 @@ 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); } @@ -1043,7 +1060,18 @@ 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 { + public static ParquetFileReader getParquetFileReaderChecked( + @NotNull final File parquetFile, + @NotNull final ParquetInstructions readInstructions) throws IOException { + final String absolutePath = parquetFile.getAbsolutePath(); + final String S3_MARKER = "s3:/"; + if (absolutePath.contains(S3_MARKER)) { + // TODO I am creating S3 URI back from the file path which is incorrect, should have passed URI only + final int index = absolutePath.indexOf(S3_MARKER); + final String s3uri = S3_MARKER + absolutePath.substring(index + S3_MARKER.length() - 1); + return new ParquetFileReader(absolutePath, + new S3BackedSeekableChannelProvider(readInstructions.getAwsRegionName(), s3uri)); + } return new ParquetFileReader( parquetFile.getAbsolutePath(), new CachedChannelProvider( @@ -1054,7 +1082,8 @@ public static ParquetFileReader getParquetFileReaderChecked(@NotNull File parque 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..e9699e2deab 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 @@ -6,6 +6,7 @@ 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.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import io.deephaven.util.annotations.VisibleForTesting; import org.jetbrains.annotations.NotNull; @@ -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..8fb14b9ad42 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,6 +4,7 @@ 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; @@ -16,12 +17,16 @@ public final class ParquetSingleFileLayout implements TableLocationKeyFinder { private final File parquetFile; + private final ParquetInstructions readInstructions; /** * @param parquetFile The single parquet file to find + * @param readInstructions the instructions for customizations while reading */ - public ParquetSingleFileLayout(@NotNull final File parquetFile) { + public ParquetSingleFileLayout(@NotNull final File parquetFile, + @NotNull final ParquetInstructions readInstructions) { this.parquetFile = parquetFile; + this.readInstructions = readInstructions; } public String toString() { @@ -30,6 +35,6 @@ public String toString() { @Override public void findKeys(@NotNull final Consumer locationKeyObserver) { - locationKeyObserver.accept(new ParquetTableLocationKey(parquetFile, 0, null)); + locationKeyObserver.accept(new ParquetTableLocationKey(parquetFile, 0, null, readInstructions)); } } 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..5dc3d051903 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 @@ -31,7 +31,8 @@ public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final ParquetTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { final File parquetFile = locationKey.getFile(); - if (parquetFile.exists()) { + // TODO Again hacky, need to keep a URI and check if its a file or not and then do existence check + if (parquetFile.getAbsolutePath().contains("s3:/") || parquetFile.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..fcb6250203b 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,6 +3,7 @@ */ package io.deephaven.parquet.table.location; +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; @@ -31,6 +32,7 @@ public class ParquetTableLocationKey extends FileTableLocationKey { 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,10 +42,13 @@ 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) { @@ -72,7 +77,7 @@ public String getImplementationName() { * * * Callers wishing to handle these cases more explicit may call - * {@link ParquetTools#getParquetFileReaderChecked(File)}. + * {@link ParquetTools#getParquetFileReaderChecked(File, ParquetInstructions)}. * * @return true if the file reader exists or was successfully created */ @@ -81,7 +86,7 @@ public synchronized boolean verifyFileReader() { return true; } try { - fileReader = ParquetTools.getParquetFileReaderChecked(file); + fileReader = ParquetTools.getParquetFileReaderChecked(file, readInstructions); } catch (IOException e) { return false; } @@ -98,7 +103,7 @@ public synchronized ParquetFileReader getFileReader() { if (fileReader != null) { return fileReader; } - return fileReader = ParquetTools.getParquetFileReader(file); + return fileReader = ParquetTools.getParquetFileReader(file, readInstructions); } /** diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java new file mode 100644 index 00000000000..f7288e3c041 --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java @@ -0,0 +1,114 @@ +package io.deephaven.parquet.table.util; + +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3URI; +import com.amazonaws.services.s3.model.GetObjectRequest; +import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.S3ObjectInputStream; +import org.jetbrains.annotations.NotNull; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.SeekableByteChannel; + +public final class S3BackedSeekableByteChannel implements SeekableByteChannel { + + private static final int CLOSED_SENTINEL = -1; + + private final AmazonS3 s3Client; + private final AmazonS3URI s3URI; + + private long size; + private long position; + + S3BackedSeekableByteChannel(@NotNull final AmazonS3 s3Client, @NotNull final AmazonS3URI s3URI, final long size) { + this.s3Client = s3Client; + this.s3URI = s3URI; + this.size = size; + this.position = 0; + } + + @Override + public int read(final ByteBuffer dst) throws IOException { + final int numBytesToRead = dst.remaining(); + final byte[] readBuf = new byte[numBytesToRead]; + long endPosition = position + numBytesToRead - 1; + if (endPosition >= size) { + endPosition = size - 1; + } + final GetObjectRequest rangeObjectRequest = + new GetObjectRequest(s3URI.getBucket(), s3URI.getKey()).withRange(position, endPosition); + + // Following will create and send a GET request over a pool of shared HTTP connections to S3. + final S3Object s3Object = s3Client.getObject(rangeObjectRequest); + final S3ObjectInputStream s3ObjectInputStream = s3Object.getObjectContent(); + int totalBytesRead = 0; + int bytesRead; + do { + if ((bytesRead = + s3ObjectInputStream.read(readBuf, totalBytesRead, numBytesToRead - totalBytesRead)) != -1) { + position += bytesRead; + totalBytesRead += bytesRead; + } + } while (totalBytesRead < numBytesToRead && bytesRead != -1); + s3ObjectInputStream.close(); + if (totalBytesRead > 0) { + dst.put(readBuf, 0, totalBytesRead); // TODO Think if we can avoid this extra copy + return totalBytesRead; + } + return bytesRead; + } + + @Override + public int write(final ByteBuffer src) throws UnsupportedEncodingException { + throw new UnsupportedEncodingException("Don't support writing to S3 yet"); + } + + @Override + public long position() throws ClosedChannelException { + final long localPosition = position; + checkClosed(localPosition); + return localPosition; + } + + @Override + public SeekableByteChannel position(final long newPosition) throws ClosedChannelException { + checkClosed(position); + position = Math.toIntExact(newPosition); + return this; + } + + @Override + public long size() throws ClosedChannelException { + checkClosed(position); + return size; + } + + @Override + public SeekableByteChannel truncate(final long newSize) throws ClosedChannelException { + checkClosed(position); + size = newSize; + if (position > newSize) { + position = newSize; + } + return this; + } + + @Override + public void close() throws IOException { + position = CLOSED_SENTINEL; + } + + private static void checkClosed(final long position) throws ClosedChannelException { + if (position == CLOSED_SENTINEL) { + throw new ClosedChannelException(); + } + } + + @Override + public boolean isOpen() { + return position != CLOSED_SENTINEL; + } +} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java new file mode 100644 index 00000000000..ee7721a3d89 --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java @@ -0,0 +1,76 @@ +package io.deephaven.parquet.table.util; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.AmazonS3URI; +import com.amazonaws.services.s3.model.AmazonS3Exception; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import org.jetbrains.annotations.NotNull; + +import java.io.UnsupportedEncodingException; +import java.nio.channels.SeekableByteChannel; +import java.nio.file.Path; + +public final class S3BackedSeekableChannelProvider implements SeekableChannelsProvider { + /** + * The maximum number of open HTTP connections. + */ + private static final int MAX_HTTP_CONNECTIONS = 50; + + /** + * Shared s3Client object which maintains a pool of HTTP connections to S3 and can be used across threads. + */ + private final AmazonS3 s3Client; + private final AmazonS3URI s3URI; + + /** + * The size of the associated object in bytes. + */ + private final long size; + + public S3BackedSeekableChannelProvider(final String awsRegionName, final String uri) { + if (awsRegionName == null || awsRegionName.isEmpty()) { + throw new IllegalArgumentException("awsRegionName cannot be null or empty"); + } + if (uri == null || uri.isEmpty()) { + throw new IllegalArgumentException("uri cannot be null or empty"); + } + // TODO There are a many other config options. Discuss and decide the values. + final ClientConfiguration clientConfig = new ClientConfiguration().withMaxConnections(MAX_HTTP_CONNECTIONS); + + // Following will automatically read credentials from aws credentials file in "~/.aws/credentials" + s3Client = AmazonS3ClientBuilder.standard() + .withRegion(Regions.fromName(awsRegionName)) + .withClientConfiguration(clientConfig) + .build(); + s3URI = new AmazonS3URI(uri); + try { + // TODO THis will send a HEAD request to S3. Discuss and decide if we want to do this right now or later + // when we make the first channel. I wanted to keep it here since channel creation can be parallel and + // we are sure that a channel will be made for this URI when we fetch in the metadata. + size = s3Client.getObjectMetadata(s3URI.getBucket(), s3URI.getKey()).getContentLength(); + } catch (final AmazonS3Exception e) { + if (e.getStatusCode() == 404) { + throw new UncheckedDeephavenException( + "Object " + uri + " in region " + awsRegionName + " does not exist", e); + } + throw new UncheckedDeephavenException("Failed to get object metadata for " + uri + " in region " + + awsRegionName + ". Please verify all inputs along with credentials are accurate. Refer the error " + + "status code for more details", e); + } + } + + @Override + public SeekableByteChannel getReadChannel(@NotNull final Path path) { + return new S3BackedSeekableByteChannel(s3Client, s3URI, size); + } + + @Override + public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boolean append) + throws UnsupportedEncodingException { + throw new UnsupportedEncodingException("Don't support writing to S3 yet"); + } +} 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..11620fbe7b0 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 @@ -554,7 +554,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 +563,28 @@ 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 readParquetFileFromS3Test() { + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setAwsRegionName("us-east-1") + .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); + } + @Test public void stringDictionaryTest() { final int nullPos = -5; @@ -670,7 +686,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(); @@ -926,7 +943,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 +977,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 +1106,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 +1157,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 +1174,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 +1265,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 +1320,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 +1344,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 +1405,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 +1866,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/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index 4ec51e7db26..f4e727ec326 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -42,6 +42,7 @@ def _build_parquet_instructions( is_refreshing: bool = False, for_read: bool = True, force_build: bool = False, + aws_region_name: str = None, ): if not any( [ @@ -53,6 +54,7 @@ def _build_parquet_instructions( is_legacy_parquet, target_page_size is not None, is_refreshing, + aws_region_name, ] ): return None @@ -89,6 +91,9 @@ def _build_parquet_instructions( if is_refreshing: builder.setIsRefreshing(is_refreshing) + if aws_region_name: + builder.setAwsRegionName(aws_region_name) + return builder.build() def _j_table_definition(table_definition: Union[Dict[str, DType], List[Column], None]) -> Optional[jpy.JType]: @@ -131,6 +136,7 @@ def read( is_refreshing: bool = False, file_layout: Optional[ParquetFileLayout] = None, table_definition: Union[Dict[str, DType], List[Column], None] = None, + aws_region_name: str = None, ) -> Table: """ Reads in a table from a single parquet, metadata file, or directory with recognized layout. @@ -147,6 +153,7 @@ def read( have that definition. This is useful for bootstrapping purposes when the initial 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. + aws_region_name (str): the AWS region name for reading parquet files stored in AWS S3, by default None Returns: a table @@ -161,6 +168,7 @@ def read( is_refreshing=is_refreshing, for_read=True, force_build=True, + aws_region_name=aws_region_name, ) j_table_definition = _j_table_definition(table_definition) if j_table_definition is not None: From 2fe31fb22b039b749bd4e4e604f7b17498b80278 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 12 Dec 2023 19:22:58 +0530 Subject: [PATCH 02/39] Moved to AWS SDK V2 --- .../parquet/base/ColumnChunkReaderImpl.java | 29 ++++-- extensions/parquet/table/build.gradle | 6 +- .../deephaven/parquet/table/ParquetTools.java | 14 ++- .../util/S3BackedSeekableByteChannel.java | 49 +++++----- .../util/S3BackedSeekableChannelProvider.java | 70 ++++++++----- .../table/ParquetTableReadWriteTest.java | 98 +++++++++++++++++++ 6 files changed, 209 insertions(+), 57 deletions(-) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 8de3d8b9281..c1bf9efac0e 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 @@ -15,14 +15,13 @@ 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.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; @@ -193,7 +192,7 @@ public String getVersion() { @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 +201,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(); @@ -230,6 +229,17 @@ public boolean hasNext() { return remainingValues > 0; } + // TODO Move to a separate file + final class PositionedBufferedInputStream extends BufferedInputStream { + PositionedBufferedInputStream(final ReadableByteChannel readChannel, final int size) { + super(Channels.newInputStream(readChannel), size); + } + + long position() throws IOException { + return this.pos; + } + } + @Override public ColumnPageReader next() { if (!hasNext()) { @@ -240,8 +250,13 @@ public ColumnPageReader next() { final long headerOffset = currentOffset; readChannel.position(currentOffset); // deliberately not closing this stream - final PageHeader pageHeader = Util.readPageHeader(Channels.newInputStream(readChannel)); - currentOffset = readChannel.position() + pageHeader.getCompressed_page_size(); + // TODO Assuming header size will be less than 16384 + final PositionedBufferedInputStream bufferedInput = + new PositionedBufferedInputStream(readChannel, 16384); + final PageHeader pageHeader = Util.readPageHeader(bufferedInput); + final long headerSize = bufferedInput.position(); + final long pageDataOffset = currentOffset + headerSize; + currentOffset += headerSize + pageHeader.getCompressed_page_size(); if (pageHeader.isSetDictionary_page_header()) { // Dictionary page; skip it return next(); @@ -271,7 +286,7 @@ public ColumnPageReader next() { ? dictionarySupplier : () -> NULL_DICTIONARY; return new ColumnPageReaderImpl(channelsProvider, decompressor, pageDictionarySupplier, - nullMaterializerFactory, path, getFilePath(), fieldTypes, readChannel.position(), pageHeader, + nullMaterializerFactory, path, getFilePath(), fieldTypes, pageDataOffset, pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES); } catch (IOException e) { throw new UncheckedDeephavenException("Error reading page header", e); diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index bab0edd3591..45dc4a6ed1b 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -27,9 +27,9 @@ dependencies { implementation project(':log-factory') implementation project(':Configuration') implementation depCommonsLang3 - implementation platform('com.amazonaws:aws-java-sdk-bom:1.12.529') - implementation 'com.amazonaws:aws-java-sdk-s3' - + implementation platform('software.amazon.awssdk:bom:2.21.43') + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:apache-client:2.21.43' Classpaths.inheritCommonsText(project, 'implementation') 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 b6c9b3639b9..d406d73c34a 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 @@ -99,6 +99,17 @@ public static Table readTable(@NotNull final String sourceFilePath) { return readTableInternal(new File(sourceFilePath), ParquetInstructions.EMPTY); } + public static Table readTable(@NotNull final String sourceFilePath, + @NotNull final ParquetInstructions readInstructions, + @NotNull final TableDefinition tableDefinition) { + if (sourceFilePath.startsWith("s3:") && sourceFilePath.endsWith(PARQUET_FILE_EXTENSION)) { + // TODO This is hacky, because here URI is getting converted to a file path and // will change to / + // We need to keep this as a URI and internally check if its a file or S3 backed URI + return readSingleFileTable(new File(sourceFilePath), readInstructions, tableDefinition); + } + return readTableInternal(new File(sourceFilePath), readInstructions); + } + /** * Reads in a table from a single parquet, metadata file, or directory with recognized layout. * @@ -1070,7 +1081,8 @@ public static ParquetFileReader getParquetFileReaderChecked( final int index = absolutePath.indexOf(S3_MARKER); final String s3uri = S3_MARKER + absolutePath.substring(index + S3_MARKER.length() - 1); return new ParquetFileReader(absolutePath, - new S3BackedSeekableChannelProvider(readInstructions.getAwsRegionName(), s3uri)); + new CachedChannelProvider( + new S3BackedSeekableChannelProvider(readInstructions.getAwsRegionName(), s3uri), 1 << 7)); } return new ParquetFileReader( parquetFile.getAbsolutePath(), diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java index f7288e3c041..fcdabebe480 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java @@ -1,11 +1,12 @@ package io.deephaven.parquet.table.util; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3URI; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectInputStream; +import io.deephaven.base.verify.Assert; import org.jetbrains.annotations.NotNull; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.S3Uri; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; import java.io.IOException; import java.io.UnsupportedEncodingException; @@ -17,13 +18,13 @@ public final class S3BackedSeekableByteChannel implements SeekableByteChannel { private static final int CLOSED_SENTINEL = -1; - private final AmazonS3 s3Client; - private final AmazonS3URI s3URI; + private final S3Client s3Client; + private final S3Uri s3URI; private long size; private long position; - S3BackedSeekableByteChannel(@NotNull final AmazonS3 s3Client, @NotNull final AmazonS3URI s3URI, final long size) { + S3BackedSeekableByteChannel(@NotNull final S3Client s3Client, @NotNull final S3Uri s3URI, final long size) { this.s3Client = s3Client; this.s3URI = s3URI; this.size = size; @@ -38,22 +39,26 @@ public int read(final ByteBuffer dst) throws IOException { if (endPosition >= size) { endPosition = size - 1; } - final GetObjectRequest rangeObjectRequest = - new GetObjectRequest(s3URI.getBucket(), s3URI.getKey()).withRange(position, endPosition); - - // Following will create and send a GET request over a pool of shared HTTP connections to S3. - final S3Object s3Object = s3Client.getObject(rangeObjectRequest); - final S3ObjectInputStream s3ObjectInputStream = s3Object.getObjectContent(); + final String byteRange = "bytes=" + position + "-" + endPosition; + Assert.eqTrue(s3URI.bucket().isPresent(), "s3URI.bucket().isPresent()"); + Assert.eqTrue(s3URI.key().isPresent(), "s3URI.key().isPresent()"); + final GetObjectRequest rangeObjectRequest = GetObjectRequest.builder() + .bucket(s3URI.bucket().get()) + .key(s3URI.key().get()) + .range(byteRange) + .build(); + + // Following will create and send a blocking GET request over a pool of shared HTTP connections to S3. int totalBytesRead = 0; int bytesRead; - do { - if ((bytesRead = - s3ObjectInputStream.read(readBuf, totalBytesRead, numBytesToRead - totalBytesRead)) != -1) { - position += bytesRead; - totalBytesRead += bytesRead; - } - } while (totalBytesRead < numBytesToRead && bytesRead != -1); - s3ObjectInputStream.close(); + try (final ResponseInputStream s3InputStream = s3Client.getObject(rangeObjectRequest)) { + do { + if ((bytesRead = s3InputStream.read(readBuf, totalBytesRead, numBytesToRead - totalBytesRead)) != -1) { + position += bytesRead; + totalBytesRead += bytesRead; + } + } while (totalBytesRead < numBytesToRead && bytesRead != -1); + } if (totalBytesRead > 0) { dst.put(readBuf, 0, totalBytesRead); // TODO Think if we can avoid this extra copy return totalBytesRead; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java index ee7721a3d89..2ae100a9723 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java @@ -1,16 +1,20 @@ package io.deephaven.parquet.table.util; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.regions.Regions; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3ClientBuilder; -import com.amazonaws.services.s3.AmazonS3URI; -import com.amazonaws.services.s3.model.AmazonS3Exception; import io.deephaven.UncheckedDeephavenException; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.S3Uri; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.NoSuchKeyException; +import java.io.IOException; import java.io.UnsupportedEncodingException; +import java.net.URI; +import java.net.URISyntaxException; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -23,39 +27,53 @@ public final class S3BackedSeekableChannelProvider implements SeekableChannelsPr /** * Shared s3Client object which maintains a pool of HTTP connections to S3 and can be used across threads. */ - private final AmazonS3 s3Client; - private final AmazonS3URI s3URI; + private final S3Client s3Client; + private final S3Uri s3URI; /** * The size of the associated object in bytes. */ private final long size; - public S3BackedSeekableChannelProvider(final String awsRegionName, final String uri) { + public S3BackedSeekableChannelProvider(final String awsRegionName, final String uriStr) { if (awsRegionName == null || awsRegionName.isEmpty()) { throw new IllegalArgumentException("awsRegionName cannot be null or empty"); } - if (uri == null || uri.isEmpty()) { + if (uriStr == null || uriStr.isEmpty()) { throw new IllegalArgumentException("uri cannot be null or empty"); } - // TODO There are a many other config options. Discuss and decide the values. - final ClientConfiguration clientConfig = new ClientConfiguration().withMaxConnections(MAX_HTTP_CONNECTIONS); + // TODO Number of other options also available, discuss with others and decide if we want to change any + final SdkHttpClient httpClient = ApacheHttpClient.builder() + .maxConnections(MAX_HTTP_CONNECTIONS) + .build(); // Following will automatically read credentials from aws credentials file in "~/.aws/credentials" - s3Client = AmazonS3ClientBuilder.standard() - .withRegion(Regions.fromName(awsRegionName)) - .withClientConfiguration(clientConfig) + s3Client = S3Client.builder() + .region(Region.of(awsRegionName)) + .httpClient(httpClient) .build(); - s3URI = new AmazonS3URI(uri); + final URI uri; + try { + uri = new URI(uriStr); + } catch (final URISyntaxException e) { + throw new UncheckedDeephavenException("Failed to parse URI " + uriStr, e); + } + s3URI = S3Uri.builder().uri(uri).bucket(uri.getHost()).key(uri.getPath().substring(1)).build(); try { - // TODO THis will send a HEAD request to S3. Discuss and decide if we want to do this right now or later - // when we make the first channel. I wanted to keep it here since channel creation can be parallel and - // we are sure that a channel will be made for this URI when we fetch in the metadata. - size = s3Client.getObjectMetadata(s3URI.getBucket(), s3URI.getKey()).getContentLength(); - } catch (final AmazonS3Exception e) { - if (e.getStatusCode() == 404) { - throw new UncheckedDeephavenException( - "Object " + uri + " in region " + awsRegionName + " does not exist", e); + // Discuss and decide if we want to do this right now or later + // when we make the first channel. I wanted to keep it here since channel creation can be done by multiple + // threads but this object creation is only in single thread. And we are sure that a channel will be made + // for this URI when we fetch in the metadata. + // Also, I am using a synchronous client here, there is also an async client, should we use that instead? + final HeadObjectRequest headObjectRequest = HeadObjectRequest.builder() + .bucket(s3URI.bucket().get()) + .key(s3URI.key().get()) + .build(); + size = s3Client.headObject(headObjectRequest).contentLength().longValue(); + } catch (final NoSuchKeyException e) { + if (e.statusCode() == 404) { + throw new UncheckedDeephavenException("Object " + uri + " in region " + awsRegionName + + " does not exist", e); } throw new UncheckedDeephavenException("Failed to get object metadata for " + uri + " in region " + awsRegionName + ". Please verify all inputs along with credentials are accurate. Refer the error " @@ -73,4 +91,8 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boole throws UnsupportedEncodingException { throw new UnsupportedEncodingException("Don't support writing to S3 yet"); } + + public void close() throws IOException { + s3Client.close(); + } } 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 11620fbe7b0..c77fc176ace 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 @@ -569,6 +569,104 @@ public void testArrayColumns() { && !firstColumnMetadata.contains("RLE_DICTIONARY")); } + @Test + public void readRefParquetFileFromS3Test() { + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setAwsRegionName("us-east-2") + .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")); + + final Table fromAws1 = ParquetTools.readTable( + "s3://aws-public-blockchain/v1.0/btc/transactions/date=2009-01-03/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet", + readInstructions, tableDefinition).head(5).select(); + final Table fromDisk1 = ParquetTools.readSingleFileTable( + new File( + "/Users/shivammalhotra/Documents/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet"), + ParquetTools.SNAPPY, + tableDefinition).head(5).select(); + assertTableEquals(fromAws1, fromDisk1); + + final Table fromAws2 = ParquetTools.readTable( + "s3://aws-public-blockchain/v1.0/btc/transactions/date=2023-11-13/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet", + readInstructions, tableDefinition).head(5).select(); + final Table fromDisk2 = ParquetTools.readSingleFileTable( + new File( + "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), + ParquetTools.SNAPPY, + tableDefinition).head(5).select(); + assertTableEquals(fromAws2, fromDisk2); + } + + @Test + public void readRefParquetFileLocally() { + 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")); + final Table fromAws1 = + ParquetTools.readSingleFileTable( + new File( + "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), + // new File( + // "/Users/shivammalhotra/Documents/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet"), + ParquetTools.SNAPPY, + tableDefinition).head(5).select(); + } + + @Test + public void profileReadingFromS3() { + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setAwsRegionName("us-east-1") + .build(); + + long totalTime = 0; + long NUM_RUNS = 5; + for (int i = 0; i < NUM_RUNS; i++) { + final long start = System.nanoTime(); + ParquetTools.readTable("s3://dh-s3-parquet-test1/multiColFile.parquet", readInstructions).select(); + final long end = System.nanoTime(); + totalTime += end - start; + System.out.println((i + 1) + ". Execution time AWS is " + (end - start) / 1000_000_000.0 + " sec"); + } + System.out.println("Average execution time AWS is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); + + NUM_RUNS = 100; + totalTime = 0; + for (int i = 0; i < NUM_RUNS; i++) { + final long start = System.nanoTime(); + ParquetTools.readTable("/Users/shivammalhotra/documents/multiColFile.parquet").select(); + final long end = System.nanoTime(); + totalTime += end - start; + System.out.println((i + 1) + ". Execution time local is " + (end - start) / 1000_000_000.0 + " sec"); + } + System.out.println("Average execution time local is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); + } + @Test public void readParquetFileFromS3Test() { final ParquetInstructions readInstructions = new ParquetInstructions.Builder() From c69f875ae9095d24b66f76a10f36e6d19f04dbb1 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 14 Dec 2023 17:45:41 +0530 Subject: [PATCH 03/39] Added support for S3 nio AWS channel --- .../parquet/base/ColumnChunkReaderImpl.java | 11 +-- extensions/parquet/table/build.gradle | 2 + .../util/S3BackedSeekableChannelProvider.java | 82 +++++++------------ 3 files changed, 33 insertions(+), 62 deletions(-) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index c1bf9efac0e..0bf71390501 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 @@ -250,13 +250,8 @@ public ColumnPageReader next() { final long headerOffset = currentOffset; readChannel.position(currentOffset); // deliberately not closing this stream - // TODO Assuming header size will be less than 16384 - final PositionedBufferedInputStream bufferedInput = - new PositionedBufferedInputStream(readChannel, 16384); - final PageHeader pageHeader = Util.readPageHeader(bufferedInput); - final long headerSize = bufferedInput.position(); - final long pageDataOffset = currentOffset + headerSize; - currentOffset += headerSize + pageHeader.getCompressed_page_size(); + final PageHeader pageHeader = Util.readPageHeader(Channels.newInputStream(readChannel)); + currentOffset = readChannel.position() + pageHeader.getCompressed_page_size(); if (pageHeader.isSetDictionary_page_header()) { // Dictionary page; skip it return next(); @@ -286,7 +281,7 @@ public ColumnPageReader next() { ? dictionarySupplier : () -> NULL_DICTIONARY; return new ColumnPageReaderImpl(channelsProvider, decompressor, pageDictionarySupplier, - nullMaterializerFactory, path, getFilePath(), fieldTypes, pageDataOffset, pageHeader, + nullMaterializerFactory, path, getFilePath(), fieldTypes, readChannel.position(), pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES); } catch (IOException e) { throw new UncheckedDeephavenException("Error reading page header", e); diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 45dc4a6ed1b..94c57c92baa 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -31,6 +31,8 @@ dependencies { implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:apache-client:2.21.43' + implementation("software.amazon.nio.s3:aws-java-nio-spi-for-s3:1.2.4") + Classpaths.inheritCommonsText(project, 'implementation') Classpaths.inheritJacksonPlatform(project) diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java index 2ae100a9723..6b06b56b91f 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java @@ -3,87 +3,60 @@ import io.deephaven.UncheckedDeephavenException; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; -import software.amazon.awssdk.http.apache.ApacheHttpClient; -import software.amazon.awssdk.http.SdkHttpClient; import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.S3Uri; -import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.NoSuchKeyException; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.nio.spi.s3.FixedS3ClientProvider; +import software.amazon.nio.spi.s3.S3FileSystem; +import software.amazon.nio.spi.s3.S3FileSystemProvider; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URI; import java.net.URISyntaxException; import java.nio.channels.SeekableByteChannel; +import java.nio.file.FileSystemAlreadyExistsException; import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.Collections; public final class S3BackedSeekableChannelProvider implements SeekableChannelsProvider { - /** - * The maximum number of open HTTP connections. - */ - private static final int MAX_HTTP_CONNECTIONS = 50; - /** - * Shared s3Client object which maintains a pool of HTTP connections to S3 and can be used across threads. - */ - private final S3Client s3Client; - private final S3Uri s3URI; + private final S3FileSystemProvider provider; + private final S3FileSystem fileSystem; + private final S3AsyncClient s3AsyncClient; + private final URI uri; - /** - * The size of the associated object in bytes. - */ - private final long size; - - public S3BackedSeekableChannelProvider(final String awsRegionName, final String uriStr) { + public S3BackedSeekableChannelProvider(final String awsRegionName, final String uriStr) throws IOException { if (awsRegionName == null || awsRegionName.isEmpty()) { throw new IllegalArgumentException("awsRegionName cannot be null or empty"); } if (uriStr == null || uriStr.isEmpty()) { throw new IllegalArgumentException("uri cannot be null or empty"); } - // TODO Number of other options also available, discuss with others and decide if we want to change any - final SdkHttpClient httpClient = ApacheHttpClient.builder() - .maxConnections(MAX_HTTP_CONNECTIONS) - .build(); - - // Following will automatically read credentials from aws credentials file in "~/.aws/credentials" - s3Client = S3Client.builder() - .region(Region.of(awsRegionName)) - .httpClient(httpClient) - .build(); - final URI uri; try { uri = new URI(uriStr); } catch (final URISyntaxException e) { throw new UncheckedDeephavenException("Failed to parse URI " + uriStr, e); } - s3URI = S3Uri.builder().uri(uri).bucket(uri.getHost()).key(uri.getPath().substring(1)).build(); + final String bucket = uri.getHost(); + provider = new S3FileSystemProvider(); + S3FileSystem tempFileSystem; try { - // Discuss and decide if we want to do this right now or later - // when we make the first channel. I wanted to keep it here since channel creation can be done by multiple - // threads but this object creation is only in single thread. And we are sure that a channel will be made - // for this URI when we fetch in the metadata. - // Also, I am using a synchronous client here, there is also an async client, should we use that instead? - final HeadObjectRequest headObjectRequest = HeadObjectRequest.builder() - .bucket(s3URI.bucket().get()) - .key(s3URI.key().get()) - .build(); - size = s3Client.headObject(headObjectRequest).contentLength().longValue(); - } catch (final NoSuchKeyException e) { - if (e.statusCode() == 404) { - throw new UncheckedDeephavenException("Object " + uri + " in region " + awsRegionName + - " does not exist", e); - } - throw new UncheckedDeephavenException("Failed to get object metadata for " + uri + " in region " - + awsRegionName + ". Please verify all inputs along with credentials are accurate. Refer the error " - + "status code for more details", e); + tempFileSystem = provider.newFileSystem(URI.create("s3://" + bucket)); + } catch (final FileSystemAlreadyExistsException e) { + tempFileSystem = provider.getFileSystem(URI.create("s3://" + bucket)); } + fileSystem = tempFileSystem; + s3AsyncClient = S3AsyncClient.builder() + .region(Region.of(awsRegionName)) + .build(); + fileSystem.clientProvider(new FixedS3ClientProvider(s3AsyncClient)); } @Override - public SeekableByteChannel getReadChannel(@NotNull final Path path) { - return new S3BackedSeekableByteChannel(s3Client, s3URI, size); + public SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { + return provider.newByteChannel(Paths.get(uri), Collections.singleton(StandardOpenOption.READ)); } @Override @@ -93,6 +66,7 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boole } public void close() throws IOException { - s3Client.close(); + fileSystem.close(); + s3AsyncClient.close(); } } From 4b985515accbca4f3e0b00333865c1563eaa41e6 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 18 Dec 2023 14:46:10 +0530 Subject: [PATCH 04/39] Moved s3 nio files to local repo --- extensions/parquet/table/build.gradle | 2 +- .../util/S3BackedSeekableByteChannel.java | 119 --------- .../util/S3BackedSeekableChannelProvider.java | 33 +-- .../table/util/S3ReadAheadByteChannel.java | 226 +++++++++++++++++ .../table/util/S3SeekableByteChannel.java | 236 ++++++++++++++++++ .../table/ParquetTableReadWriteTest.java | 38 ++- 6 files changed, 510 insertions(+), 144 deletions(-) delete mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3ReadAheadByteChannel.java create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 94c57c92baa..ba206c926cc 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -31,7 +31,7 @@ dependencies { implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:apache-client:2.21.43' - implementation("software.amazon.nio.s3:aws-java-nio-spi-for-s3:1.2.4") + implementation('com.github.ben-manes.caffeine:caffeine:2.9.3') Classpaths.inheritCommonsText(project, 'implementation') diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java deleted file mode 100644 index fcdabebe480..00000000000 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableByteChannel.java +++ /dev/null @@ -1,119 +0,0 @@ -package io.deephaven.parquet.table.util; - -import io.deephaven.base.verify.Assert; -import org.jetbrains.annotations.NotNull; -import software.amazon.awssdk.core.ResponseInputStream; -import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.S3Uri; -import software.amazon.awssdk.services.s3.model.GetObjectRequest; -import software.amazon.awssdk.services.s3.model.GetObjectResponse; - -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.nio.ByteBuffer; -import java.nio.channels.ClosedChannelException; -import java.nio.channels.SeekableByteChannel; - -public final class S3BackedSeekableByteChannel implements SeekableByteChannel { - - private static final int CLOSED_SENTINEL = -1; - - private final S3Client s3Client; - private final S3Uri s3URI; - - private long size; - private long position; - - S3BackedSeekableByteChannel(@NotNull final S3Client s3Client, @NotNull final S3Uri s3URI, final long size) { - this.s3Client = s3Client; - this.s3URI = s3URI; - this.size = size; - this.position = 0; - } - - @Override - public int read(final ByteBuffer dst) throws IOException { - final int numBytesToRead = dst.remaining(); - final byte[] readBuf = new byte[numBytesToRead]; - long endPosition = position + numBytesToRead - 1; - if (endPosition >= size) { - endPosition = size - 1; - } - final String byteRange = "bytes=" + position + "-" + endPosition; - Assert.eqTrue(s3URI.bucket().isPresent(), "s3URI.bucket().isPresent()"); - Assert.eqTrue(s3URI.key().isPresent(), "s3URI.key().isPresent()"); - final GetObjectRequest rangeObjectRequest = GetObjectRequest.builder() - .bucket(s3URI.bucket().get()) - .key(s3URI.key().get()) - .range(byteRange) - .build(); - - // Following will create and send a blocking GET request over a pool of shared HTTP connections to S3. - int totalBytesRead = 0; - int bytesRead; - try (final ResponseInputStream s3InputStream = s3Client.getObject(rangeObjectRequest)) { - do { - if ((bytesRead = s3InputStream.read(readBuf, totalBytesRead, numBytesToRead - totalBytesRead)) != -1) { - position += bytesRead; - totalBytesRead += bytesRead; - } - } while (totalBytesRead < numBytesToRead && bytesRead != -1); - } - if (totalBytesRead > 0) { - dst.put(readBuf, 0, totalBytesRead); // TODO Think if we can avoid this extra copy - return totalBytesRead; - } - return bytesRead; - } - - @Override - public int write(final ByteBuffer src) throws UnsupportedEncodingException { - throw new UnsupportedEncodingException("Don't support writing to S3 yet"); - } - - @Override - public long position() throws ClosedChannelException { - final long localPosition = position; - checkClosed(localPosition); - return localPosition; - } - - @Override - public SeekableByteChannel position(final long newPosition) throws ClosedChannelException { - checkClosed(position); - position = Math.toIntExact(newPosition); - return this; - } - - @Override - public long size() throws ClosedChannelException { - checkClosed(position); - return size; - } - - @Override - public SeekableByteChannel truncate(final long newSize) throws ClosedChannelException { - checkClosed(position); - size = newSize; - if (position > newSize) { - position = newSize; - } - return this; - } - - @Override - public void close() throws IOException { - position = CLOSED_SENTINEL; - } - - private static void checkClosed(final long position) throws ClosedChannelException { - if (position == CLOSED_SENTINEL) { - throw new ClosedChannelException(); - } - } - - @Override - public boolean isOpen() { - return position != CLOSED_SENTINEL; - } -} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java index 6b06b56b91f..a789e5bdb8a 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java @@ -1,31 +1,29 @@ package io.deephaven.parquet.table.util; import io.deephaven.UncheckedDeephavenException; +import io.deephaven.configuration.Configuration; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3AsyncClient; -import software.amazon.nio.spi.s3.FixedS3ClientProvider; -import software.amazon.nio.spi.s3.S3FileSystem; -import software.amazon.nio.spi.s3.S3FileSystemProvider; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URI; import java.net.URISyntaxException; import java.nio.channels.SeekableByteChannel; -import java.nio.file.FileSystemAlreadyExistsException; import java.nio.file.Path; -import java.nio.file.Paths; -import java.nio.file.StandardOpenOption; -import java.util.Collections; public final class S3BackedSeekableChannelProvider implements SeekableChannelsProvider { - private final S3FileSystemProvider provider; - private final S3FileSystem fileSystem; private final S3AsyncClient s3AsyncClient; private final URI uri; + private final String s3uri, bucket, key; + + public static final int MAX_FRAGMENT_SIZE = + Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-size", 512 * 1024); // 512 KB + private static final int MAX_FRAGMENT_NUMBER = + Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-number", 2); public S3BackedSeekableChannelProvider(final String awsRegionName, final String uriStr) throws IOException { if (awsRegionName == null || awsRegionName.isEmpty()) { @@ -39,24 +37,18 @@ public S3BackedSeekableChannelProvider(final String awsRegionName, final String } catch (final URISyntaxException e) { throw new UncheckedDeephavenException("Failed to parse URI " + uriStr, e); } - final String bucket = uri.getHost(); - provider = new S3FileSystemProvider(); - S3FileSystem tempFileSystem; - try { - tempFileSystem = provider.newFileSystem(URI.create("s3://" + bucket)); - } catch (final FileSystemAlreadyExistsException e) { - tempFileSystem = provider.getFileSystem(URI.create("s3://" + bucket)); - } - fileSystem = tempFileSystem; + this.s3uri = uriStr; + this.bucket = uri.getHost(); + this.key = uri.getPath().substring(1); s3AsyncClient = S3AsyncClient.builder() .region(Region.of(awsRegionName)) .build(); - fileSystem.clientProvider(new FixedS3ClientProvider(s3AsyncClient)); } @Override public SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { - return provider.newByteChannel(Paths.get(uri), Collections.singleton(StandardOpenOption.READ)); + return new S3SeekableByteChannel(s3uri, bucket, key, s3AsyncClient, 0, MAX_FRAGMENT_SIZE, MAX_FRAGMENT_NUMBER, + null, null); } @Override @@ -66,7 +58,6 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boole } public void close() throws IOException { - fileSystem.close(); s3AsyncClient.close(); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3ReadAheadByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3ReadAheadByteChannel.java new file mode 100644 index 00000000000..e00ecad21fc --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3ReadAheadByteChannel.java @@ -0,0 +1,226 @@ +package io.deephaven.parquet.table.util; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.stats.CacheStats; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.BytesWrapper; +import software.amazon.awssdk.core.async.AsyncResponseTransformer; +import software.amazon.awssdk.services.s3.S3AsyncClient; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * A {@code ReadableByteChannel} delegate for an {@code S3SeekableByteChannel} that maintains internal read ahead + * buffers to reduce the amount of high latency + * requests to S3. If the bytes required by a read are already in the buffer, they will be fulfilled from the buffer + * rather than making another S3 request. + *

As reads are made this object will update the current read position of the delegating {@code S3SeekableByteChannel}

+ */ +public class S3ReadAheadByteChannel implements ReadableByteChannel { + + private final S3AsyncClient client; + private final String s3uri, bucket, key; + private final S3SeekableByteChannel delegator; + private final int maxFragmentSize; + private final int maxNumberFragments; + private final int numFragmentsInObject; + private final long size; + private final Long timeout; + private final TimeUnit timeUnit; + private boolean open; + private final Cache> readAheadBuffersCache; + + private final Logger logger = LoggerFactory.getLogger(this.getClass()); + + + /** + * Construct a new {@code S3ReadAheadByteChannel} which is used by its parent delegator to perform read operations. + * The channel is backed by a cache that holds the buffered fragments of the object identified + * by the {@code path}. + * + * @param maxFragmentSize the maximum amount of bytes in a read ahead fragment. Must be {@code >= 1}. + * @param maxNumberFragments the maximum number of read ahead fragments to hold. Must be {@code >= 2}. + * @param client the client used to read from the {@code path} + * @param delegator the {@code S3SeekableByteChannel} that delegates reading to this object. + * @param timeout the amount of time to wait for a read ahead fragment to be available. + * @param timeUnit the {@code TimeUnit} for the {@code timeout}. + * @throws IOException if a problem occurs initializing the cached fragments + */ + public S3ReadAheadByteChannel(String s3uri, String bucket, String key, int maxFragmentSize, int maxNumberFragments, S3AsyncClient client, S3SeekableByteChannel delegator, Long timeout, TimeUnit timeUnit) throws IOException { + Objects.requireNonNull(client); + Objects.requireNonNull(delegator); + if (maxFragmentSize < 1) + throw new IllegalArgumentException("maxFragmentSize must be >= 1"); + if (maxNumberFragments < 2) + throw new IllegalArgumentException("maxNumberFragments must be >= 2"); + + logger.debug("max read ahead fragments '{}' with size '{}' bytes", maxNumberFragments, maxFragmentSize); + this.client = client; + this.s3uri = s3uri; + this.bucket = bucket; + this.key = key; + this.delegator = delegator; + this.size = delegator.size(); + this.maxFragmentSize = maxFragmentSize; + this.numFragmentsInObject = (int) Math.ceil((float) size / (float) maxFragmentSize); + this.readAheadBuffersCache = Caffeine.newBuilder().maximumSize(maxNumberFragments).recordStats().build(); + this.maxNumberFragments = maxNumberFragments; + this.open = true; + this.timeout = timeout != null ? timeout : 5L; + this.timeUnit = timeUnit != null ? timeUnit : TimeUnit.MINUTES; + } + + @Override + public int read(ByteBuffer dst) throws IOException { + Objects.requireNonNull(dst); + + long channelPosition = delegator.position(); + logger.debug("delegator position: {}", channelPosition); + + // if the position of the delegator is at the end (>= size) return -1. we're finished reading. + if (channelPosition >= size) return -1; + + //figure out the index of the fragment the bytes would start in + Integer fragmentIndex = fragmentIndexForByteNumber(channelPosition); + logger.debug("fragment index: {}", fragmentIndex); + + int fragmentOffset = (int) (channelPosition - (fragmentIndex.longValue() * maxFragmentSize)); + logger.debug("fragment {} offset: {}", fragmentIndex, fragmentOffset); + + try { + final ByteBuffer fragment = Objects.requireNonNull(readAheadBuffersCache.get(fragmentIndex, this::computeFragmentFuture)) + .get(timeout, timeUnit) + .asReadOnlyBuffer(); + + fragment.position(fragmentOffset); + logger.debug("fragment remaining: {}", fragment.remaining()); + logger.debug("dst remaining: {}", dst.remaining()); + + //put the bytes from fragment from the offset upto the min of fragment remaining or dst remaining + int limit = Math.min(fragment.remaining(), dst.remaining()); + logger.debug("byte limit: {}", limit); + + byte[] copiedBytes = new byte[limit]; + fragment.get(copiedBytes, 0, limit); + dst.put(copiedBytes); + + if (fragment.position() >= fragment.limit() / 2) { + + // clear any fragments in cache that are lower index than this one // <-- This can be an issue for us + clearPriorFragments(fragmentIndex); + + // until available cache slots are filled or number of fragments in file + int maxFragmentsToLoad = Math.min(maxNumberFragments - 1, numFragmentsInObject - fragmentIndex - 1); + + for (int i = 0; i < maxFragmentsToLoad; i++) { + final int idxToLoad = i + fragmentIndex + 1; + + // add the index if it's not already there + if (readAheadBuffersCache.asMap().containsKey(idxToLoad)) + continue; + + logger.debug("initiate pre-loading fragment with index '{}' from '{}'", idxToLoad, s3uri); + readAheadBuffersCache.put(idxToLoad, computeFragmentFuture(idxToLoad)); + } + } + + delegator.position(channelPosition + copiedBytes.length); + return copiedBytes.length; + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } catch (ExecutionException e) { + // the async execution completed exceptionally. + // not currently obvious when this will happen or if we can recover + logger.error("an exception occurred while reading bytes from {} that was not recovered by the S3 Client RetryCondition(s)", s3uri); + throw new IOException(e); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + } + + private void clearPriorFragments(int currentFragIndx) { + final Set<@NonNull Integer> priorIndexes = readAheadBuffersCache + .asMap() + .keySet().stream() + .filter(idx -> idx < currentFragIndx) + .collect(Collectors.toSet()); + + if (priorIndexes.size() > 0) { + logger.debug("invalidating fragment(s) '{}' from '{}'", + priorIndexes.stream().map(Objects::toString).collect(Collectors.joining(", ")), s3uri); + + readAheadBuffersCache.invalidateAll(priorIndexes); + } + } + + @Override + public boolean isOpen() { + return open; + } + + @Override + public void close() { + open = false; + readAheadBuffersCache.invalidateAll(); + readAheadBuffersCache.cleanUp(); + } + + /** + * The number of fragments currently in the cache. + * + * @return the size of the cache after any async evictions or reloads have happened. + */ + protected int numberOfCachedFragments() { + readAheadBuffersCache.cleanUp(); + return (int) readAheadBuffersCache.estimatedSize(); + } + + /** + * Obtain a snapshot of the statistics of the internal cache, provides information about hits, misses, requests, evictions etc. + * that are useful for tuning. + * + * @return the statistics of the internal cache. + */ + protected CacheStats cacheStatistics() { + return readAheadBuffersCache.stats(); + } + + private CompletableFuture computeFragmentFuture(int fragmentIndex) { + long readFrom = (long) fragmentIndex * maxFragmentSize; + long readTo = Math.min(readFrom + maxFragmentSize, size) - 1; + String range = "bytes=" + readFrom + "-" + readTo; + logger.debug("byte range for {} is '{}'", key, range); + + return client.getObject( + builder -> builder + .bucket(bucket) + .key(key) + .range(range), + AsyncResponseTransformer.toBytes()) + .thenApply(BytesWrapper::asByteBuffer); + } + + /** + * Compute which buffer a byte should be in + * + * @param byteNumber the number of the byte in the object accessed by this channel + * @return the index of the fragment in which {@code byteNumber} will be found. + */ + protected Integer fragmentIndexForByteNumber(long byteNumber) { + return Math.toIntExact(Math.floorDiv(byteNumber, (long) maxFragmentSize)); + } +} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java new file mode 100644 index 00000000000..5db6a63a86f --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -0,0 +1,236 @@ +package io.deephaven.parquet.table.util; + +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.NonReadableChannelException; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.nio.file.StandardOpenOption; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class S3SeekableByteChannel implements SeekableByteChannel { + + private long position; + private final S3AsyncClient s3Client; + private final String bucket; + private final String key; + private final ReadableByteChannel readDelegate; + + private boolean closed; + private long size = -1L; + + public S3SeekableByteChannel(String s3uri, String bucket, String key, S3AsyncClient s3Client, long startAt, int maxFragmentSize, int maxNumberFragments, Long timeout, TimeUnit timeUnit) throws IOException { + position = startAt; + this.bucket = bucket; + this.key = key; + closed = false; + this.s3Client = s3Client; + readDelegate = new S3ReadAheadByteChannel(s3uri, bucket, key, maxFragmentSize, maxNumberFragments, s3Client, this, timeout, timeUnit); + } + + /** + * Reads a sequence of bytes from this channel into the given buffer. + * + *

Bytes are read starting at this channel's current position, and + * then the position is updated with the number of bytes actually read. + * Otherwise, this method behaves exactly as specified in the {@link + * ReadableByteChannel} interface. + * + * @param dst the destination buffer + * @return the number of bytes read or -1 if no more bytes can be read. + */ + @Override + public int read(ByteBuffer dst) throws IOException { + validateOpen(); + + if (readDelegate == null) { + throw new NonReadableChannelException(); + } + + return readDelegate.read(dst); + } + + /** + * Writes a sequence of bytes to this channel from the given buffer. + * + *

Bytes are written starting at this channel's current position, unless + * the channel is connected to an entity such as a file that is opened with + * the {@link StandardOpenOption#APPEND APPEND} option, in + * which case the position is first advanced to the end. The entity to which + * the channel is connected will grow to accommodate the + * written bytes, and the position updates with the number of bytes + * actually written. Otherwise, this method behaves exactly as specified by + * the {@link WritableByteChannel} interface. + * + * @param src the src of the bytes to write to this channel + */ + @Override + public int write(ByteBuffer src) throws IOException { + throw new UnsupportedOperationException("Don't support writing to S3 yet"); + } + + /** + * Returns this channel's position. + * + * @return This channel's position, + * a non-negative integer counting the number of bytes + * from the beginning of the entity to the current position + */ + @Override + public long position() throws IOException { + validateOpen(); + + synchronized (this) { + return position; + } + } + + /** + * Sets this channel's position. + * + *

Setting the position to a value that is greater than the current size + * is legal but does not change the size of the entity. A later attempt to + * read bytes at such a position will immediately return an end-of-file + * indication. A later attempt to write bytes at such a position will cause + * the entity to grow to accommodate the new bytes; the values of any bytes + * between the previous end-of-file and the newly-written bytes are + * unspecified. + * + *

Setting the channel's position is not recommended when connected to + * an entity, typically a file, that is opened with the {@link + * StandardOpenOption#APPEND APPEND} option. When opened for + * append, the position is first advanced to the end before writing. + * + * @param newPosition The new position, a non-negative integer counting + * the number of bytes from the beginning of the entity + * @return This channel + * @throws ClosedChannelException If this channel is closed + * @throws IllegalArgumentException If the new position is negative + * @throws IOException If some other I/O error occurs + */ + @Override + public SeekableByteChannel position(long newPosition) throws IOException { + if (newPosition < 0) + throw new IllegalArgumentException("newPosition cannot be < 0"); + + if (!isOpen()) { + throw new ClosedChannelException(); + } + + // this is only valid to read channels + if (readDelegate == null) { + throw new NonReadableChannelException(); + } + + synchronized (this) { + position = newPosition; + return this; + } + } + + /** + * Returns the current size of entity to which this channel is connected. + * + * @return The current size, measured in bytes + * @throws IOException If some other I/O error occurs + */ + @Override + public long size() throws IOException { + validateOpen(); + + if (size < 0) { + + long timeOut = 1L; + TimeUnit unit = TimeUnit.MINUTES; + + synchronized (this) { + final HeadObjectResponse headObjectResponse; + try { + headObjectResponse = s3Client.headObject(builder -> builder + .bucket(bucket) + .key(key)).get(timeOut, unit); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } + + this.size = headObjectResponse.contentLength(); + } + } + return this.size; + } + + /** + * Truncates the entity, to which this channel is connected, to the given + * size. + * + *

If the given size is less than the current size then the entity is + * truncated, discarding any bytes beyond the new end. If the given size is + * greater than or equal to the current size then the entity is not modified. + * In either case, if the current position is greater than the given size + * then it is set to that size. + * + *

An implementation of this interface may prohibit truncation when + * connected to an entity, typically a file, opened with the {@link + * StandardOpenOption#APPEND APPEND} option. + * + * @param size The new size, a non-negative byte count + * @return This channel + */ + @Override + public SeekableByteChannel truncate(long size) { + throw new UnsupportedOperationException("Currently not supported"); + } + + /** + * Tells whether this channel is open. + * + * @return {@code true} if, and only if, this channels delegate is open + */ + @Override + public boolean isOpen() { + synchronized (this) { + return !this.closed; + } + } + + /** + * Closes this channel. + * + *

After a channel is closed, any further attempt to invoke I/O + * operations upon it will cause a {@link ClosedChannelException} to be + * thrown. + * + *

If this channel is already closed then invoking this method has no + * effect. + * + *

This method may be invoked at any time. If some other thread has + * already invoked it, however, then another invocation will block until + * the first invocation is complete, after which it will return without + * effect.

+ */ + @Override + public void close() throws IOException { + synchronized (this) { + if (readDelegate != null) { + readDelegate.close(); + } + closed = true; + } + } + + private void validateOpen() throws ClosedChannelException { + if (this.closed) { + throw new ClosedChannelException(); + } + } +} 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 c77fc176ace..5941b6686a8 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 @@ -640,12 +640,12 @@ public void readRefParquetFileLocally() { @Test public void profileReadingFromS3() { - final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setAwsRegionName("us-east-1") .build(); long totalTime = 0; - long NUM_RUNS = 5; + long NUM_RUNS = 10; for (int i = 0; i < NUM_RUNS; i++) { final long start = System.nanoTime(); ParquetTools.readTable("s3://dh-s3-parquet-test1/multiColFile.parquet", readInstructions).select(); @@ -655,6 +655,38 @@ public void profileReadingFromS3() { } System.out.println("Average execution time AWS is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); + + readInstructions = new ParquetInstructions.Builder() + .setAwsRegionName("us-east-2") + .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")); + + totalTime = 0; + for (int i = 0; i < NUM_RUNS; i++) { + final long start = System.nanoTime(); + ParquetTools.readTable( + "s3://aws-public-blockchain/v1.0/btc/transactions/date=2009-01-03/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet", + readInstructions, tableDefinition).head(5).select(); + final long end = System.nanoTime(); + totalTime += end - start; + System.out.println((i + 1) + ". Execution time AWS is " + (end - start) / 1000_000_000.0 + " sec"); + } + System.out.println("Average execution time AWS is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); + NUM_RUNS = 100; totalTime = 0; for (int i = 0; i < NUM_RUNS; i++) { @@ -662,7 +694,7 @@ public void profileReadingFromS3() { ParquetTools.readTable("/Users/shivammalhotra/documents/multiColFile.parquet").select(); final long end = System.nanoTime(); totalTime += end - start; - System.out.println((i + 1) + ". Execution time local is " + (end - start) / 1000_000_000.0 + " sec"); + // System.out.println((i + 1) + ". Execution time local is " + (end - start) / 1000_000_000.0 + " sec"); } System.out.println("Average execution time local is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); } From 6d727230dc5a86f335bdade480f3d34f6f3b93f8 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 18 Dec 2023 15:39:58 +0530 Subject: [PATCH 05/39] Made a number of improvements to s3 nio channel files - Added an async response transfer to save on copies - Moved HEAD request to constructor - Saved on one copy while reading - Using a CRT based client - Merged the two read channels - Updated caffeine version - Added a shared cache between channels - Removed delete old entries from cache logic - Made cache size configurable --- .../parquet/base/ColumnChunkReaderImpl.java | 11 - .../parquet/base/RowGroupReaderImpl.java | 4 - extensions/parquet/table/build.gradle | 4 +- .../util/AsyncAWSResponseTransformer.java | 97 ++++++++ .../util/S3BackedSeekableChannelProvider.java | 66 ++++- .../table/util/S3ReadAheadByteChannel.java | 226 ------------------ .../table/util/S3SeekableByteChannel.java | 188 +++++++++++---- .../table/ParquetTableReadWriteTest.java | 33 +++ 8 files changed, 330 insertions(+), 299 deletions(-) create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/AsyncAWSResponseTransformer.java delete mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3ReadAheadByteChannel.java 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 0bf71390501..8d8d0668f97 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 @@ -229,17 +229,6 @@ public boolean hasNext() { return remainingValues > 0; } - // TODO Move to a separate file - final class PositionedBufferedInputStream extends BufferedInputStream { - PositionedBufferedInputStream(final ReadableByteChannel readChannel, final int size) { - super(Channels.newInputStream(readChannel), size); - } - - long position() throws IOException { - return this.pos; - } - } - @Override public ColumnPageReader next() { if (!hasNext()) { 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 9b582888ffa..90d4e20add3 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 @@ -78,10 +78,6 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { if (columnChunk.isSetOffset_index_offset()) { try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(rootPath)) { readChannel.position(columnChunk.getOffset_index_offset()); - // TODO Think if we need to reduce the buffer size. - // We read BUFFER_SIZE (=65536) number of bytes from the channel, which leads to a big read request to - // aws, even if the offset index is much smaller. Same thing happens for non aws parquet files too but - // reads are less expensive there. offsetIndex = ParquetMetadataConverter.fromParquetOffsetIndex(Util.readOffsetIndex( new BufferedInputStream(Channels.newInputStream(readChannel), BUFFER_SIZE))); } catch (IOException e) { diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index ba206c926cc..84a23b93c59 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -29,9 +29,9 @@ dependencies { implementation depCommonsLang3 implementation platform('software.amazon.awssdk:bom:2.21.43') implementation 'software.amazon.awssdk:s3' - implementation 'software.amazon.awssdk:apache-client:2.21.43' + implementation 'software.amazon.awssdk:aws-crt-client' - implementation('com.github.ben-manes.caffeine:caffeine:2.9.3') + implementation("com.github.ben-manes.caffeine:caffeine:3.1.8") Classpaths.inheritCommonsText(project, 'implementation') diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/AsyncAWSResponseTransformer.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/AsyncAWSResponseTransformer.java new file mode 100644 index 00000000000..7fb26be8c0e --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/AsyncAWSResponseTransformer.java @@ -0,0 +1,97 @@ +package io.deephaven.parquet.table.util; + +import io.deephaven.base.verify.Assert; +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; + +/** + * An {@link AsyncResponseTransformer} that transforms a response into a {@link ByteBuffer}. + * + * @param POJO response type. + */ +public final class AsyncAWSResponseTransformer implements AsyncResponseTransformer { + + private volatile CompletableFuture cf; + private ResponseT response; + private final ByteBuffer byteBuffer; + + AsyncAWSResponseTransformer(final int bufferSize) { + // TODO Can be improved with a buffer pool + byteBuffer = ByteBuffer.allocate(bufferSize); + } + + @Override + public CompletableFuture prepare() { + cf = new CompletableFuture<>(); + return cf; + } + + @Override + public void onResponse(ResponseT response) { + this.response = response; + } + + /** + * @return the unmarshalled response object from the service. + */ + public ResponseT response() { + return response; + } + + @Override + public void onStream(SdkPublisher publisher) { + publisher.subscribe(new ByteBuferSubscriber(cf, byteBuffer)); + } + + @Override + public void exceptionOccurred(Throwable throwable) { + cf.completeExceptionally(throwable); + } + + final static class ByteBuferSubscriber implements Subscriber { + private final CompletableFuture resultFuture; + + private Subscription subscription; + + private final ByteBuffer byteBuffer; + + ByteBuferSubscriber(CompletableFuture resultFuture, ByteBuffer byteBuffer) { + this.resultFuture = resultFuture; + this.byteBuffer = byteBuffer; + } + + @Override + public void onSubscribe(final Subscription s) { + if (subscription != null) { + 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() <= byteBuffer.remaining(), + "responseBytes.remaining() <= byteBuffer.remaining()"); + byteBuffer.put(responseBytes); + subscription.request(1); + } + + @Override + public void onError(final Throwable throwable) { + resultFuture.completeExceptionally(throwable); + } + + @Override + public void onComplete() { + resultFuture.complete(byteBuffer); + } + } +} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java index a789e5bdb8a..5d2a9fb46ae 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java @@ -1,29 +1,43 @@ package io.deephaven.parquet.table.util; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; import io.deephaven.UncheckedDeephavenException; import io.deephaven.configuration.Configuration; import io.deephaven.parquet.base.util.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.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URI; import java.net.URISyntaxException; +import java.nio.ByteBuffer; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public final class S3BackedSeekableChannelProvider implements SeekableChannelsProvider { private final S3AsyncClient s3AsyncClient; private final URI uri; private final String s3uri, bucket, key; + private final long size; - public static final int MAX_FRAGMENT_SIZE = - Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-size", 512 * 1024); // 512 KB - private static final int MAX_FRAGMENT_NUMBER = - Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-number", 2); + private static final int MAX_CACHE_SIZE = + Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-cache-size", 10); + private static final int MAX_AWS_CONCURRENT_REQUESTS = + Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-concurrency", 20); + + private final Cache> readAheadBuffersCache; public S3BackedSeekableChannelProvider(final String awsRegionName, final String uriStr) throws IOException { if (awsRegionName == null || awsRegionName.isEmpty()) { @@ -32,23 +46,55 @@ public S3BackedSeekableChannelProvider(final String awsRegionName, final String if (uriStr == null || uriStr.isEmpty()) { throw new IllegalArgumentException("uri cannot be null or empty"); } + if (MAX_AWS_CONCURRENT_REQUESTS < 1) { + throw new IllegalArgumentException("maxConcurrency must be >= 1"); + } + try { uri = new URI(uriStr); } catch (final URISyntaxException e) { throw new UncheckedDeephavenException("Failed to parse URI " + uriStr, e); } - this.s3uri = uriStr; - this.bucket = uri.getHost(); - this.key = uri.getPath().substring(1); + + final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() + .maxConcurrency(MAX_AWS_CONCURRENT_REQUESTS) + .connectionTimeout(Duration.ofSeconds(5)) + .build(); s3AsyncClient = S3AsyncClient.builder() .region(Region.of(awsRegionName)) + .httpClient(asyncHttpClient) .build(); + + if (MAX_CACHE_SIZE < 1) + throw new IllegalArgumentException("maxCacheSize must be >= 1"); + this.readAheadBuffersCache = Caffeine.newBuilder().maximumSize(MAX_CACHE_SIZE).recordStats().build(); + + this.s3uri = uriStr; + this.bucket = uri.getHost(); + this.key = uri.getPath().substring(1); + // Send HEAD request to S3 to get the size of the file + { + final long timeOut = 1L; + final TimeUnit unit = TimeUnit.MINUTES; + + final HeadObjectResponse headObjectResponse; + try { + headObjectResponse = s3AsyncClient.headObject(builder -> builder + .bucket(bucket) + .key(key)).get(timeOut, unit); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } catch (final ExecutionException | TimeoutException e) { + throw new IOException(e); + } + this.size = headObjectResponse.contentLength(); + } } @Override public SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { - return new S3SeekableByteChannel(s3uri, bucket, key, s3AsyncClient, 0, MAX_FRAGMENT_SIZE, MAX_FRAGMENT_NUMBER, - null, null); + return new S3SeekableByteChannel(s3uri, bucket, key, s3AsyncClient, 0, size, null, null, readAheadBuffersCache); } @Override @@ -59,5 +105,7 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boole public void close() throws IOException { s3AsyncClient.close(); + readAheadBuffersCache.invalidateAll(); + readAheadBuffersCache.cleanUp(); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3ReadAheadByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3ReadAheadByteChannel.java deleted file mode 100644 index e00ecad21fc..00000000000 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3ReadAheadByteChannel.java +++ /dev/null @@ -1,226 +0,0 @@ -package io.deephaven.parquet.table.util; - -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; -import com.github.benmanes.caffeine.cache.stats.CacheStats; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.core.BytesWrapper; -import software.amazon.awssdk.core.async.AsyncResponseTransformer; -import software.amazon.awssdk.services.s3.S3AsyncClient; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; - -/** - * A {@code ReadableByteChannel} delegate for an {@code S3SeekableByteChannel} that maintains internal read ahead - * buffers to reduce the amount of high latency - * requests to S3. If the bytes required by a read are already in the buffer, they will be fulfilled from the buffer - * rather than making another S3 request. - *

As reads are made this object will update the current read position of the delegating {@code S3SeekableByteChannel}

- */ -public class S3ReadAheadByteChannel implements ReadableByteChannel { - - private final S3AsyncClient client; - private final String s3uri, bucket, key; - private final S3SeekableByteChannel delegator; - private final int maxFragmentSize; - private final int maxNumberFragments; - private final int numFragmentsInObject; - private final long size; - private final Long timeout; - private final TimeUnit timeUnit; - private boolean open; - private final Cache> readAheadBuffersCache; - - private final Logger logger = LoggerFactory.getLogger(this.getClass()); - - - /** - * Construct a new {@code S3ReadAheadByteChannel} which is used by its parent delegator to perform read operations. - * The channel is backed by a cache that holds the buffered fragments of the object identified - * by the {@code path}. - * - * @param maxFragmentSize the maximum amount of bytes in a read ahead fragment. Must be {@code >= 1}. - * @param maxNumberFragments the maximum number of read ahead fragments to hold. Must be {@code >= 2}. - * @param client the client used to read from the {@code path} - * @param delegator the {@code S3SeekableByteChannel} that delegates reading to this object. - * @param timeout the amount of time to wait for a read ahead fragment to be available. - * @param timeUnit the {@code TimeUnit} for the {@code timeout}. - * @throws IOException if a problem occurs initializing the cached fragments - */ - public S3ReadAheadByteChannel(String s3uri, String bucket, String key, int maxFragmentSize, int maxNumberFragments, S3AsyncClient client, S3SeekableByteChannel delegator, Long timeout, TimeUnit timeUnit) throws IOException { - Objects.requireNonNull(client); - Objects.requireNonNull(delegator); - if (maxFragmentSize < 1) - throw new IllegalArgumentException("maxFragmentSize must be >= 1"); - if (maxNumberFragments < 2) - throw new IllegalArgumentException("maxNumberFragments must be >= 2"); - - logger.debug("max read ahead fragments '{}' with size '{}' bytes", maxNumberFragments, maxFragmentSize); - this.client = client; - this.s3uri = s3uri; - this.bucket = bucket; - this.key = key; - this.delegator = delegator; - this.size = delegator.size(); - this.maxFragmentSize = maxFragmentSize; - this.numFragmentsInObject = (int) Math.ceil((float) size / (float) maxFragmentSize); - this.readAheadBuffersCache = Caffeine.newBuilder().maximumSize(maxNumberFragments).recordStats().build(); - this.maxNumberFragments = maxNumberFragments; - this.open = true; - this.timeout = timeout != null ? timeout : 5L; - this.timeUnit = timeUnit != null ? timeUnit : TimeUnit.MINUTES; - } - - @Override - public int read(ByteBuffer dst) throws IOException { - Objects.requireNonNull(dst); - - long channelPosition = delegator.position(); - logger.debug("delegator position: {}", channelPosition); - - // if the position of the delegator is at the end (>= size) return -1. we're finished reading. - if (channelPosition >= size) return -1; - - //figure out the index of the fragment the bytes would start in - Integer fragmentIndex = fragmentIndexForByteNumber(channelPosition); - logger.debug("fragment index: {}", fragmentIndex); - - int fragmentOffset = (int) (channelPosition - (fragmentIndex.longValue() * maxFragmentSize)); - logger.debug("fragment {} offset: {}", fragmentIndex, fragmentOffset); - - try { - final ByteBuffer fragment = Objects.requireNonNull(readAheadBuffersCache.get(fragmentIndex, this::computeFragmentFuture)) - .get(timeout, timeUnit) - .asReadOnlyBuffer(); - - fragment.position(fragmentOffset); - logger.debug("fragment remaining: {}", fragment.remaining()); - logger.debug("dst remaining: {}", dst.remaining()); - - //put the bytes from fragment from the offset upto the min of fragment remaining or dst remaining - int limit = Math.min(fragment.remaining(), dst.remaining()); - logger.debug("byte limit: {}", limit); - - byte[] copiedBytes = new byte[limit]; - fragment.get(copiedBytes, 0, limit); - dst.put(copiedBytes); - - if (fragment.position() >= fragment.limit() / 2) { - - // clear any fragments in cache that are lower index than this one // <-- This can be an issue for us - clearPriorFragments(fragmentIndex); - - // until available cache slots are filled or number of fragments in file - int maxFragmentsToLoad = Math.min(maxNumberFragments - 1, numFragmentsInObject - fragmentIndex - 1); - - for (int i = 0; i < maxFragmentsToLoad; i++) { - final int idxToLoad = i + fragmentIndex + 1; - - // add the index if it's not already there - if (readAheadBuffersCache.asMap().containsKey(idxToLoad)) - continue; - - logger.debug("initiate pre-loading fragment with index '{}' from '{}'", idxToLoad, s3uri); - readAheadBuffersCache.put(idxToLoad, computeFragmentFuture(idxToLoad)); - } - } - - delegator.position(channelPosition + copiedBytes.length); - return copiedBytes.length; - - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } catch (ExecutionException e) { - // the async execution completed exceptionally. - // not currently obvious when this will happen or if we can recover - logger.error("an exception occurred while reading bytes from {} that was not recovered by the S3 Client RetryCondition(s)", s3uri); - throw new IOException(e); - } catch (TimeoutException e) { - throw new RuntimeException(e); - } - } - - private void clearPriorFragments(int currentFragIndx) { - final Set<@NonNull Integer> priorIndexes = readAheadBuffersCache - .asMap() - .keySet().stream() - .filter(idx -> idx < currentFragIndx) - .collect(Collectors.toSet()); - - if (priorIndexes.size() > 0) { - logger.debug("invalidating fragment(s) '{}' from '{}'", - priorIndexes.stream().map(Objects::toString).collect(Collectors.joining(", ")), s3uri); - - readAheadBuffersCache.invalidateAll(priorIndexes); - } - } - - @Override - public boolean isOpen() { - return open; - } - - @Override - public void close() { - open = false; - readAheadBuffersCache.invalidateAll(); - readAheadBuffersCache.cleanUp(); - } - - /** - * The number of fragments currently in the cache. - * - * @return the size of the cache after any async evictions or reloads have happened. - */ - protected int numberOfCachedFragments() { - readAheadBuffersCache.cleanUp(); - return (int) readAheadBuffersCache.estimatedSize(); - } - - /** - * Obtain a snapshot of the statistics of the internal cache, provides information about hits, misses, requests, evictions etc. - * that are useful for tuning. - * - * @return the statistics of the internal cache. - */ - protected CacheStats cacheStatistics() { - return readAheadBuffersCache.stats(); - } - - private CompletableFuture computeFragmentFuture(int fragmentIndex) { - long readFrom = (long) fragmentIndex * maxFragmentSize; - long readTo = Math.min(readFrom + maxFragmentSize, size) - 1; - String range = "bytes=" + readFrom + "-" + readTo; - logger.debug("byte range for {} is '{}'", key, range); - - return client.getObject( - builder -> builder - .bucket(bucket) - .key(key) - .range(range), - AsyncResponseTransformer.toBytes()) - .thenApply(BytesWrapper::asByteBuffer); - } - - /** - * Compute which buffer a byte should be in - * - * @param byteNumber the number of the byte in the object accessed by this channel - * @return the index of the fragment in which {@code byteNumber} will be found. - */ - protected Integer fragmentIndexForByteNumber(long byteNumber) { - return Math.toIntExact(Math.floorDiv(byteNumber, (long) maxFragmentSize)); - } -} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java index 5db6a63a86f..cdeef998496 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -1,38 +1,70 @@ package io.deephaven.parquet.table.util; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.stats.CacheStats; +import io.deephaven.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.S3AsyncClient; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; -import java.nio.channels.NonReadableChannelException; import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; import java.nio.channels.WritableByteChannel; import java.nio.file.StandardOpenOption; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public class S3SeekableByteChannel implements SeekableByteChannel { - +public final class S3SeekableByteChannel implements SeekableByteChannel { private long position; private final S3AsyncClient s3Client; - private final String bucket; - private final String key; - private final ReadableByteChannel readDelegate; + private volatile boolean closed; + private final String s3uri, bucket, key; + private final int maxFragmentSize; + private final int maxNumberFragments; + private final int numFragmentsInObject; + private final long size; + private final Long timeout; + private final TimeUnit timeUnit; + private final Cache> readAheadBuffersCache; + + private final Logger logger = LoggerFactory.getLogger(this.getClass()); + + private static final int MAX_FRAGMENT_SIZE = + Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-size", 512 * 1024); // 512 KB + private static final int MAX_FRAGMENT_NUMBER = + Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-number", 2); - private boolean closed; - private long size = -1L; - public S3SeekableByteChannel(String s3uri, String bucket, String key, S3AsyncClient s3Client, long startAt, int maxFragmentSize, int maxNumberFragments, Long timeout, TimeUnit timeUnit) throws IOException { - position = startAt; + S3SeekableByteChannel(String s3uri, String bucket, String key, S3AsyncClient s3Client, long startAt, long size, + Long timeout, TimeUnit timeUnit, final Cache> readAheadBuffersCache) throws IOException { + Objects.requireNonNull(s3Client); + if (MAX_FRAGMENT_SIZE < 1) + throw new IllegalArgumentException("maxFragmentSize must be >= 1"); + if (MAX_FRAGMENT_NUMBER < 1) + throw new IllegalArgumentException("maxNumberFragments must be >= 1"); + if (size < 1) + throw new IllegalArgumentException("size must be >= 1"); + + this.position = startAt; this.bucket = bucket; this.key = key; - closed = false; + this.closed = false; this.s3Client = s3Client; - readDelegate = new S3ReadAheadByteChannel(s3uri, bucket, key, maxFragmentSize, maxNumberFragments, s3Client, this, timeout, timeUnit); + + this.s3uri = s3uri; + this.maxFragmentSize = MAX_FRAGMENT_SIZE; + this.readAheadBuffersCache = readAheadBuffersCache; + this.maxNumberFragments = MAX_FRAGMENT_NUMBER; + this.timeout = timeout != null ? timeout : 5L; + this.timeUnit = timeUnit != null ? timeUnit : TimeUnit.MINUTES; + this.size = size; + this.numFragmentsInObject = (int) Math.ceil((double) size / maxFragmentSize); } /** @@ -47,14 +79,86 @@ public S3SeekableByteChannel(String s3uri, String bucket, String key, S3AsyncCli * @return the number of bytes read or -1 if no more bytes can be read. */ @Override - public int read(ByteBuffer dst) throws IOException { + public int read(final ByteBuffer dst) throws IOException { validateOpen(); - if (readDelegate == null) { - throw new NonReadableChannelException(); + Objects.requireNonNull(dst); + + final long channelPosition = this.position(); + + // if the position of the delegator is at the end (>= size) return -1. we're finished reading. + if (channelPosition >= size) { + return -1; + } + + //figure out the index of the fragment the bytes would start in + final int fragmentIndex = fragmentIndexForByteNumber(channelPosition); + final int fragmentOffset = (int) (channelPosition - ((long) fragmentIndex * maxFragmentSize)); + try { + final ByteBuffer fragment = Objects.requireNonNull(readAheadBuffersCache.get(fragmentIndex, this::computeFragmentFuture)) + .get(timeout, timeUnit) + .asReadOnlyBuffer(); + + //put the bytes from fragment from the offset upto the min of fragment remaining or dst remaining + fragment.position(fragmentOffset); + final int limit = Math.min(fragment.remaining(), dst.remaining()); + fragment.limit(fragment.position() + limit); + dst.put(fragment); + + if (fragment.position() >= fragment.limit() / 2) { + + // until available cache slots are filled or number of fragments in file + final int maxFragmentsToLoad = Math.min(maxNumberFragments - 1, numFragmentsInObject - fragmentIndex - 1); + + for (int i = 0; i < maxFragmentsToLoad; i++) { + final int idxToLoad = i + fragmentIndex + 1; + + // add the index if it's not already there + if (readAheadBuffersCache.asMap().containsKey(idxToLoad)) + continue; + + readAheadBuffersCache.put(idxToLoad, computeFragmentFuture(idxToLoad)); + } + } + + position(channelPosition + limit); + return limit; + + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } catch (final ExecutionException e) { + // the async execution completed exceptionally. + // not currently obvious when this will happen or if we can recover + logger.error("an exception occurred while reading bytes from {} that was not recovered by the S3 Client RetryCondition(s)", s3uri); + throw new IOException(e); + } catch (final TimeoutException e) { + throw new RuntimeException(e); } + } + + /** + * Compute which buffer a byte should be in + * + * @param byteNumber the number of the byte in the object accessed by this channel + * @return the index of the fragment in which {@code byteNumber} will be found. + */ + private int fragmentIndexForByteNumber(final long byteNumber) { + return Math.toIntExact(Math.floorDiv(byteNumber, (long) maxFragmentSize)); + } - return readDelegate.read(dst); + private CompletableFuture computeFragmentFuture(final int fragmentIndex) { + final long readFrom = (long) fragmentIndex * maxFragmentSize; + final long readTo = Math.min(readFrom + maxFragmentSize, size) - 1; + final String range = "bytes=" + readFrom + "-" + readTo; + logger.debug("byte range for {} is '{}'", key, range); + + return s3Client.getObject( + builder -> builder + .bucket(bucket) + .key(key) + .range(range), + new AsyncAWSResponseTransformer<>(maxFragmentSize)); } /** @@ -124,11 +228,6 @@ public SeekableByteChannel position(long newPosition) throws IOException { throw new ClosedChannelException(); } - // this is only valid to read channels - if (readDelegate == null) { - throw new NonReadableChannelException(); - } - synchronized (this) { position = newPosition; return this; @@ -144,28 +243,6 @@ public SeekableByteChannel position(long newPosition) throws IOException { @Override public long size() throws IOException { validateOpen(); - - if (size < 0) { - - long timeOut = 1L; - TimeUnit unit = TimeUnit.MINUTES; - - synchronized (this) { - final HeadObjectResponse headObjectResponse; - try { - headObjectResponse = s3Client.headObject(builder -> builder - .bucket(bucket) - .key(key)).get(timeOut, unit); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } catch (ExecutionException | TimeoutException e) { - throw new IOException(e); - } - - this.size = headObjectResponse.contentLength(); - } - } return this.size; } @@ -203,6 +280,26 @@ public boolean isOpen() { } } + /** + * The number of fragments currently in the cache. + * + * @return the size of the cache after any async evictions or reloads have happened. + */ + int numberOfCachedFragments() { + readAheadBuffersCache.cleanUp(); + return (int) readAheadBuffersCache.estimatedSize(); + } + + /** + * Obtain a snapshot of the statistics of the internal cache, provides information about hits, misses, requests, evictions etc. + * that are useful for tuning. + * + * @return the statistics of the internal cache. + */ + CacheStats cacheStatistics() { + return readAheadBuffersCache.stats(); + } + /** * Closes this channel. * @@ -221,9 +318,6 @@ public boolean isOpen() { @Override public void close() throws IOException { synchronized (this) { - if (readDelegate != null) { - readDelegate.close(); - } closed = true; } } 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 5941b6686a8..57805742e18 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 @@ -569,6 +569,39 @@ public void testArrayColumns() { && !firstColumnMetadata.contains("RLE_DICTIONARY")); } + @Test + public void readLongParquetFileFromS3Test() { + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setAwsRegionName("us-east-2") + .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")); + + final Table fromAws1 = ParquetTools.readTable( + "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(); + final Table fromDisk1 = ParquetTools.readSingleFileTable( + new File( + "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), + ParquetTools.SNAPPY, + tableDefinition).select(); + assertTableEquals(fromAws1, fromDisk1); + } + @Test public void readRefParquetFileFromS3Test() { final ParquetInstructions readInstructions = new ParquetInstructions.Builder() From cca5051d553eeaab43dc29dc826dde71f2add0f5 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 26 Dec 2023 22:18:32 +0530 Subject: [PATCH 06/39] Moved to a context based cache Removed Caffeine --- .../engine/table/impl/SourceTable.java | 1 + .../impl/sources/RedirectedColumnSource.java | 2 +- .../sources/regioned/RegionContextHolder.java | 1 - .../parquet/base/ColumnChunkReaderImpl.java | 6 +- .../parquet/base/ColumnPageReaderImpl.java | 12 +- .../parquet/base/ParquetFileReader.java | 3 +- .../parquet/base/RowGroupReaderImpl.java | 3 +- .../base/util/CachedChannelProvider.java | 26 ++- .../base/util/LocalFSChannelProvider.java | 9 +- .../base/util/SeekableChannelsProvider.java | 30 ++- .../base/util/CachedChannelProviderTest.java | 23 ++- extensions/parquet/table/build.gradle | 2 - .../deephaven/parquet/table/ParquetTools.java | 4 +- ...> ByteBufferAsyncResponseTransformer.java} | 4 +- .../table/util/S3SeekableByteChannel.java | 191 +++++++++++------- ...er.java => S3SeekableChannelProvider.java} | 52 +++-- .../util/TrackedSeekableChannelsProvider.java | 10 +- 17 files changed, 259 insertions(+), 120 deletions(-) rename extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/{AsyncAWSResponseTransformer.java => ByteBufferAsyncResponseTransformer.java} (93%) rename extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/{S3BackedSeekableChannelProvider.java => S3SeekableChannelProvider.java} (69%) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index f2870f10631..679a6e63f59 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -116,6 +116,7 @@ private void initialize() { @TestUseOnly public final TableLocationProvider tableLocationProvider() { return locationProvider; + // Ignore } /** 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/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java index 50502bd1676..61843b127d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java @@ -14,7 +14,6 @@ public class RegionContextHolder implements ChunkSource.FillContext { private Context innerContext; public RegionContextHolder(final int chunkCapacity, @Nullable final SharedContext sharedContext) { - this.chunkCapacity = chunkCapacity; this.sharedContext = sharedContext; } 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 8d8d0668f97..65c4cddae1b 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 @@ -171,7 +171,8 @@ private Dictionary getDictionary() { } else { return NULL_DICTIONARY; } - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(getFilePath())) { + try (final SeekableByteChannel readChannel = + channelsProvider.getReadChannel(channelsProvider.makeContext(), getFilePath())) { readChannel.position(dictionaryPageOffset); return readDictionary(readChannel); } catch (IOException e) { @@ -235,7 +236,8 @@ public ColumnPageReader next() { 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(channelsProvider.makeContext(), getFilePath())) { final long headerOffset = currentOffset; readChannel.position(currentOffset); // deliberately not closing this stream 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..ae04fe87bcd 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 @@ -105,14 +105,16 @@ public class ColumnPageReaderImpl implements ColumnPageReader { @Override public Object materialize(Object nullValue) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + try (final SeekableByteChannel readChannel = + channelsProvider.getReadChannel(channelsProvider.makeContext(), filePath)) { ensurePageHeader(readChannel); return readDataPage(nullValue, readChannel); } } public int readRowCount() throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + try (final SeekableByteChannel readChannel = + channelsProvider.getReadChannel(channelsProvider.makeContext(), filePath)) { ensurePageHeader(readChannel); return readRowCountFromDataPage(readChannel); } @@ -121,7 +123,8 @@ public int readRowCount() throws IOException { @Override public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + try (final SeekableByteChannel readChannel = + channelsProvider.getReadChannel(channelsProvider.makeContext(), filePath)) { ensurePageHeader(readChannel); return readKeyFromDataPage(keyDest, nullPlaceholder, readChannel); } @@ -619,7 +622,8 @@ public int numValues() throws IOException { if (numValues >= 0) { return numValues; } - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + try (final SeekableByteChannel readChannel = + channelsProvider.getReadChannel(channelsProvider.makeContext(), filePath)) { ensurePageHeader(readChannel); // Above will block till it populates numValues Assert.geqZero(numValues, "numValues"); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java index 4545c52b855..fe6ef46df6d 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 @@ -42,7 +42,8 @@ public ParquetFileReader(final String filePath, final SeekableChannelsProvider c filePath.endsWith(".parquet") ? Paths.get(filePath) : Paths.get(filePath).getParent(); final byte[] footer; - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(filePath)) { + try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); + final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, filePath)) { final long fileLen = readChannel.size(); if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + // footerIndex + MAGIC 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..e08146ae23a 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 @@ -76,7 +76,8 @@ 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(channelsProvider.makeContext(), rootPath)) { readChannel.position(columnChunk.getOffset_index_offset()); offsetIndex = ParquetMetadataConverter.fromParquetOffsetIndex(Util.readOffsetIndex( new BufferedInputStream(Channels.newInputStream(readChannel), BUFFER_SIZE))); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java index b8e7ef70268..529e61b9d3c 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java @@ -52,13 +52,21 @@ public CachedChannelProvider(@NotNull final SeekableChannelsProvider wrappedProv } @Override - public SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { + public ChannelContext makeContext() { + return wrappedProvider.makeContext(); + } + + @Override + public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final Path path) + throws IOException { final String pathKey = path.toAbsolutePath().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 channel = result == null + ? new CachedChannel(wrappedProvider.getReadChannel(context, path), ChannelType.Read, pathKey) : result.position(0); + channel.setContext(context); + return channel; } @Override @@ -128,7 +136,7 @@ private long advanceClock() { /** * {@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 +171,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 +204,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 void setContext(ChannelContext context) { + if (wrappedChannel instanceof ContextHolder) { + ((ContextHolder) wrappedChannel).setContext(context); + } + } } /** diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java index 29322e25489..697e6d6e4e4 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java @@ -14,7 +14,14 @@ public class LocalFSChannelProvider implements SeekableChannelsProvider { @Override - public SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { + public ChannelContext makeContext() { + return ChannelContext.NULL; + } + + @Override + public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final Path path) + throws IOException { + // context is unused here because it is NULL return FileChannel.open(path, StandardOpenOption.READ); } 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 index 2da3f131879..c0b268c8b4f 100644 --- 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 @@ -3,6 +3,8 @@ */ package io.deephaven.parquet.base.util; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.io.IOException; @@ -12,11 +14,33 @@ public interface SeekableChannelsProvider { - default SeekableByteChannel getReadChannel(@NotNull final String path) throws IOException { - return getReadChannel(Paths.get(path)); + interface ChannelContext extends SafeCloseable { + + ChannelContext NULL = new ChannelContext() {}; + + /** + * Release any resources associated with this context. The context should not be used afterward. + */ + default void close() {} + } + + ChannelContext makeContext(); + + interface ContextHolder { + void setContext(ChannelContext context); + + @FinalDefault + default void clearContext() { + setContext(null); + } + } + + default SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final String path) + throws IOException { + return getReadChannel(context, Paths.get(path)); } - SeekableByteChannel getReadChannel(@NotNull Path path) throws IOException; + SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull Path path) throws IOException; default SeekableByteChannel getWriteChannel(@NotNull final String path, final boolean append) throws IOException { return getWriteChannel(Paths.get(path), append); diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java index 29cd046d782..f7d5606a539 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java @@ -31,7 +31,7 @@ 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); } for (int jj = 0; jj < 10; ++jj) { sameFile[jj].close(); @@ -48,8 +48,9 @@ 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); @@ -91,7 +92,7 @@ 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(); @@ -111,14 +112,15 @@ public void testReuse() throws IOException { final CachedChannelProvider cachedChannelProvider = new CachedChannelProvider(wrappedProvider, 50); final SeekableByteChannel[] someResult = new SeekableByteChannel[50]; for (int ci = 0; ci < someResult.length; ++ci) { - someResult[ci] = cachedChannelProvider.getReadChannel("r" + ci); + someResult[ci] = cachedChannelProvider.getReadChannel(wrappedProvider.makeContext(), "r" + ci); } 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)); + Assert.assertSame(someResult[ci], + cachedChannelProvider.getReadChannel(wrappedProvider.makeContext(), "r" + ci)); } for (int ci = 0; ci < someResult.length; ++ci) { someResult[someResult.length - ci - 1].close(); @@ -160,12 +162,17 @@ private class TestChannelProvider implements SeekableChannelsProvider { AtomicInteger count = new AtomicInteger(0); @Override - public SeekableByteChannel getReadChannel(@NotNull String path) { + public ChannelContext makeContext() { + return ChannelContext.NULL; + } + + @Override + public SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull String path) { return new TestMockChannel(count.getAndIncrement(), path); } @Override - public SeekableByteChannel getReadChannel(@NotNull Path path) { + public SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull Path path) { return new TestMockChannel(count.getAndIncrement(), path.toString()); } diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 84a23b93c59..0132f20a257 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -31,8 +31,6 @@ dependencies { implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:aws-crt-client' - implementation("com.github.ben-manes.caffeine:caffeine:3.1.8") - Classpaths.inheritCommonsText(project, 'implementation') Classpaths.inheritJacksonPlatform(project) 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 d406d73c34a..f55b17a0d25 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,7 +14,7 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; -import io.deephaven.parquet.table.util.S3BackedSeekableChannelProvider; +import io.deephaven.parquet.table.util.S3SeekableChannelProvider; import io.deephaven.vector.*; import io.deephaven.stringset.StringSet; import io.deephaven.engine.util.file.TrackedFileHandleFactory; @@ -1082,7 +1082,7 @@ public static ParquetFileReader getParquetFileReaderChecked( final String s3uri = S3_MARKER + absolutePath.substring(index + S3_MARKER.length() - 1); return new ParquetFileReader(absolutePath, new CachedChannelProvider( - new S3BackedSeekableChannelProvider(readInstructions.getAwsRegionName(), s3uri), 1 << 7)); + new S3SeekableChannelProvider(readInstructions.getAwsRegionName(), s3uri), 1 << 7)); } return new ParquetFileReader( parquetFile.getAbsolutePath(), diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/AsyncAWSResponseTransformer.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java similarity index 93% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/AsyncAWSResponseTransformer.java rename to extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java index 7fb26be8c0e..8f56bbc9ab6 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/AsyncAWSResponseTransformer.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java @@ -14,13 +14,13 @@ * * @param POJO response type. */ -public final class AsyncAWSResponseTransformer implements AsyncResponseTransformer { +public final class ByteBufferAsyncResponseTransformer implements AsyncResponseTransformer { private volatile CompletableFuture cf; private ResponseT response; private final ByteBuffer byteBuffer; - AsyncAWSResponseTransformer(final int bufferSize) { + ByteBufferAsyncResponseTransformer(final int bufferSize) { // TODO Can be improved with a buffer pool byteBuffer = ByteBuffer.allocate(bufferSize); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java index cdeef998496..350a609be0c 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -1,12 +1,13 @@ package io.deephaven.parquet.table.util; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.stats.CacheStats; +import io.deephaven.base.verify.Assert; import io.deephaven.configuration.Configuration; +import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.S3AsyncClient; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; @@ -14,40 +15,85 @@ import java.nio.channels.SeekableByteChannel; import java.nio.channels.WritableByteChannel; import java.nio.file.StandardOpenOption; +import java.security.InvalidParameterException; +import java.util.ArrayList; +import java.util.List; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public final class S3SeekableByteChannel implements SeekableByteChannel { +public final class S3SeekableByteChannel implements SeekableByteChannel, SeekableChannelsProvider.ContextHolder { + + /** + * Context object used to store read-ahead buffers for efficiently reading from S3. + */ + static final class ChannelContext implements SeekableChannelsProvider.ChannelContext { + + /** + * Used to store context information for fetching a single fragment from S3 + */ + static class FragmentContext { + private final int fragmentIndex; + private final CompletableFuture future; + + private FragmentContext(final int fragmentIndex, final CompletableFuture future) { + this.fragmentIndex = fragmentIndex; + this.future = future; + } + } + + private final List readAheadBuffers; + + ChannelContext(final int readAheadCount, final int maxCacheSize) { + if (maxCacheSize < 1 + readAheadCount) { + throw new InvalidParameterException("maxCacheSize must be >= 1 + readAheadCount"); + } + readAheadBuffers = new ArrayList<>(maxCacheSize); + for (int i = 0; i < maxCacheSize; i++) { + readAheadBuffers.add(null); + } + } + + private int getIndex(final int fragmentIndex) { + return fragmentIndex % readAheadBuffers.size(); + } + + void setFragmentContext(final int fragmentIndex, final CompletableFuture future) { + readAheadBuffers.set(getIndex(fragmentIndex), new FragmentContext(fragmentIndex, future)); + } + + FragmentContext getFragmentContext(final int fragmentIndex) { + return readAheadBuffers.get(getIndex(fragmentIndex)); + } + } + private long position; private final S3AsyncClient s3Client; private volatile boolean closed; private final String s3uri, bucket, key; private final int maxFragmentSize; - private final int maxNumberFragments; +// private final int maxNumberFragments; private final int numFragmentsInObject; private final long size; private final Long timeout; private final TimeUnit timeUnit; - private final Cache> readAheadBuffersCache; + private ChannelContext context; private final Logger logger = LoggerFactory.getLogger(this.getClass()); + static final int READ_AHEAD_COUNT = + Configuration.getInstance().getIntegerWithDefault("s3.read-ahead-count", 1); + static final int MAX_CACHE_SIZE = + Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-cache-size", 50); private static final int MAX_FRAGMENT_SIZE = Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-size", 512 * 1024); // 512 KB - private static final int MAX_FRAGMENT_NUMBER = - Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-number", 2); - - S3SeekableByteChannel(String s3uri, String bucket, String key, S3AsyncClient s3Client, long startAt, long size, - Long timeout, TimeUnit timeUnit, final Cache> readAheadBuffersCache) throws IOException { + S3SeekableByteChannel(SeekableChannelsProvider.ChannelContext context, String s3uri, String bucket, String key, S3AsyncClient s3Client, long startAt, long size) { Objects.requireNonNull(s3Client); if (MAX_FRAGMENT_SIZE < 1) throw new IllegalArgumentException("maxFragmentSize must be >= 1"); - if (MAX_FRAGMENT_NUMBER < 1) - throw new IllegalArgumentException("maxNumberFragments must be >= 1"); if (size < 1) throw new IllegalArgumentException("size must be >= 1"); @@ -59,12 +105,18 @@ public final class S3SeekableByteChannel implements SeekableByteChannel { this.s3uri = s3uri; this.maxFragmentSize = MAX_FRAGMENT_SIZE; - this.readAheadBuffersCache = readAheadBuffersCache; - this.maxNumberFragments = MAX_FRAGMENT_NUMBER; - this.timeout = timeout != null ? timeout : 5L; - this.timeUnit = timeUnit != null ? timeUnit : TimeUnit.MINUTES; + this.timeout = 5L; + this.timeUnit = TimeUnit.MINUTES; this.size = size; this.numFragmentsInObject = (int) Math.ceil((double) size / maxFragmentSize); + this.context = (ChannelContext) context; + } + + @Override + public void setContext(@Nullable SeekableChannelsProvider.ChannelContext context) { + // null context is allowed for clearing the context + Assert.assertion(context == null || context instanceof ChannelContext, "context == null || context instanceof ChannelContext"); + this.context = (ChannelContext) context; } /** @@ -91,39 +143,23 @@ public int read(final ByteBuffer dst) throws IOException { return -1; } - //figure out the index of the fragment the bytes would start in - final int fragmentIndex = fragmentIndexForByteNumber(channelPosition); - final int fragmentOffset = (int) (channelPosition - ((long) fragmentIndex * maxFragmentSize)); + // Figure out the index of the fragment the bytes would start in + final int currFragmentIndex = fragmentIndexForByteNumber(channelPosition); + final int fragmentOffset = (int) (channelPosition - ((long) currFragmentIndex * maxFragmentSize)); + Assert.neqNull(context, "context"); + + // Blocking fetch the current fragment if it's not already in the cache + final ChannelContext.FragmentContext fragmentContext = context.getFragmentContext(currFragmentIndex); + final CompletableFuture fetchCurrFragment; + if (fragmentContext != null && fragmentContext.fragmentIndex == currFragmentIndex) { + fetchCurrFragment = fragmentContext.future; + } else { + fetchCurrFragment = computeFragmentFuture(currFragmentIndex); + context.setFragmentContext(currFragmentIndex, fetchCurrFragment); + } + final ByteBuffer currentFragment; try { - final ByteBuffer fragment = Objects.requireNonNull(readAheadBuffersCache.get(fragmentIndex, this::computeFragmentFuture)) - .get(timeout, timeUnit) - .asReadOnlyBuffer(); - - //put the bytes from fragment from the offset upto the min of fragment remaining or dst remaining - fragment.position(fragmentOffset); - final int limit = Math.min(fragment.remaining(), dst.remaining()); - fragment.limit(fragment.position() + limit); - dst.put(fragment); - - if (fragment.position() >= fragment.limit() / 2) { - - // until available cache slots are filled or number of fragments in file - final int maxFragmentsToLoad = Math.min(maxNumberFragments - 1, numFragmentsInObject - fragmentIndex - 1); - - for (int i = 0; i < maxFragmentsToLoad; i++) { - final int idxToLoad = i + fragmentIndex + 1; - - // add the index if it's not already there - if (readAheadBuffersCache.asMap().containsKey(idxToLoad)) - continue; - - readAheadBuffersCache.put(idxToLoad, computeFragmentFuture(idxToLoad)); - } - } - - position(channelPosition + limit); - return limit; - + currentFragment = fetchCurrFragment.get(timeout, timeUnit).asReadOnlyBuffer(); } catch (final InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); @@ -135,6 +171,25 @@ public int read(final ByteBuffer dst) throws IOException { } catch (final TimeoutException e) { throw new RuntimeException(e); } + + // Put the bytes from fragment from the offset up to the min of fragment remaining or dst remaining + currentFragment.position(fragmentOffset); + final int limit = Math.min(currentFragment.remaining(), dst.remaining()); + currentFragment.limit(currentFragment.position() + limit); + dst.put(currentFragment); + + // Send requests for read-ahead buffers + final int numFragmentsToLoad = Math.min(READ_AHEAD_COUNT, numFragmentsInObject - currFragmentIndex - 1); + for (int i = 0; i < numFragmentsToLoad; i++) { + final int readAheadFragmentIndex = i + currFragmentIndex + 1; + final ChannelContext.FragmentContext readAheadFragmentContext = context.getFragmentContext(readAheadFragmentIndex); + if (readAheadFragmentContext == null || readAheadFragmentContext.fragmentIndex != readAheadFragmentIndex) { + context.setFragmentContext(readAheadFragmentIndex, computeFragmentFuture(readAheadFragmentIndex)); + } + } + + position(channelPosition + limit); + return limit; } /** @@ -158,7 +213,7 @@ private CompletableFuture computeFragmentFuture(final int fragmentIn .bucket(bucket) .key(key) .range(range), - new AsyncAWSResponseTransformer<>(maxFragmentSize)); + new ByteBufferAsyncResponseTransformer<>(maxFragmentSize)); } /** @@ -280,25 +335,25 @@ public boolean isOpen() { } } - /** - * The number of fragments currently in the cache. - * - * @return the size of the cache after any async evictions or reloads have happened. - */ - int numberOfCachedFragments() { - readAheadBuffersCache.cleanUp(); - return (int) readAheadBuffersCache.estimatedSize(); - } - - /** - * Obtain a snapshot of the statistics of the internal cache, provides information about hits, misses, requests, evictions etc. - * that are useful for tuning. - * - * @return the statistics of the internal cache. - */ - CacheStats cacheStatistics() { - return readAheadBuffersCache.stats(); - } +// /** +// * The number of fragments currently in the cache. +// * +// * @return the size of the cache after any async evictions or reloads have happened. +// */ +// int numberOfCachedFragments() { +// readAheadBuffersCache.cleanUp(); +// return (int) readAheadBuffersCache.estimatedSize(); +// } + +// /** +// * Obtain a snapshot of the statistics of the internal cache, provides information about hits, misses, requests, evictions etc. +// * that are useful for tuning. +// * +// * @return the statistics of the internal cache. +// */ +// CacheStats cacheStatistics() { +// return readAheadBuffersCache.stats(); +// } /** * Closes this channel. diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java similarity index 69% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java rename to extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java index 5d2a9fb46ae..f21bcc5a618 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3BackedSeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java @@ -1,7 +1,5 @@ package io.deephaven.parquet.table.util; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; import io.deephaven.UncheckedDeephavenException; import io.deephaven.configuration.Configuration; import io.deephaven.parquet.base.util.SeekableChannelsProvider; @@ -16,30 +14,27 @@ import java.io.UnsupportedEncodingException; import java.net.URI; import java.net.URISyntaxException; -import java.nio.ByteBuffer; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; import java.time.Duration; -import java.util.concurrent.CompletableFuture; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public final class S3BackedSeekableChannelProvider implements SeekableChannelsProvider { +public final class S3SeekableChannelProvider implements SeekableChannelsProvider { private final S3AsyncClient s3AsyncClient; private final URI uri; private final String s3uri, bucket, key; private final long size; + private final Map contextMap = new HashMap<>(); - private static final int MAX_CACHE_SIZE = - Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-cache-size", 10); private static final int MAX_AWS_CONCURRENT_REQUESTS = Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-concurrency", 20); - private final Cache> readAheadBuffersCache; - - public S3BackedSeekableChannelProvider(final String awsRegionName, final String uriStr) throws IOException { + public S3SeekableChannelProvider(final String awsRegionName, final String uriStr) throws IOException { if (awsRegionName == null || awsRegionName.isEmpty()) { throw new IllegalArgumentException("awsRegionName cannot be null or empty"); } @@ -65,10 +60,6 @@ public S3BackedSeekableChannelProvider(final String awsRegionName, final String .httpClient(asyncHttpClient) .build(); - if (MAX_CACHE_SIZE < 1) - throw new IllegalArgumentException("maxCacheSize must be >= 1"); - this.readAheadBuffersCache = Caffeine.newBuilder().maximumSize(MAX_CACHE_SIZE).recordStats().build(); - this.s3uri = uriStr; this.bucket = uri.getHost(); this.key = uri.getPath().substring(1); @@ -93,8 +84,29 @@ public S3BackedSeekableChannelProvider(final String awsRegionName, final String } @Override - public SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { - return new S3SeekableByteChannel(s3uri, bucket, key, s3AsyncClient, 0, size, null, null, readAheadBuffersCache); + public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, + @NotNull final Path path) throws IOException { + // Ignore the context provided here, will be set properly before reading + return new S3SeekableByteChannel(context, s3uri, bucket, key, s3AsyncClient, 0, size); + } + + @Override + public ChannelContext makeContext() { + final Long tid = Long.valueOf(Thread.currentThread().getId()); + if (contextMap.containsKey(tid)) { + return contextMap.get(tid); + } else { + final ChannelContext context; + synchronized (contextMap) { + if (contextMap.containsKey(tid)) { + return contextMap.get(tid); + } + context = new S3SeekableByteChannel.ChannelContext(S3SeekableByteChannel.READ_AHEAD_COUNT, + S3SeekableByteChannel.MAX_CACHE_SIZE); + contextMap.put(tid, context); + } + return context; + } } @Override @@ -105,7 +117,11 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boole public void close() throws IOException { s3AsyncClient.close(); - readAheadBuffersCache.invalidateAll(); - readAheadBuffersCache.cleanUp(); + synchronized (contextMap) { + for (final ChannelContext context : contextMap.values()) { + context.close(); + } + contextMap.clear(); + } } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java index fcf15c1d971..00820475ce3 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java @@ -41,10 +41,18 @@ public TrackedSeekableChannelsProvider(@NotNull final TrackedFileHandleFactory f } @Override - public final SeekableByteChannel getReadChannel(@NotNull final Path path) throws IOException { + public ChannelContext makeContext() { + return ChannelContext.NULL; + } + + @Override + public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final Path path) + throws IOException { + // context is unused here because it is NULL return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, path.toFile()); } + @Override public final SeekableByteChannel getWriteChannel(@NotNull final Path filePath, final boolean append) throws IOException { From b5e2c966a9ddc6bae491381b01b3308f44b72e29 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Wed, 27 Dec 2023 18:53:42 +0530 Subject: [PATCH 07/39] Added s3 specific parquet instructions --- .../parquet/table/ParquetInstructions.java | 26 +++++- .../deephaven/parquet/table/ParquetTools.java | 5 ++ .../parquet/table/S3ParquetInstructions.java | 83 +++++++++++++++++++ 3 files changed, 112 insertions(+), 2 deletions(-) create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java 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 ca0e34bb6b3..ce51c1a227a 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 @@ -143,6 +143,8 @@ public final String getColumnNameFromParquetColumnNameOrDefault(final String par */ public abstract String getAwsRegionName(); + public abstract Object getSpecialInstructions(); + public abstract String getCompressionCodecName(); /** @@ -212,6 +214,11 @@ public boolean useDictionary(final String columnName) { return null; } + @Override + public @Nullable String getSpecialInstructions() { + return null; + } + @Override public String getCompressionCodecName() { return defaultCompressionCodecName; @@ -309,6 +316,7 @@ private static final class ReadOnly extends ParquetInstructions { private final int targetPageSize; private final boolean isRefreshing; private final String awsRegionName; + private final Object specialInstructions; private ReadOnly( final KeyedObjectHashMap columnNameToInstructions, @@ -319,7 +327,8 @@ private ReadOnly( final boolean isLegacyParquet, final int targetPageSize, final boolean isRefreshing, - final String awsRegionName) { + final String awsRegionName, + final Object specialInstructions) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; this.compressionCodecName = compressionCodecName; @@ -329,6 +338,7 @@ private ReadOnly( this.targetPageSize = targetPageSize; this.isRefreshing = isRefreshing; this.awsRegionName = awsRegionName; + this.specialInstructions = specialInstructions; } private String getOrDefault(final String columnName, final String defaultValue, @@ -422,6 +432,12 @@ public String getAwsRegionName() { return awsRegionName; } + @Override + public @Nullable Object getSpecialInstructions() { + return specialInstructions; + } + + KeyedObjectHashMap copyColumnNameToInstructions() { // noinspection unchecked return (columnNameToInstructions == null) @@ -473,6 +489,7 @@ public static class Builder { private int targetPageSize = defaultTargetPageSize; private boolean isRefreshing = DEFAULT_IS_REFRESHING; private String awsRegionName; + private Object specialInstructions; public Builder() {} @@ -649,6 +666,11 @@ public Builder setAwsRegionName(final String awsRegionName) { return this; } + public Builder setSpecialInstructions(final Object specialInstructions) { + this.specialInstructions = specialInstructions; + return this; + } + public ParquetInstructions build() { final KeyedObjectHashMap columnNameToInstructionsOut = columnNameToInstructions; columnNameToInstructions = null; @@ -657,7 +679,7 @@ public ParquetInstructions build() { parquetColumnNameToInstructions = null; return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, - awsRegionName); + awsRegionName, specialInstructions); } } 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 f55b17a0d25..80bec9ffa6e 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 @@ -7,6 +7,7 @@ import io.deephaven.base.ClassUtil; import io.deephaven.base.FileUtils; import io.deephaven.base.Pair; +import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.ColumnDefinition; @@ -103,6 +104,10 @@ public static Table readTable(@NotNull final String sourceFilePath, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { if (sourceFilePath.startsWith("s3:") && sourceFilePath.endsWith(PARQUET_FILE_EXTENSION)) { + final Object specialInstructions = readInstructions.getSpecialInstructions(); + Assert.instanceOf(specialInstructions, "specialInstructions", S3ParquetInstructions.class); + final S3ParquetInstructions s3Instructions = (S3ParquetInstructions) specialInstructions; + // TODO This is hacky, because here URI is getting converted to a file path and // will change to / // We need to keep this as a URI and internally check if its a file or S3 backed URI return readSingleFileTable(new File(sourceFilePath), readInstructions, tableDefinition); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java new file mode 100644 index 00000000000..d9c46f1cb77 --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java @@ -0,0 +1,83 @@ +package io.deephaven.parquet.table; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value; + +@Value.Immutable +@BuildableStyle +public abstract class S3ParquetInstructions { + + private final static int DEFAULT_MAX_CONCURRENT_REQUESTS = 20; + private final static int DEFAULT_READ_AHEAD_COUNT = 1; + private final static int DEFAULT_MAX_FRAGMENT_SIZE = 512 << 20; // 5 MB + private final static int MIN_MAX_FRAGMENT_SIZE = 8 << 10; // 8 KB + private final static int DEFAULT_MAX_CACHE_SIZE = 50; + + public static Builder builder() { + return S3ParquetInstructions.builder(); + } + + public abstract String awsRegionName(); + + @Value.Default + public int maxConcurrentRequests() { + return DEFAULT_MAX_CONCURRENT_REQUESTS; + } + + @Value.Default + public int readAheadCount() { + return DEFAULT_READ_AHEAD_COUNT; + } + + @Value.Default + public int maxFragmentSize() { + return DEFAULT_MAX_FRAGMENT_SIZE; + } + + @Value.Default + public int maxCacheSize() { + return DEFAULT_MAX_CACHE_SIZE; + } + + @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 (maxFragmentSize() < MIN_MAX_FRAGMENT_SIZE) { + throw new IllegalArgumentException("maxFragmentSize(=" + maxFragmentSize() + ") must be >= 8*1024 or 8 KB"); + } + } + + @Value.Check + final void boundsCheckMaxCacheSize() { + if (maxCacheSize() < readAheadCount() + 1) { + throw new IllegalArgumentException("maxCacheSize(=" + maxCacheSize() + ") must be >= 1 + readAheadCount"); + } + } + + public interface Builder { + Builder awsRegionName(String awsRegionName); + + Builder maxConcurrentRequests(int maxConcurrentRequests); + + Builder readAheadCount(int readAheadCount); + + Builder maxFragmentSize(int maxFragmentSize); + + Builder maxCacheSize(int maxCacheSize); + + S3ParquetInstructions build(); + } +} From 2cb1c3fcc77d4927e7df9b4a3988069ed43073bd Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 28 Dec 2023 17:42:13 +0530 Subject: [PATCH 08/39] Moved to URIs from files for reading single parquet file --- .../locations/local/FileTableLocationKey.java | 22 +-- .../table/impl/QueryTableAggregationTest.java | 3 +- .../impl/TestKeyValuePartitionLayout.java | 40 ++--- .../parquet/base/ParquetFileReader.java | 42 ++++- .../deephaven/parquet/table/ParquetTools.java | 167 ++++++++++++------ .../table/layout/ParquetSingleFileLayout.java | 15 +- .../table/location/ParquetColumnLocation.java | 9 +- .../table/location/ParquetTableLocation.java | 15 +- .../location/ParquetTableLocationFactory.java | 9 +- .../location/ParquetTableLocationKey.java | 39 +++- .../table/ParquetTableReadWriteTest.java | 63 ++++++- 11 files changed, 299 insertions(+), 125 deletions(-) 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..42ac0637696 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 @@ -12,6 +12,7 @@ import org.jetbrains.annotations.Nullable; import java.io.File; +import java.net.URI; import java.util.Map; /** @@ -24,7 +25,7 @@ public class FileTableLocationKey extends PartitionedTableLocationKey { private static final String IMPLEMENTATION_NAME = FileTableLocationKey.class.getSimpleName(); - protected final File file; + protected final URI parquetFileURI; private final int order; private int cachedHashCode; @@ -32,7 +33,8 @@ public class FileTableLocationKey extends PartitionedTableLocationKey { /** * Construct a new FileTableLocationKey for the supplied {@code file} and {@code partitions}. * - * @param file The file (or directory) that backs the keyed location. Will be adjusted to an absolute path. + * @param parquetFileURI The URI for file (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 * FileTableLocationKeys with a lower {@code order} before other keys. Comparing this ordering value takes * precedence over other fields. @@ -41,21 +43,21 @@ public class FileTableLocationKey extends PartitionedTableLocationKey { * 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 FileTableLocationKey(@NotNull final File file, final int order, + public FileTableLocationKey(@NotNull final URI parquetFileURI, final int order, @Nullable final Map> partitions) { super(partitions); - this.file = file.getAbsoluteFile(); + this.parquetFileURI = parquetFileURI; this.order = order; } - public final File getFile() { - return file; + public final URI getURI() { + return parquetFileURI; } @Override public LogOutput append(@NotNull final LogOutput logOutput) { return logOutput.append(getImplementationName()) - .append(":[file=").append(file.getPath()) + .append(":[file=").append(parquetFileURI.toString()) .append(",partitions=").append(PartitionsFormatter.INSTANCE, partitions) .append(']'); } @@ -84,7 +86,7 @@ public int compareTo(@NotNull final TableLocationKey other) { if (partitionComparisonResult != 0) { return partitionComparisonResult; } - return file.compareTo(otherTyped.file); + return parquetFileURI.compareTo(otherTyped.parquetFileURI); } throw new ClassCastException("Cannot compare " + getClass() + " to " + other.getClass()); } @@ -92,7 +94,7 @@ public int compareTo(@NotNull final TableLocationKey other) { @Override public int hashCode() { if (cachedHashCode == 0) { - final int computedHashCode = 31 * partitions.hashCode() + file.hashCode(); + final int computedHashCode = 31 * partitions.hashCode() + parquetFileURI.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 = FileTableLocationKey.class.hashCode(); @@ -113,7 +115,7 @@ public boolean equals(@Nullable final Object other) { return false; } final FileTableLocationKey otherTyped = (FileTableLocationKey) other; - return file.equals(otherTyped.file) && partitions.equals(otherTyped.partitions); + return parquetFileURI.equals(otherTyped.parquetFileURI) && partitions.equals(otherTyped.partitions); } @Override 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 d62c00397e9..511b6fa4086 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 @@ -3897,7 +3897,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/TestKeyValuePartitionLayout.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestKeyValuePartitionLayout.java index 6bcde5137ac..ddc06dd1169 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestKeyValuePartitionLayout.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestKeyValuePartitionLayout.java @@ -58,14 +58,14 @@ public void testFlat() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 0).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 0).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); TestCase.assertEquals(2, results.size()); - TestCase.assertEquals(file1.getAbsoluteFile(), results.get(0).getFile()); - TestCase.assertEquals(file2.getAbsoluteFile(), results.get(1).getFile()); + TestCase.assertEquals(file1.toURI(), results.get(0).getURI()); + TestCase.assertEquals(file2.toURI(), results.get(1).getURI()); TestCase.assertTrue(results.get(0).getPartitionKeys().isEmpty()); TestCase.assertTrue(results.get(1).getPartitionKeys().isEmpty()); @@ -82,14 +82,14 @@ public void testOneLevel() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 1).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 1).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); TestCase.assertEquals(2, results.size()); - TestCase.assertEquals(file2.getAbsoluteFile(), results.get(0).getFile()); - TestCase.assertEquals(file1.getAbsoluteFile(), results.get(1).getFile()); + TestCase.assertEquals(file2.toURI(), results.get(0).getURI()); + TestCase.assertEquals(file1.toURI(), results.get(1).getURI()); TestCase.assertEquals(1, results.get(0).getPartitionKeys().size()); TestCase.assertEquals(1, results.get(1).getPartitionKeys().size()); @@ -115,15 +115,15 @@ public void testThreeLevels() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); TestCase.assertEquals(3, results.size()); - TestCase.assertEquals(file2.getAbsoluteFile(), results.get(0).getFile()); - TestCase.assertEquals(file3.getAbsoluteFile(), results.get(1).getFile()); - TestCase.assertEquals(file1.getAbsoluteFile(), results.get(2).getFile()); + TestCase.assertEquals(file2.toURI(), results.get(0).getURI()); + TestCase.assertEquals(file3.toURI(), results.get(1).getURI()); + TestCase.assertEquals(file1.toURI(), results.get(2).getURI()); TestCase.assertEquals(3, results.get(0).getPartitionKeys().size()); TestCase.assertEquals(3, results.get(1).getPartitionKeys().size()); @@ -166,7 +166,7 @@ public void testTypesAndNameLegalization() throws IOException { for (final Supplier locationTableBuilderSupplier : locationTableBuilderSuppliers) { final TableLocationKeyFinder finder = new KeyValuePartitionLayout<>( dataDirectory, path -> true, locationTableBuilderSupplier, - (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3); + (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3); final RecordingLocationKeyFinder recorder1 = new RecordingLocationKeyFinder<>(); finder.findKeys(recorder1); @@ -180,9 +180,9 @@ public void testTypesAndNameLegalization() throws IOException { TestCase.assertEquals(3, results1.size()); - TestCase.assertEquals(file2.getAbsoluteFile(), results1.get(0).getFile()); - TestCase.assertEquals(file3.getAbsoluteFile(), results1.get(1).getFile()); - TestCase.assertEquals(file1.getAbsoluteFile(), results1.get(2).getFile()); + TestCase.assertEquals(file2.toURI(), results1.get(0).getURI()); + TestCase.assertEquals(file3.toURI(), results1.get(1).getURI()); + TestCase.assertEquals(file1.toURI(), results1.get(2).getURI()); TestCase.assertEquals(3, results1.get(0).getPartitionKeys().size()); TestCase.assertEquals(3, results1.get(1).getPartitionKeys().size()); @@ -219,7 +219,7 @@ public void testMaxDepthEmpty() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); @@ -247,15 +247,15 @@ public void testMaxDepth() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); TestCase.assertEquals(3, results.size()); - TestCase.assertEquals(file2.getAbsoluteFile(), results.get(0).getFile()); - TestCase.assertEquals(file3.getAbsoluteFile(), results.get(1).getFile()); - TestCase.assertEquals(file1.getAbsoluteFile(), results.get(2).getFile()); + TestCase.assertEquals(file2.toURI(), results.get(0).getURI()); + TestCase.assertEquals(file3.toURI(), results.get(1).getURI()); + TestCase.assertEquals(file1.toURI(), results.get(2).getURI()); } @Test @@ -275,7 +275,7 @@ public void testMismatch() throws IOException { try { new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3).findKeys(ftlk -> { + (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3).findKeys(ftlk -> { }); TestCase.fail("Expected exception"); } catch (TableDataException expected) { 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 fe6ef46df6d..dcf4523a5af 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 @@ -3,6 +3,7 @@ */ package io.deephaven.parquet.base; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.parquet.base.util.Helpers; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.apache.parquet.format.*; @@ -12,12 +13,13 @@ import java.io.ByteArrayInputStream; import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; 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.*; /** @@ -27,20 +29,44 @@ 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 S3_PARQUET_FILE_URI_SCHEME = "s3"; + private static final String S3_PARQUET_FILE_URI_PREFIX = "s3://"; public final FileMetaData fileMetaData; private final SeekableChannelsProvider channelsProvider; private final Path rootPath; private final MessageType type; + // TODO Where should I keep it? + public static URI convertToURI(final String filePath) { + // We need to encode spaces in the path string that are not allowed in a URI + if (filePath.startsWith(S3_PARQUET_FILE_URI_PREFIX)) { + try { + return new URI(filePath.replace(" ", "%20")); + } catch (final URISyntaxException e) { + throw new UncheckedDeephavenException("Failed to convert file path " + filePath + " to URI", e); + } + } else { + return Path.of(filePath).toUri(); + } + } + public ParquetFileReader(final String filePath, 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(filePath), channelsProvider); + } + public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvider channelsProvider) + throws IOException { + this.channelsProvider = channelsProvider; + final Path filePath; + if (parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { + filePath = Path.of(parquetFileURI.toString()); + } else { + filePath = Path.of(parquetFileURI); + } + // Root path should be this file if a single file, else the parent directory for a metadata file + rootPath = parquetFileURI.getRawPath().endsWith(".parquet") ? filePath : filePath.getParent(); final byte[] footer; try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, filePath)) { @@ -48,7 +74,7 @@ public ParquetFileReader(final String filePath, final SeekableChannelsProvider c 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; @@ -59,7 +85,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; 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 80bec9ffa6e..930c8352641 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 @@ -7,7 +7,6 @@ import io.deephaven.base.ClassUtil; import io.deephaven.base.FileUtils; import io.deephaven.base.Pair; -import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.ColumnDefinition; @@ -48,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.S3_PARQUET_FILE_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetFileReader.convertToURI; import static io.deephaven.parquet.table.ParquetTableWriter.PARQUET_FILE_EXTENSION; import static io.deephaven.util.type.TypeUtils.getUnboxedTypeIfBoxed; @@ -92,27 +94,7 @@ private ParquetTools() {} * @see ParquetFlatPartitionedLayout */ public static Table readTable(@NotNull final String sourceFilePath) { - if (sourceFilePath.startsWith("s3:") && sourceFilePath.endsWith(PARQUET_FILE_EXTENSION)) { - // TODO This is hacky, because here URI is getting converted to a file path and // will change to / - // We need to keep this as a URI and internally check if its a file or S3 backed URI - return readSingleFileTable(new File(sourceFilePath), ParquetInstructions.EMPTY); - } - return readTableInternal(new File(sourceFilePath), ParquetInstructions.EMPTY); - } - - public static Table readTable(@NotNull final String sourceFilePath, - @NotNull final ParquetInstructions readInstructions, - @NotNull final TableDefinition tableDefinition) { - if (sourceFilePath.startsWith("s3:") && sourceFilePath.endsWith(PARQUET_FILE_EXTENSION)) { - final Object specialInstructions = readInstructions.getSpecialInstructions(); - Assert.instanceOf(specialInstructions, "specialInstructions", S3ParquetInstructions.class); - final S3ParquetInstructions s3Instructions = (S3ParquetInstructions) specialInstructions; - - // TODO This is hacky, because here URI is getting converted to a file path and // will change to / - // We need to keep this as a URI and internally check if its a file or S3 backed URI - return readSingleFileTable(new File(sourceFilePath), readInstructions, tableDefinition); - } - return readTableInternal(new File(sourceFilePath), readInstructions); + return readTableInternal(convertToURI(sourceFilePath), ParquetInstructions.EMPTY); } /** @@ -141,12 +123,7 @@ public static Table readTable(@NotNull final String sourceFilePath, public static Table readTable( @NotNull final String sourceFilePath, @NotNull final ParquetInstructions readInstructions) { - if (sourceFilePath.startsWith("s3:") && sourceFilePath.endsWith(PARQUET_FILE_EXTENSION)) { - // TODO This is hacky, because here URI is getting converted to a file path and // will change to / - // We need to keep this as a URI and internally check if its a file or S3 backed URI - return readSingleFileTable(new File(sourceFilePath), readInstructions); - } - return readTableInternal(new File(sourceFilePath), readInstructions); + return readTableInternal(convertToURI(sourceFilePath), readInstructions); } /** @@ -172,7 +149,7 @@ public static Table readTable( * @see ParquetFlatPartitionedLayout */ public static Table readTable(@NotNull final File sourceFile) { - return readTableInternal(sourceFile, ParquetInstructions.EMPTY); + return readTableInternal(sourceFile.toURI(), ParquetInstructions.EMPTY); } /** @@ -201,7 +178,7 @@ public static Table readTable(@NotNull final File sourceFile) { public static Table readTable( @NotNull final File sourceFile, @NotNull final ParquetInstructions readInstructions) { - return readTableInternal(sourceFile, readInstructions); + return readTableInternal(sourceFile.toURI(), readInstructions); } /** @@ -629,41 +606,45 @@ public static void deleteTable(File path) { * key} order) location found will be used to infer schema. * *

- * Delegates to one of {@link #readSingleFileTable(File, ParquetInstructions)}, + * Delegates to one of {@link #readSingleFileTable(URI, ParquetInstructions)}, * {@link #readPartitionedTableWithMetadata(File, ParquetInstructions)}, * {@link #readFlatPartitionedTable(File, ParquetInstructions)}, or * {@link #readKeyValuePartitionedTable(File, ParquetInstructions)}. * - * @param source The source file or directory + * @param source The URI for source file or directory * @param instructions Instructions for reading * @return A {@link Table} */ private static Table readTableInternal( - @NotNull final File source, + @NotNull final URI source, @NotNull final ParquetInstructions instructions) { - final Path sourcePath = source.toPath(); + if (source.getScheme() != null && source.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { + return readSingleFileTable(source, instructions); + } + final Path sourcePath = Path.of(source.getRawPath()); if (!Files.exists(sourcePath)) { throw new TableDataException("Source file " + source + " does not exist"); } 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 @@ -680,10 +661,10 @@ 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); } @@ -728,7 +709,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); } @@ -956,13 +937,37 @@ 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(File, 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, readInstructions); + return readSingleFileTable(file.toURI(), readInstructions); + } + + /** + * Creates a single table via the parquet {@code filePath} using the provided {@code tableDefinition}. + *

+ * Callers wishing to be more explicit (for example, to skip some columns) may prefer to call + * {@link #readSingleFileTable(File, ParquetInstructions, TableDefinition)}. + * + * @param filePath the parquet file path + * @param readInstructions the instructions for customizations while reading + * @return the table + * @see ParquetTableLocationKey#ParquetTableLocationKey(File, int, Map, ParquetInstructions) + * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) + */ + public static Table readSingleFileTable( + @NotNull final String filePath, + @NotNull final ParquetInstructions readInstructions) { + return readSingleFileTable(convertToURI(filePath), 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(), @@ -976,15 +981,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(File, 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), readInstructions, - tableDefinition); + return readSingleFileTable(file.toURI(), readInstructions, tableDefinition); + } + + /** + * Creates a single table via the parquet {@code filePath} using the provided {@code tableDefinition}. API used by + * Python code. + * + * @param filePath the parquet file path + * @param readInstructions the instructions for customizations while reading + * @param tableDefinition the table definition + * @return the table + * @see ParquetTableLocationKey#ParquetTableLocationKey(File, int, Map, ParquetInstructions) + * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) + */ + public static Table readSingleFileTable( + @NotNull final String filePath, + @NotNull final ParquetInstructions readInstructions, + @NotNull final TableDefinition tableDefinition) { + return readSingleFileTable(convertToURI(filePath), 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( @@ -1069,6 +1099,23 @@ public static ParquetFileReader getParquetFileReader(@NotNull final File parquet } } + /** + * 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}. * @@ -1079,18 +1126,28 @@ public static ParquetFileReader getParquetFileReader(@NotNull final File parquet public static ParquetFileReader getParquetFileReaderChecked( @NotNull final File parquetFile, @NotNull final ParquetInstructions readInstructions) throws IOException { - final String absolutePath = parquetFile.getAbsolutePath(); - final String S3_MARKER = "s3:/"; - if (absolutePath.contains(S3_MARKER)) { - // TODO I am creating S3 URI back from the file path which is incorrect, should have passed URI only - final int index = absolutePath.indexOf(S3_MARKER); - final String s3uri = S3_MARKER + absolutePath.substring(index + S3_MARKER.length() - 1); - return new ParquetFileReader(absolutePath, + 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 { + if (parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { + return new ParquetFileReader(parquetFileURI, new CachedChannelProvider( - new S3SeekableChannelProvider(readInstructions.getAwsRegionName(), s3uri), 1 << 7)); + new S3SeekableChannelProvider(readInstructions.getAwsRegionName(), + parquetFileURI.toString()), + 1 << 7)); } return new ParquetFileReader( - parquetFile.getAbsolutePath(), + parquetFileURI, new CachedChannelProvider( new TrackedSeekableChannelsProvider(TrackedFileHandleFactory.getInstance()), 1 << 7)); } 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 8fb14b9ad42..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 @@ -8,33 +8,32 @@ 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, + public ParquetSingleFileLayout(@NotNull final URI parquetFileUri, @NotNull final ParquetInstructions readInstructions) { - this.parquetFile = parquetFile; + 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, readInstructions)); + 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..d677d20b144 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 @@ -3,6 +3,7 @@ */ package io.deephaven.parquet.table.location; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.chunk.*; @@ -50,6 +51,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 +60,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.S3_PARQUET_FILE_URI_SCHEME; import static io.deephaven.parquet.table.ParquetTableWriter.*; final class ParquetColumnLocation extends AbstractColumnLocation { @@ -163,7 +166,11 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition if (!hasGroupingTable) { return null; } - final File parquetFile = tl().getParquetFile(); + final URI parquetFileURI = tl().getParquetFile(); + if (parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { + throw new UncheckedDeephavenException("Parquet files in S3 are not expected to have indexing files"); + } + final File parquetFile = new File(parquetFileURI); try { ParquetFileReader parquetFileReader; final String indexFilePath; 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..897017b7766 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 @@ -26,9 +26,12 @@ import org.jetbrains.annotations.NotNull; import java.io.File; +import java.net.URI; import java.util.*; import java.util.stream.IntStream; +import static io.deephaven.parquet.base.ParquetFileReader.S3_PARQUET_FILE_URI_SCHEME; + public class ParquetTableLocation extends AbstractTableLocation { private static final String IMPLEMENTATION_NAME = ParquetColumnLocation.class.getSimpleName(); @@ -87,7 +90,13 @@ 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()); + final String uriScheme = tableLocationKey.getURI().getScheme(); + if (uriScheme != null && uriScheme.equals(S3_PARQUET_FILE_URI_SCHEME)) { + handleUpdate(computeIndex(), 0L); // TODO What should I put here? + } else { + handleUpdate(computeIndex(), new File(tableLocationKey.getURI().toString()).lastModified()); + } + } @Override @@ -98,8 +107,8 @@ public String getImplementationName() { @Override public void refresh() {} - File getParquetFile() { - return ((ParquetTableLocationKey) getKey()).getFile(); + URI getParquetFile() { + return ((ParquetTableLocationKey) getKey()).getURI(); } ParquetInstructions getReadInstructions() { 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 5dc3d051903..c395212a5e2 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.S3_PARQUET_FILE_URI_SCHEME; /** * {@link TableLocationFactory} for {@link ParquetTableLocation}s. @@ -30,9 +33,9 @@ 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(); - // TODO Again hacky, need to keep a URI and check if its a file or not and then do existence check - if (parquetFile.getAbsolutePath().contains("s3:/") || parquetFile.exists()) { + final URI parquetFileURI = locationKey.getURI(); + if ((parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) + || new File(parquetFileURI.getPath()).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 fcb6250203b..d3ac2dac038 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 @@ -18,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; @@ -51,11 +52,32 @@ public ParquetTableLocationKey(@NotNull final File file, final int order, 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 @@ -77,7 +99,7 @@ public String getImplementationName() { * * * Callers wishing to handle these cases more explicit may call - * {@link ParquetTools#getParquetFileReaderChecked(File, ParquetInstructions)}. + * {@link ParquetTools#getParquetFileReaderChecked(URI, ParquetInstructions)}. * * @return true if the file reader exists or was successfully created */ @@ -86,7 +108,7 @@ public synchronized boolean verifyFileReader() { return true; } try { - fileReader = ParquetTools.getParquetFileReaderChecked(file, readInstructions); + fileReader = ParquetTools.getParquetFileReaderChecked(parquetFileURI, readInstructions); } catch (IOException e) { return false; } @@ -103,7 +125,7 @@ public synchronized ParquetFileReader getFileReader() { if (fileReader != null) { return fileReader; } - return fileReader = ParquetTools.getParquetFileReader(file, readInstructions); + return fileReader = ParquetTools.getParquetFileReader(parquetFileURI, readInstructions); } /** @@ -132,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); } } @@ -167,7 +189,8 @@ 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().equals(new File(parquetFileURI).getAbsoluteFile()); }).toArray(); } 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 57805742e18..6b31827748a 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 @@ -394,7 +394,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 @@ -591,7 +591,7 @@ public void readLongParquetFileFromS3Test() { ColumnDefinition.ofTime("last_modified"), ColumnDefinition.ofDouble("input_value")); - final Table fromAws1 = ParquetTools.readTable( + final Table fromAws1 = 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(); final Table fromDisk1 = ParquetTools.readSingleFileTable( @@ -623,7 +623,7 @@ public void readRefParquetFileFromS3Test() { ColumnDefinition.ofTime("last_modified"), ColumnDefinition.ofDouble("input_value")); - final Table fromAws1 = ParquetTools.readTable( + final Table fromAws1 = 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).head(5).select(); final Table fromDisk1 = ParquetTools.readSingleFileTable( @@ -633,7 +633,7 @@ public void readRefParquetFileFromS3Test() { tableDefinition).head(5).select(); assertTableEquals(fromAws1, fromDisk1); - final Table fromAws2 = ParquetTools.readTable( + final Table fromAws2 = 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).head(5).select(); final Table fromDisk2 = ParquetTools.readSingleFileTable( @@ -665,8 +665,6 @@ public void readRefParquetFileLocally() { ParquetTools.readSingleFileTable( new File( "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), - // new File( - // "/Users/shivammalhotra/Documents/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet"), ParquetTools.SNAPPY, tableDefinition).head(5).select(); } @@ -678,7 +676,7 @@ public void profileReadingFromS3() { .build(); long totalTime = 0; - long NUM_RUNS = 10; + long NUM_RUNS = 1; for (int i = 0; i < NUM_RUNS; i++) { final long start = System.nanoTime(); ParquetTools.readTable("s3://dh-s3-parquet-test1/multiColFile.parquet", readInstructions).select(); @@ -711,7 +709,7 @@ public void profileReadingFromS3() { totalTime = 0; for (int i = 0; i < NUM_RUNS; i++) { final long start = System.nanoTime(); - ParquetTools.readTable( + 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).head(5).select(); final long end = System.nanoTime(); @@ -746,6 +744,11 @@ public void readParquetFileFromS3Test() { 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 col file with spaces in name.parquet", + readInstructions).select(); + assertTableEquals(fromAws3, dhTable2); } @Test @@ -1074,6 +1077,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 = 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. From af10a1a2740162b19b8a022f69bd9208b3270476 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 29 Dec 2023 18:11:40 +0530 Subject: [PATCH 09/39] Added s3 specific parquet instructions Also added support for Python --- .../parquet/table/ParquetInstructions.java | 26 +-- .../deephaven/parquet/table/ParquetTools.java | 4 +- .../parquet/table/S3ParquetInstructions.java | 18 +- .../table/util/S3SeekableByteChannel.java | 173 +++--------------- .../table/util/S3SeekableChannelProvider.java | 60 +++--- .../table/ParquetTableReadWriteTest.java | 52 +++++- py/server/deephaven/parquet.py | 127 +++++++++---- 7 files changed, 198 insertions(+), 262 deletions(-) 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 ce51c1a227a..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 @@ -138,11 +138,6 @@ public final String getColumnNameFromParquetColumnNameOrDefault(final String par */ public abstract boolean useDictionary(String columnName); - /** - * @return The AWS region name to use for S3 operations; defaults to null - */ - public abstract String getAwsRegionName(); - public abstract Object getSpecialInstructions(); public abstract String getCompressionCodecName(); @@ -209,11 +204,6 @@ public boolean useDictionary(final String columnName) { return false; } - @Override - public @Nullable String getAwsRegionName() { - return null; - } - @Override public @Nullable String getSpecialInstructions() { return null; @@ -315,7 +305,6 @@ private static final class ReadOnly extends ParquetInstructions { private final boolean isLegacyParquet; private final int targetPageSize; private final boolean isRefreshing; - private final String awsRegionName; private final Object specialInstructions; private ReadOnly( @@ -327,7 +316,6 @@ private ReadOnly( final boolean isLegacyParquet, final int targetPageSize, final boolean isRefreshing, - final String awsRegionName, final Object specialInstructions) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; @@ -337,7 +325,6 @@ private ReadOnly( this.isLegacyParquet = isLegacyParquet; this.targetPageSize = targetPageSize; this.isRefreshing = isRefreshing; - this.awsRegionName = awsRegionName; this.specialInstructions = specialInstructions; } @@ -427,11 +414,6 @@ public boolean isRefreshing() { return isRefreshing; } - @Override - public String getAwsRegionName() { - return awsRegionName; - } - @Override public @Nullable Object getSpecialInstructions() { return specialInstructions; @@ -488,7 +470,6 @@ public static class Builder { private boolean isLegacyParquet; private int targetPageSize = defaultTargetPageSize; private boolean isRefreshing = DEFAULT_IS_REFRESHING; - private String awsRegionName; private Object specialInstructions; public Builder() {} @@ -661,11 +642,6 @@ public Builder setIsRefreshing(final boolean isRefreshing) { return this; } - public Builder setAwsRegionName(final String awsRegionName) { - this.awsRegionName = awsRegionName; - return this; - } - public Builder setSpecialInstructions(final Object specialInstructions) { this.specialInstructions = specialInstructions; return this; @@ -679,7 +655,7 @@ public ParquetInstructions build() { parquetColumnNameToInstructions = null; return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, - awsRegionName, specialInstructions); + specialInstructions); } } 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 930c8352641..caaa2eb2815 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 @@ -1142,9 +1142,7 @@ public static ParquetFileReader getParquetFileReaderChecked( if (parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { return new ParquetFileReader(parquetFileURI, new CachedChannelProvider( - new S3SeekableChannelProvider(readInstructions.getAwsRegionName(), - parquetFileURI.toString()), - 1 << 7)); + new S3SeekableChannelProvider(parquetFileURI, readInstructions), 1 << 7)); } return new ParquetFileReader( parquetFileURI, diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java index d9c46f1cb77..3f35a13c141 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java @@ -7,14 +7,14 @@ @BuildableStyle public abstract class S3ParquetInstructions { - private final static int DEFAULT_MAX_CONCURRENT_REQUESTS = 20; + private final static int DEFAULT_MAX_CONCURRENT_REQUESTS = 50; private final static int DEFAULT_READ_AHEAD_COUNT = 1; - private final static int DEFAULT_MAX_FRAGMENT_SIZE = 512 << 20; // 5 MB - private final static int MIN_MAX_FRAGMENT_SIZE = 8 << 10; // 8 KB + private final static int DEFAULT_FRAGMENT_SIZE = 512 << 20; // 5 MB + private final static int MIN_FRAGMENT_SIZE = 8 << 10; // 8 KB private final static int DEFAULT_MAX_CACHE_SIZE = 50; public static Builder builder() { - return S3ParquetInstructions.builder(); + return ImmutableS3ParquetInstructions.builder(); } public abstract String awsRegionName(); @@ -30,8 +30,8 @@ public int readAheadCount() { } @Value.Default - public int maxFragmentSize() { - return DEFAULT_MAX_FRAGMENT_SIZE; + public int fragmentSize() { + return DEFAULT_FRAGMENT_SIZE; } @Value.Default @@ -55,8 +55,8 @@ final void boundsCheckReadAheadCount() { @Value.Check final void boundsCheckMaxFragmentSize() { - if (maxFragmentSize() < MIN_MAX_FRAGMENT_SIZE) { - throw new IllegalArgumentException("maxFragmentSize(=" + maxFragmentSize() + ") must be >= 8*1024 or 8 KB"); + if (fragmentSize() < MIN_FRAGMENT_SIZE) { + throw new IllegalArgumentException("fragmentSize(=" + fragmentSize() + ") must be >= 8*1024 or 8 KB"); } } @@ -74,7 +74,7 @@ public interface Builder { Builder readAheadCount(int readAheadCount); - Builder maxFragmentSize(int maxFragmentSize); + Builder fragmentSize(int fragmentSize); Builder maxCacheSize(int maxCacheSize); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java index 350a609be0c..6a050ccc9ab 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -1,10 +1,7 @@ package io.deephaven.parquet.table.util; import io.deephaven.base.verify.Assert; -import io.deephaven.configuration.Configuration; import io.deephaven.parquet.base.util.SeekableChannelsProvider; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.S3AsyncClient; import javax.annotation.Nullable; @@ -72,50 +69,37 @@ FragmentContext getFragmentContext(final int fragmentIndex) { private long position; private final S3AsyncClient s3Client; private volatile boolean closed; - private final String s3uri, bucket, key; - private final int maxFragmentSize; -// private final int maxNumberFragments; + private final String bucket, key; + private final int fragmentSize; + private final int readAheadCount; private final int numFragmentsInObject; private final long size; private final Long timeout; private final TimeUnit timeUnit; private ChannelContext context; - private final Logger logger = LoggerFactory.getLogger(this.getClass()); - - static final int READ_AHEAD_COUNT = - Configuration.getInstance().getIntegerWithDefault("s3.read-ahead-count", 1); - static final int MAX_CACHE_SIZE = - Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-cache-size", 50); - private static final int MAX_FRAGMENT_SIZE = - Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-fragment-size", 512 * 1024); // 512 KB - - S3SeekableByteChannel(SeekableChannelsProvider.ChannelContext context, String s3uri, String bucket, String key, S3AsyncClient s3Client, long startAt, long size) { - Objects.requireNonNull(s3Client); - if (MAX_FRAGMENT_SIZE < 1) - throw new IllegalArgumentException("maxFragmentSize must be >= 1"); - if (size < 1) - throw new IllegalArgumentException("size must be >= 1"); - - this.position = startAt; + S3SeekableByteChannel(SeekableChannelsProvider.ChannelContext context, String bucket, String key, S3AsyncClient s3Client, long size, int fragmentSize, int readAheadCount) { + this.position = 0; this.bucket = bucket; this.key = key; this.closed = false; this.s3Client = s3Client; - this.s3uri = s3uri; - this.maxFragmentSize = MAX_FRAGMENT_SIZE; + this.fragmentSize = fragmentSize; + this.readAheadCount = readAheadCount; this.timeout = 5L; this.timeUnit = TimeUnit.MINUTES; this.size = size; - this.numFragmentsInObject = (int) Math.ceil((double) size / maxFragmentSize); + this.numFragmentsInObject = (int) Math.ceil((double) size / fragmentSize); this.context = (ChannelContext) context; } @Override public void setContext(@Nullable SeekableChannelsProvider.ChannelContext context) { // null context is allowed for clearing the context - Assert.assertion(context == null || context instanceof ChannelContext, "context == null || context instanceof ChannelContext"); + if (context != null && !(context instanceof ChannelContext)) { + throw new IllegalArgumentException("context must be null or an instance of ChannelContext"); + } this.context = (ChannelContext) context; } @@ -145,7 +129,7 @@ public int read(final ByteBuffer dst) throws IOException { // Figure out the index of the fragment the bytes would start in final int currFragmentIndex = fragmentIndexForByteNumber(channelPosition); - final int fragmentOffset = (int) (channelPosition - ((long) currFragmentIndex * maxFragmentSize)); + final int fragmentOffset = (int) (channelPosition - ((long) currFragmentIndex * fragmentSize)); Assert.neqNull(context, "context"); // Blocking fetch the current fragment if it's not already in the cache @@ -164,9 +148,6 @@ public int read(final ByteBuffer dst) throws IOException { Thread.currentThread().interrupt(); throw new RuntimeException(e); } catch (final ExecutionException e) { - // the async execution completed exceptionally. - // not currently obvious when this will happen or if we can recover - logger.error("an exception occurred while reading bytes from {} that was not recovered by the S3 Client RetryCondition(s)", s3uri); throw new IOException(e); } catch (final TimeoutException e) { throw new RuntimeException(e); @@ -179,7 +160,7 @@ public int read(final ByteBuffer dst) throws IOException { dst.put(currentFragment); // Send requests for read-ahead buffers - final int numFragmentsToLoad = Math.min(READ_AHEAD_COUNT, numFragmentsInObject - currFragmentIndex - 1); + final int numFragmentsToLoad = Math.min(readAheadCount, numFragmentsInObject - currFragmentIndex - 1); for (int i = 0; i < numFragmentsToLoad; i++) { final int readAheadFragmentIndex = i + currFragmentIndex + 1; final ChannelContext.FragmentContext readAheadFragmentContext = context.getFragmentContext(readAheadFragmentIndex); @@ -193,57 +174,35 @@ public int read(final ByteBuffer dst) throws IOException { } /** - * Compute which buffer a byte should be in + * Compute which fragment a byte should be in * * @param byteNumber the number of the byte in the object accessed by this channel * @return the index of the fragment in which {@code byteNumber} will be found. */ private int fragmentIndexForByteNumber(final long byteNumber) { - return Math.toIntExact(Math.floorDiv(byteNumber, (long) maxFragmentSize)); + return Math.toIntExact(Math.floorDiv(byteNumber, (long) fragmentSize)); } private CompletableFuture computeFragmentFuture(final int fragmentIndex) { - final long readFrom = (long) fragmentIndex * maxFragmentSize; - final long readTo = Math.min(readFrom + maxFragmentSize, size) - 1; + final long readFrom = (long) fragmentIndex * fragmentSize; + final long readTo = Math.min(readFrom + fragmentSize, size) - 1; final String range = "bytes=" + readFrom + "-" + readTo; - logger.debug("byte range for {} is '{}'", key, range); return s3Client.getObject( builder -> builder .bucket(bucket) .key(key) .range(range), - new ByteBufferAsyncResponseTransformer<>(maxFragmentSize)); + new ByteBufferAsyncResponseTransformer<>(fragmentSize)); } - /** - * Writes a sequence of bytes to this channel from the given buffer. - * - *

Bytes are written starting at this channel's current position, unless - * the channel is connected to an entity such as a file that is opened with - * the {@link StandardOpenOption#APPEND APPEND} option, in - * which case the position is first advanced to the end. The entity to which - * the channel is connected will grow to accommodate the - * written bytes, and the position updates with the number of bytes - * actually written. Otherwise, this method behaves exactly as specified by - * the {@link WritableByteChannel} interface. - * - * @param src the src of the bytes to write to this channel - */ @Override - public int write(ByteBuffer src) throws IOException { + public int write(final ByteBuffer src) throws IOException { throw new UnsupportedOperationException("Don't support writing to S3 yet"); } - /** - * Returns this channel's position. - * - * @return This channel's position, - * a non-negative integer counting the number of bytes - * from the beginning of the entity to the current position - */ @Override - public long position() throws IOException { + public long position() throws ClosedChannelException { validateOpen(); synchronized (this) { @@ -251,31 +210,8 @@ public long position() throws IOException { } } - /** - * Sets this channel's position. - * - *

Setting the position to a value that is greater than the current size - * is legal but does not change the size of the entity. A later attempt to - * read bytes at such a position will immediately return an end-of-file - * indication. A later attempt to write bytes at such a position will cause - * the entity to grow to accommodate the new bytes; the values of any bytes - * between the previous end-of-file and the newly-written bytes are - * unspecified. - * - *

Setting the channel's position is not recommended when connected to - * an entity, typically a file, that is opened with the {@link - * StandardOpenOption#APPEND APPEND} option. When opened for - * append, the position is first advanced to the end before writing. - * - * @param newPosition The new position, a non-negative integer counting - * the number of bytes from the beginning of the entity - * @return This channel - * @throws ClosedChannelException If this channel is closed - * @throws IllegalArgumentException If the new position is negative - * @throws IOException If some other I/O error occurs - */ @Override - public SeekableByteChannel position(long newPosition) throws IOException { + public SeekableByteChannel position(final long newPosition) throws ClosedChannelException { if (newPosition < 0) throw new IllegalArgumentException("newPosition cannot be < 0"); @@ -289,45 +225,17 @@ public SeekableByteChannel position(long newPosition) throws IOException { } } - /** - * Returns the current size of entity to which this channel is connected. - * - * @return The current size, measured in bytes - * @throws IOException If some other I/O error occurs - */ @Override - public long size() throws IOException { + public long size() throws ClosedChannelException { validateOpen(); return this.size; } - /** - * Truncates the entity, to which this channel is connected, to the given - * size. - * - *

If the given size is less than the current size then the entity is - * truncated, discarding any bytes beyond the new end. If the given size is - * greater than or equal to the current size then the entity is not modified. - * In either case, if the current position is greater than the given size - * then it is set to that size. - * - *

An implementation of this interface may prohibit truncation when - * connected to an entity, typically a file, opened with the {@link - * StandardOpenOption#APPEND APPEND} option. - * - * @param size The new size, a non-negative byte count - * @return This channel - */ @Override - public SeekableByteChannel truncate(long size) { + public SeekableByteChannel truncate(final long size) { throw new UnsupportedOperationException("Currently not supported"); } - /** - * Tells whether this channel is open. - * - * @return {@code true} if, and only if, this channels delegate is open - */ @Override public boolean isOpen() { synchronized (this) { @@ -335,41 +243,6 @@ public boolean isOpen() { } } -// /** -// * The number of fragments currently in the cache. -// * -// * @return the size of the cache after any async evictions or reloads have happened. -// */ -// int numberOfCachedFragments() { -// readAheadBuffersCache.cleanUp(); -// return (int) readAheadBuffersCache.estimatedSize(); -// } - -// /** -// * Obtain a snapshot of the statistics of the internal cache, provides information about hits, misses, requests, evictions etc. -// * that are useful for tuning. -// * -// * @return the statistics of the internal cache. -// */ -// CacheStats cacheStatistics() { -// return readAheadBuffersCache.stats(); -// } - - /** - * Closes this channel. - * - *

After a channel is closed, any further attempt to invoke I/O - * operations upon it will cause a {@link ClosedChannelException} to be - * thrown. - * - *

If this channel is already closed then invoking this method has no - * effect. - * - *

This method may be invoked at any time. If some other thread has - * already invoked it, however, then another invocation will block until - * the first invocation is complete, after which it will return without - * effect.

- */ @Override public void close() throws IOException { synchronized (this) { diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java index f21bcc5a618..b90c4dc756d 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java @@ -1,8 +1,8 @@ package io.deephaven.parquet.table.util; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.configuration.Configuration; import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.S3ParquetInstructions; import org.jetbrains.annotations.NotNull; import software.amazon.awssdk.http.async.SdkAsyncHttpClient; import software.amazon.awssdk.regions.Region; @@ -13,7 +13,6 @@ import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URI; -import java.net.URISyntaxException; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; import java.time.Duration; @@ -26,33 +25,25 @@ public final class S3SeekableChannelProvider implements SeekableChannelsProvider { private final S3AsyncClient s3AsyncClient; - private final URI uri; - private final String s3uri, bucket, key; + private final String bucket, key; private final long size; + private final int maxConcurrentRequests; + private final int fragmentSize; + private final int maxCacheSize; + private final int readAheadCount; private final Map contextMap = new HashMap<>(); - private static final int MAX_AWS_CONCURRENT_REQUESTS = - Configuration.getInstance().getIntegerWithDefault("s3.spi.read.max-concurrency", 20); - public S3SeekableChannelProvider(final String awsRegionName, final String uriStr) throws IOException { - if (awsRegionName == null || awsRegionName.isEmpty()) { - throw new IllegalArgumentException("awsRegionName cannot be null or empty"); + public S3SeekableChannelProvider(final URI parquetFileURI, final ParquetInstructions readInstructions) + throws IOException { + if (!(readInstructions.getSpecialInstructions() instanceof S3ParquetInstructions)) { + throw new IllegalArgumentException("Must provide S3ParquetInstructions to read files from S3"); } - if (uriStr == null || uriStr.isEmpty()) { - throw new IllegalArgumentException("uri cannot be null or empty"); - } - if (MAX_AWS_CONCURRENT_REQUESTS < 1) { - throw new IllegalArgumentException("maxConcurrency must be >= 1"); - } - - try { - uri = new URI(uriStr); - } catch (final URISyntaxException e) { - throw new UncheckedDeephavenException("Failed to parse URI " + uriStr, e); - } - + final S3ParquetInstructions s3Instructions = (S3ParquetInstructions) readInstructions.getSpecialInstructions(); + final String awsRegionName = s3Instructions.awsRegionName(); + maxConcurrentRequests = s3Instructions.maxConcurrentRequests(); final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() - .maxConcurrency(MAX_AWS_CONCURRENT_REQUESTS) + .maxConcurrency(maxConcurrentRequests) .connectionTimeout(Duration.ofSeconds(5)) .build(); s3AsyncClient = S3AsyncClient.builder() @@ -60,9 +51,12 @@ public S3SeekableChannelProvider(final String awsRegionName, final String uriStr .httpClient(asyncHttpClient) .build(); - this.s3uri = uriStr; - this.bucket = uri.getHost(); - this.key = uri.getPath().substring(1); + fragmentSize = s3Instructions.fragmentSize(); + maxCacheSize = s3Instructions.maxCacheSize(); + readAheadCount = s3Instructions.readAheadCount(); + + this.bucket = parquetFileURI.getHost(); + this.key = parquetFileURI.getPath().substring(1); // Send HEAD request to S3 to get the size of the file { final long timeOut = 1L; @@ -70,9 +64,8 @@ public S3SeekableChannelProvider(final String awsRegionName, final String uriStr final HeadObjectResponse headObjectResponse; try { - headObjectResponse = s3AsyncClient.headObject(builder -> builder - .bucket(bucket) - .key(key)).get(timeOut, unit); + headObjectResponse = s3AsyncClient.headObject( + builder -> builder.bucket(bucket).key(key)).get(timeOut, unit); } catch (final InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); @@ -85,9 +78,9 @@ public S3SeekableChannelProvider(final String awsRegionName, final String uriStr @Override public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, - @NotNull final Path path) throws IOException { + @NotNull final Path path) { // Ignore the context provided here, will be set properly before reading - return new S3SeekableByteChannel(context, s3uri, bucket, key, s3AsyncClient, 0, size); + return new S3SeekableByteChannel(context, bucket, key, s3AsyncClient, size, fragmentSize, readAheadCount); } @Override @@ -101,8 +94,7 @@ public ChannelContext makeContext() { if (contextMap.containsKey(tid)) { return contextMap.get(tid); } - context = new S3SeekableByteChannel.ChannelContext(S3SeekableByteChannel.READ_AHEAD_COUNT, - S3SeekableByteChannel.MAX_CACHE_SIZE); + context = new S3SeekableByteChannel.ChannelContext(readAheadCount, maxCacheSize); contextMap.put(tid, context); } return context; 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 6b31827748a..c89bd82ecba 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 @@ -571,8 +571,15 @@ public void testArrayColumns() { @Test public void readLongParquetFileFromS3Test() { + final S3ParquetInstructions s3ParquetInstructions = S3ParquetInstructions.builder() + .awsRegionName("us-east-2") + .readAheadCount(1) + .fragmentSize(5 * 1024 * 1024) + .maxConcurrentRequests(50) + .maxCacheSize(50) + .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setAwsRegionName("us-east-2") + .setSpecialInstructions(s3ParquetInstructions) .build(); final TableDefinition tableDefinition = TableDefinition.of( @@ -604,8 +611,15 @@ public void readLongParquetFileFromS3Test() { @Test public void readRefParquetFileFromS3Test() { + final S3ParquetInstructions s3ParquetInstructions = S3ParquetInstructions.builder() + .awsRegionName("us-east-2") + .readAheadCount(1) + .fragmentSize(5 * 1024 * 1024) + .maxConcurrentRequests(50) + .maxCacheSize(50) + .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setAwsRegionName("us-east-2") + .setSpecialInstructions(s3ParquetInstructions) .build(); final TableDefinition tableDefinition = TableDefinition.of( ColumnDefinition.ofString("hash"), @@ -671,8 +685,15 @@ public void readRefParquetFileLocally() { @Test public void profileReadingFromS3() { - ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setAwsRegionName("us-east-1") + final S3ParquetInstructions s3ParquetInstructions = S3ParquetInstructions.builder() + .awsRegionName("us-east-1") + .readAheadCount(1) + .fragmentSize(5 * 1024 * 1024) + .maxConcurrentRequests(50) + .maxCacheSize(50) + .build(); + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setSpecialInstructions(s3ParquetInstructions) .build(); long totalTime = 0; @@ -686,9 +707,15 @@ public void profileReadingFromS3() { } System.out.println("Average execution time AWS is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); - - readInstructions = new ParquetInstructions.Builder() - .setAwsRegionName("us-east-2") + final S3ParquetInstructions s3ParquetInstructions2 = S3ParquetInstructions.builder() + .awsRegionName("us-east-2") + .readAheadCount(1) + .fragmentSize(5 * 1024 * 1024) + .maxConcurrentRequests(50) + .maxCacheSize(50) + .build(); + final ParquetInstructions readInstructions2 = new ParquetInstructions.Builder() + .setSpecialInstructions(s3ParquetInstructions2) .build(); final TableDefinition tableDefinition = TableDefinition.of( ColumnDefinition.ofString("hash"), @@ -711,7 +738,7 @@ public void profileReadingFromS3() { final long start = System.nanoTime(); 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).head(5).select(); + readInstructions2, tableDefinition).head(5).select(); final long end = System.nanoTime(); totalTime += end - start; System.out.println((i + 1) + ". Execution time AWS is " + (end - start) / 1000_000_000.0 + " sec"); @@ -732,8 +759,15 @@ public void profileReadingFromS3() { @Test public void readParquetFileFromS3Test() { + final S3ParquetInstructions s3ParquetInstructions = S3ParquetInstructions.builder() + .awsRegionName("us-east-1") + .readAheadCount(1) + .fragmentSize(5 * 1024 * 1024) + .maxConcurrentRequests(50) + .maxCacheSize(50) + .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setAwsRegionName("us-east-1") + .setSpecialInstructions(s3ParquetInstructions) .build(); final Table fromAws1 = ParquetTools.readTable("s3://dh-s3-parquet-test1/multiColFile.parquet", readInstructions).select(); diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index f4e727ec326..ceb2ef38cad 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -19,8 +19,46 @@ _JFile = jpy.get_type("java.io.File") _JCompressionCodecName = jpy.get_type("org.apache.parquet.hadoop.metadata.CompressionCodecName") _JParquetInstructions = jpy.get_type("io.deephaven.parquet.table.ParquetInstructions") +_JS3ParquetInstructions = jpy.get_type("io.deephaven.parquet.table.S3ParquetInstructions") _JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") +def _build_s3_parquet_instructions( + aws_region_name: str, # TODO This is a required parameter, so is this okay? + max_concurrent_requests: int = None, + read_ahead_count: int = None, + fragment_size: int = None, + max_cache_size: int = None, +): + if not any( + [ + aws_region_name, + max_concurrent_requests, + read_ahead_count, + fragment_size, + max_cache_size, + ] + ): + return None + + builder = _JS3ParquetInstructions.builder() + + if aws_region_name: + 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) + + return builder.build() + @dataclass class ColumnInstruction: @@ -33,29 +71,37 @@ class ColumnInstruction: def _build_parquet_instructions( - col_instructions: List[ColumnInstruction] = None, - compression_codec_name: str = None, - max_dictionary_keys: int = None, - max_dictionary_size: int = None, - is_legacy_parquet: bool = False, - target_page_size: int = None, - is_refreshing: bool = False, - for_read: bool = True, - force_build: bool = False, - aws_region_name: str = None, + col_instructions: List[ColumnInstruction] = None, + compression_codec_name: str = None, + max_dictionary_keys: int = None, + max_dictionary_size: int = None, + is_legacy_parquet: bool = False, + target_page_size: int = None, + is_refreshing: bool = False, + for_read: bool = True, + force_build: bool = False, + aws_region_name: str = None, + max_concurrent_requests: int = None, + read_ahead_count: int = None, + fragment_size: int = None, + max_cache_size: int = None, ): if not any( - [ - force_build, - col_instructions, - compression_codec_name, - max_dictionary_keys is not None, - max_dictionary_size is not None, - is_legacy_parquet, - target_page_size is not None, - is_refreshing, - aws_region_name, - ] + [ + force_build, + col_instructions, + compression_codec_name, + max_dictionary_keys is not None, + max_dictionary_size is not None, + is_legacy_parquet, + target_page_size is not None, + is_refreshing, + aws_region_name is not None, + max_concurrent_requests is not None, + read_ahead_count is not None, + fragment_size is not None, + max_cache_size is not None, + ] ): return None @@ -91,8 +137,15 @@ def _build_parquet_instructions( if is_refreshing: builder.setIsRefreshing(is_refreshing) - if aws_region_name: - builder.setAwsRegionName(aws_region_name) + if aws_region_name is not None: + s3_parquet_instructions = _build_s3_parquet_instructions( + aws_region_name=aws_region_name, + max_concurrent_requests=max_concurrent_requests, + read_ahead_count=read_ahead_count, + fragment_size=fragment_size, + max_cache_size=max_cache_size, + ) + builder.setSpecialInstructions(s3_parquet_instructions) return builder.build() @@ -113,6 +166,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. """ @@ -130,13 +184,17 @@ class ParquetFileLayout(Enum): def read( - path: str, - col_instructions: Optional[List[ColumnInstruction]] = None, - is_legacy_parquet: bool = False, - is_refreshing: bool = False, - file_layout: Optional[ParquetFileLayout] = None, - table_definition: Union[Dict[str, DType], List[Column], None] = None, - aws_region_name: str = None, + path: str, + col_instructions: Optional[List[ColumnInstruction]] = None, + is_legacy_parquet: bool = False, + is_refreshing: bool = False, + file_layout: Optional[ParquetFileLayout] = None, + table_definition: Union[Dict[str, DType], List[Column], None] = None, + aws_region_name: str = None, + max_concurrent_requests: int = None, + read_ahead_count: int = None, + fragment_size: int = None, + max_cache_size: int = None, ) -> Table: """ Reads in a table from a single parquet, metadata file, or directory with recognized layout. @@ -154,6 +212,7 @@ def read( 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. aws_region_name (str): the AWS region name for reading parquet files stored in AWS S3, by default None + TODO Add docstrings for the more parameters Returns: a table @@ -169,13 +228,17 @@ def read( for_read=True, force_build=True, aws_region_name=aws_region_name, + max_concurrent_requests=max_concurrent_requests, + read_ahead_count=read_ahead_count, + fragment_size=fragment_size, + max_cache_size=max_cache_size, ) 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: @@ -188,7 +251,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: From edbd29a9ad4631a6e9e78d9f40c1c71ce1d96e35 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 28 Dec 2023 19:57:22 +0530 Subject: [PATCH 10/39] Improved comments and Javadocs --- .../engine/table/impl/SourceTable.java | 1 - .../sources/regioned/RegionContextHolder.java | 1 + .../parquet/base/ParquetFileReader.java | 5 +- .../parquet/table/S3ParquetInstructions.java | 19 ++ .../ByteBufferAsyncResponseTransformer.java | 14 +- .../table/util/S3SeekableByteChannel.java | 192 +++++++++--------- .../table/util/S3SeekableChannelProvider.java | 69 ++++--- 7 files changed, 158 insertions(+), 143 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index 679a6e63f59..f2870f10631 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -116,7 +116,6 @@ private void initialize() { @TestUseOnly public final TableLocationProvider tableLocationProvider() { return locationProvider; - // Ignore } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java index 61843b127d9..50502bd1676 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java @@ -14,6 +14,7 @@ public class RegionContextHolder implements ChunkSource.FillContext { private Context innerContext; public RegionContextHolder(final int chunkCapacity, @Nullable final SharedContext sharedContext) { + this.chunkCapacity = chunkCapacity; this.sharedContext = sharedContext; } 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 dcf4523a5af..df9c509c1ce 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 @@ -67,9 +67,10 @@ public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvide } // Root path should be this file if a single file, else the parent directory for a metadata file rootPath = parquetFileURI.getRawPath().endsWith(".parquet") ? filePath : filePath.getParent(); + // TODO Close this context after Ryan's patch + final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); final byte[] footer; - try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); - final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, filePath)) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, filePath)) { final long fileLen = readChannel.size(); if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + // footerIndex + MAGIC diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java index 3f35a13c141..0e4de2f725c 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java @@ -3,6 +3,9 @@ import io.deephaven.annotations.BuildableStyle; import org.immutables.value.Value; +/** + * This class provides instructions intended for reading and writing parquet files to AWS S3 instances. + */ @Value.Immutable @BuildableStyle public abstract class S3ParquetInstructions { @@ -17,23 +20,39 @@ public static Builder builder() { return ImmutableS3ParquetInstructions.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. + */ @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. + */ @Value.Default public int readAheadCount() { return DEFAULT_READ_AHEAD_COUNT; } + /** + * The maximum size of each fragment to read from S3. The fetched fragment can be smaller than this in case fewer + * bytes remaining in the file. + */ @Value.Default public int fragmentSize() { return DEFAULT_FRAGMENT_SIZE; } + /** + * The maximum number of fragments to cache in memory. + */ @Value.Default public int maxCacheSize() { return DEFAULT_MAX_CACHE_SIZE; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java index 8f56bbc9ab6..af46463ce86 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java @@ -11,13 +11,14 @@ /** * An {@link AsyncResponseTransformer} that transforms a response into a {@link ByteBuffer}. + * This class is inspired from {@link software.amazon.awssdk.core.internal.async.ByteArrayAsyncResponseTransformer} + * but avoids a number of extra copies done by the former. * * @param POJO response type. */ public final class ByteBufferAsyncResponseTransformer implements AsyncResponseTransformer { private volatile CompletableFuture cf; - private ResponseT response; private final ByteBuffer byteBuffer; ByteBufferAsyncResponseTransformer(final int bufferSize) { @@ -33,14 +34,7 @@ public CompletableFuture prepare() { @Override public void onResponse(ResponseT response) { - this.response = response; - } - - /** - * @return the unmarshalled response object from the service. - */ - public ResponseT response() { - return response; + // No need to store the response object as we are only interested in the byte buffer } @Override @@ -55,9 +49,7 @@ public void exceptionOccurred(Throwable throwable) { final static class ByteBuferSubscriber implements Subscriber { private final CompletableFuture resultFuture; - private Subscription subscription; - private final ByteBuffer byteBuffer; ByteBuferSubscriber(CompletableFuture resultFuture, ByteBuffer byteBuffer) { diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java index 6a050ccc9ab..00f19a8245a 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -1,38 +1,56 @@ package io.deephaven.parquet.table.util; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import org.jetbrains.annotations.NotNull; import software.amazon.awssdk.services.s3.S3AsyncClient; import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.ClosedChannelException; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; -import java.nio.channels.WritableByteChannel; -import java.nio.file.StandardOpenOption; import java.security.InvalidParameterException; import java.util.ArrayList; import java.util.List; -import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; 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. + */ public final class S3SeekableByteChannel implements SeekableByteChannel, SeekableChannelsProvider.ContextHolder { + private static final long CLOSED_SENTINEL = -1; + + /** + * Maximum time to wait while fetching fragments from S3. + */ + static final Long CONNECTION_TIMEOUT_MINUTES = 1L; + static final TimeUnit CONNECTION_TIMEOUT_UNIT = TimeUnit.MINUTES; + /** * Context object used to store read-ahead buffers for efficiently reading from S3. */ static final class ChannelContext implements SeekableChannelsProvider.ChannelContext { /** - * Used to store context information for fetching a single fragment from S3 + * Used to store information related to a single fragment */ static class FragmentContext { + /** + * The index of the fragment in the object + */ private final int fragmentIndex; + + /** + * The future that will be completed with the fragment's bytes + */ private final CompletableFuture future; private FragmentContext(final int fragmentIndex, final CompletableFuture future) { @@ -64,75 +82,78 @@ void setFragmentContext(final int fragmentIndex, final CompletableFuture Bytes are read starting at this channel's current position, and - * then the position is updated with the number of bytes actually read. - * Otherwise, this method behaves exactly as specified in the {@link - * ReadableByteChannel} interface. - * - * @param dst the destination buffer - * @return the number of bytes read or -1 if no more bytes can be read. - */ @Override - public int read(final ByteBuffer dst) throws IOException { - validateOpen(); - - Objects.requireNonNull(dst); - - final long channelPosition = this.position(); - - // if the position of the delegator is at the end (>= size) return -1. we're finished reading. - if (channelPosition >= size) { + public int read(@NotNull final ByteBuffer destination) throws ClosedChannelException { + Assert.neqNull(context, "context"); + if (!destination.hasRemaining()) { + return 0; + } + final long localPosition = position; + checkClosed(localPosition); + if (localPosition >= size) { + // We are finished reading return -1; } - // Figure out the index of the fragment the bytes would start in - final int currFragmentIndex = fragmentIndexForByteNumber(channelPosition); - final int fragmentOffset = (int) (channelPosition - ((long) currFragmentIndex * fragmentSize)); - Assert.neqNull(context, "context"); + final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); + final int fragmentOffset = (int) (localPosition - (currFragmentIndex * fragmentSize)); - // Blocking fetch the current fragment if it's not already in the cache + // Send async read request the current fragment, if it's not already in the cache final ChannelContext.FragmentContext fragmentContext = context.getFragmentContext(currFragmentIndex); final CompletableFuture fetchCurrFragment; if (fragmentContext != null && fragmentContext.fragmentIndex == currFragmentIndex) { @@ -141,25 +162,8 @@ public int read(final ByteBuffer dst) throws IOException { fetchCurrFragment = computeFragmentFuture(currFragmentIndex); context.setFragmentContext(currFragmentIndex, fetchCurrFragment); } - final ByteBuffer currentFragment; - try { - currentFragment = fetchCurrFragment.get(timeout, timeUnit).asReadOnlyBuffer(); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } catch (final ExecutionException e) { - throw new IOException(e); - } catch (final TimeoutException e) { - throw new RuntimeException(e); - } - // Put the bytes from fragment from the offset up to the min of fragment remaining or dst remaining - currentFragment.position(fragmentOffset); - final int limit = Math.min(currentFragment.remaining(), dst.remaining()); - currentFragment.limit(currentFragment.position() + limit); - dst.put(currentFragment); - - // Send requests for read-ahead buffers + // Send async requests for read-ahead buffers and store them in the cache final int numFragmentsToLoad = Math.min(readAheadCount, numFragmentsInObject - currFragmentIndex - 1); for (int i = 0; i < numFragmentsToLoad; i++) { final int readAheadFragmentIndex = i + currFragmentIndex + 1; @@ -169,16 +173,26 @@ public int read(final ByteBuffer dst) throws IOException { } } - position(channelPosition + limit); + // Wait till the current fragment is fetched + final ByteBuffer currentFragment; + try { + currentFragment = fetchCurrFragment.get(timeout, timeUnit).asReadOnlyBuffer(); + } catch (final InterruptedException | ExecutionException | TimeoutException | RuntimeException e) { + throw new UncheckedDeephavenException("Failed to fetch fragment " + currFragmentIndex + " at byte offset " + + fragmentOffset + " for file " + key + " in S3 bucket " + bucket, e); + } + + // 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. + currentFragment.position(fragmentOffset); + final int limit = Math.min(currentFragment.remaining(), destination.remaining()); + currentFragment.limit(currentFragment.position() + limit); + destination.put(currentFragment); + position = localPosition + limit; return limit; } - /** - * Compute which fragment a byte should be in - * - * @param byteNumber the number of the byte in the object accessed by this channel - * @return the index of the fragment in which {@code byteNumber} will be found. - */ private int fragmentIndexForByteNumber(final long byteNumber) { return Math.toIntExact(Math.floorDiv(byteNumber, (long) fragmentSize)); } @@ -187,13 +201,8 @@ private CompletableFuture computeFragmentFuture(final int fragmentIn final long readFrom = (long) fragmentIndex * fragmentSize; final long readTo = Math.min(readFrom + fragmentSize, size) - 1; final String range = "bytes=" + readFrom + "-" + readTo; - - return s3Client.getObject( - builder -> builder - .bucket(bucket) - .key(key) - .range(range), - new ByteBufferAsyncResponseTransformer<>(fragmentSize)); + return s3Client.getObject(builder -> builder.bucket(bucket).key(key).range(range), + new ByteBufferAsyncResponseTransformer<>(fragmentSize)); } @Override @@ -203,32 +212,25 @@ public int write(final ByteBuffer src) throws IOException { @Override public long position() throws ClosedChannelException { - validateOpen(); - - synchronized (this) { - return position; - } + 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"); - - if (!isOpen()) { - throw new ClosedChannelException(); - } - - synchronized (this) { - position = newPosition; - return this; + if (newPosition < 0) { + throw new IllegalArgumentException("newPosition cannot be < 0, provided newPosition=" + newPosition); } + checkClosed(position); + position = newPosition; + return this; } @Override public long size() throws ClosedChannelException { - validateOpen(); - return this.size; + checkClosed(position); + return size; } @Override @@ -238,20 +240,16 @@ public SeekableByteChannel truncate(final long size) { @Override public boolean isOpen() { - synchronized (this) { - return !this.closed; - } + return position != CLOSED_SENTINEL; } @Override public void close() throws IOException { - synchronized (this) { - closed = true; - } + position = CLOSED_SENTINEL; } - private void validateOpen() throws ClosedChannelException { - if (this.closed) { + private static void checkClosed(final long position) throws ClosedChannelException { + if (position == CLOSED_SENTINEL) { throw new ClosedChannelException(); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java index b90c4dc756d..24e8e1f8da1 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java @@ -1,5 +1,7 @@ package io.deephaven.parquet.table.util; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.base.verify.Assert; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.S3ParquetInstructions; @@ -11,7 +13,6 @@ import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient; import java.io.IOException; -import java.io.UnsupportedEncodingException; import java.net.URI; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -19,67 +20,71 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import static io.deephaven.parquet.table.util.S3SeekableByteChannel.CONNECTION_TIMEOUT_MINUTES; +import static io.deephaven.parquet.table.util.S3SeekableByteChannel.CONNECTION_TIMEOUT_UNIT; + +/** + * {@link SeekableChannelsProvider} implementation that is used to fetch objects from AWS S3 instances. + */ public final class S3SeekableChannelProvider implements SeekableChannelsProvider { private final S3AsyncClient s3AsyncClient; + + /** + * Parquet file {@link URI} stored as a {@link Path} to save on parsing time at the time of constructing new read + * channels. Note that this conversion is lossy, as the {@link URI} will contain characters "s3://" but {@link Path} + * will have "s3:/". + */ + private final Path parquetFilePath; private final String bucket, key; private final long size; - private final int maxConcurrentRequests; private final int fragmentSize; private final int maxCacheSize; private final int readAheadCount; - private final Map contextMap = new HashMap<>(); + private final Map contextMap = new HashMap<>(); // TODO Remove this - public S3SeekableChannelProvider(final URI parquetFileURI, final ParquetInstructions readInstructions) - throws IOException { + public S3SeekableChannelProvider(final URI parquetFileURI, final ParquetInstructions readInstructions) { if (!(readInstructions.getSpecialInstructions() instanceof S3ParquetInstructions)) { throw new IllegalArgumentException("Must provide S3ParquetInstructions to read files from S3"); } final S3ParquetInstructions s3Instructions = (S3ParquetInstructions) readInstructions.getSpecialInstructions(); final String awsRegionName = s3Instructions.awsRegionName(); - maxConcurrentRequests = s3Instructions.maxConcurrentRequests(); + final int maxConcurrentRequests = s3Instructions.maxConcurrentRequests(); final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() .maxConcurrency(maxConcurrentRequests) - .connectionTimeout(Duration.ofSeconds(5)) + .connectionTimeout(Duration.ofMinutes(CONNECTION_TIMEOUT_MINUTES)) .build(); - s3AsyncClient = S3AsyncClient.builder() + this.s3AsyncClient = S3AsyncClient.builder() .region(Region.of(awsRegionName)) .httpClient(asyncHttpClient) .build(); - fragmentSize = s3Instructions.fragmentSize(); - maxCacheSize = s3Instructions.maxCacheSize(); - readAheadCount = s3Instructions.readAheadCount(); + this.fragmentSize = s3Instructions.fragmentSize(); + this.maxCacheSize = s3Instructions.maxCacheSize(); + this.readAheadCount = s3Instructions.readAheadCount(); + this.parquetFilePath = Path.of(parquetFileURI.toString()); this.bucket = parquetFileURI.getHost(); this.key = parquetFileURI.getPath().substring(1); - // Send HEAD request to S3 to get the size of the file - { - final long timeOut = 1L; - final TimeUnit unit = TimeUnit.MINUTES; - - final HeadObjectResponse headObjectResponse; - try { - headObjectResponse = s3AsyncClient.headObject( - builder -> builder.bucket(bucket).key(key)).get(timeOut, unit); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } catch (final ExecutionException | TimeoutException e) { - throw new IOException(e); - } - this.size = headObjectResponse.contentLength(); + // Send a blocking HEAD request to S3 to get the size of the file + final HeadObjectResponse headObjectResponse; + try { + headObjectResponse = s3AsyncClient.headObject( + builder -> builder.bucket(bucket).key(key)) + .get(CONNECTION_TIMEOUT_MINUTES, CONNECTION_TIMEOUT_UNIT); + } catch (final InterruptedException | ExecutionException | TimeoutException | RuntimeException e) { + throw new UncheckedDeephavenException("Failed to fetch HEAD for file " + parquetFileURI, e); } + this.size = headObjectResponse.contentLength(); } @Override public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, @NotNull final Path path) { - // Ignore the context provided here, will be set properly before reading + Assert.equals(parquetFilePath, "parquetFilePath", path, "path"); return new S3SeekableByteChannel(context, bucket, key, s3AsyncClient, size, fragmentSize, readAheadCount); } @@ -90,6 +95,7 @@ public ChannelContext makeContext() { return contextMap.get(tid); } else { final ChannelContext context; + // TODO Remove this part synchronized (contextMap) { if (contextMap.containsKey(tid)) { return contextMap.get(tid); @@ -102,9 +108,8 @@ public ChannelContext makeContext() { } @Override - public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boolean append) - throws UnsupportedEncodingException { - throw new UnsupportedEncodingException("Don't support writing to S3 yet"); + public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boolean append) { + throw new UnsupportedOperationException("Don't support writing to S3 yet"); } public void close() throws IOException { From 404af994d87048b8251e22607292399b1f163b93 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 8 Jan 2024 20:13:22 +0530 Subject: [PATCH 11/39] Resolving Devin's comments --- .../locations/local/FileTableLocationKey.java | 24 +- .../locations/local/URITableLocationKey.java | 127 +++++++++++ .../impl/TestKeyValuePartitionLayout.java | 40 ++-- .../parquet/base/ColumnChunkReaderImpl.java | 32 +-- .../parquet/base/ColumnPageReaderImpl.java | 17 +- .../parquet/base/ParquetFileReader.java | 47 ++-- .../parquet/base/RowGroupReaderImpl.java | 15 +- .../base/util/CachedChannelProvider.java | 9 +- .../base/util/LocalFSChannelProvider.java | 5 +- .../base/util/SeekableChannelsProvider.java | 13 +- .../base/util/CachedChannelProviderTest.java | 5 +- .../deephaven/parquet/table/ParquetTools.java | 13 +- .../parquet/table/S3ParquetInstructions.java | 40 +++- .../DeephavenNestedPartitionLayout.java | 6 +- .../table/location/ParquetColumnLocation.java | 4 +- .../table/location/ParquetTableLocation.java | 4 +- .../location/ParquetTableLocationFactory.java | 5 +- .../location/ParquetTableLocationKey.java | 12 +- .../ByteBufferAsyncResponseTransformer.java | 27 +-- .../table/util/S3SeekableByteChannel.java | 208 ++++++++++++------ .../table/util/S3SeekableChannelProvider.java | 51 +---- .../util/TrackedSeekableChannelsProvider.java | 5 +- .../table/ParquetTableReadWriteTest.java | 35 ++- py/server/deephaven/parquet.py | 50 +++-- .../ReplicateParquetTransferObjects.java | 3 + .../ReplicateTableLocationKey.java | 27 +++ 26 files changed, 546 insertions(+), 278 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URITableLocationKey.java create mode 100644 replication/static/src/main/java/io/deephaven/replicators/ReplicateTableLocationKey.java 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 42ac0637696..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 @@ -12,7 +12,6 @@ import org.jetbrains.annotations.Nullable; import java.io.File; -import java.net.URI; import java.util.Map; /** @@ -25,7 +24,7 @@ public class FileTableLocationKey extends PartitionedTableLocationKey { private static final String IMPLEMENTATION_NAME = FileTableLocationKey.class.getSimpleName(); - protected final URI parquetFileURI; + protected final File file; private final int order; private int cachedHashCode; @@ -33,8 +32,7 @@ public class FileTableLocationKey extends PartitionedTableLocationKey { /** * Construct a new FileTableLocationKey for the supplied {@code file} and {@code partitions}. * - * @param parquetFileURI The URI for file (or directory) that backs the keyed location. Will be adjusted to an - * absolute path. + * @param file The file (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 * FileTableLocationKeys with a lower {@code order} before other keys. Comparing this ordering value takes * precedence over other fields. @@ -43,21 +41,21 @@ public class FileTableLocationKey extends PartitionedTableLocationKey { * 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 FileTableLocationKey(@NotNull final URI parquetFileURI, final int order, + public FileTableLocationKey(@NotNull final File file, final int order, @Nullable final Map> partitions) { super(partitions); - this.parquetFileURI = parquetFileURI; + this.file = file.getAbsoluteFile(); this.order = order; } - public final URI getURI() { - return parquetFileURI; + public final File getFile() { + return file; } @Override public LogOutput append(@NotNull final LogOutput logOutput) { return logOutput.append(getImplementationName()) - .append(":[file=").append(parquetFileURI.toString()) + .append(":[file=").append(file.getPath()) .append(",partitions=").append(PartitionsFormatter.INSTANCE, partitions) .append(']'); } @@ -70,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 @@ -86,7 +84,7 @@ public int compareTo(@NotNull final TableLocationKey other) { if (partitionComparisonResult != 0) { return partitionComparisonResult; } - return parquetFileURI.compareTo(otherTyped.parquetFileURI); + return file.compareTo(otherTyped.file); } throw new ClassCastException("Cannot compare " + getClass() + " to " + other.getClass()); } @@ -94,7 +92,7 @@ public int compareTo(@NotNull final TableLocationKey other) { @Override public int hashCode() { if (cachedHashCode == 0) { - final int computedHashCode = 31 * partitions.hashCode() + parquetFileURI.hashCode(); + final int computedHashCode = 31 * partitions.hashCode() + file.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 = FileTableLocationKey.class.hashCode(); @@ -115,7 +113,7 @@ public boolean equals(@Nullable final Object other) { return false; } final FileTableLocationKey otherTyped = (FileTableLocationKey) other; - return parquetFileURI.equals(otherTyped.parquetFileURI) && partitions.equals(otherTyped.partitions); + return file.equals(otherTyped.file) && partitions.equals(otherTyped.partitions); } @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..baa388e4c9a --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URITableLocationKey.java @@ -0,0 +1,127 @@ +/** + * 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); + 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/test/java/io/deephaven/engine/table/impl/locations/impl/TestKeyValuePartitionLayout.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestKeyValuePartitionLayout.java index ddc06dd1169..6bcde5137ac 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestKeyValuePartitionLayout.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestKeyValuePartitionLayout.java @@ -58,14 +58,14 @@ public void testFlat() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 0).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 0).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); TestCase.assertEquals(2, results.size()); - TestCase.assertEquals(file1.toURI(), results.get(0).getURI()); - TestCase.assertEquals(file2.toURI(), results.get(1).getURI()); + TestCase.assertEquals(file1.getAbsoluteFile(), results.get(0).getFile()); + TestCase.assertEquals(file2.getAbsoluteFile(), results.get(1).getFile()); TestCase.assertTrue(results.get(0).getPartitionKeys().isEmpty()); TestCase.assertTrue(results.get(1).getPartitionKeys().isEmpty()); @@ -82,14 +82,14 @@ public void testOneLevel() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 1).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 1).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); TestCase.assertEquals(2, results.size()); - TestCase.assertEquals(file2.toURI(), results.get(0).getURI()); - TestCase.assertEquals(file1.toURI(), results.get(1).getURI()); + TestCase.assertEquals(file2.getAbsoluteFile(), results.get(0).getFile()); + TestCase.assertEquals(file1.getAbsoluteFile(), results.get(1).getFile()); TestCase.assertEquals(1, results.get(0).getPartitionKeys().size()); TestCase.assertEquals(1, results.get(1).getPartitionKeys().size()); @@ -115,15 +115,15 @@ public void testThreeLevels() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); TestCase.assertEquals(3, results.size()); - TestCase.assertEquals(file2.toURI(), results.get(0).getURI()); - TestCase.assertEquals(file3.toURI(), results.get(1).getURI()); - TestCase.assertEquals(file1.toURI(), results.get(2).getURI()); + TestCase.assertEquals(file2.getAbsoluteFile(), results.get(0).getFile()); + TestCase.assertEquals(file3.getAbsoluteFile(), results.get(1).getFile()); + TestCase.assertEquals(file1.getAbsoluteFile(), results.get(2).getFile()); TestCase.assertEquals(3, results.get(0).getPartitionKeys().size()); TestCase.assertEquals(3, results.get(1).getPartitionKeys().size()); @@ -166,7 +166,7 @@ public void testTypesAndNameLegalization() throws IOException { for (final Supplier locationTableBuilderSupplier : locationTableBuilderSuppliers) { final TableLocationKeyFinder finder = new KeyValuePartitionLayout<>( dataDirectory, path -> true, locationTableBuilderSupplier, - (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3); + (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3); final RecordingLocationKeyFinder recorder1 = new RecordingLocationKeyFinder<>(); finder.findKeys(recorder1); @@ -180,9 +180,9 @@ public void testTypesAndNameLegalization() throws IOException { TestCase.assertEquals(3, results1.size()); - TestCase.assertEquals(file2.toURI(), results1.get(0).getURI()); - TestCase.assertEquals(file3.toURI(), results1.get(1).getURI()); - TestCase.assertEquals(file1.toURI(), results1.get(2).getURI()); + TestCase.assertEquals(file2.getAbsoluteFile(), results1.get(0).getFile()); + TestCase.assertEquals(file3.getAbsoluteFile(), results1.get(1).getFile()); + TestCase.assertEquals(file1.getAbsoluteFile(), results1.get(2).getFile()); TestCase.assertEquals(3, results1.get(0).getPartitionKeys().size()); TestCase.assertEquals(3, results1.get(1).getPartitionKeys().size()); @@ -219,7 +219,7 @@ public void testMaxDepthEmpty() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); @@ -247,15 +247,15 @@ public void testMaxDepth() throws IOException { final RecordingLocationKeyFinder recorder = new RecordingLocationKeyFinder<>(); new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3).findKeys(recorder); + (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3).findKeys(recorder); final List results = recorder.getRecordedKeys().stream().sorted().collect(Collectors.toList()); TestCase.assertEquals(3, results.size()); - TestCase.assertEquals(file2.toURI(), results.get(0).getURI()); - TestCase.assertEquals(file3.toURI(), results.get(1).getURI()); - TestCase.assertEquals(file1.toURI(), results.get(2).getURI()); + TestCase.assertEquals(file2.getAbsoluteFile(), results.get(0).getFile()); + TestCase.assertEquals(file3.getAbsoluteFile(), results.get(1).getFile()); + TestCase.assertEquals(file1.getAbsoluteFile(), results.get(2).getFile()); } @Test @@ -275,7 +275,7 @@ public void testMismatch() throws IOException { try { new KeyValuePartitionLayout<>(dataDirectory, path -> true, () -> new LocationTableBuilderCsv(dataDirectory), - (path, partitions) -> new FileTableLocationKey(path.toUri(), 0, partitions), 3).findKeys(ftlk -> { + (path, partitions) -> new FileTableLocationKey(path.toFile(), 0, partitions), 3).findKeys(ftlk -> { }); TestCase.fail("Expected exception"); } catch (TableDataException expected) { 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 65c4cddae1b..005e5b7625b 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 @@ -23,6 +23,7 @@ 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.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; @@ -39,7 +40,10 @@ 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; @@ -47,7 +51,7 @@ public class ColumnChunkReaderImpl implements ColumnChunkReader { private final Supplier dictionarySupplier; private final PageMaterializer.Factory nullMaterializerFactory; - private Path filePath; + private URI uri; /** * Number of rows in the row group of this column chunk. */ @@ -57,12 +61,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()) { @@ -116,14 +120,14 @@ 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()); + return uri = rootURI.resolve(columnChunk.getFile_path()); } else { - return filePath = rootPath; + return uri = rootURI; } } @@ -172,7 +176,7 @@ private Dictionary getDictionary() { return NULL_DICTIONARY; } try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), getFilePath())) { + channelsProvider.getReadChannel(channelsProvider.makeContext(), getURI())) { readChannel.position(dictionaryPageOffset); return readDictionary(readChannel); } catch (IOException e) { @@ -237,7 +241,7 @@ public ColumnPageReader next() { } // NB: The channels provider typically caches channels; this avoids maintaining a handle per column chunk try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), getFilePath())) { + channelsProvider.getReadChannel(channelsProvider.makeContext(), getURI())) { final long headerOffset = currentOffset; readChannel.position(currentOffset); // deliberately not closing this stream @@ -272,7 +276,7 @@ public ColumnPageReader next() { ? dictionarySupplier : () -> NULL_DICTIONARY; return new ColumnPageReaderImpl(channelsProvider, decompressor, pageDictionarySupplier, - nullMaterializerFactory, path, getFilePath(), fieldTypes, readChannel.position(), pageHeader, + nullMaterializerFactory, path, getURI(), fieldTypes, readChannel.position(), pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES); } catch (IOException e) { throw new UncheckedDeephavenException("Error reading page header", e); @@ -305,7 +309,7 @@ public ColumnPageReader next() { - offsetIndex.getFirstRowIndex(pos) + 1); ColumnPageReaderImpl 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; @@ -324,7 +328,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/ColumnPageReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnPageReaderImpl.java index ae04fe87bcd..c27a955b01e 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 @@ -27,6 +27,7 @@ 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; @@ -52,7 +53,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { private final Supplier 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. @@ -86,7 +87,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { Supplier 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; @@ -106,7 +107,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { @Override public Object materialize(Object nullValue) throws IOException { try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), filePath)) { + channelsProvider.getReadChannel(channelsProvider.makeContext(), uri)) { ensurePageHeader(readChannel); return readDataPage(nullValue, readChannel); } @@ -114,7 +115,7 @@ public Object materialize(Object nullValue) throws IOException { public int readRowCount() throws IOException { try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), filePath)) { + channelsProvider.getReadChannel(channelsProvider.makeContext(), uri)) { ensurePageHeader(readChannel); return readRowCountFromDataPage(readChannel); } @@ -124,7 +125,7 @@ public int readRowCount() throws IOException { @Override public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder) throws IOException { try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), filePath)) { + channelsProvider.getReadChannel(channelsProvider.makeContext(), uri)) { ensurePageHeader(readChannel); return readKeyFromDataPage(keyDest, nullPlaceholder, readChannel); } @@ -623,7 +624,7 @@ public int numValues() throws IOException { return numValues; } try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), filePath)) { + channelsProvider.getReadChannel(channelsProvider.makeContext(), uri)) { ensurePageHeader(readChannel); // Above will block till it populates numValues Assert.geqZero(numValues, "numValues"); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java index df9c509c1ce..db417a919f9 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 @@ -12,6 +12,7 @@ import org.apache.parquet.schema.*; import java.io.ByteArrayInputStream; +import java.io.File; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -19,35 +20,40 @@ import java.nio.ByteOrder; import java.nio.channels.SeekableByteChannel; import java.nio.charset.StandardCharsets; -import java.nio.file.Path; import java.util.*; /** - * 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 S3_PARQUET_FILE_URI_SCHEME = "s3"; - private static final String S3_PARQUET_FILE_URI_PREFIX = "s3://"; + public static final String S3_URI_SCHEME = "s3"; + private static final String S3_URI_PREFIX = "s3://"; 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; // TODO Where should I keep it? public static URI convertToURI(final String filePath) { - // We need to encode spaces in the path string that are not allowed in a URI - if (filePath.startsWith(S3_PARQUET_FILE_URI_PREFIX)) { + if (filePath.startsWith(S3_URI_PREFIX)) { try { - return new URI(filePath.replace(" ", "%20")); + return new URI(filePath); } catch (final URISyntaxException e) { - throw new UncheckedDeephavenException("Failed to convert file path " + filePath + " to URI", e); + throw new UncheckedDeephavenException("Failed to convert file path " + filePath + " to URI, we expect " + + "CLI-style URIs, e.g., \"s3://bucket/key\" as input", e); } } else { - return Path.of(filePath).toUri(); + // Resolve to get an absolute file path and convert to URI + return new File(filePath).getAbsoluteFile().toURI(); } } @@ -59,18 +65,21 @@ public ParquetFileReader(final String filePath, final SeekableChannelsProvider c public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvider channelsProvider) throws IOException { this.channelsProvider = channelsProvider; - final Path filePath; - if (parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { - filePath = Path.of(parquetFileURI.toString()); + if (!parquetFileURI.getRawPath().endsWith(".parquet")) { + // Construct a new URI for the parent directory + try { + rootURI = new URI(parquetFileURI.getScheme(), parquetFileURI.getHost(), + new File(parquetFileURI.getPath()).getParent(), parquetFileURI.getRawFragment()); + } catch (final URISyntaxException e) { + throw new UncheckedDeephavenException("Cannot construct URI for parent directory of " + parquetFileURI, + e); + } } else { - filePath = Path.of(parquetFileURI); + rootURI = parquetFileURI; } - // Root path should be this file if a single file, else the parent directory for a metadata file - rootPath = parquetFileURI.getRawPath().endsWith(".parquet") ? filePath : filePath.getParent(); - // TODO Close this context after Ryan's patch final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); final byte[] footer; - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, filePath)) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, parquetFileURI)) { final long fileLen = readChannel.size(); if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + // footerIndex + MAGIC @@ -210,7 +219,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/RowGroupReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java index e08146ae23a..bab8938cfc3 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 @@ -17,9 +17,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 +34,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; @@ -77,7 +80,7 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { OffsetIndex offsetIndex = null; if (columnChunk.isSetOffset_index_offset()) { try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), rootPath)) { + channelsProvider.getReadChannel(channelsProvider.makeContext(), rootURI)) { readChannel.position(columnChunk.getOffset_index_offset()); offsetIndex = ParquetMetadataConverter.fromParquetOffsetIndex(Util.readOffsetIndex( new BufferedInputStream(Channels.newInputStream(readChannel), BUFFER_SIZE))); @@ -85,7 +88,7 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { 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/CachedChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java index 529e61b9d3c..139c01e7c6b 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java @@ -12,6 +12,7 @@ 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; @@ -57,13 +58,13 @@ public ChannelContext makeContext() { } @Override - public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final Path path) + public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { - final String pathKey = path.toAbsolutePath().toString(); + final String uriString = uri.toString(); final KeyedObjectHashMap channelPool = channelPools.get(ChannelType.Read); - final CachedChannel result = tryGetPooledChannel(pathKey, channelPool); + final CachedChannel result = tryGetPooledChannel(uriString, channelPool); final CachedChannel channel = result == null - ? new CachedChannel(wrappedProvider.getReadChannel(context, path), ChannelType.Read, pathKey) + ? new CachedChannel(wrappedProvider.getReadChannel(context, uri), ChannelType.Read, uriString) : result.position(0); channel.setContext(context); return channel; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java index 697e6d6e4e4..61274f543a4 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java @@ -6,6 +6,7 @@ import org.jetbrains.annotations.NotNull; import java.io.IOException; +import java.net.URI; import java.nio.channels.FileChannel; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -19,10 +20,10 @@ public ChannelContext makeContext() { } @Override - public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final Path path) + public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { // context is unused here because it is NULL - return FileChannel.open(path, StandardOpenOption.READ); + return FileChannel.open(Path.of(uri), StandardOpenOption.READ); } @Override 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 index c0b268c8b4f..28ff49bd995 100644 --- 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 @@ -3,11 +3,14 @@ */ package io.deephaven.parquet.base.util; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; 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; @@ -35,12 +38,16 @@ default void clearContext() { } } - default SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final String path) + default SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final String uriStr) throws IOException { - return getReadChannel(context, Paths.get(path)); + try { + return getReadChannel(context, new URI(uriStr)); + } catch (final URISyntaxException e) { + throw new UncheckedDeephavenException("Cannot convert path string to URI: " + uriStr, e); + } } - SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull Path path) throws IOException; + SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull URI uri) throws IOException; default SeekableByteChannel getWriteChannel(@NotNull final String path, final boolean append) throws IOException { return getWriteChannel(Paths.get(path), append); diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java index f7d5606a539..f1067347149 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java @@ -8,6 +8,7 @@ import org.junit.Test; import java.io.IOException; +import java.net.URI; import java.nio.ByteBuffer; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -172,8 +173,8 @@ public SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotN } @Override - public SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull Path path) { - return new TestMockChannel(count.getAndIncrement(), path.toString()); + public SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull URI uri) { + return new TestMockChannel(count.getAndIncrement(), uri.toString()); } @Override 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 caaa2eb2815..f04bc1b5bbc 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 @@ -54,7 +54,7 @@ import java.nio.file.attribute.BasicFileAttributes; import java.util.*; -import static io.deephaven.parquet.base.ParquetFileReader.S3_PARQUET_FILE_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetFileReader.S3_URI_SCHEME; import static io.deephaven.parquet.base.ParquetFileReader.convertToURI; import static io.deephaven.parquet.table.ParquetTableWriter.PARQUET_FILE_EXTENSION; import static io.deephaven.util.type.TypeUtils.getUnboxedTypeIfBoxed; @@ -618,7 +618,7 @@ public static void deleteTable(File path) { private static Table readTableInternal( @NotNull final URI source, @NotNull final ParquetInstructions instructions) { - if (source.getScheme() != null && source.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { + if (S3_URI_SCHEME.equals(source.getScheme())) { return readSingleFileTable(source, instructions); } final Path sourcePath = Path.of(source.getRawPath()); @@ -1139,10 +1139,15 @@ public static ParquetFileReader getParquetFileReaderChecked( public static ParquetFileReader getParquetFileReaderChecked( @NotNull final URI parquetFileURI, @NotNull final ParquetInstructions readInstructions) throws IOException { - if (parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { + if (S3_URI_SCHEME.equals(parquetFileURI.getScheme())) { + if (!(readInstructions.getSpecialInstructions() instanceof S3ParquetInstructions)) { + throw new IllegalArgumentException("Must provide S3ParquetInstructions to read files from S3"); + } + final S3ParquetInstructions s3Instructions = + (S3ParquetInstructions) readInstructions.getSpecialInstructions(); return new ParquetFileReader(parquetFileURI, new CachedChannelProvider( - new S3SeekableChannelProvider(parquetFileURI, readInstructions), 1 << 7)); + new S3SeekableChannelProvider(s3Instructions), 1 << 7)); } return new ParquetFileReader( parquetFileURI, diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java index 0e4de2f725c..017bc3487fd 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java @@ -3,6 +3,8 @@ import io.deephaven.annotations.BuildableStyle; import org.immutables.value.Value; +import java.time.Duration; + /** * This class provides instructions intended for reading and writing parquet files to AWS S3 instances. */ @@ -14,7 +16,9 @@ public abstract class S3ParquetInstructions { private final static int DEFAULT_READ_AHEAD_COUNT = 1; private final static int DEFAULT_FRAGMENT_SIZE = 512 << 20; // 5 MB private final static int MIN_FRAGMENT_SIZE = 8 << 10; // 8 KB - private final static int DEFAULT_MAX_CACHE_SIZE = 50; + 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 ImmutableS3ParquetInstructions.builder(); @@ -26,7 +30,7 @@ public static Builder builder() { public abstract String awsRegionName(); /** - * The maximum number of concurrent requests to make to S3. + * The maximum number of concurrent requests to make to S3, defaults to {@value #DEFAULT_MAX_CONCURRENT_REQUESTS}. */ @Value.Default public int maxConcurrentRequests() { @@ -34,7 +38,8 @@ public int maxConcurrentRequests() { } /** - * The number of fragments to send asynchronous read requests for while reading the current fragment. + * The number of fragments to send asynchronous read requests for while reading the current fragment, defaults to + * {@value #DEFAULT_READ_AHEAD_COUNT}. */ @Value.Default public int readAheadCount() { @@ -43,7 +48,7 @@ public int readAheadCount() { /** * The maximum size of each fragment to read from S3. The fetched fragment can be smaller than this in case fewer - * bytes remaining in the file. + * bytes remaining in the file, defaults to {@value #DEFAULT_FRAGMENT_SIZE} bytes. */ @Value.Default public int fragmentSize() { @@ -51,13 +56,31 @@ public int fragmentSize() { } /** - * The maximum number of fragments to cache in memory. + * The maximum number of fragments to cache in memory, defaults to {@value #DEFAULT_MAX_CACHE_SIZE}. */ @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; + } + + @Value.Check final void boundsCheckMaxConcurrentRequests() { if (maxConcurrentRequests() < 1) { @@ -82,7 +105,8 @@ final void boundsCheckMaxFragmentSize() { @Value.Check final void boundsCheckMaxCacheSize() { if (maxCacheSize() < readAheadCount() + 1) { - throw new IllegalArgumentException("maxCacheSize(=" + maxCacheSize() + ") must be >= 1 + readAheadCount"); + throw new IllegalArgumentException("maxCacheSize(=" + maxCacheSize() + ") must be >= 1 + " + + "readAheadCount(=" + readAheadCount() + ")"); } } @@ -97,6 +121,10 @@ public interface Builder { Builder maxCacheSize(int maxCacheSize); + Builder connectionTimeout(Duration connectionTimeout); + + Builder readTimeout(Duration connectionTimeout); + S3ParquetInstructions build(); } } 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 e9699e2deab..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,7 @@ 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; @@ -32,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 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 d677d20b144..f78373f805c 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 @@ -60,7 +60,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.S3_PARQUET_FILE_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetFileReader.S3_URI_SCHEME; import static io.deephaven.parquet.table.ParquetTableWriter.*; final class ParquetColumnLocation extends AbstractColumnLocation { @@ -167,7 +167,7 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition return null; } final URI parquetFileURI = tl().getParquetFile(); - if (parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) { + if (S3_URI_SCHEME.equals(parquetFileURI.getScheme())) { throw new UncheckedDeephavenException("Parquet files in S3 are not expected to have indexing files"); } final File parquetFile = new File(parquetFileURI); 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 897017b7766..5e255e532b1 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 @@ -30,7 +30,7 @@ import java.util.*; import java.util.stream.IntStream; -import static io.deephaven.parquet.base.ParquetFileReader.S3_PARQUET_FILE_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetFileReader.S3_URI_SCHEME; public class ParquetTableLocation extends AbstractTableLocation { @@ -91,7 +91,7 @@ public ParquetTableLocation(@NotNull final TableKey tableKey, version = tableInfo.map(TableInfo::version).orElse(null); final String uriScheme = tableLocationKey.getURI().getScheme(); - if (uriScheme != null && uriScheme.equals(S3_PARQUET_FILE_URI_SCHEME)) { + if (S3_URI_SCHEME.equals(uriScheme)) { handleUpdate(computeIndex(), 0L); // TODO What should I put here? } else { handleUpdate(computeIndex(), new File(tableLocationKey.getURI().toString()).lastModified()); 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 c395212a5e2..09589c333e5 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 @@ -15,7 +15,7 @@ import java.io.File; import java.net.URI; -import static io.deephaven.parquet.base.ParquetFileReader.S3_PARQUET_FILE_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetFileReader.S3_URI_SCHEME; /** * {@link TableLocationFactory} for {@link ParquetTableLocation}s. @@ -34,8 +34,7 @@ public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final ParquetTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { final URI parquetFileURI = locationKey.getURI(); - if ((parquetFileURI.getScheme() != null && parquetFileURI.getScheme().equals(S3_PARQUET_FILE_URI_SCHEME)) - || new File(parquetFileURI.getPath()).exists()) { + if (S3_URI_SCHEME.equals(parquetFileURI.getScheme()) || new File(parquetFileURI.getPath()).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 d3ac2dac038..4bec4f8438c 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,11 +3,12 @@ */ package io.deephaven.parquet.table.location; +import io.deephaven.base.verify.Assert; +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; @@ -26,7 +27,7 @@ /** * {@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(); @@ -108,7 +109,7 @@ public synchronized boolean verifyFileReader() { return true; } try { - fileReader = ParquetTools.getParquetFileReaderChecked(parquetFileURI, readInstructions); + fileReader = ParquetTools.getParquetFileReaderChecked(uri, readInstructions); } catch (IOException e) { return false; } @@ -125,7 +126,7 @@ public synchronized ParquetFileReader getFileReader() { if (fileReader != null) { return fileReader; } - return fileReader = ParquetTools.getParquetFileReader(parquetFileURI, readInstructions); + return fileReader = ParquetTools.getParquetFileReader(uri, readInstructions); } /** @@ -189,8 +190,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(new File(parquetFileURI).getAbsoluteFile()); + return filePath == null || new File(filePath).getAbsoluteFile().toURI().equals(uri); }).toArray(); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java index af46463ce86..b9781648418 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java @@ -9,21 +9,13 @@ import java.nio.ByteBuffer; import java.util.concurrent.CompletableFuture; -/** - * An {@link AsyncResponseTransformer} that transforms a response into a {@link ByteBuffer}. - * This class is inspired from {@link software.amazon.awssdk.core.internal.async.ByteArrayAsyncResponseTransformer} - * but avoids a number of extra copies done by the former. - * - * @param POJO response type. - */ public final class ByteBufferAsyncResponseTransformer implements AsyncResponseTransformer { + private final int bufferSize; private volatile CompletableFuture cf; - private final ByteBuffer byteBuffer; ByteBufferAsyncResponseTransformer(final int bufferSize) { - // TODO Can be improved with a buffer pool - byteBuffer = ByteBuffer.allocate(bufferSize); + this.bufferSize = bufferSize; } @Override @@ -33,26 +25,27 @@ public CompletableFuture prepare() { } @Override - public void onResponse(ResponseT response) { + 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(SdkPublisher publisher) { - publisher.subscribe(new ByteBuferSubscriber(cf, byteBuffer)); + public void onStream(final SdkPublisher publisher) { + // TODO Can be improved with a buffer pool + publisher.subscribe(new ByteBufferSubscriber(cf, ByteBuffer.allocate(bufferSize))); } @Override - public void exceptionOccurred(Throwable throwable) { + public void exceptionOccurred(final Throwable throwable) { cf.completeExceptionally(throwable); } - final static class ByteBuferSubscriber implements Subscriber { + final static class ByteBufferSubscriber implements Subscriber { private final CompletableFuture resultFuture; private Subscription subscription; private final ByteBuffer byteBuffer; - ByteBuferSubscriber(CompletableFuture resultFuture, ByteBuffer byteBuffer) { + ByteBufferSubscriber(CompletableFuture resultFuture, ByteBuffer byteBuffer) { this.resultFuture = resultFuture; this.byteBuffer = byteBuffer; } @@ -83,7 +76,7 @@ public void onError(final Throwable throwable) { @Override public void onComplete() { - resultFuture.complete(byteBuffer); + resultFuture.complete(byteBuffer.asReadOnlyBuffer()); } } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java index 00f19a8245a..b4957b9f60d 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -2,16 +2,21 @@ import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.exceptions.CancellationException; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Uri; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import javax.annotation.Nullable; 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.security.InvalidParameterException; +import java.time.Duration; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -28,16 +33,10 @@ public final class S3SeekableByteChannel implements SeekableByteChannel, Seekabl private static final long CLOSED_SENTINEL = -1; - /** - * Maximum time to wait while fetching fragments from S3. - */ - static final Long CONNECTION_TIMEOUT_MINUTES = 1L; - static final TimeUnit CONNECTION_TIMEOUT_UNIT = TimeUnit.MINUTES; - /** * Context object used to store read-ahead buffers for efficiently reading from S3. */ - static final class ChannelContext implements SeekableChannelsProvider.ChannelContext { + static final class S3ChannelContext implements SeekableChannelsProvider.ChannelContext { /** * Used to store information related to a single fragment @@ -46,12 +45,12 @@ static class FragmentContext { /** * The index of the fragment in the object */ - private final int fragmentIndex; + private int fragmentIndex; /** * The future that will be completed with the fragment's bytes */ - private final CompletableFuture future; + private CompletableFuture future; private FragmentContext(final int fragmentIndex, final CompletableFuture future) { this.fragmentIndex = fragmentIndex; @@ -59,82 +58,111 @@ private FragmentContext(final int fragmentIndex, final CompletableFuture readAheadBuffers; + /** + * Used to cache recently fetched fragments for faster lookup + */ + private final List bufferCache; - ChannelContext(final int readAheadCount, final int maxCacheSize) { - if (maxCacheSize < 1 + readAheadCount) { - throw new InvalidParameterException("maxCacheSize must be >= 1 + readAheadCount"); - } - readAheadBuffers = new ArrayList<>(maxCacheSize); + private long size; + + S3ChannelContext(final int maxCacheSize) { + bufferCache = new ArrayList<>(maxCacheSize); for (int i = 0; i < maxCacheSize; i++) { - readAheadBuffers.add(null); + bufferCache.add(null); } + size = -1; } private int getIndex(final int fragmentIndex) { - return fragmentIndex % readAheadBuffers.size(); + return fragmentIndex % bufferCache.size(); } void setFragmentContext(final int fragmentIndex, final CompletableFuture future) { - readAheadBuffers.set(getIndex(fragmentIndex), new FragmentContext(fragmentIndex, future)); + final int cacheIdx = getIndex(fragmentIndex); + final FragmentContext cachedEntry = bufferCache.get(cacheIdx); + if (cachedEntry == null) { + bufferCache.set(cacheIdx, new FragmentContext(fragmentIndex, future)); + } else { + // We should not cache an already cached fragment + Assert.neq(cachedEntry.fragmentIndex, "cachedEntry.fragmentIndex", fragmentIndex, "fragmentIdx"); + + // Cancel any outstanding requests for this cached fragment + cachedEntry.future.cancel(true); + + // Reuse the existing entry + cachedEntry.fragmentIndex = fragmentIndex; + cachedEntry.future = future; + } + } + + /** + * Will return the {@link CompletableFuture} corresponding to provided fragment index if present in the cache, + * else will return {@code null} + */ + @Nullable + CompletableFuture getCachedFuture(final int fragmentIndex) { + final FragmentContext cachedFragment = bufferCache.get(getIndex(fragmentIndex)); + if (cachedFragment != null && cachedFragment.fragmentIndex == fragmentIndex) { + return cachedFragment.future; + } + return null; } - FragmentContext getFragmentContext(final int fragmentIndex) { - return readAheadBuffers.get(getIndex(fragmentIndex)); + long getSize() { + return size; + } + + void setSize(final long size) { + this.size = size; } @Override public void close() { - readAheadBuffers.clear(); + bufferCache.clear(); } } - private ChannelContext context; - - private volatile long position; - - private final S3AsyncClient s3Client; - private final String bucket, key; - private final int numFragmentsInObject; + private final S3AsyncClient s3AsyncClient; + private final String bucket; + private final String key; private final int fragmentSize; private final int readAheadCount; + private final Duration readTimeout; /** - * The size of the object in bytes + * The size of the object in bytes, fetched at the time of first read */ - private final long size; + private long size; + private int numFragmentsInObject; - /** - * The maximum time and units to wait while fetching an object - **/ - private final Long timeout; - private final TimeUnit timeUnit; + private S3ChannelContext context; + private long position; - S3SeekableByteChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, @NotNull final String bucket, - @NotNull final String key, @NotNull final S3AsyncClient s3Client, final long size, - final int fragmentSize, final int readAheadCount) { - this.position = 0; - this.bucket = bucket; - this.key = key; - this.s3Client = s3Client; - + S3SeekableByteChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, @NotNull final URI uri, + @NotNull final S3AsyncClient s3AsyncClient, final int fragmentSize, final int readAheadCount, + final Duration readTimeout) { + final S3Uri s3Uri = s3AsyncClient.utilities().parseUri(uri); + this.bucket = s3Uri.bucket().orElse(null); + this.key = s3Uri.key().orElse(null); + this.s3AsyncClient = s3AsyncClient; this.fragmentSize = fragmentSize; this.readAheadCount = readAheadCount; - this.timeout = CONNECTION_TIMEOUT_MINUTES; - this.timeUnit = CONNECTION_TIMEOUT_UNIT; - this.size = size; - this.numFragmentsInObject = (int) Math.ceil((double) size / fragmentSize); - this.context = (ChannelContext) context; + this.readTimeout = readTimeout; + Assert.instanceOf(context, "context", S3ChannelContext.class); + this.context = (S3ChannelContext) context; + this.size = -1; + this.position = 0; } @Override public void setContext(@Nullable final SeekableChannelsProvider.ChannelContext context) { // null context equivalent to clearing the context - if (context != null && !(context instanceof ChannelContext)) { - throw new IllegalArgumentException("context must be null or an instance of ChannelContext"); + if (context != null && !(context instanceof S3ChannelContext)) { + throw new IllegalArgumentException("context must be null or an instance of ChannelContext, provided context " + + " of class " + context.getClass().getName()); } - this.context = (ChannelContext) context; + this.context = (S3ChannelContext) context; } @Override @@ -145,30 +173,28 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep } final long localPosition = position; checkClosed(localPosition); + + // Fetch the file size if this is the first read + populateSize(); if (localPosition >= size) { // We are finished reading return -1; } - final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); - final int fragmentOffset = (int) (localPosition - (currFragmentIndex * fragmentSize)); - // Send async read request the current fragment, if it's not already in the cache - final ChannelContext.FragmentContext fragmentContext = context.getFragmentContext(currFragmentIndex); - final CompletableFuture fetchCurrFragment; - if (fragmentContext != null && fragmentContext.fragmentIndex == currFragmentIndex) { - fetchCurrFragment = fragmentContext.future; - } else { - fetchCurrFragment = computeFragmentFuture(currFragmentIndex); - context.setFragmentContext(currFragmentIndex, fetchCurrFragment); + final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); + CompletableFuture currFragmentFuture = context.getCachedFuture(currFragmentIndex); + if (currFragmentFuture == null) { + currFragmentFuture = computeFragmentFuture(currFragmentIndex); + context.setFragmentContext(currFragmentIndex, currFragmentFuture); } // Send async requests for read-ahead buffers and store them in the cache final int numFragmentsToLoad = Math.min(readAheadCount, numFragmentsInObject - currFragmentIndex - 1); for (int i = 0; i < numFragmentsToLoad; i++) { final int readAheadFragmentIndex = i + currFragmentIndex + 1; - final ChannelContext.FragmentContext readAheadFragmentContext = context.getFragmentContext(readAheadFragmentIndex); - if (readAheadFragmentContext == null || readAheadFragmentContext.fragmentIndex != readAheadFragmentIndex) { + final CompletableFuture readAheadFragmentFuture = context.getCachedFuture(readAheadFragmentIndex); + if (readAheadFragmentFuture == null) { context.setFragmentContext(readAheadFragmentIndex, computeFragmentFuture(readAheadFragmentIndex)); } } @@ -176,38 +202,55 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep // Wait till the current fragment is fetched final ByteBuffer currentFragment; try { - currentFragment = fetchCurrFragment.get(timeout, timeUnit).asReadOnlyBuffer(); - } catch (final InterruptedException | ExecutionException | TimeoutException | RuntimeException e) { - throw new UncheckedDeephavenException("Failed to fetch fragment " + currFragmentIndex + " at byte offset " - + fragmentOffset + " for file " + key + " in S3 bucket " + bucket, e); + currentFragment = currFragmentFuture.get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); + } catch (final InterruptedException | ExecutionException | TimeoutException e) { + final String operation = "fetching fragment " + currFragmentIndex + " for file " + key + " in S3 bucket " + bucket; + handleS3Exception(e, operation); + throw new UncheckedDeephavenException("Exception caught while " + operation, e); } // 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 * fragmentSize)); currentFragment.position(fragmentOffset); final int limit = Math.min(currentFragment.remaining(), destination.remaining()); + final int originalBufferLimit = currentFragment.limit(); currentFragment.limit(currentFragment.position() + limit); destination.put(currentFragment); + // Need to reset buffer limit so we can read from the same buffer again in future + currentFragment.limit(originalBufferLimit); position = localPosition + limit; return limit; } private int fragmentIndexForByteNumber(final long byteNumber) { - return Math.toIntExact(Math.floorDiv(byteNumber, (long) fragmentSize)); + return Math.toIntExact(byteNumber / fragmentSize); } private CompletableFuture computeFragmentFuture(final int fragmentIndex) { final long readFrom = (long) fragmentIndex * fragmentSize; final long readTo = Math.min(readFrom + fragmentSize, size) - 1; final String range = "bytes=" + readFrom + "-" + readTo; - return s3Client.getObject(builder -> builder.bucket(bucket).key(key).range(range), + return s3AsyncClient.getObject(builder -> builder.bucket(bucket).key(key).range(range), new ByteBufferAsyncResponseTransformer<>(fragmentSize)); } + private void handleS3Exception(final Exception e, final String operationDescription) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + throw new CancellationException("Thread interrupted while " + operationDescription, e); + } else if (e instanceof ExecutionException) { + throw new UncheckedDeephavenException("Execution exception occurred while " + operationDescription, e); + } else if (e instanceof TimeoutException) { + throw new UncheckedDeephavenException("Operation timeout while " + operationDescription + " after waiting " + + "for duration " + readTimeout, e); + } + } + @Override public int write(final ByteBuffer src) throws IOException { - throw new UnsupportedOperationException("Don't support writing to S3 yet"); + throw new NonWritableChannelException(); } @Override @@ -230,12 +273,33 @@ public SeekableByteChannel position(final long newPosition) throws ClosedChannel @Override public long size() throws ClosedChannelException { checkClosed(position); + if (size < 0){ + populateSize(); + } return size; } + private void populateSize() { + if (context.getSize() < 0) { + // Fetch the size of the file on the first read using a blocking HEAD request + final HeadObjectResponse headObjectResponse; + try { + headObjectResponse = s3AsyncClient.headObject(builder -> builder.bucket(bucket).key(key)) + .get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); + } catch (final InterruptedException | ExecutionException | TimeoutException e) { + final String operation = "fetching HEAD for file " + key + " in S3 bucket " + bucket; + handleS3Exception(e, operation); + throw new UncheckedDeephavenException("Exception caught while " + operation, e); + } + context.setSize(headObjectResponse.contentLength()); + } + this.size = context.getSize(); + this.numFragmentsInObject = (int) ((size + fragmentSize - 1) / fragmentSize); // = ceil(size / fragmentSize) + } + @Override public SeekableByteChannel truncate(final long size) { - throw new UnsupportedOperationException("Currently not supported"); + throw new NonWritableChannelException(); } @Override diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java index 24e8e1f8da1..cc9ea4beb32 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java @@ -1,15 +1,11 @@ package io.deephaven.parquet.table.util; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.base.verify.Assert; import io.deephaven.parquet.base.util.SeekableChannelsProvider; -import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.S3ParquetInstructions; 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.services.s3.model.HeadObjectResponse; import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient; import java.io.IOException; @@ -19,11 +15,6 @@ import java.time.Duration; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; - -import static io.deephaven.parquet.table.util.S3SeekableByteChannel.CONNECTION_TIMEOUT_MINUTES; -import static io.deephaven.parquet.table.util.S3SeekableByteChannel.CONNECTION_TIMEOUT_UNIT; /** * {@link SeekableChannelsProvider} implementation that is used to fetch objects from AWS S3 instances. @@ -31,61 +22,33 @@ public final class S3SeekableChannelProvider implements SeekableChannelsProvider { private final S3AsyncClient s3AsyncClient; - - /** - * Parquet file {@link URI} stored as a {@link Path} to save on parsing time at the time of constructing new read - * channels. Note that this conversion is lossy, as the {@link URI} will contain characters "s3://" but {@link Path} - * will have "s3:/". - */ - private final Path parquetFilePath; - private final String bucket, key; - private final long size; private final int fragmentSize; private final int maxCacheSize; private final int readAheadCount; + private final Duration readTimeout; private final Map contextMap = new HashMap<>(); // TODO Remove this - - public S3SeekableChannelProvider(final URI parquetFileURI, final ParquetInstructions readInstructions) { - if (!(readInstructions.getSpecialInstructions() instanceof S3ParquetInstructions)) { - throw new IllegalArgumentException("Must provide S3ParquetInstructions to read files from S3"); - } - final S3ParquetInstructions s3Instructions = (S3ParquetInstructions) readInstructions.getSpecialInstructions(); + public S3SeekableChannelProvider(final S3ParquetInstructions s3Instructions) { final String awsRegionName = s3Instructions.awsRegionName(); final int maxConcurrentRequests = s3Instructions.maxConcurrentRequests(); final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() .maxConcurrency(maxConcurrentRequests) - .connectionTimeout(Duration.ofMinutes(CONNECTION_TIMEOUT_MINUTES)) + .connectionTimeout(s3Instructions.connectionTimeout()) .build(); this.s3AsyncClient = S3AsyncClient.builder() .region(Region.of(awsRegionName)) .httpClient(asyncHttpClient) .build(); - this.fragmentSize = s3Instructions.fragmentSize(); this.maxCacheSize = s3Instructions.maxCacheSize(); this.readAheadCount = s3Instructions.readAheadCount(); - - this.parquetFilePath = Path.of(parquetFileURI.toString()); - this.bucket = parquetFileURI.getHost(); - this.key = parquetFileURI.getPath().substring(1); - // Send a blocking HEAD request to S3 to get the size of the file - final HeadObjectResponse headObjectResponse; - try { - headObjectResponse = s3AsyncClient.headObject( - builder -> builder.bucket(bucket).key(key)) - .get(CONNECTION_TIMEOUT_MINUTES, CONNECTION_TIMEOUT_UNIT); - } catch (final InterruptedException | ExecutionException | TimeoutException | RuntimeException e) { - throw new UncheckedDeephavenException("Failed to fetch HEAD for file " + parquetFileURI, e); - } - this.size = headObjectResponse.contentLength(); + this.readTimeout = s3Instructions.readTimeout(); } @Override public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, - @NotNull final Path path) { - Assert.equals(parquetFilePath, "parquetFilePath", path, "path"); - return new S3SeekableByteChannel(context, bucket, key, s3AsyncClient, size, fragmentSize, readAheadCount); + @NotNull final URI uri) { + return new S3SeekableByteChannel(context, uri, s3AsyncClient, fragmentSize, readAheadCount, readTimeout); } @Override @@ -100,7 +63,7 @@ public ChannelContext makeContext() { if (contextMap.containsKey(tid)) { return contextMap.get(tid); } - context = new S3SeekableByteChannel.ChannelContext(readAheadCount, maxCacheSize); + context = new S3SeekableByteChannel.S3ChannelContext(maxCacheSize); contextMap.put(tid, context); } return context; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java index 00820475ce3..84ab8ca01f5 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java @@ -12,6 +12,7 @@ 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; @@ -46,10 +47,10 @@ public ChannelContext makeContext() { } @Override - public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final Path path) + public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { // context is unused here because it is NULL - return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, path.toFile()); + return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, new File(uri)); } 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 c89bd82ecba..98e13420074 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 @@ -64,6 +64,7 @@ import java.io.Serializable; import java.math.BigDecimal; import java.math.BigInteger; +import java.time.Duration; import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; @@ -576,7 +577,9 @@ public void readLongParquetFileFromS3Test() { .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) .maxConcurrentRequests(50) - .maxCacheSize(50) + .maxCacheSize(32) + .connectionTimeout(Duration.ofSeconds(1)) + .readTimeout(Duration.ofSeconds(60)) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3ParquetInstructions) @@ -616,7 +619,9 @@ public void readRefParquetFileFromS3Test() { .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) .maxConcurrentRequests(50) - .maxCacheSize(50) + .maxCacheSize(32) + .connectionTimeout(Duration.ofSeconds(1)) + .readTimeout(Duration.ofSeconds(60)) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3ParquetInstructions) @@ -690,7 +695,8 @@ public void profileReadingFromS3() { .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) .maxConcurrentRequests(50) - .maxCacheSize(50) + .maxCacheSize(32) + .readTimeout(Duration.ofMinutes(5)) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3ParquetInstructions) @@ -712,7 +718,8 @@ public void profileReadingFromS3() { .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) .maxConcurrentRequests(50) - .maxCacheSize(50) + .maxCacheSize(32) + .readTimeout(Duration.ofMinutes(5)) .build(); final ParquetInstructions readInstructions2 = new ParquetInstructions.Builder() .setSpecialInstructions(s3ParquetInstructions2) @@ -764,7 +771,8 @@ public void readParquetFileFromS3Test() { .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) .maxConcurrentRequests(50) - .maxCacheSize(50) + .maxCacheSize(32) + .readTimeout(Duration.ofSeconds(60)) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3ParquetInstructions) @@ -779,10 +787,17 @@ public void readParquetFileFromS3Test() { final Table dhTable2 = TableTools.emptyTable(5).update("A=(int)i"); assertTableEquals(fromAws2, dhTable2); - final Table fromAws3 = - ParquetTools.readTable("s3://dh-s3-parquet-test1/single col file with spaces in name.parquet", - readInstructions).select(); + 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 @@ -862,7 +877,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()
@@ -1147,7 +1162,7 @@ private void writingParquetFilesWithSpacesInNameHelper(final Table table, final
         assertTableEquals(table, fromDisk);
         FileUtils.deleteRecursively(parentDir);
 
-        final String destRelativePathStr = parentDirName + "/" + parquetFileName;
+        final String destRelativePathStr = rootFile.getName() + "/" + parentDirName + "/" + parquetFileName;
         ParquetTools.writeTable(table, destRelativePathStr);
         fromDisk = readSingleFileTable(destRelativePathStr, ParquetInstructions.EMPTY);
         assertTableEquals(table, fromDisk);
diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py
index ceb2ef38cad..8fd2b234e44 100644
--- a/py/server/deephaven/parquet.py
+++ b/py/server/deephaven/parquet.py
@@ -21,29 +21,19 @@
 _JParquetInstructions = jpy.get_type("io.deephaven.parquet.table.ParquetInstructions")
 _JS3ParquetInstructions = jpy.get_type("io.deephaven.parquet.table.S3ParquetInstructions")
 _JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition")
+_JDuration = jpy.get_type("java.time.Duration")
 
 def _build_s3_parquet_instructions(
-        aws_region_name: str,  # TODO This is a required parameter, so is this okay?
+        aws_region_name: str,
         max_concurrent_requests: int = None,
         read_ahead_count: int = None,
         fragment_size: int = None,
         max_cache_size: int = None,
+        connection_timeout: _JDuration = None,
+        read_timeout: _JDuration = None,
 ):
-    if not any(
-            [
-                aws_region_name,
-                max_concurrent_requests,
-                read_ahead_count,
-                fragment_size,
-                max_cache_size,
-            ]
-    ):
-        return None
-
     builder = _JS3ParquetInstructions.builder()
-
-    if aws_region_name:
-        builder.awsRegionName(aws_region_name)
+    builder.awsRegionName(aws_region_name)
 
     if max_concurrent_requests is not None:
         builder.maxConcurrentRequests(max_concurrent_requests)
@@ -57,6 +47,12 @@ def _build_s3_parquet_instructions(
     if max_cache_size is not None:
         builder.maxCacheSize(max_cache_size)
 
+    if connection_timeout is not None:
+        builder.connectionTimeout(connection_timeout)
+
+    if read_timeout is not None:
+        builder.readTimeout(read_timeout)
+
     return builder.build()
 
 
@@ -85,6 +81,8 @@ def _build_parquet_instructions(
         read_ahead_count: int = None,
         fragment_size: int = None,
         max_cache_size: int = None,
+        connection_timeout: _JDuration = None,
+        read_timeout: _JDuration = None,
 ):
     if not any(
             [
@@ -101,6 +99,8 @@ def _build_parquet_instructions(
                 read_ahead_count is not None,
                 fragment_size is not None,
                 max_cache_size is not None,
+                connection_timeout is not None,
+                read_timeout is not None,
             ]
     ):
         return None
@@ -144,6 +144,8 @@ def _build_parquet_instructions(
             read_ahead_count=read_ahead_count,
             fragment_size=fragment_size,
             max_cache_size=max_cache_size,
+            connection_timeout=connection_timeout,
+            read_timeout=read_timeout,
         )
         builder.setSpecialInstructions(s3_parquet_instructions)
 
@@ -195,6 +197,8 @@ def read(
         read_ahead_count: int = None,
         fragment_size: int = None,
         max_cache_size: int = None,
+        connection_timeout: _JDuration = None,
+        read_timeout: _JDuration = None,
 ) -> Table:
     """ Reads in a table from a single parquet, metadata file, or directory with recognized layout.
 
@@ -212,7 +216,19 @@ def read(
             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.
         aws_region_name (str): the AWS region name for reading parquet files stored in AWS S3, by default None
-        TODO Add docstrings for the more parameters
+        max_concurrent_requests (int): the maximum number of concurrent requests for reading parquet files stored in S3,
+            by default 50.
+        read_ahead_count (int): the number of fragments to send asynchronous read requests for while reading the current
+            fragment, defaults to 1.
+        fragment_size (int): the maximum size of each fragment to read from S3. The fetched fragment can be smaller than
+            this in case fewer bytes remaining in the file, defaults to 5 MB.
+        max_cache_size (int): the maximum number of fragments to cache in memory, defaults to 32.
+        connection_timeout (Duration): the amount of time to wait when initially establishing a connection before giving
+            up and timing out, defaults to 2 seconds.
+        read_timeout (Duration): the amount of time to wait when reading a fragment before giving up and timing out,
+            defaults to 2 seconds
+        # TODO Make sure all the defaults are correct
+
     Returns:
         a table
 
@@ -232,6 +248,8 @@ def read(
             read_ahead_count=read_ahead_count,
             fragment_size=fragment_size,
             max_cache_size=max_cache_size,
+            connection_timeout=connection_timeout,
+            read_timeout=read_timeout,
         )
         j_table_definition = _j_table_definition(table_definition)
         if j_table_definition is not None:
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);
+    }
+}

From 9aa75497fb84332fd7d7c20960e8e1155708727e Mon Sep 17 00:00:00 2001
From: Shivam Malhotra 
Date: Tue, 9 Jan 2024 18:29:38 +0530
Subject: [PATCH 12/39] Code review with Devin part 2

---
 .../parquet/base/ParquetFileReader.java       | 38 +++++---
 .../deephaven/parquet/table/ParquetTools.java | 94 ++++++++++++-------
 ...tInstructions.java => S3Instructions.java} |  8 +-
 .../table/util/S3SeekableByteChannel.java     | 23 ++---
 .../table/util/S3SeekableChannelProvider.java | 11 +--
 .../table/ParquetTableReadWriteTest.java      | 48 +++++++---
 py/server/deephaven/parquet.py                |  4 +-
 7 files changed, 140 insertions(+), 86 deletions(-)
 rename extensions/parquet/table/src/main/java/io/deephaven/parquet/table/{S3ParquetInstructions.java => S3Instructions.java} (95%)

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 db417a919f9..2a6504f4d6c 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
@@ -31,7 +31,7 @@ public class ParquetFileReader {
     private static final String MAGIC_STR = "PAR1";
     static final byte[] MAGIC = MAGIC_STR.getBytes(StandardCharsets.US_ASCII);
     public static final String S3_URI_SCHEME = "s3";
-    private static final String S3_URI_PREFIX = "s3://";
+    public static final String FILE_URI_SCHEME = "file";
 
     public final FileMetaData fileMetaData;
     private final SeekableChannelsProvider channelsProvider;
@@ -42,24 +42,32 @@ public class ParquetFileReader {
     private final URI rootURI;
     private final MessageType type;
 
-    // TODO Where should I keep it?
-    public static URI convertToURI(final String filePath) {
-        if (filePath.startsWith(S3_URI_PREFIX)) {
-            try {
-                return new URI(filePath);
-            } catch (final URISyntaxException e) {
-                throw new UncheckedDeephavenException("Failed to convert file path " + filePath + " to URI, we expect "
-                        + "CLI-style URIs, e.g., \"s3://bucket/key\" as input", e);
-            }
-        } else {
-            // Resolve to get an absolute file path and convert to URI
-            return new File(filePath).getAbsoluteFile().toURI();
+    // TODO Where should I keep this method?
+    /**
+     * Take the parquet file source path or URI and convert it to a URI object.
+     *
+     * @param source The parquet file source path or URI
+     * @return The URI object
+     */
+    public static URI convertToURI(final String source) {
+        final URI ret;
+        try {
+            ret = new URI(source);
+        } catch (final URISyntaxException e) {
+            throw new UncheckedDeephavenException("Failed to convert source string " + source + " to URI, we expect "
+                    + " either local file paths or CLI-style S3 URIs, e.g., \"s3://bucket/key\" as input", e);
+        }
+        final String scheme = ret.getScheme();
+        if (scheme != null && !scheme.isEmpty()) {
+            return ret;
         }
+        // Resolve to get an absolute file path and convert to URI
+        return new File(ret.getPath()).getAbsoluteFile().toURI();
     }
 
-    public ParquetFileReader(final String filePath, final SeekableChannelsProvider channelsProvider)
+    public ParquetFileReader(final String source, final SeekableChannelsProvider channelsProvider)
             throws IOException {
-        this(convertToURI(filePath), channelsProvider);
+        this(convertToURI(source), channelsProvider);
     }
 
     public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvider channelsProvider)
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 f04bc1b5bbc..fc4421ec37c 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
@@ -54,6 +54,7 @@
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.*;
 
+import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME;
 import static io.deephaven.parquet.base.ParquetFileReader.S3_URI_SCHEME;
 import static io.deephaven.parquet.base.ParquetFileReader.convertToURI;
 import static io.deephaven.parquet.table.ParquetTableWriter.PARQUET_FILE_EXTENSION;
@@ -72,7 +73,8 @@ 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, metadata file, or directory with recognized layout. The source provided
+     * can be a local file path or can be a CLI-style AWS S3 URI, e.g., "s3://bucket/key".
      *
      * 

* This method attempts to "do the right thing." It examines the source to determine if it's a single parquet file, @@ -81,24 +83,26 @@ 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(convertToURI(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, metadata file, or directory with recognized layout. The source provided + * can be a local file path or can be a CLI-style AWS S3 URI, e.g., "s3://bucket/key". * *

* This method attempts to "do the right thing." It examines the source to determine if it's a single parquet file, @@ -112,7 +116,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 @@ -121,9 +125,9 @@ 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(convertToURI(sourceFilePath), readInstructions); + return readTableInternal(convertToURI(source), readInstructions); } /** @@ -149,7 +153,7 @@ public static Table readTable( * @see ParquetFlatPartitionedLayout */ public static Table readTable(@NotNull final File sourceFile) { - return readTableInternal(sourceFile.toURI(), ParquetInstructions.EMPTY); + return readTableInternal(sourceFile, ParquetInstructions.EMPTY); } /** @@ -178,7 +182,7 @@ public static Table readTable(@NotNull final File sourceFile) { public static Table readTable( @NotNull final File sourceFile, @NotNull final ParquetInstructions readInstructions) { - return readTableInternal(sourceFile.toURI(), readInstructions); + return readTableInternal(sourceFile, readInstructions); } /** @@ -606,22 +610,19 @@ public static void deleteTable(File path) { * key} order) location found will be used to infer schema. * *

- * Delegates to one of {@link #readSingleFileTable(URI, ParquetInstructions)}, + * Delegates to one of {@link #readSingleFileTable(File, ParquetInstructions)}, * {@link #readPartitionedTableWithMetadata(File, ParquetInstructions)}, * {@link #readFlatPartitionedTable(File, ParquetInstructions)}, or * {@link #readKeyValuePartitionedTable(File, ParquetInstructions)}. * - * @param source The URI for source file or directory + * @param source The source file or directory * @param instructions Instructions for reading * @return A {@link Table} */ private static Table readTableInternal( - @NotNull final URI source, + @NotNull final File source, @NotNull final ParquetInstructions instructions) { - if (S3_URI_SCHEME.equals(source.getScheme())) { - return readSingleFileTable(source, instructions); - } - final Path sourcePath = Path.of(source.getRawPath()); + final Path sourcePath = source.toPath(); if (!Files.exists(sourcePath)) { throw new TableDataException("Source file " + source + " does not exist"); } @@ -671,6 +672,27 @@ private static Table readTableInternal( 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) { + final String scheme = source.getScheme(); + if (scheme != null && !scheme.isEmpty() && !scheme.equals(FILE_URI_SCHEME)) { + if (!scheme.equals(S3_URI_SCHEME)) { + throw new IllegalArgumentException( + "We only support reading single parquet file URI hosted on S3, but got " + source); + } + 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) != '.'; @@ -937,7 +959,7 @@ 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, ParquetInstructions) + * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) */ public static Table readSingleFileTable( @@ -947,21 +969,22 @@ public static Table readSingleFileTable( } /** - * Creates a single table via the parquet {@code filePath} using the provided {@code 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 can be a CLI-style AWS S3 URI, e.g., "s3://bucket/key". *

* Callers wishing to be more explicit (for example, to skip some columns) may prefer to call - * {@link #readSingleFileTable(File, ParquetInstructions, TableDefinition)}. + * {@link #readSingleFileTable(String, ParquetInstructions, TableDefinition)}. * - * @param filePath the parquet file path + * @param source the path or URI for the parquet file * @param readInstructions the instructions for customizations while reading * @return the table - * @see ParquetTableLocationKey#ParquetTableLocationKey(File, int, Map, ParquetInstructions) + * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) */ public static Table readSingleFileTable( - @NotNull final String filePath, + @NotNull final String source, @NotNull final ParquetInstructions readInstructions) { - return readSingleFileTable(convertToURI(filePath), readInstructions); + return readSingleFileTable(convertToURI(source), readInstructions); } private static Table readSingleFileTable( @@ -981,7 +1004,7 @@ private 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, ParquetInstructions) + * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) */ public static Table readSingleFileTable( @@ -992,21 +1015,21 @@ public static Table readSingleFileTable( } /** - * Creates a single table via the parquet {@code filePath} using the provided {@code tableDefinition}. API used by - * Python code. + * Creates a single table via the parquet {@code source} using the provided {@code tableDefinition}. The source + * provided can be a local file path or can be a CLI-style AWS S3 URI, e.g., "s3://bucket/key". * - * @param filePath the parquet file path + * @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(File, int, Map, ParquetInstructions) + * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) */ public static Table readSingleFileTable( - @NotNull final String filePath, + @NotNull final String source, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { - return readSingleFileTable(convertToURI(filePath), readInstructions, tableDefinition); + return readSingleFileTable(convertToURI(source), readInstructions, tableDefinition); } private static Table readSingleFileTable( @@ -1140,11 +1163,10 @@ public static ParquetFileReader getParquetFileReaderChecked( @NotNull final URI parquetFileURI, @NotNull final ParquetInstructions readInstructions) throws IOException { if (S3_URI_SCHEME.equals(parquetFileURI.getScheme())) { - if (!(readInstructions.getSpecialInstructions() instanceof S3ParquetInstructions)) { - throw new IllegalArgumentException("Must provide S3ParquetInstructions to read files from S3"); + if (!(readInstructions.getSpecialInstructions() instanceof S3Instructions)) { + throw new IllegalArgumentException("Must provide S3Instructions to read files from S3"); } - final S3ParquetInstructions s3Instructions = - (S3ParquetInstructions) readInstructions.getSpecialInstructions(); + final S3Instructions s3Instructions = (S3Instructions) readInstructions.getSpecialInstructions(); return new ParquetFileReader(parquetFileURI, new CachedChannelProvider( new S3SeekableChannelProvider(s3Instructions), 1 << 7)); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3Instructions.java similarity index 95% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java rename to extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3Instructions.java index 017bc3487fd..0ce489f3e8c 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3ParquetInstructions.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3Instructions.java @@ -6,11 +6,11 @@ import java.time.Duration; /** - * This class provides instructions intended for reading and writing parquet files to AWS S3 instances. + * This class provides instructions intended for reading and writing data to AWS S3 instances. */ @Value.Immutable @BuildableStyle -public abstract class S3ParquetInstructions { +public abstract class S3Instructions { private final static int DEFAULT_MAX_CONCURRENT_REQUESTS = 50; private final static int DEFAULT_READ_AHEAD_COUNT = 1; @@ -21,7 +21,7 @@ public abstract class S3ParquetInstructions { private final static Duration DEFAULT_READ_TIMEOUT = Duration.ofSeconds(2); public static Builder builder() { - return ImmutableS3ParquetInstructions.builder(); + return ImmutableS3Instructions.builder(); } /** @@ -125,6 +125,6 @@ public interface Builder { Builder readTimeout(Duration connectionTimeout); - S3ParquetInstructions build(); + S3Instructions build(); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java index b4957b9f60d..c5e2f949d94 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -203,10 +203,9 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep final ByteBuffer currentFragment; try { currentFragment = currFragmentFuture.get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); - } catch (final InterruptedException | ExecutionException | TimeoutException e) { + } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { final String operation = "fetching fragment " + currFragmentIndex + " for file " + key + " in S3 bucket " + bucket; - handleS3Exception(e, operation); - throw new UncheckedDeephavenException("Exception caught while " + operation, e); + throw handleS3Exception(e, operation); } // Copy the bytes from fragment from the offset up to the min of remaining fragment and destination bytes. @@ -236,16 +235,19 @@ private CompletableFuture computeFragmentFuture(final int fragmentIn new ByteBufferAsyncResponseTransformer<>(fragmentSize)); } - private void handleS3Exception(final Exception e, final String operationDescription) { + private UncheckedDeephavenException handleS3Exception(final Exception e, final String operationDescription) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); - throw new CancellationException("Thread interrupted while " + operationDescription, e); + return new CancellationException("Thread interrupted while " + operationDescription, e); } else if (e instanceof ExecutionException) { - throw new UncheckedDeephavenException("Execution exception occurred while " + operationDescription, e); + return new UncheckedDeephavenException("Execution exception occurred while " + operationDescription, e); } else if (e instanceof TimeoutException) { - throw new UncheckedDeephavenException("Operation timeout while " + operationDescription + " after waiting " + + return new UncheckedDeephavenException("Operation timeout while " + operationDescription + " after waiting " + "for duration " + readTimeout, e); + } else if (e instanceof CancellationException) { + return new UncheckedDeephavenException("Cancelled an operation while " + operationDescription, e); } + throw new UncheckedDeephavenException("Exception caught while " + operationDescription, e); } @Override @@ -286,12 +288,11 @@ private void populateSize() { try { headObjectResponse = s3AsyncClient.headObject(builder -> builder.bucket(bucket).key(key)) .get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); - } catch (final InterruptedException | ExecutionException | TimeoutException e) { + } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { final String operation = "fetching HEAD for file " + key + " in S3 bucket " + bucket; - handleS3Exception(e, operation); - throw new UncheckedDeephavenException("Exception caught while " + operation, e); + throw handleS3Exception(e, operation); } - context.setSize(headObjectResponse.contentLength()); + context.setSize(headObjectResponse.contentLength().longValue()); } this.size = context.getSize(); this.numFragmentsInObject = (int) ((size + fragmentSize - 1) / fragmentSize); // = ceil(size / fragmentSize) diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java index cc9ea4beb32..6988c8f5f54 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java @@ -1,7 +1,7 @@ package io.deephaven.parquet.table.util; import io.deephaven.parquet.base.util.SeekableChannelsProvider; -import io.deephaven.parquet.table.S3ParquetInstructions; +import io.deephaven.parquet.table.S3Instructions; import org.jetbrains.annotations.NotNull; import software.amazon.awssdk.http.async.SdkAsyncHttpClient; import software.amazon.awssdk.regions.Region; @@ -28,15 +28,14 @@ public final class S3SeekableChannelProvider implements SeekableChannelsProvider private final Duration readTimeout; private final Map contextMap = new HashMap<>(); // TODO Remove this - public S3SeekableChannelProvider(final S3ParquetInstructions s3Instructions) { - final String awsRegionName = s3Instructions.awsRegionName(); - final int maxConcurrentRequests = s3Instructions.maxConcurrentRequests(); + public S3SeekableChannelProvider(final S3Instructions s3Instructions) { final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() - .maxConcurrency(maxConcurrentRequests) + .maxConcurrency(Integer.valueOf(s3Instructions.maxConcurrentRequests())) .connectionTimeout(s3Instructions.connectionTimeout()) .build(); + // TODO Should we cache and reuse the client object? this.s3AsyncClient = S3AsyncClient.builder() - .region(Region.of(awsRegionName)) + .region(Region.of(s3Instructions.awsRegionName())) .httpClient(asyncHttpClient) .build(); this.fragmentSize = s3Instructions.fragmentSize(); 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 98e13420074..6e385ab7cfe 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 @@ -64,6 +64,7 @@ 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; @@ -572,7 +573,7 @@ public void testArrayColumns() { @Test public void readLongParquetFileFromS3Test() { - final S3ParquetInstructions s3ParquetInstructions = S3ParquetInstructions.builder() + final S3Instructions s3Instructions = S3Instructions.builder() .awsRegionName("us-east-2") .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) @@ -582,7 +583,7 @@ public void readLongParquetFileFromS3Test() { .readTimeout(Duration.ofSeconds(60)) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setSpecialInstructions(s3ParquetInstructions) + .setSpecialInstructions(s3Instructions) .build(); final TableDefinition tableDefinition = TableDefinition.of( @@ -614,7 +615,7 @@ public void readLongParquetFileFromS3Test() { @Test public void readRefParquetFileFromS3Test() { - final S3ParquetInstructions s3ParquetInstructions = S3ParquetInstructions.builder() + final S3Instructions s3Instructions = S3Instructions.builder() .awsRegionName("us-east-2") .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) @@ -624,7 +625,7 @@ public void readRefParquetFileFromS3Test() { .readTimeout(Duration.ofSeconds(60)) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setSpecialInstructions(s3ParquetInstructions) + .setSpecialInstructions(s3Instructions) .build(); final TableDefinition tableDefinition = TableDefinition.of( ColumnDefinition.ofString("hash"), @@ -690,7 +691,7 @@ public void readRefParquetFileLocally() { @Test public void profileReadingFromS3() { - final S3ParquetInstructions s3ParquetInstructions = S3ParquetInstructions.builder() + final S3Instructions s3Instructions = S3Instructions.builder() .awsRegionName("us-east-1") .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) @@ -699,7 +700,7 @@ public void profileReadingFromS3() { .readTimeout(Duration.ofMinutes(5)) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setSpecialInstructions(s3ParquetInstructions) + .setSpecialInstructions(s3Instructions) .build(); long totalTime = 0; @@ -713,7 +714,7 @@ public void profileReadingFromS3() { } System.out.println("Average execution time AWS is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); - final S3ParquetInstructions s3ParquetInstructions2 = S3ParquetInstructions.builder() + final S3Instructions s3Instructions2 = S3Instructions.builder() .awsRegionName("us-east-2") .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) @@ -722,7 +723,7 @@ public void profileReadingFromS3() { .readTimeout(Duration.ofMinutes(5)) .build(); final ParquetInstructions readInstructions2 = new ParquetInstructions.Builder() - .setSpecialInstructions(s3ParquetInstructions2) + .setSpecialInstructions(s3Instructions2) .build(); final TableDefinition tableDefinition = TableDefinition.of( ColumnDefinition.ofString("hash"), @@ -766,7 +767,7 @@ public void profileReadingFromS3() { @Test public void readParquetFileFromS3Test() { - final S3ParquetInstructions s3ParquetInstructions = S3ParquetInstructions.builder() + final S3Instructions s3Instructions = S3Instructions.builder() .awsRegionName("us-east-1") .readAheadCount(1) .fragmentSize(5 * 1024 * 1024) @@ -775,7 +776,7 @@ public void readParquetFileFromS3Test() { .readTimeout(Duration.ofSeconds(60)) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setSpecialInstructions(s3ParquetInstructions) + .setSpecialInstructions(s3Instructions) .build(); final Table fromAws1 = ParquetTools.readTable("s3://dh-s3-parquet-test1/multiColFile.parquet", readInstructions).select(); @@ -1063,6 +1064,27 @@ 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); + final Table fromDisk = ParquetTools.readTable(fileURI.toString()); + assertTableEquals(tableToSave, fromDisk); + final Table fromDisk2 = ParquetTools.readTable("file://" + absolutePath); + assertTableEquals(tableToSave, fromDisk2); + + try { + ParquetTools.readTable("https://" + absolutePath); + TestCase.fail("Exception expected for invalid scheme"); + } catch (final RuntimeException e) { + assertTrue(e instanceof IllegalArgumentException); + } + } + /** * These are tests for writing multiple parquet tables in a single call. */ @@ -1156,14 +1178,16 @@ private void writingParquetFilesWithSpacesInNameHelper(final Table table, final assertTableEquals(table, fromDisk); FileUtils.deleteRecursively(parentDir); - final String destAbsolutePathStr = dest.getAbsolutePath(); + String destAbsolutePathStr = dest.getAbsolutePath(); ParquetTools.writeTable(table, destAbsolutePathStr); + destAbsolutePathStr = destAbsolutePathStr.replace(" ", "%20"); fromDisk = readSingleFileTable(destAbsolutePathStr, ParquetInstructions.EMPTY); assertTableEquals(table, fromDisk); FileUtils.deleteRecursively(parentDir); - final String destRelativePathStr = rootFile.getName() + "/" + parentDirName + "/" + parquetFileName; + String destRelativePathStr = rootFile.getName() + "/" + parentDirName + "/" + parquetFileName; ParquetTools.writeTable(table, destRelativePathStr); + destRelativePathStr = destRelativePathStr.replace(" ", "%20"); fromDisk = readSingleFileTable(destRelativePathStr, ParquetInstructions.EMPTY); assertTableEquals(table, fromDisk); FileUtils.deleteRecursively(parentDir); diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index 8fd2b234e44..c127de70036 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -19,7 +19,7 @@ _JFile = jpy.get_type("java.io.File") _JCompressionCodecName = jpy.get_type("org.apache.parquet.hadoop.metadata.CompressionCodecName") _JParquetInstructions = jpy.get_type("io.deephaven.parquet.table.ParquetInstructions") -_JS3ParquetInstructions = jpy.get_type("io.deephaven.parquet.table.S3ParquetInstructions") +_JS3Instructions = jpy.get_type("io.deephaven.parquet.table.S3Instructions") _JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") _JDuration = jpy.get_type("java.time.Duration") @@ -32,7 +32,7 @@ def _build_s3_parquet_instructions( connection_timeout: _JDuration = None, read_timeout: _JDuration = None, ): - builder = _JS3ParquetInstructions.builder() + builder = _JS3Instructions.builder() builder.awsRegionName(aws_region_name) if max_concurrent_requests is not None: From b85043e7a4942f8220218f0deb130b07c1df25cc Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 9 Jan 2024 19:31:06 +0530 Subject: [PATCH 13/39] Integrated with Ryan's patch --- .../datastructures/LazyCachingFunction.java | 45 ++++++++++++ .../parquet/base/ColumnChunkReader.java | 33 +++++++-- .../parquet/base/ColumnChunkReaderImpl.java | 68 ++++++++++++------- .../parquet/base/ColumnPageReader.java | 17 +++++ .../parquet/base/ColumnPageReaderImpl.java | 33 +++++---- .../parquet/base/ParquetFileReader.java | 4 +- .../parquet/base/RowGroupReaderImpl.java | 4 +- .../table/pagestore/ColumnChunkPageStore.java | 57 +++++++++++++--- .../OffsetIndexBasedColumnChunkPageStore.java | 18 +++-- .../VariablePageSizeColumnChunkPageStore.java | 41 +++++++---- .../pagestore/topage/ChunkDictionary.java | 9 ++- .../table/pagestore/topage/ToObjectPage.java | 5 +- .../table/pagestore/topage/ToStringPage.java | 5 +- .../table/util/S3SeekableByteChannel.java | 8 +-- .../table/util/S3SeekableChannelProvider.java | 28 ++------ 15 files changed, 262 insertions(+), 113 deletions(-) create mode 100644 Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java 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..094072d8c7b --- /dev/null +++ b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java @@ -0,0 +1,45 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +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 suppliers that are + * safely repeatable and don't return {@code null}. + */ +public final class LazyCachingFunction implements Function { + + private final Function internalSupplier; + + private volatile SoftReference cachedResultRef; + + /** + * Construct a {@link Function} wrapper. + * + * @param internalSupplier The {@link Function} to wrap. Must be safely repeatable and must not return {@code null}. + */ + public LazyCachingFunction(@NotNull final Function internalSupplier) { + this.internalSupplier = internalSupplier; + } + + @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 = internalSupplier.apply(arg)); + } + return current; + } +} 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..5b19dfb13f3 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,16 @@ */ package io.deephaven.parquet.base; +import io.deephaven.parquet.base.util.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.NotNull; 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 { /** @@ -35,16 +37,34 @@ public interface ColumnChunkReader { @Nullable OffsetIndex getOffsetIndex(); + /** + * Used to iterate over column page readers for each page with the capability to set channel context to for reading + * the pages. + */ + interface ColumnPageReaderIterator extends Iterator { + + /** + * Set the {@code channelContext} to be used only for a single {@code next()} call. If not set, + * {@code SeekableChannelsProvider.ChannelContext.NULL} will be used + */ + void setChannelContext(final SeekableChannelsProvider.ChannelContext channelContext); + + default void clearChannelContext() { + setChannelContext(SeekableChannelsProvider.ChannelContext.NULL); + } + } + /** * @return An iterator over individual parquet pages */ - Iterator getPageIterator() throws IOException; + ColumnPageReaderIterator getPageIterator() throws IOException; interface ColumnPageDirectAccessor { /** * Directly access a page reader for a given page number. */ - ColumnPageReader getPageReader(final int pageNum); + ColumnPageReader getPageReader(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, + final int pageNum); } /** @@ -61,7 +81,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 +105,9 @@ public int getMaxId() { */ @Nullable String getVersion(); + + /** + * Create a new channel context for this column chunk reader. + */ + SeekableChannelsProvider.ChannelContext makeChannelContext(); } 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 005e5b7625b..52d3ecaa4b5 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 @@ -7,7 +7,7 @@ import io.deephaven.parquet.base.util.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; @@ -27,11 +27,9 @@ 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 org.apache.parquet.format.Encoding.PLAIN_DICTIONARY; import static org.apache.parquet.format.Encoding.RLE_DICTIONARY; @@ -48,7 +46,7 @@ public class ColumnChunkReaderImpl implements ColumnChunkReader { 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 URI uri; @@ -77,7 +75,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; @@ -103,7 +101,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()); @@ -154,12 +152,12 @@ public boolean usesDictionaryOnEveryPage() { } @Override - public Supplier getDictionarySupplier() { + public Function getDictionarySupplier() { return dictionarySupplier; } @NotNull - private Dictionary getDictionary() { + private Dictionary getDictionary(final SeekableChannelsProvider.ChannelContext channelContext) { final long dictionaryPageOffset; final ColumnMetaData chunkMeta = columnChunk.getMeta_data(); if (chunkMeta.isSetDictionary_page_offset()) { @@ -175,8 +173,7 @@ private Dictionary getDictionary() { } else { return NULL_DICTIONARY; } - try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), getURI())) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, getURI())) { readChannel.position(dictionaryPageOffset); return readDictionary(readChannel); } catch (IOException e) { @@ -194,6 +191,11 @@ public String getVersion() { return version; } + @Override + public SeekableChannelsProvider.ChannelContext makeChannelContext() { + return channelsProvider.makeContext(); + } + @NotNull private Dictionary readDictionary(ReadableByteChannel file) throws IOException { // explicitly not closing this, caller is responsible @@ -220,13 +222,20 @@ 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; + private SeekableChannelsProvider.ChannelContext channelContext; ColumnPageReaderIteratorImpl(final long startOffset, final long numValues) { this.remainingValues = numValues; this.currentOffset = startOffset; + this.channelContext = SeekableChannelsProvider.ChannelContext.NULL; + } + + @Override + public void setChannelContext(SeekableChannelsProvider.ChannelContext channelContext) { + this.channelContext = channelContext; } @Override @@ -240,8 +249,7 @@ public ColumnPageReader next() { 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(channelsProvider.makeContext(), getURI())) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, getURI())) { final long headerOffset = currentOffset; readChannel.position(currentOffset); // deliberately not closing this stream @@ -271,26 +279,36 @@ 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, getURI(), fieldTypes, readChannel.position(), pageHeader, - ColumnPageReaderImpl.NULL_NUM_VALUES); + : (SeekableChannelsProvider.ChannelContext context) -> NULL_DICTIONARY; + final ColumnPageReader nextReader = new ColumnPageReaderImpl(channelsProvider, decompressor, + pageDictionarySupplier, nullMaterializerFactory, path, getURI(), fieldTypes, + readChannel.position(), pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES, channelContext); + clearChannelContext(); + 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; + private SeekableChannelsProvider.ChannelContext channelContext; ColumnPageReaderIteratorIndexImpl() { pos = 0; + channelContext = SeekableChannelsProvider.ChannelContext.NULL; } + @Override + public void setChannelContext(SeekableChannelsProvider.ChannelContext channelContext) { + this.channelContext = channelContext; + } + + @Override public boolean hasNext() { return offsetIndex.getPageCount() > pos; @@ -307,11 +325,12 @@ 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, getURI(), fieldTypes, offsetIndex.getOffset(pos), null, - numValues); + numValues, channelContext); pos++; + clearChannelContext(); return columnPageReader; } } @@ -321,7 +340,8 @@ private final class ColumnPageDirectAccessorImpl implements ColumnPageDirectAcce ColumnPageDirectAccessorImpl() {} @Override - public ColumnPageReader getPageReader(final int pageNum) { + public ColumnPageReader getPageReader(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, + final int pageNum) { if (pageNum < 0 || pageNum >= offsetIndex.getPageCount()) { throw new IndexOutOfBoundsException( "pageNum=" + pageNum + ", offsetIndex.getPageCount()=" + offsetIndex.getPageCount()); @@ -329,7 +349,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, getURI(), fieldTypes, offsetIndex.getOffset(pageNum), null, - ColumnPageReaderImpl.NULL_NUM_VALUES); + ColumnPageReaderImpl.NULL_NUM_VALUES, channelContext); } } } 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..d29818d3cc3 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.parquet.base.util.SeekableChannelsProvider; import org.apache.parquet.column.Dictionary; import org.jetbrains.annotations.NotNull; @@ -52,4 +53,20 @@ default long numRows() throws IOException { */ @NotNull Dictionary getDictionary(); + + /** + * Set the channel context to use while reading the parquet file. This will cache a copy of the object for use + * across reads. Therefore, its is recommended to use {@link #clearChannelContext()} when done with the + * reader/context. + * + * @param context The channel context to use. + */ + void setChannelContext(@NotNull final SeekableChannelsProvider.ChannelContext context); + + /** + * Clear the cached channel context. + */ + default void clearChannelContext() { + setChannelContext(SeekableChannelsProvider.ChannelContext.NULL); + } } 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 c27a955b01e..e69920739eb 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 @@ -34,10 +34,9 @@ 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; @@ -50,7 +49,7 @@ 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 URI uri; @@ -64,6 +63,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { private PageHeader pageHeader; private int numValues; private int rowCount = -1; + private SeekableChannelsProvider.ChannelContext context; /** * Returns a {@link ColumnPageReader} object for reading the column page data from the file. @@ -84,14 +84,15 @@ public class ColumnPageReaderImpl implements ColumnPageReader { */ ColumnPageReaderImpl(SeekableChannelsProvider channelsProvider, CompressorAdapter compressorAdapter, - Supplier dictionarySupplier, + Function dictionarySupplier, PageMaterializer.Factory materializerFactory, ColumnDescriptor path, URI uri, List fieldTypes, long offset, PageHeader pageHeader, - int numValues) { + int numValues, + SeekableChannelsProvider.ChannelContext context) { this.channelsProvider = channelsProvider; this.compressorAdapter = compressorAdapter; this.dictionarySupplier = dictionarySupplier; @@ -102,20 +103,19 @@ public class ColumnPageReaderImpl implements ColumnPageReader { this.offset = offset; this.pageHeader = pageHeader; this.numValues = numValues; + this.context = context; } @Override public Object materialize(Object nullValue) throws IOException { - try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), uri)) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, uri)) { ensurePageHeader(readChannel); return readDataPage(nullValue, readChannel); } } public int readRowCount() throws IOException { - try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), uri)) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, uri)) { ensurePageHeader(readChannel); return readRowCountFromDataPage(readChannel); } @@ -124,8 +124,7 @@ public int readRowCount() throws IOException { @Override public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder) throws IOException { - try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), uri)) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, uri)) { ensurePageHeader(readChannel); return readKeyFromDataPage(keyDest, nullPlaceholder, readChannel); } @@ -600,7 +599,7 @@ private ValuesReader getDataReader(Encoding dataEncoding, ByteBuffer in, int val } ValuesReader dataReader; if (dataEncoding.usesDictionary()) { - final Dictionary dictionary = dictionarySupplier.get(); + final Dictionary dictionary = dictionarySupplier.apply(context); if (dictionary == ColumnChunkReader.NULL_DICTIONARY) { throw new ParquetDecodingException("Could not read page in col " + path + " as the dictionary was " + "missing for encoding " + dataEncoding); @@ -623,8 +622,7 @@ public int numValues() throws IOException { if (numValues >= 0) { return numValues; } - try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), uri)) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, uri)) { ensurePageHeader(readChannel); // Above will block till it populates numValues Assert.geqZero(numValues, "numValues"); @@ -635,7 +633,12 @@ public int numValues() throws IOException { @NotNull @Override public Dictionary getDictionary() { - return dictionarySupplier.get(); + return dictionarySupplier.apply(context); + } + + @Override + public void setChannelContext(@NotNull final SeekableChannelsProvider.ChannelContext context) { + this.context = context; } @Override 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 2a6504f4d6c..333a0d58c1e 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 @@ -85,9 +85,9 @@ public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvide } else { rootURI = parquetFileURI; } - final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); final byte[] footer; - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, parquetFileURI)) { + try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); + final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, parquetFileURI)) { final long fileLen = readChannel.size(); if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC + data + footer + // footerIndex + MAGIC 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 bab8938cfc3..a495fe4abfd 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 @@ -79,8 +79,8 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { OffsetIndex offsetIndex = null; if (columnChunk.isSetOffset_index_offset()) { - try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelsProvider.makeContext(), rootURI)) { + try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); + final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, rootURI)) { readChannel.position(columnChunk.getOffset_index_offset()); offsetIndex = ParquetMetadataConverter.fromParquetOffsetIndex(Util.readOffsetIndex( new BufferedInputStream(Channels.newInputStream(readChannel), BUFFER_SIZE))); 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..f059c3157e1 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,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Context; import io.deephaven.engine.table.SharedContext; +import io.deephaven.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.engine.table.Releasable; import io.deephaven.chunk.attributes.Any; @@ -171,17 +172,52 @@ 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); + /** + * Take an object of {@link PagingContextHolder} and populate the inner context with values from + * {@link #columnChunkReader} + */ + final void innerFillContext(@Nullable final FillContext context) { + if (context != null) { + ((PagingContextHolder) context).updateInnerContext(this::fillContextUpdater); + } + } + + /** + * Return true if this class allocated the current inner context and therefore the context can be reused for reading + * pages. + */ + private static boolean isFillContextCompatible(@Nullable final Context currentInnerContext) { + return currentInnerContext instanceof ChannelContextWrapper; + } + + static SeekableChannelsProvider.ChannelContext getChannelContext(@Nullable final FillContext fillContext) { + if (fillContext instanceof PagingContextHolder) { + final Context innerContext = ((PagingContextHolder) fillContext).getInnerContext(); + if (innerContext instanceof ChannelContextWrapper) { + return ((ChannelContextWrapper) innerContext).getChannelContext(); + } + } + return SeekableChannelsProvider.ChannelContext.NULL; } - 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; + /** + * Wrapper class which takes a {@link SeekableChannelsProvider.ChannelContext} and implements {@link FillContext} + */ + private static class ChannelContextWrapper implements FillContext { + private @NotNull final SeekableChannelsProvider.ChannelContext context; + + ChannelContextWrapper(@NotNull SeekableChannelsProvider.ChannelContext context) { + this.context = context; + } + + SeekableChannelsProvider.ChannelContext getChannelContext() { + return context; + } + + @Override + public void close() { + context.close(); + } } private T fillContextUpdater( @@ -191,7 +227,6 @@ private T fillContextUpdater( // noinspection unchecked return (T) (isFillContextCompatible(currentInnerContext) ? currentInnerContext - // TODO(deephaven-core#4836): Replace this with getting a context from this.ColumnChunkReader - : makeFillContext(chunkCapacity, sharedContext)); + : new ChannelContextWrapper(columnChunkReader.makeChannelContext())); } } 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..74611052d95 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.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; @@ -98,7 +99,8 @@ 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(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, + final int pageNum) { if (pageNum < 0 || pageNum >= numPages) { throw new IllegalArgumentException("pageNum " + pageNum + " is out of range [0, " + numPages + ")"); } @@ -115,15 +117,15 @@ 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. - final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(pageNum); + final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(channelContext, pageNum); try { page = new PageCache.IntrusivePage<>(toPage(offsetIndex.getFirstRowIndex(pageNum), reader)); } catch (final IOException except) { throw new UncheckedIOException(except); } pageState.pageRef = new WeakReference<>(page); + // Clear out the context to avoid retaining old copies + reader.clearChannelContext(); } } } @@ -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,10 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext pageNum = (numPages - 1); } } - return getPage(pageNum); + + // Use the latest context while reading the page + innerFillContext(fillContext); + final SeekableChannelsProvider.ChannelContext channelContext = getChannelContext(fillContext); + return getPage(channelContext, 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..c165b71d0f6 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.parquet.base.util.SeekableChannelsProvider; 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,12 @@ final class VariablePageSizeColumnChunkPageStore extends Colum pages = (WeakReference>[]) new WeakReference[INIT_ARRAY_SIZE]; } - private void extendOnePage(final int prevNumPages) { + private void extendOnePage(@NotNull final SeekableChannelsProvider.ChannelContext 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,18 +65,19 @@ 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); } + columnPageReaderIterator.setChannelContext(channelContext); final ColumnPageReader columnPageReader = columnPageReaderIterator.next(); long numRows; WeakReference> pageRef = PageCache.getNullPage(); - long prevRowOffset = pageRowOffsets[localNumPages]; + final long prevRowOffset = pageRowOffsets[localNumPages]; try { numRows = columnPageReader.numRows(); @@ -85,10 +87,12 @@ private void extendOnePage(final int prevNumPages) { pageRef = new WeakReference<>(page); numRows = page.getPage().size(); } - } catch (IOException except) { + } catch (final IOException except) { throw new UncheckedIOException(except); } + // Clear out the context to avoid retaining old copies + columnPageReader.clearChannelContext(); columnPageReaders[localNumPages] = columnPageReader; pages[localNumPages] = pageRef; pageRowOffsets[localNumPages + 1] = prevRowOffset + numRows; @@ -101,19 +105,21 @@ private void extendOnePage(final int prevNumPages) { } } - private int fillToRow(int minPageNum, long row) { + private int fillToRow(@NotNull final SeekableChannelsProvider.ChannelContext 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 SeekableChannelsProvider.ChannelContext channelContext, + final int pageNum) { PageCache.IntrusivePage page = pages[pageNum].get(); if (page == null) { @@ -123,8 +129,13 @@ private ChunkPage getPage(final int pageNum) { if (page == null) { try { + // Use the latest context while reading the page + final ColumnPageReader columnPageReader = columnPageReaders[pageNum]; + columnPageReader.setChannelContext(channelContext); page = new PageCache.IntrusivePage<>( toPage(pageRowOffsets[pageNum], columnPageReaders[pageNum])); + // Clear out the context to avoid retaining old copies + columnPageReader.clearChannelContext(); } catch (IOException except) { throw new UncheckedIOException(except); } @@ -144,7 +155,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 +164,12 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext pageNum = -2 - pageNum; } + // Use the latest channel context while reading page headers + innerFillContext(fillContext); + final SeekableChannelsProvider.ChannelContext channelContext = getChannelContext(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 +178,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..7e85674ea77 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,12 +6,14 @@ import gnu.trove.map.TObjectIntMap; import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.chunk.attributes.Any; +import io.deephaven.parquet.base.util.SeekableChannelsProvider; 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; /** @@ -35,7 +37,7 @@ public interface Lookup { } private final Lookup lookup; - private final Supplier dictionarySupplier; + private final Function dictionarySupplier; private final Supplier> valuesSupplier; private final Supplier> reverseMapSupplier; @@ -48,11 +50,12 @@ public interface Lookup { */ ChunkDictionary( @NotNull final Lookup lookup, - @NotNull final Supplier dictionarySupplier) { + @NotNull final Function dictionarySupplier) { this.lookup = lookup; this.dictionarySupplier = dictionarySupplier; this.valuesSupplier = new LazyCachingSupplier<>(() -> { - final Dictionary dictionary = dictionarySupplier.get(); + // Dictionary is already materialized till this point, therefore we can safely use NULL context + final Dictionary dictionary = dictionarySupplier.apply(SeekableChannelsProvider.ChannelContext.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..0a973753658 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.parquet.base.util.SeekableChannelsProvider; 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/ToStringPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringPage.java index b9a43add297..7ce3d819fa2 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.parquet.base.util.SeekableChannelsProvider; 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/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java index c5e2f949d94..9e8099ae723 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -1,3 +1,6 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ package io.deephaven.parquet.table.util; import io.deephaven.UncheckedDeephavenException; @@ -115,11 +118,6 @@ long getSize() { void setSize(final long size) { this.size = size; } - - @Override - public void close() { - bufferCache.clear(); - } } private final S3AsyncClient s3AsyncClient; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java index 6988c8f5f54..cc77e2a0206 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java @@ -1,3 +1,6 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ package io.deephaven.parquet.table.util; import io.deephaven.parquet.base.util.SeekableChannelsProvider; @@ -13,8 +16,6 @@ import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; import java.time.Duration; -import java.util.HashMap; -import java.util.Map; /** * {@link SeekableChannelsProvider} implementation that is used to fetch objects from AWS S3 instances. @@ -26,7 +27,6 @@ public final class S3SeekableChannelProvider implements SeekableChannelsProvider private final int maxCacheSize; private final int readAheadCount; private final Duration readTimeout; - private final Map contextMap = new HashMap<>(); // TODO Remove this public S3SeekableChannelProvider(final S3Instructions s3Instructions) { final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() @@ -52,21 +52,7 @@ public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelsProvide @Override public ChannelContext makeContext() { - final Long tid = Long.valueOf(Thread.currentThread().getId()); - if (contextMap.containsKey(tid)) { - return contextMap.get(tid); - } else { - final ChannelContext context; - // TODO Remove this part - synchronized (contextMap) { - if (contextMap.containsKey(tid)) { - return contextMap.get(tid); - } - context = new S3SeekableByteChannel.S3ChannelContext(maxCacheSize); - contextMap.put(tid, context); - } - return context; - } + return new S3SeekableByteChannel.S3ChannelContext(maxCacheSize); } @Override @@ -76,11 +62,5 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boole public void close() throws IOException { s3AsyncClient.close(); - synchronized (contextMap) { - for (final ChannelContext context : contextMap.values()) { - context.close(); - } - contextMap.clear(); - } } } From 0e5ff035818f20741612d9560d548d5624961b68 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Wed, 10 Jan 2024 17:32:02 +0530 Subject: [PATCH 14/39] Resolving more comments --- .../parquet/base/ColumnChunkReaderImpl.java | 21 +++++++++++++++---- .../parquet/base/ParquetFileReader.java | 13 +++--------- .../deephaven/parquet/table/ParquetTools.java | 16 ++++++++------ .../table/util/S3SeekableByteChannel.java | 2 ++ .../table/util/S3SeekableChannelProvider.java | 2 +- .../util/TrackedSeekableChannelsProvider.java | 6 +++++- .../table/ParquetTableReadWriteTest.java | 6 ++---- 7 files changed, 40 insertions(+), 26 deletions(-) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 52d3ecaa4b5..c6d5196900b 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 @@ -173,10 +173,23 @@ private Dictionary getDictionary(final SeekableChannelsProvider.ChannelContext c } else { return NULL_DICTIONARY; } - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, getURI())) { - readChannel.position(dictionaryPageOffset); - return readDictionary(readChannel); - } catch (IOException e) { + try { + if (channelContext == SeekableChannelsProvider.ChannelContext.NULL) { + // Create a new context object and use that for reading the dictionary + try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); + final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, getURI())) { + readChannel.position(dictionaryPageOffset); + return readDictionary(readChannel); + } + } else { + // Use the context object provided by the caller + try (final SeekableByteChannel readChannel = + channelsProvider.getReadChannel(channelContext, getURI())) { + readChannel.position(dictionaryPageOffset); + return readDictionary(readChannel); + } + } + } catch (final IOException e) { throw new UncheckedIOException(e); } } 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 333a0d58c1e..db3c775bd21 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 @@ -50,19 +50,12 @@ public class ParquetFileReader { * @return The URI object */ public static URI convertToURI(final String source) { - final URI ret; try { - ret = new URI(source); + return new URI(source); } catch (final URISyntaxException e) { - throw new UncheckedDeephavenException("Failed to convert source string " + source + " to URI, we expect " - + " either local file paths or CLI-style S3 URIs, e.g., \"s3://bucket/key\" as input", e); + // Assuming the source is a file path, resolve to get the absolute path and convert to URI + return new File(source).getAbsoluteFile().toURI(); } - final String scheme = ret.getScheme(); - if (scheme != null && !scheme.isEmpty()) { - return ret; - } - // Resolve to get an absolute file path and convert to URI - return new File(ret.getPath()).getAbsoluteFile().toURI(); } public ParquetFileReader(final String source, final SeekableChannelsProvider channelsProvider) 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 fc4421ec37c..7864bdc6940 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 @@ -7,6 +7,7 @@ import io.deephaven.base.ClassUtil; import io.deephaven.base.FileUtils; import io.deephaven.base.Pair; +import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.ColumnDefinition; @@ -74,7 +75,7 @@ private ParquetTools() {} /** * Reads in a table from a single parquet, metadata file, or directory with recognized layout. The source provided - * can be a local file path or can be a CLI-style AWS S3 URI, e.g., "s3://bucket/key". + * 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, @@ -102,7 +103,7 @@ public static Table readTable(@NotNull final String source) { /** * Reads in a table from a single parquet, metadata file, or directory with recognized layout. The source provided - * can be a local file path or can be a CLI-style AWS S3 URI, e.g., "s3://bucket/key". + * 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, @@ -683,14 +684,14 @@ private static Table readTableInternal( @NotNull final URI source, @NotNull final ParquetInstructions instructions) { final String scheme = source.getScheme(); - if (scheme != null && !scheme.isEmpty() && !scheme.equals(FILE_URI_SCHEME)) { + if (scheme != null && !scheme.equals(FILE_URI_SCHEME)) { if (!scheme.equals(S3_URI_SCHEME)) { throw new IllegalArgumentException( "We only support reading single parquet file URI hosted on S3, but got " + source); } return readSingleFileTable(source, instructions); } - return readTableInternal(new File(source), instructions); + return readTableInternal(new File(source.getPath()), instructions); } private static boolean ignoreDotFiles(Path path) { @@ -970,7 +971,7 @@ public static Table readSingleFileTable( /** * Creates a single table via the parquet {@code source} using the provided {@code tableDefinition}. The source - * provided can be a local file path or can be a CLI-style AWS S3 URI, e.g., "s3://bucket/key". + * 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)}. @@ -1016,7 +1017,7 @@ public static Table readSingleFileTable( /** * Creates a single table via the parquet {@code source} using the provided {@code tableDefinition}. The source - * provided can be a local file path or can be a CLI-style AWS S3 URI, e.g., "s3://bucket/key". + * 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 @@ -1171,6 +1172,9 @@ public static ParquetFileReader getParquetFileReaderChecked( new CachedChannelProvider( new S3SeekableChannelProvider(s3Instructions), 1 << 7)); } + Assert.assertion(parquetFileURI.getScheme() == null + || parquetFileURI.getScheme().equals(ParquetFileReader.FILE_URI_SCHEME), + "Expected uri scheme to be null or \"file\", got uri as " + parquetFileURI); return new ParquetFileReader( parquetFileURI, new CachedChannelProvider( diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java index 9e8099ae723..a718247cae0 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java @@ -166,6 +166,8 @@ public void setContext(@Nullable final SeekableChannelsProvider.ChannelContext c @Override public int read(@NotNull final ByteBuffer destination) throws ClosedChannelException { Assert.neqNull(context, "context"); + Assert.neq(context, "context", SeekableChannelsProvider.ChannelContext.NULL, + "SeekableChannelsProvider.ChannelContext.NULL"); if (!destination.hasRemaining()) { return 0; } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java index cc77e2a0206..44dbb438c89 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java @@ -30,7 +30,7 @@ public final class S3SeekableChannelProvider implements SeekableChannelsProvider public S3SeekableChannelProvider(final S3Instructions s3Instructions) { final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() - .maxConcurrency(Integer.valueOf(s3Instructions.maxConcurrentRequests())) + .maxConcurrency(s3Instructions.maxConcurrentRequests()) .connectionTimeout(s3Instructions.connectionTimeout()) .build(); // TODO Should we cache and reuse the client object? diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java index 84ab8ca01f5..ab246a9c82b 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java @@ -3,10 +3,12 @@ */ package io.deephaven.parquet.table.util; +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.ParquetFileReader; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; @@ -50,7 +52,9 @@ public ChannelContext makeContext() { public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { // context is unused here because it is NULL - return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, new File(uri)); + Assert.assertion(uri.getScheme() == null || uri.getScheme().equals(ParquetFileReader.FILE_URI_SCHEME), + "Expected uri scheme to be null or \"file\", got uri as " + uri); + return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, new File(uri.getPath())); } 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 6e385ab7cfe..c85f30f36be 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 @@ -1178,16 +1178,14 @@ private void writingParquetFilesWithSpacesInNameHelper(final Table table, final assertTableEquals(table, fromDisk); FileUtils.deleteRecursively(parentDir); - String destAbsolutePathStr = dest.getAbsolutePath(); + final String destAbsolutePathStr = dest.getAbsolutePath(); ParquetTools.writeTable(table, destAbsolutePathStr); - destAbsolutePathStr = destAbsolutePathStr.replace(" ", "%20"); fromDisk = readSingleFileTable(destAbsolutePathStr, ParquetInstructions.EMPTY); assertTableEquals(table, fromDisk); FileUtils.deleteRecursively(parentDir); - String destRelativePathStr = rootFile.getName() + "/" + parentDirName + "/" + parquetFileName; + final String destRelativePathStr = rootFile.getName() + "/" + parentDirName + "/" + parquetFileName; ParquetTools.writeTable(table, destRelativePathStr); - destRelativePathStr = destRelativePathStr.replace(" ", "%20"); fromDisk = readSingleFileTable(destRelativePathStr, ParquetInstructions.EMPTY); assertTableEquals(table, fromDisk); FileUtils.deleteRecursively(parentDir); From 98b6f31aed7868f0088515ea0955d3e44811e704 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Wed, 10 Jan 2024 22:27:06 +0530 Subject: [PATCH 15/39] Moved to a service loader style pattern for channel providers --- .../util/SeekableChannelsProviderLoader.java | 50 +++++++++++++++++++ .../util/SeekableChannelsProviderPlugin.java | 22 ++++++++ extensions/parquet/table/build.gradle | 5 +- .../plugin/channelprovider/s3/build.gradle | 20 ++++++++ .../channelprovider/s3/gradle.properties | 1 + .../ByteBufferAsyncResponseTransformer.java | 8 ++- .../s3}/S3SeekableByteChannel.java | 35 +++++++------ .../s3}/S3SeekableChannelProvider.java | 8 ++- .../s3/S3SeekableChannelProviderPlugin.java | 37 ++++++++++++++ .../channelprovider/tracked/build.gradle | 16 ++++++ .../channelprovider/tracked/gradle.properties | 1 + .../TrackedSeekableChannelsProvider.java | 2 +- ...kedSeekableChannelsProviderPluginImpl.java | 36 +++++++++++++ .../parquet/table/ParquetTableWriter.java | 2 +- .../deephaven/parquet/table/ParquetTools.java | 30 ++++------- server/build.gradle | 1 + server/jetty-app/build.gradle | 6 +++ settings.gradle | 6 +++ 18 files changed, 242 insertions(+), 44 deletions(-) create mode 100644 extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java create mode 100644 extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java create mode 100644 extensions/parquet/table/plugin/channelprovider/s3/build.gradle create mode 100644 extensions/parquet/table/plugin/channelprovider/s3/gradle.properties rename extensions/parquet/table/{src/main/java/io/deephaven/parquet/table/util => plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3}/ByteBufferAsyncResponseTransformer.java (92%) rename extensions/parquet/table/{src/main/java/io/deephaven/parquet/table/util => plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3}/S3SeekableByteChannel.java (90%) rename extensions/parquet/table/{src/main/java/io/deephaven/parquet/table/util => plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3}/S3SeekableChannelProvider.java (90%) create mode 100644 extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProviderPlugin.java create mode 100644 extensions/parquet/table/plugin/channelprovider/tracked/build.gradle create mode 100644 extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties rename extensions/parquet/table/{src/main/java/io/deephaven/parquet/table/util => plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked}/TrackedSeekableChannelsProvider.java (98%) create mode 100644 extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProviderPluginImpl.java diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java new file mode 100644 index 00000000000..e30ce663bca --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java @@ -0,0 +1,50 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.parquet.base.util; + +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<>(); + } + + public SeekableChannelsProvider fromServiceLoader(@NotNull final URI uri, @Nullable final Object object) { + if (providers.isEmpty()) { + // Load the plugins + for (final SeekableChannelsProviderPlugin plugin : ServiceLoader + .load(SeekableChannelsProviderPlugin.class)) { + providers.add(plugin); + } + } + for (final SeekableChannelsProviderPlugin plugin : providers) { + if (plugin.isCompatible(uri, object)) { + return plugin.impl(uri, object); + } + } + throw new UnsupportedOperationException("No plugin found for uri: " + uri); + } +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java new file mode 100644 index 00000000000..1409585ecbb --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java @@ -0,0 +1,22 @@ +package io.deephaven.parquet.base.util; + +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 final URI uri, @Nullable final Object config); + + /** + * Create a {@link SeekableChannelsProvider} for the given URI and config object. + */ + SeekableChannelsProvider impl(@NotNull final URI uri, @Nullable final Object object); +} diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 2e4dc8be216..30254f23e91 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') @@ -51,6 +51,9 @@ dependencies { project(path: ':test-configs') Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') + implementation project(':plugin-trackedseekablechannel') + testRuntimeOnly project(':plugin-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/plugin/channelprovider/s3/build.gradle b/extensions/parquet/table/plugin/channelprovider/s3/build.gradle new file mode 100644 index 00000000000..18b6b9eaf4e --- /dev/null +++ b/extensions/parquet/table/plugin/channelprovider/s3/build.gradle @@ -0,0 +1,20 @@ +plugins { + id 'java-library' + id 'io.deephaven.project.register' +} + +dependencies { + api project(':extensions-parquet-base') + + implementation project(':Base') + implementation project(':Util') + implementation project(':extensions-parquet-table') + + 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) +} \ No newline at end of file diff --git a/extensions/parquet/table/plugin/channelprovider/s3/gradle.properties b/extensions/parquet/table/plugin/channelprovider/s3/gradle.properties new file mode 100644 index 00000000000..1a106ad8ae0 --- /dev/null +++ b/extensions/parquet/table/plugin/channelprovider/s3/gradle.properties @@ -0,0 +1 @@ +io.deephaven.project.ProjectType=JAVA_PUBLIC \ No newline at end of file diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java b/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/ByteBufferAsyncResponseTransformer.java similarity index 92% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java rename to extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/ByteBufferAsyncResponseTransformer.java index b9781648418..7f5633a16e2 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/ByteBufferAsyncResponseTransformer.java +++ b/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/ByteBufferAsyncResponseTransformer.java @@ -1,4 +1,7 @@ -package io.deephaven.parquet.table.util; +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.parquet.table.plugin.channelprovider.s3; import io.deephaven.base.verify.Assert; import org.reactivestreams.Subscriber; @@ -9,7 +12,8 @@ import java.nio.ByteBuffer; import java.util.concurrent.CompletableFuture; -public final class ByteBufferAsyncResponseTransformer implements AsyncResponseTransformer { +public final class ByteBufferAsyncResponseTransformer + implements AsyncResponseTransformer { private final int bufferSize; private volatile CompletableFuture cf; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java b/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableByteChannel.java similarity index 90% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java rename to extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableByteChannel.java index a718247cae0..47e0f573232 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableByteChannel.java +++ b/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableByteChannel.java @@ -1,7 +1,7 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.util; +package io.deephaven.parquet.table.plugin.channelprovider.s3; import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; @@ -29,8 +29,8 @@ /** - * {@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. + * {@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. */ public final class S3SeekableByteChannel implements SeekableByteChannel, SeekableChannelsProvider.ContextHolder { @@ -138,8 +138,8 @@ void setSize(final long size) { private long position; S3SeekableByteChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, @NotNull final URI uri, - @NotNull final S3AsyncClient s3AsyncClient, final int fragmentSize, final int readAheadCount, - final Duration readTimeout) { + @NotNull final S3AsyncClient s3AsyncClient, final int fragmentSize, final int readAheadCount, + final Duration readTimeout) { final S3Uri s3Uri = s3AsyncClient.utilities().parseUri(uri); this.bucket = s3Uri.bucket().orElse(null); this.key = s3Uri.key().orElse(null); @@ -157,8 +157,9 @@ void setSize(final long size) { public void setContext(@Nullable final SeekableChannelsProvider.ChannelContext context) { // null context equivalent to clearing the context if (context != null && !(context instanceof S3ChannelContext)) { - throw new IllegalArgumentException("context must be null or an instance of ChannelContext, provided context " + - " of class " + context.getClass().getName()); + throw new IllegalArgumentException( + "context must be null or an instance of ChannelContext, provided context" + + " of class " + context.getClass().getName()); } this.context = (S3ChannelContext) context; } @@ -193,7 +194,8 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep final int numFragmentsToLoad = Math.min(readAheadCount, numFragmentsInObject - currFragmentIndex - 1); for (int i = 0; i < numFragmentsToLoad; i++) { final int readAheadFragmentIndex = i + currFragmentIndex + 1; - final CompletableFuture readAheadFragmentFuture = context.getCachedFuture(readAheadFragmentIndex); + final CompletableFuture readAheadFragmentFuture = + context.getCachedFuture(readAheadFragmentIndex); if (readAheadFragmentFuture == null) { context.setFragmentContext(readAheadFragmentIndex, computeFragmentFuture(readAheadFragmentIndex)); } @@ -203,8 +205,10 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep final ByteBuffer currentFragment; try { currentFragment = currFragmentFuture.get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); - } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { - final String operation = "fetching fragment " + currFragmentIndex + " for file " + key + " in S3 bucket " + bucket; + } catch (final InterruptedException | ExecutionException | TimeoutException + | java.util.concurrent.CancellationException e) { + final String operation = + "fetching fragment " + currFragmentIndex + " for file " + key + " in S3 bucket " + bucket; throw handleS3Exception(e, operation); } @@ -242,8 +246,8 @@ private UncheckedDeephavenException handleS3Exception(final Exception e, final S } else if (e instanceof ExecutionException) { return new UncheckedDeephavenException("Execution exception occurred while " + operationDescription, e); } else if (e instanceof TimeoutException) { - return new UncheckedDeephavenException("Operation timeout while " + operationDescription + " after waiting " + - "for duration " + readTimeout, e); + return new UncheckedDeephavenException("Operation timeout while " + operationDescription + " after waiting " + + "for duration " + readTimeout, e); } else if (e instanceof CancellationException) { return new UncheckedDeephavenException("Cancelled an operation while " + operationDescription, e); } @@ -275,7 +279,7 @@ public SeekableByteChannel position(final long newPosition) throws ClosedChannel @Override public long size() throws ClosedChannelException { checkClosed(position); - if (size < 0){ + if (size < 0) { populateSize(); } return size; @@ -288,14 +292,15 @@ private void populateSize() { try { headObjectResponse = s3AsyncClient.headObject(builder -> builder.bucket(bucket).key(key)) .get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); - } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { + } catch (final InterruptedException | ExecutionException | TimeoutException + | java.util.concurrent.CancellationException e) { final String operation = "fetching HEAD for file " + key + " in S3 bucket " + bucket; throw handleS3Exception(e, operation); } context.setSize(headObjectResponse.contentLength().longValue()); } this.size = context.getSize(); - this.numFragmentsInObject = (int) ((size + fragmentSize - 1) / fragmentSize); // = ceil(size / fragmentSize) + this.numFragmentsInObject = (int) ((size + fragmentSize - 1) / fragmentSize); // = ceil(size / fragmentSize) } @Override diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java b/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProvider.java similarity index 90% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java rename to extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProvider.java index 44dbb438c89..8b829993d2e 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/S3SeekableChannelProvider.java +++ b/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProvider.java @@ -1,7 +1,7 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.util; +package io.deephaven.parquet.table.plugin.channelprovider.s3; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.table.S3Instructions; @@ -11,7 +11,6 @@ import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.http.crt.AwsCrtAsyncHttpClient; -import java.io.IOException; import java.net.URI; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; @@ -28,12 +27,11 @@ public final class S3SeekableChannelProvider implements SeekableChannelsProvider private final int readAheadCount; private final Duration readTimeout; - public S3SeekableChannelProvider(final S3Instructions s3Instructions) { + S3SeekableChannelProvider(final S3Instructions s3Instructions) { final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() .maxConcurrency(s3Instructions.maxConcurrentRequests()) .connectionTimeout(s3Instructions.connectionTimeout()) .build(); - // TODO Should we cache and reuse the client object? this.s3AsyncClient = S3AsyncClient.builder() .region(Region.of(s3Instructions.awsRegionName())) .httpClient(asyncHttpClient) @@ -60,7 +58,7 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boole throw new UnsupportedOperationException("Don't support writing to S3 yet"); } - public void close() throws IOException { + public void close() { s3AsyncClient.close(); } } diff --git a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProviderPlugin.java b/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProviderPlugin.java new file mode 100644 index 00000000000..138e4054ce2 --- /dev/null +++ b/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProviderPlugin.java @@ -0,0 +1,37 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.parquet.table.plugin.channelprovider.s3; + +import com.google.auto.service.AutoService; +import io.deephaven.parquet.base.ParquetFileReader; +import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelsProviderPlugin; +import io.deephaven.parquet.table.S3Instructions; +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 { + @Override + public boolean isCompatible(@NotNull final URI uri, @Nullable final Object config) { + return config instanceof S3Instructions && ParquetFileReader.S3_URI_SCHEME.equals(uri.getScheme()); + } + + @Override + public SeekableChannelsProvider impl(@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/parquet/table/plugin/channelprovider/tracked/build.gradle b/extensions/parquet/table/plugin/channelprovider/tracked/build.gradle new file mode 100644 index 00000000000..fa466439238 --- /dev/null +++ b/extensions/parquet/table/plugin/channelprovider/tracked/build.gradle @@ -0,0 +1,16 @@ +plugins { + id 'java-library' + id 'io.deephaven.project.register' +} + +dependencies { + api project(':extensions-parquet-base') + + implementation project(':Base') + implementation project(':Util') + implementation project(':engine-table') + + compileOnly depAnnotations + + Classpaths.inheritAutoService(project) +} \ No newline at end of file diff --git a/extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties b/extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties new file mode 100644 index 00000000000..1a106ad8ae0 --- /dev/null +++ b/extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties @@ -0,0 +1 @@ +io.deephaven.project.ProjectType=JAVA_PUBLIC \ No newline at end of file diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java b/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProvider.java similarity index 98% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java rename to extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProvider.java index ab246a9c82b..243fe7ab7cf 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/util/TrackedSeekableChannelsProvider.java +++ b/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProvider.java @@ -1,7 +1,7 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.util; +package io.deephaven.parquet.table.plugin.channelprovider.tracked; import io.deephaven.base.verify.Assert; import io.deephaven.engine.util.file.FileHandle; diff --git a/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProviderPluginImpl.java b/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProviderPluginImpl.java new file mode 100644 index 00000000000..31179bd0fef --- /dev/null +++ b/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProviderPluginImpl.java @@ -0,0 +1,36 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.parquet.table.plugin.channelprovider.tracked; + +import com.google.auto.service.AutoService; +import io.deephaven.engine.util.file.TrackedFileHandleFactory; +import io.deephaven.parquet.base.ParquetFileReader; +import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.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 TrackedSeekableChannelsProviderPluginImpl implements SeekableChannelsProviderPlugin { + @Override + public boolean isCompatible(@NotNull final URI uri, @Nullable final Object object) { + return object == null && (uri.getScheme() == null || uri.getScheme().equals(ParquetFileReader.FILE_URI_SCHEME)); + } + + @Override + public SeekableChannelsProvider impl(@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/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..ea66fa0d2c8 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 @@ -24,8 +24,8 @@ 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.plugin.channelprovider.tracked.TrackedSeekableChannelsProvider; 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; 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 7864bdc6940..fc60fe78448 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 @@ -7,7 +7,6 @@ import io.deephaven.base.ClassUtil; import io.deephaven.base.FileUtils; import io.deephaven.base.Pair; -import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.ColumnDefinition; @@ -15,10 +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.parquet.table.util.S3SeekableChannelProvider; +import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelsProviderLoader; +import io.deephaven.parquet.base.util.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; @@ -30,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; @@ -1163,22 +1162,15 @@ public static ParquetFileReader getParquetFileReaderChecked( public static ParquetFileReader getParquetFileReaderChecked( @NotNull final URI parquetFileURI, @NotNull final ParquetInstructions readInstructions) throws IOException { - if (S3_URI_SCHEME.equals(parquetFileURI.getScheme())) { - if (!(readInstructions.getSpecialInstructions() instanceof S3Instructions)) { - throw new IllegalArgumentException("Must provide S3Instructions to read files from S3"); - } - final S3Instructions s3Instructions = (S3Instructions) readInstructions.getSpecialInstructions(); - return new ParquetFileReader(parquetFileURI, - new CachedChannelProvider( - new S3SeekableChannelProvider(s3Instructions), 1 << 7)); + final SeekableChannelsProvider provider; + if (parquetFileURI.getScheme() != null && !parquetFileURI.getScheme().equals(FILE_URI_SCHEME)) { + // Need additional instructions to read from non-local file systems + provider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader(parquetFileURI, + readInstructions.getSpecialInstructions()); + } else { + provider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader(parquetFileURI, null); } - Assert.assertion(parquetFileURI.getScheme() == null - || parquetFileURI.getScheme().equals(ParquetFileReader.FILE_URI_SCHEME), - "Expected uri scheme to be null or \"file\", got uri as " + parquetFileURI); - return new ParquetFileReader( - parquetFileURI, - new CachedChannelProvider( - new TrackedSeekableChannelsProvider(TrackedFileHandleFactory.getInstance()), 1 << 7)); + return new ParquetFileReader(parquetFileURI, new CachedChannelProvider(provider, 1 << 7)); } @VisibleForTesting diff --git a/server/build.gradle b/server/build.gradle index 539b7996ea4..3ba13e39ce7 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -112,6 +112,7 @@ dependencies { Classpaths.inheritImmutables(project, true) runtimeOnly project(':authentication:example-providers:psk') + runtimeOnly project(':plugin-trackedseekablechannel') } TestTools.addEngineOutOfBandTest(project) diff --git a/server/jetty-app/build.gradle b/server/jetty-app/build.gradle index f795f21a6df..3847d492f64 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(':plugin-s3') + } +} + def authHandlers = [] def authConfigs = ['AuthHandlers'] if (hasProperty('anonymous')) { diff --git a/settings.gradle b/settings.gradle index fdd30f42cd8..2006aae8561 100644 --- a/settings.gradle +++ b/settings.gradle @@ -274,6 +274,12 @@ project(':plugin-hierarchicaltable').projectDir = file('plugin/hierarchicaltable include(':plugin-gc-app') project(':plugin-gc-app').projectDir = file('plugin/gc-app') +include(':plugin-trackedseekablechannel') +project(':plugin-trackedseekablechannel').projectDir = file('extensions/parquet/table/plugin/channelprovider/tracked') + +include(':plugin-s3') +project(':plugin-s3').projectDir = file('extensions/parquet/table/plugin/channelprovider/s3') + include(':uri') project(':uri').projectDir = file('java-client/uri') From d7fc6a123aafd65fb28463cb502c44a7f17b3d0d Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Wed, 10 Jan 2024 22:43:30 +0530 Subject: [PATCH 16/39] Minor fixes in gradle files --- extensions/parquet/table/build.gradle | 3 --- .../parquet/table/plugin/channelprovider/s3/gradle.properties | 2 +- .../table/plugin/channelprovider/tracked/gradle.properties | 2 +- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 30254f23e91..951af46586a 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -27,9 +27,6 @@ dependencies { implementation project(':log-factory') implementation project(':Configuration') implementation depCommonsLang3 - implementation platform('software.amazon.awssdk:bom:2.21.43') - implementation 'software.amazon.awssdk:s3' - implementation 'software.amazon.awssdk:aws-crt-client' Classpaths.inheritCommonsText(project, 'implementation') diff --git a/extensions/parquet/table/plugin/channelprovider/s3/gradle.properties b/extensions/parquet/table/plugin/channelprovider/s3/gradle.properties index 1a106ad8ae0..c186bbfdde1 100644 --- a/extensions/parquet/table/plugin/channelprovider/s3/gradle.properties +++ b/extensions/parquet/table/plugin/channelprovider/s3/gradle.properties @@ -1 +1 @@ -io.deephaven.project.ProjectType=JAVA_PUBLIC \ No newline at end of file +io.deephaven.project.ProjectType=JAVA_PUBLIC diff --git a/extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties b/extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties index 1a106ad8ae0..c186bbfdde1 100644 --- a/extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties +++ b/extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties @@ -1 +1 @@ -io.deephaven.project.ProjectType=JAVA_PUBLIC \ No newline at end of file +io.deephaven.project.ProjectType=JAVA_PUBLIC From aefa5ef579c36554935fea76b1fdb2eb7efb2015 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 11 Jan 2024 19:27:04 +0530 Subject: [PATCH 17/39] Resolved more comments --- .../parquet/base/ColumnChunkReaderImpl.java | 30 +++++++++--------- .../parquet/base/ParquetFileReader.java | 19 ++---------- .../base/util/CachedChannelProvider.java | 5 +++ .../base/util/LocalFSChannelProvider.java | 3 ++ .../base/util/SeekableChannelsProvider.java | 31 ++++++++++++++----- .../util/SeekableChannelsProviderLoader.java | 11 ++++++- .../util/SeekableChannelsProviderPlugin.java | 2 +- .../base/util/CachedChannelProviderTest.java | 3 ++ extensions/parquet/table/build.gradle | 4 +-- .../parquet/table/ParquetTableWriter.java | 7 +++-- .../deephaven/parquet/table/ParquetTools.java | 22 +++---------- .../table/location/ParquetColumnLocation.java | 7 ++--- .../table/location/ParquetTableLocation.java | 6 ++-- .../location/ParquetTableLocationFactory.java | 4 +-- .../table/ParquetTableReadWriteTest.java | 17 +++++++++- .../channelprovider => }/s3/build.gradle | 6 ++-- .../channelprovider => }/s3/gradle.properties | 0 .../ByteBufferAsyncResponseTransformer.java | 5 ++- .../extensions/s3}/S3Instructions.java | 5 ++- .../extensions}/s3/S3SeekableByteChannel.java | 22 ++++++------- .../s3/S3SeekableChannelProvider.java | 5 ++- .../s3/S3SeekableChannelProviderPlugin.java | 11 ++++--- .../tracked => trackedfile}/build.gradle | 4 ++- .../tracked => trackedfile}/gradle.properties | 0 .../TrackedSeekableChannelsProvider.java | 20 ++++++------ ...rackedSeekableChannelsProviderPlugin.java} | 13 +++++--- py/server/deephaven/parquet.py | 2 +- server/build.gradle | 1 - server/jetty-app/build.gradle | 2 +- settings.gradle | 12 +++---- 30 files changed, 156 insertions(+), 123 deletions(-) rename extensions/{parquet/table/plugin/channelprovider => }/s3/build.gradle (75%) rename extensions/{parquet/table/plugin/channelprovider => }/s3/gradle.properties (100%) rename extensions/{parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider => s3/src/main/java/io/deephaven/extensions}/s3/ByteBufferAsyncResponseTransformer.java (93%) rename extensions/{parquet/table/src/main/java/io/deephaven/parquet/table => s3/src/main/java/io/deephaven/extensions/s3}/S3Instructions.java (97%) rename extensions/{parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider => s3/src/main/java/io/deephaven/extensions}/s3/S3SeekableByteChannel.java (94%) rename extensions/{parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider => s3/src/main/java/io/deephaven/extensions}/s3/S3SeekableChannelProvider.java (92%) rename extensions/{parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider => s3/src/main/java/io/deephaven/extensions}/s3/S3SeekableChannelProviderPlugin.java (76%) rename extensions/{parquet/table/plugin/channelprovider/tracked => trackedfile}/build.gradle (77%) rename extensions/{parquet/table/plugin/channelprovider/tracked => trackedfile}/gradle.properties (100%) rename extensions/{parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked => trackedfile/src/main/java/io/deephaven/extensions/trackedfile}/TrackedSeekableChannelsProvider.java (84%) rename extensions/{parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProviderPluginImpl.java => trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java} (71%) 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 c6d5196900b..b01e94bc8c7 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 @@ -173,22 +173,22 @@ private Dictionary getDictionary(final SeekableChannelsProvider.ChannelContext c } else { return NULL_DICTIONARY; } - try { - if (channelContext == SeekableChannelsProvider.ChannelContext.NULL) { - // Create a new context object and use that for reading the dictionary - try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); - final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, getURI())) { - readChannel.position(dictionaryPageOffset); - return readDictionary(readChannel); - } - } else { - // Use the context object provided by the caller - try (final SeekableByteChannel readChannel = - channelsProvider.getReadChannel(channelContext, getURI())) { - readChannel.position(dictionaryPageOffset); - return readDictionary(readChannel); - } + if (channelContext == SeekableChannelsProvider.ChannelContext.NULL) { + // Create a new context object and use that for reading the dictionary + try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext()) { + return getDictionaryHelper(context, dictionaryPageOffset); } + } else { + // Use the context object provided by the caller + return getDictionaryHelper(channelContext, dictionaryPageOffset); + } + } + + private Dictionary getDictionaryHelper(final SeekableChannelsProvider.ChannelContext context, + final long dictionaryPageOffset) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, getURI())) { + readChannel.position(dictionaryPageOffset); + return readDictionary(readChannel); } catch (final IOException e) { throw new UncheckedIOException(e); } 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 db3c775bd21..67321244255 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 @@ -22,6 +22,8 @@ import java.nio.charset.StandardCharsets; import java.util.*; +import static io.deephaven.parquet.base.util.SeekableChannelsProvider.convertToURI; + /** * 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". @@ -30,7 +32,6 @@ 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 S3_URI_SCHEME = "s3"; public static final String FILE_URI_SCHEME = "file"; public final FileMetaData fileMetaData; @@ -42,22 +43,6 @@ public class ParquetFileReader { private final URI rootURI; private final MessageType type; - // TODO Where should I keep this method? - /** - * Take the parquet file source path or URI and convert it to a URI object. - * - * @param source The parquet file source path or URI - * @return The URI object - */ - public static URI convertToURI(final String source) { - try { - return new URI(source); - } catch (final URISyntaxException e) { - // Assuming the source is a file path, resolve to get the absolute path and convert to URI - return new File(source).getAbsoluteFile().toURI(); - } - } - public ParquetFileReader(final String source, final SeekableChannelsProvider channelsProvider) throws IOException { this(convertToURI(source), channelsProvider); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java index 139c01e7c6b..4f4bcdda67c 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java @@ -134,6 +134,11 @@ private long advanceClock() { return logicalClock = 1; } + @Override + public void close() { + wrappedProvider.close(); + } + /** * {@link SeekableByteChannel Channel} wrapper for pooled usage. */ diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java index 61274f543a4..5c04b63b98e 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java @@ -39,4 +39,7 @@ public SeekableByteChannel getWriteChannel(@NotNull final Path filePath, final b } return result; } + + @Override + public void close() {} } 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 index 28ff49bd995..95911f50af6 100644 --- 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 @@ -3,11 +3,11 @@ */ package io.deephaven.parquet.base.util; -import io.deephaven.UncheckedDeephavenException; import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; +import java.io.File; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -15,7 +15,28 @@ import java.nio.file.Path; import java.nio.file.Paths; -public interface SeekableChannelsProvider { +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).getAbsoluteFile().toURI(); + } + if (uri.getScheme() == null) { + // Need to convert to a "file" URI + return new File(source).getAbsoluteFile().toURI(); + } + return uri; + } interface ChannelContext extends SafeCloseable { @@ -40,11 +61,7 @@ default void clearContext() { default SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final String uriStr) throws IOException { - try { - return getReadChannel(context, new URI(uriStr)); - } catch (final URISyntaxException e) { - throw new UncheckedDeephavenException("Cannot convert path string to URI: " + uriStr, e); - } + return getReadChannel(context, convertToURI(uriStr)); } SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull URI uri) throws IOException; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java index e30ce663bca..fc3cd787fb5 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java @@ -32,6 +32,15 @@ private SeekableChannelsProviderLoader() { providers = new ArrayList<>(); } + /** + * 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) { if (providers.isEmpty()) { // Load the plugins @@ -42,7 +51,7 @@ public SeekableChannelsProvider fromServiceLoader(@NotNull final URI uri, @Nulla } for (final SeekableChannelsProviderPlugin plugin : providers) { if (plugin.isCompatible(uri, object)) { - return plugin.impl(uri, object); + return plugin.createProvider(uri, object); } } throw new UnsupportedOperationException("No plugin found for uri: " + uri); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java index 1409585ecbb..0644fffd46f 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java @@ -18,5 +18,5 @@ public interface SeekableChannelsProviderPlugin { /** * Create a {@link SeekableChannelsProvider} for the given URI and config object. */ - SeekableChannelsProvider impl(@NotNull final URI uri, @Nullable final Object object); + SeekableChannelsProvider createProvider(@NotNull final URI uri, @Nullable final Object object); } diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java index f1067347149..851fbc078a9 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java @@ -186,6 +186,9 @@ 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 { diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 951af46586a..d6cbff09f22 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -48,8 +48,8 @@ dependencies { project(path: ':test-configs') Classpaths.inheritSlf4j(project, 'slf4j-simple', 'testRuntimeOnly') - implementation project(':plugin-trackedseekablechannel') - testRuntimeOnly project(':plugin-s3') + runtimeOnly project(':extensions-trackedfile') + testImplementation project(':extensions-s3') brotliTestImplementation project(':extensions-parquet-table') brotliTestImplementation('com.github.rdblue:brotli-codec:0.1.1') 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 ea66fa0d2c8..5b1b2f2310f 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,11 +20,11 @@ import io.deephaven.parquet.base.ColumnWriter; import io.deephaven.parquet.base.ParquetFileWriter; import io.deephaven.parquet.base.RowGroupWriter; +import io.deephaven.parquet.base.util.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.plugin.channelprovider.tracked.TrackedSeekableChannelsProvider; import io.deephaven.parquet.table.transfer.*; import io.deephaven.stringset.StringSet; import io.deephaven.util.QueryConstants; @@ -45,6 +45,8 @@ import java.nio.file.Paths; import java.util.*; +import static io.deephaven.parquet.base.util.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 fc60fe78448..80b997199f1 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 @@ -55,8 +55,7 @@ import java.util.*; import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; -import static io.deephaven.parquet.base.ParquetFileReader.S3_URI_SCHEME; -import static io.deephaven.parquet.base.ParquetFileReader.convertToURI; +import static io.deephaven.parquet.base.util.SeekableChannelsProvider.convertToURI; import static io.deephaven.parquet.table.ParquetTableWriter.PARQUET_FILE_EXTENSION; import static io.deephaven.util.type.TypeUtils.getUnboxedTypeIfBoxed; @@ -682,15 +681,10 @@ private static Table readTableInternal( private static Table readTableInternal( @NotNull final URI source, @NotNull final ParquetInstructions instructions) { - final String scheme = source.getScheme(); - if (scheme != null && !scheme.equals(FILE_URI_SCHEME)) { - if (!scheme.equals(S3_URI_SCHEME)) { - throw new IllegalArgumentException( - "We only support reading single parquet file URI hosted on S3, but got " + source); - } + if (!source.getScheme().equals(FILE_URI_SCHEME)) { return readSingleFileTable(source, instructions); } - return readTableInternal(new File(source.getPath()), instructions); + return readTableInternal(new File(source), instructions); } private static boolean ignoreDotFiles(Path path) { @@ -1162,14 +1156,8 @@ public static ParquetFileReader getParquetFileReaderChecked( public static ParquetFileReader getParquetFileReaderChecked( @NotNull final URI parquetFileURI, @NotNull final ParquetInstructions readInstructions) throws IOException { - final SeekableChannelsProvider provider; - if (parquetFileURI.getScheme() != null && !parquetFileURI.getScheme().equals(FILE_URI_SCHEME)) { - // Need additional instructions to read from non-local file systems - provider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader(parquetFileURI, - readInstructions.getSpecialInstructions()); - } else { - provider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader(parquetFileURI, null); - } + final SeekableChannelsProvider provider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader( + parquetFileURI, readInstructions.getSpecialInstructions()); return new ParquetFileReader(parquetFileURI, new CachedChannelProvider(provider, 1 << 7)); } 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 f78373f805c..1e90f6d4033 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 @@ -60,7 +60,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.S3_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; import static io.deephaven.parquet.table.ParquetTableWriter.*; final class ParquetColumnLocation extends AbstractColumnLocation { @@ -167,9 +167,8 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition return null; } final URI parquetFileURI = tl().getParquetFile(); - if (S3_URI_SCHEME.equals(parquetFileURI.getScheme())) { - throw new UncheckedDeephavenException("Parquet files in S3 are not expected to have indexing files"); - } + Assert.assertion(parquetFileURI.getScheme().equals(FILE_URI_SCHEME), + "Expected a file uri, got " + parquetFileURI); final File parquetFile = new File(parquetFileURI); try { ParquetFileReader parquetFileReader; 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 5e255e532b1..25f7f0409a3 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 @@ -30,7 +30,7 @@ import java.util.*; import java.util.stream.IntStream; -import static io.deephaven.parquet.base.ParquetFileReader.S3_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; public class ParquetTableLocation extends AbstractTableLocation { @@ -90,13 +90,11 @@ public ParquetTableLocation(@NotNull final TableKey tableKey, columnTypes = tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); version = tableInfo.map(TableInfo::version).orElse(null); - final String uriScheme = tableLocationKey.getURI().getScheme(); - if (S3_URI_SCHEME.equals(uriScheme)) { + if (!tableLocationKey.getURI().getScheme().equals(FILE_URI_SCHEME)) { handleUpdate(computeIndex(), 0L); // TODO What should I put here? } else { handleUpdate(computeIndex(), new File(tableLocationKey.getURI().toString()).lastModified()); } - } @Override 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 09589c333e5..8222c10b961 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 @@ -15,7 +15,7 @@ import java.io.File; import java.net.URI; -import static io.deephaven.parquet.base.ParquetFileReader.S3_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; /** * {@link TableLocationFactory} for {@link ParquetTableLocation}s. @@ -34,7 +34,7 @@ public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final ParquetTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { final URI parquetFileURI = locationKey.getURI(); - if (S3_URI_SCHEME.equals(parquetFileURI.getScheme()) || new File(parquetFileURI.getPath()).exists()) { + if (!parquetFileURI.getScheme().equals(FILE_URI_SCHEME) || new File(parquetFileURI).exists()) { return new ParquetTableLocation(tableKey, locationKey, readInstructions); } else { return new NonexistentTableLocation(tableKey, locationKey); 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 c85f30f36be..6280a24831f 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 @@ -31,6 +31,7 @@ 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; @@ -1072,16 +1073,30 @@ public void basicWriteAndReadFromFileURITests() { 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 IllegalArgumentException); + assertTrue(e instanceof UnsupportedOperationException); } } diff --git a/extensions/parquet/table/plugin/channelprovider/s3/build.gradle b/extensions/s3/build.gradle similarity index 75% rename from extensions/parquet/table/plugin/channelprovider/s3/build.gradle rename to extensions/s3/build.gradle index 18b6b9eaf4e..ac2921e3397 100644 --- a/extensions/parquet/table/plugin/channelprovider/s3/build.gradle +++ b/extensions/s3/build.gradle @@ -3,12 +3,13 @@ plugins { id 'io.deephaven.project.register' } +description 'Used to create a channel provider plugin for reading and writing files stored in S3.' + dependencies { api project(':extensions-parquet-base') implementation project(':Base') implementation project(':Util') - implementation project(':extensions-parquet-table') implementation platform('software.amazon.awssdk:bom:2.21.43') implementation 'software.amazon.awssdk:s3' @@ -17,4 +18,5 @@ dependencies { compileOnly depAnnotations Classpaths.inheritAutoService(project) -} \ No newline at end of file + Classpaths.inheritImmutables(project) +} diff --git a/extensions/parquet/table/plugin/channelprovider/s3/gradle.properties b/extensions/s3/gradle.properties similarity index 100% rename from extensions/parquet/table/plugin/channelprovider/s3/gradle.properties rename to extensions/s3/gradle.properties diff --git a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/ByteBufferAsyncResponseTransformer.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java similarity index 93% rename from extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/ByteBufferAsyncResponseTransformer.java rename to extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java index 7f5633a16e2..a26d8f55233 100644 --- a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/ByteBufferAsyncResponseTransformer.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java @@ -1,7 +1,7 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.plugin.channelprovider.s3; +package io.deephaven.extensions.s3; import io.deephaven.base.verify.Assert; import org.reactivestreams.Subscriber; @@ -12,8 +12,7 @@ import java.nio.ByteBuffer; import java.util.concurrent.CompletableFuture; -public final class ByteBufferAsyncResponseTransformer - implements AsyncResponseTransformer { +final class ByteBufferAsyncResponseTransformer implements AsyncResponseTransformer { private final int bufferSize; private volatile CompletableFuture cf; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java similarity index 97% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3Instructions.java rename to extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index 0ce489f3e8c..5b9c699f016 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -1,4 +1,7 @@ -package io.deephaven.parquet.table; +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.extensions.s3; import io.deephaven.annotations.BuildableStyle; import org.immutables.value.Value; diff --git a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java similarity index 94% rename from extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableByteChannel.java rename to extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index 47e0f573232..d6018cb6b65 100644 --- a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -1,18 +1,18 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.plugin.channelprovider.s3; +package io.deephaven.extensions.s3; import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; -import io.deephaven.engine.exceptions.CancellationException; +import java.util.concurrent.CancellationException; import io.deephaven.parquet.base.util.SeekableChannelsProvider; 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.HeadObjectResponse; -import javax.annotation.Nullable; import java.io.IOException; import java.net.URI; import java.nio.ByteBuffer; @@ -32,7 +32,7 @@ * {@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. */ -public final class S3SeekableByteChannel implements SeekableByteChannel, SeekableChannelsProvider.ContextHolder { +final class S3SeekableByteChannel implements SeekableByteChannel, SeekableChannelsProvider.ContextHolder { private static final long CLOSED_SENTINEL = -1; @@ -205,8 +205,7 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep final ByteBuffer currentFragment; try { currentFragment = currFragmentFuture.get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); - } catch (final InterruptedException | ExecutionException | TimeoutException - | java.util.concurrent.CancellationException e) { + } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { final String operation = "fetching fragment " + currFragmentIndex + " for file " + key + " in S3 bucket " + bucket; throw handleS3Exception(e, operation); @@ -221,7 +220,7 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep final int originalBufferLimit = currentFragment.limit(); currentFragment.limit(currentFragment.position() + limit); destination.put(currentFragment); - // Need to reset buffer limit so we can read from the same buffer again in future + // Need to reset buffer limit, so we can read from the same buffer again in future currentFragment.limit(originalBufferLimit); position = localPosition + limit; return limit; @@ -242,7 +241,7 @@ private CompletableFuture computeFragmentFuture(final int fragmentIn private UncheckedDeephavenException handleS3Exception(final Exception e, final String operationDescription) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); - return new CancellationException("Thread interrupted while " + operationDescription, e); + return new UncheckedDeephavenException("Thread interrupted while " + operationDescription, e); } else if (e instanceof ExecutionException) { return new UncheckedDeephavenException("Execution exception occurred while " + operationDescription, e); } else if (e instanceof TimeoutException) { @@ -292,12 +291,11 @@ private void populateSize() { try { headObjectResponse = s3AsyncClient.headObject(builder -> builder.bucket(bucket).key(key)) .get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); - } catch (final InterruptedException | ExecutionException | TimeoutException - | java.util.concurrent.CancellationException e) { + } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { final String operation = "fetching HEAD for file " + key + " in S3 bucket " + bucket; throw handleS3Exception(e, operation); } - context.setSize(headObjectResponse.contentLength().longValue()); + context.setSize(headObjectResponse.contentLength()); } this.size = context.getSize(); this.numFragmentsInObject = (int) ((size + fragmentSize - 1) / fragmentSize); // = ceil(size / fragmentSize) @@ -314,7 +312,7 @@ public boolean isOpen() { } @Override - public void close() throws IOException { + public void close() { position = CLOSED_SENTINEL; } diff --git a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java similarity index 92% rename from extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProvider.java rename to extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index 8b829993d2e..f9fefd2beee 100644 --- a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -1,10 +1,9 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.plugin.channelprovider.s3; +package io.deephaven.extensions.s3; import io.deephaven.parquet.base.util.SeekableChannelsProvider; -import io.deephaven.parquet.table.S3Instructions; import org.jetbrains.annotations.NotNull; import software.amazon.awssdk.http.async.SdkAsyncHttpClient; import software.amazon.awssdk.regions.Region; @@ -19,7 +18,7 @@ /** * {@link SeekableChannelsProvider} implementation that is used to fetch objects from AWS S3 instances. */ -public final class S3SeekableChannelProvider implements SeekableChannelsProvider { +final class S3SeekableChannelProvider implements SeekableChannelsProvider { private final S3AsyncClient s3AsyncClient; private final int fragmentSize; diff --git a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProviderPlugin.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProviderPlugin.java similarity index 76% rename from extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProviderPlugin.java rename to extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProviderPlugin.java index 138e4054ce2..67de3592c40 100644 --- a/extensions/parquet/table/plugin/channelprovider/s3/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/s3/S3SeekableChannelProviderPlugin.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProviderPlugin.java @@ -1,13 +1,11 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.plugin.channelprovider.s3; +package io.deephaven.extensions.s3; import com.google.auto.service.AutoService; -import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.base.util.SeekableChannelsProviderPlugin; -import io.deephaven.parquet.table.S3Instructions; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -18,13 +16,16 @@ */ @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 config instanceof S3Instructions && ParquetFileReader.S3_URI_SCHEME.equals(uri.getScheme()); + return S3_URI_SCHEME.equals(uri.getScheme()); } @Override - public SeekableChannelsProvider impl(@NotNull final URI uri, @Nullable final Object config) { + 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"); diff --git a/extensions/parquet/table/plugin/channelprovider/tracked/build.gradle b/extensions/trackedfile/build.gradle similarity index 77% rename from extensions/parquet/table/plugin/channelprovider/tracked/build.gradle rename to extensions/trackedfile/build.gradle index fa466439238..9d3f32cfd01 100644 --- a/extensions/parquet/table/plugin/channelprovider/tracked/build.gradle +++ b/extensions/trackedfile/build.gradle @@ -3,6 +3,8 @@ plugins { id 'io.deephaven.project.register' } +description 'Used to create a channel provider plugin for reading and writing local files.' + dependencies { api project(':extensions-parquet-base') @@ -13,4 +15,4 @@ dependencies { compileOnly depAnnotations Classpaths.inheritAutoService(project) -} \ No newline at end of file +} diff --git a/extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties b/extensions/trackedfile/gradle.properties similarity index 100% rename from extensions/parquet/table/plugin/channelprovider/tracked/gradle.properties rename to extensions/trackedfile/gradle.properties diff --git a/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java similarity index 84% rename from extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProvider.java rename to extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index 243fe7ab7cf..3a62f2d7aa1 100644 --- a/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -1,14 +1,13 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.plugin.channelprovider.tracked; +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.ParquetFileReader; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; @@ -19,14 +18,16 @@ 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; - public static SeekableChannelsProvider getInstance() { + static SeekableChannelsProvider getInstance() { if (instance == null) { synchronized (TrackedSeekableChannelsProvider.class) { if (instance == null) { @@ -39,7 +40,7 @@ public static SeekableChannelsProvider getInstance() { private final TrackedFileHandleFactory fileHandleFactory; - public TrackedSeekableChannelsProvider(@NotNull final TrackedFileHandleFactory fileHandleFactory) { + TrackedSeekableChannelsProvider(@NotNull final TrackedFileHandleFactory fileHandleFactory) { this.fileHandleFactory = fileHandleFactory; } @@ -52,12 +53,10 @@ public ChannelContext makeContext() { public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { // context is unused here because it is NULL - Assert.assertion(uri.getScheme() == null || uri.getScheme().equals(ParquetFileReader.FILE_URI_SCHEME), - "Expected uri scheme to be null or \"file\", got uri as " + uri); - return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, new File(uri.getPath())); + Assert.assertion(uri.getScheme().equals(FILE_URI_SCHEME), "Expected a file uri, got " + uri); + return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, new File(uri)); } - @Override public final SeekableByteChannel getWriteChannel(@NotNull final Path filePath, final boolean append) throws IOException { @@ -90,4 +89,7 @@ public final FileHandle invoke(@NotNull final File file) throws IOException { return fileHandleFactory.writeAppendCreateHandleCreator.invoke(file); } } + + @Override + public void close() {} } diff --git a/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProviderPluginImpl.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java similarity index 71% rename from extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProviderPluginImpl.java rename to extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java index 31179bd0fef..5f6a3c449b0 100644 --- a/extensions/parquet/table/plugin/channelprovider/tracked/src/main/java/io/deephaven/parquet/table/plugin/channelprovider/tracked/TrackedSeekableChannelsProviderPluginImpl.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java @@ -1,11 +1,10 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.table.plugin.channelprovider.tracked; +package io.deephaven.extensions.trackedfile; import com.google.auto.service.AutoService; import io.deephaven.engine.util.file.TrackedFileHandleFactory; -import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.base.util.SeekableChannelsProviderPlugin; import org.jetbrains.annotations.NotNull; @@ -17,20 +16,24 @@ * {@link SeekableChannelsProviderPlugin} implementation used for reading files from local disk. */ @AutoService(SeekableChannelsProviderPlugin.class) -public final class TrackedSeekableChannelsProviderPluginImpl implements SeekableChannelsProviderPlugin { +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 object == null && (uri.getScheme() == null || uri.getScheme().equals(ParquetFileReader.FILE_URI_SCHEME)); + return FILE_URI_SCHEME.equals(uri.getScheme()); } @Override - public SeekableChannelsProvider impl(@NotNull final URI uri, @Nullable final Object object) { + 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); } + // TODO I am confused which constructor should I use, this or getInstance()?? return new TrackedSeekableChannelsProvider(TrackedFileHandleFactory.getInstance()); } } diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index c127de70036..acb3f5a75f7 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -19,7 +19,7 @@ _JFile = jpy.get_type("java.io.File") _JCompressionCodecName = jpy.get_type("org.apache.parquet.hadoop.metadata.CompressionCodecName") _JParquetInstructions = jpy.get_type("io.deephaven.parquet.table.ParquetInstructions") -_JS3Instructions = jpy.get_type("io.deephaven.parquet.table.S3Instructions") +_JS3Instructions = jpy.get_type("io.deephaven.extensions.s3.S3Instructions") _JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") _JDuration = jpy.get_type("java.time.Duration") diff --git a/server/build.gradle b/server/build.gradle index 3ba13e39ce7..539b7996ea4 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -112,7 +112,6 @@ dependencies { Classpaths.inheritImmutables(project, true) runtimeOnly project(':authentication:example-providers:psk') - runtimeOnly project(':plugin-trackedseekablechannel') } TestTools.addEngineOutOfBandTest(project) diff --git a/server/jetty-app/build.gradle b/server/jetty-app/build.gradle index 3847d492f64..ee51f12d41e 100644 --- a/server/jetty-app/build.gradle +++ b/server/jetty-app/build.gradle @@ -53,7 +53,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { - runtimeOnly project(':plugin-s3') + runtimeOnly project(':extensions-s3') } } diff --git a/settings.gradle b/settings.gradle index 2006aae8561..0188fca02ff 100644 --- a/settings.gradle +++ b/settings.gradle @@ -254,6 +254,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') @@ -274,12 +280,6 @@ project(':plugin-hierarchicaltable').projectDir = file('plugin/hierarchicaltable include(':plugin-gc-app') project(':plugin-gc-app').projectDir = file('plugin/gc-app') -include(':plugin-trackedseekablechannel') -project(':plugin-trackedseekablechannel').projectDir = file('extensions/parquet/table/plugin/channelprovider/tracked') - -include(':plugin-s3') -project(':plugin-s3').projectDir = file('extensions/parquet/table/plugin/channelprovider/s3') - include(':uri') project(':uri').projectDir = file('java-client/uri') From 82a4ef30fed1f6ed63c04ca7224fcc8dff4468a5 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 12 Jan 2024 15:24:13 +0530 Subject: [PATCH 18/39] Resolving Ryan's comments --- .../datastructures/LazyCachingFunction.java | 17 ++-- .../parquet/base/ColumnChunkReader.java | 31 ++++--- .../parquet/base/ColumnChunkReaderImpl.java | 46 +++-------- .../parquet/base/ColumnPageReader.java | 34 +++----- .../parquet/base/ColumnPageReaderImpl.java | 74 ++++++++--------- .../base/util/CachedChannelProvider.java | 5 ++ .../base/util/LocalFSChannelProvider.java | 6 ++ .../base/util/SeekableChannelsProvider.java | 15 +++- .../util/SeekableChannelsProviderPlugin.java | 4 +- .../table/pagestore/ColumnChunkPageStore.java | 80 ++++++++++--------- .../OffsetIndexBasedColumnChunkPageStore.java | 17 ++-- .../VariablePageSizeColumnChunkPageStore.java | 24 ++---- .../pagestore/topage/ChunkDictionary.java | 2 +- .../table/pagestore/topage/ToPage.java | 16 ++-- .../topage/ToPageWithDictionary.java | 18 +++-- .../pagestore/topage/ToStringSetPage.java | 6 +- .../table/ParquetTableReadWriteTest.java | 42 +++++----- .../s3/S3SeekableChannelProvider.java | 5 ++ .../TrackedSeekableChannelsProvider.java | 6 ++ .../replicators/ReplicateCachingSupplier.java | 29 +++++++ 20 files changed, 251 insertions(+), 226 deletions(-) create mode 100644 replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java diff --git a/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java index 094072d8c7b..da138aa5f40 100644 --- a/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java +++ b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java @@ -1,6 +1,11 @@ /** * 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; @@ -9,22 +14,22 @@ import java.util.function.Function; /** - * {@link Function} wrapper that caches the result in a {@link SoftReference}. Only suitable to wrap suppliers that are + * {@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}. */ public final class LazyCachingFunction implements Function { - private final Function internalSupplier; + private final Function internalFunction; private volatile SoftReference cachedResultRef; /** * Construct a {@link Function} wrapper. * - * @param internalSupplier The {@link Function} to wrap. Must be safely repeatable and must not return {@code null}. + * @param internalFunction The {@link Function} to wrap. Must be safely repeatable and must not return {@code null}. */ - public LazyCachingFunction(@NotNull final Function internalSupplier) { - this.internalSupplier = internalSupplier; + public LazyCachingFunction(@NotNull final Function internalFunction) { + this.internalFunction = internalFunction; } @Override @@ -38,7 +43,7 @@ public OUTPUT_TYPE apply(final INPUT_TYPE arg) { if ((currentRef = cachedResultRef) != null && (current = currentRef.get()) != null) { return current; } - cachedResultRef = new SoftReference<>(current = internalSupplier.apply(arg)); + cachedResultRef = new SoftReference<>(current = internalFunction.apply(arg)); } return current; } 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 5b19dfb13f3..48c7a1bdb10 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 @@ -21,7 +21,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(); @@ -41,21 +41,21 @@ public interface ColumnChunkReader { * Used to iterate over column page readers for each page with the capability to set channel context to for reading * the pages. */ - interface ColumnPageReaderIterator extends Iterator { - + interface ColumnPageReaderIterator { /** - * Set the {@code channelContext} to be used only for a single {@code next()} call. If not set, - * {@code SeekableChannelsProvider.ChannelContext.NULL} will be used + * @return Whether there are more pages to iterate over. */ - void setChannelContext(final SeekableChannelsProvider.ChannelContext channelContext); + boolean hasNext(); - default void clearChannelContext() { - setChannelContext(SeekableChannelsProvider.ChannelContext.NULL); - } + /** + * @param channelContext The channel context to use for constructing the reader + * @return The next page reader. + */ + ColumnPageReader next(SeekableChannelsProvider.ChannelContext channelContext); } /** - * @return An iterator over individual parquet pages + * @return An iterator over individual parquet pages. */ ColumnPageReaderIterator getPageIterator() throws IOException; @@ -63,17 +63,16 @@ interface ColumnPageDirectAccessor { /** * Directly access a page reader for a given page number. */ - ColumnPageReader getPageReader(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, - 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(); @@ -107,7 +106,7 @@ public int getMaxId() { String getVersion(); /** - * Create a new channel context for this column chunk reader. + * @return The channel provider for this column chunk reader. */ - SeekableChannelsProvider.ChannelContext makeChannelContext(); + 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 b01e94bc8c7..812951eff2d 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 @@ -173,15 +173,7 @@ private Dictionary getDictionary(final SeekableChannelsProvider.ChannelContext c } else { return NULL_DICTIONARY; } - if (channelContext == SeekableChannelsProvider.ChannelContext.NULL) { - // Create a new context object and use that for reading the dictionary - try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext()) { - return getDictionaryHelper(context, dictionaryPageOffset); - } - } else { - // Use the context object provided by the caller - return getDictionaryHelper(channelContext, dictionaryPageOffset); - } + return getDictionaryHelper(channelContext, dictionaryPageOffset); } private Dictionary getDictionaryHelper(final SeekableChannelsProvider.ChannelContext context, @@ -205,8 +197,8 @@ public String getVersion() { } @Override - public SeekableChannelsProvider.ChannelContext makeChannelContext() { - return channelsProvider.makeContext(); + public SeekableChannelsProvider getChannelsProvider() { + return channelsProvider; } @NotNull @@ -238,17 +230,10 @@ private Dictionary readDictionary(ReadableByteChannel file) throws IOException { private final class ColumnPageReaderIteratorImpl implements ColumnPageReaderIterator { private long currentOffset; private long remainingValues; - private SeekableChannelsProvider.ChannelContext channelContext; ColumnPageReaderIteratorImpl(final long startOffset, final long numValues) { this.remainingValues = numValues; this.currentOffset = startOffset; - this.channelContext = SeekableChannelsProvider.ChannelContext.NULL; - } - - @Override - public void setChannelContext(SeekableChannelsProvider.ChannelContext channelContext) { - this.channelContext = channelContext; } @Override @@ -257,7 +242,7 @@ public boolean hasNext() { } @Override - public ColumnPageReader next() { + public ColumnPageReader next(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) { if (!hasNext()) { throw new NoSuchElementException("No next element"); } @@ -270,7 +255,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( @@ -298,8 +283,7 @@ public ColumnPageReader next() { : (SeekableChannelsProvider.ChannelContext context) -> NULL_DICTIONARY; final ColumnPageReader nextReader = new ColumnPageReaderImpl(channelsProvider, decompressor, pageDictionarySupplier, nullMaterializerFactory, path, getURI(), fieldTypes, - readChannel.position(), pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES, channelContext); - clearChannelContext(); + readChannel.position(), pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES); return nextReader; } catch (IOException e) { throw new UncheckedDeephavenException("Error reading page header", e); @@ -309,26 +293,18 @@ pageDictionarySupplier, nullMaterializerFactory, path, getURI(), fieldTypes, private final class ColumnPageReaderIteratorIndexImpl implements ColumnPageReaderIterator { private int pos; - private SeekableChannelsProvider.ChannelContext channelContext; ColumnPageReaderIteratorIndexImpl() { pos = 0; - channelContext = SeekableChannelsProvider.ChannelContext.NULL; } - @Override - public void setChannelContext(SeekableChannelsProvider.ChannelContext channelContext) { - this.channelContext = channelContext; - } - - @Override public boolean hasNext() { return offsetIndex.getPageCount() > pos; } @Override - public ColumnPageReader next() { + public ColumnPageReader next(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) { if (!hasNext()) { throw new NoSuchElementException("No next element"); } @@ -341,9 +317,8 @@ public ColumnPageReader next() { final ColumnPageReader columnPageReader = new ColumnPageReaderImpl(channelsProvider, decompressor, dictionarySupplier, nullMaterializerFactory, path, getURI(), fieldTypes, offsetIndex.getOffset(pos), null, - numValues, channelContext); + numValues); pos++; - clearChannelContext(); return columnPageReader; } } @@ -353,8 +328,7 @@ private final class ColumnPageDirectAccessorImpl implements ColumnPageDirectAcce ColumnPageDirectAccessorImpl() {} @Override - public ColumnPageReader getPageReader(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, - final int pageNum) { + public ColumnPageReader getPageReader(final int pageNum) { if (pageNum < 0 || pageNum >= offsetIndex.getPageCount()) { throw new IndexOutOfBoundsException( "pageNum=" + pageNum + ", offsetIndex.getPageCount()=" + offsetIndex.getPageCount()); @@ -362,7 +336,7 @@ public ColumnPageReader getPageReader(@NotNull final SeekableChannelsProvider.Ch // 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, getURI(), fieldTypes, offsetIndex.getOffset(pageNum), null, - ColumnPageReaderImpl.NULL_NUM_VALUES, channelContext); + 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 d29818d3cc3..aca84ac6b29 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 @@ -17,56 +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 SeekableChannelsProvider.ChannelContext 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, SeekableChannelsProvider.ChannelContext 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, + SeekableChannelsProvider.ChannelContext 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(SeekableChannelsProvider.ChannelContext 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(); - - /** - * Set the channel context to use while reading the parquet file. This will cache a copy of the object for use - * across reads. Therefore, its is recommended to use {@link #clearChannelContext()} when done with the - * reader/context. - * - * @param context The channel context to use. - */ - void setChannelContext(@NotNull final SeekableChannelsProvider.ChannelContext context); - - /** - * Clear the cached channel context. - */ - default void clearChannelContext() { - setChannelContext(SeekableChannelsProvider.ChannelContext.NULL); - } + Dictionary getDictionary(SeekableChannelsProvider.ChannelContext 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 e69920739eb..cab06534788 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 @@ -63,7 +63,6 @@ public class ColumnPageReaderImpl implements ColumnPageReader { private PageHeader pageHeader; private int numValues; private int rowCount = -1; - private SeekableChannelsProvider.ChannelContext context; /** * Returns a {@link ColumnPageReader} object for reading the column page data from the file. @@ -91,8 +90,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { List fieldTypes, long offset, PageHeader pageHeader, - int numValues, - SeekableChannelsProvider.ChannelContext context) { + int numValues) { this.channelsProvider = channelsProvider; this.compressorAdapter = compressorAdapter; this.dictionarySupplier = dictionarySupplier; @@ -103,19 +101,19 @@ public class ColumnPageReaderImpl implements ColumnPageReader { this.offset = offset; this.pageHeader = pageHeader; this.numValues = numValues; - this.context = context; } @Override - public Object materialize(Object nullValue) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, uri)) { + public Object materialize(@NotNull final Object nullValue, + @NotNull final SeekableChannelsProvider.ChannelContext 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(context, uri)) { + public int readRowCount(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { ensurePageHeader(readChannel); return readRowCountFromDataPage(readChannel); } @@ -123,10 +121,11 @@ public int readRowCount() throws IOException { @Override - public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder) throws IOException { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, uri)) { + public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder, + @NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { ensurePageHeader(readChannel); - return readKeyFromDataPage(keyDest, nullPlaceholder, readChannel); + return readKeyFromDataPage(keyDest, nullPlaceholder, readChannel, channelContext); } } @@ -215,7 +214,8 @@ private int readRowCountFromDataPage(ReadableByteChannel file) throws IOExceptio } private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, - ReadableByteChannel file) throws IOException { + ReadableByteChannel file, @NotNull final SeekableChannelsProvider.ChannelContext channelContext) + throws IOException { int uncompressedPageSize = pageHeader.getUncompressed_page_size(); int compressedPageSize = pageHeader.getCompressed_page_size(); @@ -233,7 +233,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(); @@ -258,7 +258,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(), @@ -266,7 +266,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 SeekableChannelsProvider.ChannelContext channelContext) throws IOException { final int uncompressedPageSize = pageHeader.getUncompressed_page_size(); final int compressedPageSize = pageHeader.getCompressed_page_size(); switch (pageHeader.type) { @@ -283,7 +284,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() @@ -332,7 +333,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 SeekableChannelsProvider.ChannelContext channelContext) { RunLengthBitPackingHybridBufferDecoder rlDecoder = null; RunLengthBitPackingHybridBufferDecoder dlDecoder = null; try { @@ -363,7 +365,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) { @@ -398,7 +400,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 SeekableChannelsProvider.ChannelContext channelContext) { RunLengthBitPackingHybridBufferDecoder dlDecoder = null; try { ByteBuffer bytes = page.getBytes().toByteBuffer(); // TODO - move away from page and use @@ -418,7 +421,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) { @@ -438,7 +441,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 SeekableChannelsProvider.ChannelContext channelContext) throws IOException { if (path.getMaxRepetitionLevel() > 0) { throw new RuntimeException("Repeating levels not supported"); @@ -453,7 +457,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 { @@ -593,13 +597,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 SeekableChannelsProvider.ChannelContext 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.apply(context); + 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); @@ -618,11 +623,11 @@ private ValuesReader getDataReader(Encoding dataEncoding, ByteBuffer in, int val } @Override - public int numValues() throws IOException { + public int numValues(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { if (numValues >= 0) { return numValues; } - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, uri)) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { ensurePageHeader(readChannel); // Above will block till it populates numValues Assert.geqZero(numValues, "numValues"); @@ -632,13 +637,8 @@ public int numValues() throws IOException { @NotNull @Override - public Dictionary getDictionary() { - return dictionarySupplier.apply(context); - } - - @Override - public void setChannelContext(@NotNull final SeekableChannelsProvider.ChannelContext context) { - this.context = context; + public Dictionary getDictionary(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) { + return dictionarySupplier.apply(channelContext); } @Override @@ -647,12 +647,12 @@ public void close() throws Exception { } @Override - public long numRows() throws IOException { + public long numRows(@NotNull final SeekableChannelsProvider.ChannelContext 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/util/CachedChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java index 4f4bcdda67c..047060c5dff 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java @@ -57,6 +57,11 @@ public ChannelContext makeContext() { return wrappedProvider.makeContext(); } + @Override + public boolean isCompatibleWith(@NotNull final ChannelContext context) { + return wrappedProvider.isCompatibleWith(context); + } + @Override public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java index 5c04b63b98e..c83237954f9 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java @@ -16,9 +16,15 @@ public class LocalFSChannelProvider implements SeekableChannelsProvider { @Override public ChannelContext makeContext() { + // No additional context required for local FS return ChannelContext.NULL; } + @Override + public boolean isCompatibleWith(@NotNull final ChannelContext context) { + return context == ChannelContext.NULL; + } + @Override public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { 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 index 95911f50af6..624a88f58e0 100644 --- 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 @@ -48,8 +48,17 @@ interface ChannelContext extends SafeCloseable { default void close() {} } + /** + * Create a new {@link ChannelContext} object for creating read and write channels via this provider. + */ ChannelContext 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 ChannelContext context); + interface ContextHolder { void setContext(ChannelContext context); @@ -59,14 +68,14 @@ default void clearContext() { } } - default SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final String uriStr) + default SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull String uriStr) throws IOException { return getReadChannel(context, convertToURI(uriStr)); } - SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull URI uri) throws IOException; + SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull URI uri) throws IOException; - default SeekableByteChannel getWriteChannel(@NotNull final String path, final boolean append) throws IOException { + default SeekableByteChannel getWriteChannel(@NotNull String path, final boolean append) throws IOException { return getWriteChannel(Paths.get(path), append); } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java index 0644fffd46f..92990fb94f5 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java @@ -13,10 +13,10 @@ public interface SeekableChannelsProviderPlugin { /** * Check if this plugin is compatible with the given URI and config object. */ - boolean isCompatible(@NotNull final URI uri, @Nullable final Object config); + boolean isCompatible(@NotNull URI uri, @Nullable Object config); /** * Create a {@link SeekableChannelsProvider} for the given URI and config object. */ - SeekableChannelsProvider createProvider(@NotNull final URI uri, @Nullable final Object object); + SeekableChannelsProvider createProvider(@NotNull URI uri, @Nullable Object object); } 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 f059c3157e1..0536125f662 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 @@ -134,9 +134,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 SeekableChannelsProvider.ChannelContext channelContext) throws IOException { - return toPage.toPage(offset, columnPageReader, mask); + return toPage.toPage(offset, columnPageReader, channelContext, mask); } @Override @@ -173,60 +174,61 @@ public boolean usesDictionaryOnEveryPage() { public void close() {} /** - * Take an object of {@link PagingContextHolder} and populate the inner context with values from - * {@link #columnChunkReader} - */ - final void innerFillContext(@Nullable final FillContext context) { - if (context != null) { - ((PagingContextHolder) context).updateInnerContext(this::fillContextUpdater); - } - } - - /** - * Return true if this class allocated the current inner context and therefore the context can be reused for reading - * pages. - */ - private static boolean isFillContextCompatible(@Nullable final Context currentInnerContext) { - return currentInnerContext instanceof ChannelContextWrapper; - } - - static SeekableChannelsProvider.ChannelContext getChannelContext(@Nullable final FillContext fillContext) { - if (fillContext instanceof PagingContextHolder) { - final Context innerContext = ((PagingContextHolder) fillContext).getInnerContext(); - if (innerContext instanceof ChannelContextWrapper) { - return ((ChannelContextWrapper) innerContext).getChannelContext(); - } - } - return SeekableChannelsProvider.ChannelContext.NULL; - } - - /** - * Wrapper class which takes a {@link SeekableChannelsProvider.ChannelContext} and implements {@link FillContext} + * Wrapper class which takes a {@link SeekableChannelsProvider.ChannelContext} and implements {@link FillContext}, + * used to update the inner context of {@link PagingContextHolder}. */ private static class ChannelContextWrapper implements FillContext { - private @NotNull final SeekableChannelsProvider.ChannelContext context; + @NotNull + private final SeekableChannelsProvider.ChannelContext channelContext; - ChannelContextWrapper(@NotNull SeekableChannelsProvider.ChannelContext context) { - this.context = context; + ChannelContextWrapper(@NotNull final SeekableChannelsProvider.ChannelContext context) { + this.channelContext = context; } SeekableChannelsProvider.ChannelContext getChannelContext() { - return context; + return channelContext; } @Override public void close() { - context.close(); + channelContext.close(); } } + /** + * 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 SeekableChannelsProvider.ChannelContext} to use for reading pages via + * {@link #columnChunkReader}. + */ + final SeekableChannelsProvider.ChannelContext 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 SeekableChannelsProvider.ChannelContext.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 SeekableChannelsProvider.ChannelContext 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 - : new ChannelContextWrapper(columnChunkReader.makeChannelContext())); + return (T) new ChannelContextWrapper(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 74611052d95..7b7e75acb66 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 @@ -99,8 +99,7 @@ private static int findPageNumUsingOffsetIndex(final OffsetIndex offsetIndex, fi return (low - 1); // 'row' is somewhere in the middle of page } - private ChunkPage getPage(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, - 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 + ")"); } @@ -117,15 +116,16 @@ private ChunkPage getPage(@NotNull final SeekableChannelsProvider.ChannelC synchronized (pageState) { // Make sure no one materialized this page as we waited for the lock if ((localRef = pageState.pageRef) == null || (page = localRef.get()) == null) { - final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(channelContext, pageNum); + // Use the latest context while reading the page + final SeekableChannelsProvider.ChannelContext 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); } pageState.pageRef = new WeakReference<>(page); - // Clear out the context to avoid retaining old copies - reader.clearChannelContext(); } } } @@ -153,9 +153,6 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext } } - // Use the latest context while reading the page - innerFillContext(fillContext); - final SeekableChannelsProvider.ChannelContext channelContext = getChannelContext(fillContext); - return getPage(channelContext, 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 c165b71d0f6..c8ef551fd0c 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 @@ -72,18 +72,16 @@ private void extendOnePage(@NotNull final SeekableChannelsProvider.ChannelContex pages = Arrays.copyOf(pages, newSize); } - columnPageReaderIterator.setChannelContext(channelContext); - final ColumnPageReader columnPageReader = columnPageReaderIterator.next(); - + final ColumnPageReader columnPageReader = columnPageReaderIterator.next(channelContext); long numRows; WeakReference> pageRef = PageCache.getNullPage(); 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(); } @@ -91,8 +89,6 @@ private void extendOnePage(@NotNull final SeekableChannelsProvider.ChannelContex throw new UncheckedIOException(except); } - // Clear out the context to avoid retaining old copies - columnPageReader.clearChannelContext(); columnPageReaders[localNumPages] = columnPageReader; pages[localNumPages] = pageRef; pageRowOffsets[localNumPages + 1] = prevRowOffset + numRows; @@ -129,14 +125,9 @@ private ChunkPage getPage(@NotNull final SeekableChannelsProvider.ChannelC if (page == null) { try { - // Use the latest context while reading the page - final ColumnPageReader columnPageReader = columnPageReaders[pageNum]; - columnPageReader.setChannelContext(channelContext); - page = new PageCache.IntrusivePage<>( - toPage(pageRowOffsets[pageNum], columnPageReaders[pageNum])); - // Clear out the context to avoid retaining old copies - columnPageReader.clearChannelContext(); - } catch (IOException except) { + page = new PageCache.IntrusivePage<>(toPage(pageRowOffsets[pageNum], columnPageReaders[pageNum], + channelContext)); + } catch (final IOException except) { throw new UncheckedIOException(except); } @@ -165,8 +156,7 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext } // Use the latest channel context while reading page headers - innerFillContext(fillContext); - final SeekableChannelsProvider.ChannelContext channelContext = getChannelContext(fillContext); + final SeekableChannelsProvider.ChannelContext channelContext = innerFillContext(fillContext); if (pageNum >= localNumPages) { final int minPageNum = fillToRow(channelContext, localNumPages, rowKey); 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 7e85674ea77..13e1fc47874 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 @@ -54,7 +54,7 @@ public interface Lookup { this.lookup = lookup; this.dictionarySupplier = dictionarySupplier; this.valuesSupplier = new LazyCachingSupplier<>(() -> { - // Dictionary is already materialized till this point, therefore we can safely use NULL context + // Dictionary is already materialized at this point, therefore, we can safely use NULL context final Dictionary dictionary = dictionarySupplier.apply(SeekableChannelsProvider.ChannelContext.NULL); final T[] values = ObjectChunk.makeArray(dictionary.getMaxId() + 1); for (int ki = 0; ki < values.length; ++ki) { 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..9f9b73f2545 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.parquet.base.util.SeekableChannelsProvider; 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, + SeekableChannelsProvider.ChannelContext 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 SeekableChannelsProvider.ChannelContext 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, + SeekableChannelsProvider.ChannelContext 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..ceb39e98ef6 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.parquet.base.util.SeekableChannelsProvider; 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 SeekableChannelsProvider.ChannelContext 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 SeekableChannelsProvider.ChannelContext 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/ToStringSetPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToStringSetPage.java index 66166a6f9da..8d3776fd02c 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.parquet.base.util.SeekableChannelsProvider; 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, + SeekableChannelsProvider.ChannelContext 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 6280a24831f..4b6655018bc 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 @@ -573,7 +573,7 @@ public void testArrayColumns() { } @Test - public void readLongParquetFileFromS3Test() { + public void readRefParquetFileFromS3Test() { final S3Instructions s3Instructions = S3Instructions.builder() .awsRegionName("us-east-2") .readAheadCount(1) @@ -586,7 +586,6 @@ public void readLongParquetFileFromS3Test() { final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3Instructions) .build(); - final TableDefinition tableDefinition = TableDefinition.of( ColumnDefinition.ofString("hash"), ColumnDefinition.ofLong("version"), @@ -604,18 +603,28 @@ public void readLongParquetFileFromS3Test() { ColumnDefinition.ofDouble("input_value")); final Table fromAws1 = 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(); + "s3://aws-public-blockchain/v1.0/btc/transactions/date=2009-01-03/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet", + readInstructions, tableDefinition).head(5).select(); final Table fromDisk1 = ParquetTools.readSingleFileTable( new File( - "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), + "/Users/shivammalhotra/Documents/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet"), ParquetTools.SNAPPY, - tableDefinition).select(); + tableDefinition).head(5).select(); assertTableEquals(fromAws1, fromDisk1); + + final Table fromAws2 = 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).head(5).select(); + final Table fromDisk2 = ParquetTools.readSingleFileTable( + new File( + "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), + ParquetTools.SNAPPY, + tableDefinition).head(5).select(); + assertTableEquals(fromAws2, fromDisk2); } @Test - public void readRefParquetFileFromS3Test() { + public void readLongParquetFileFromS3Test() { final S3Instructions s3Instructions = S3Instructions.builder() .awsRegionName("us-east-2") .readAheadCount(1) @@ -628,6 +637,7 @@ public void readRefParquetFileFromS3Test() { final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3Instructions) .build(); + final TableDefinition tableDefinition = TableDefinition.of( ColumnDefinition.ofString("hash"), ColumnDefinition.ofLong("version"), @@ -645,24 +655,14 @@ public void readRefParquetFileFromS3Test() { ColumnDefinition.ofDouble("input_value")); final Table fromAws1 = 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).head(5).select(); - final Table fromDisk1 = ParquetTools.readSingleFileTable( - new File( - "/Users/shivammalhotra/Documents/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet"), - ParquetTools.SNAPPY, - tableDefinition).head(5).select(); - assertTableEquals(fromAws1, fromDisk1); - - final Table fromAws2 = 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).head(5).select(); - final Table fromDisk2 = ParquetTools.readSingleFileTable( + readInstructions, tableDefinition).select(); + final Table fromDisk1 = ParquetTools.readSingleFileTable( new File( "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), ParquetTools.SNAPPY, - tableDefinition).head(5).select(); - assertTableEquals(fromAws2, fromDisk2); + tableDefinition).select(); + assertTableEquals(fromAws1, fromDisk1); } @Test diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index f9fefd2beee..e76e9f159e1 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -52,6 +52,11 @@ public ChannelContext makeContext() { return new S3SeekableByteChannel.S3ChannelContext(maxCacheSize); } + @Override + public boolean isCompatibleWith(@NotNull ChannelContext context) { + return context instanceof S3SeekableByteChannel.S3ChannelContext; + } + @Override public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boolean append) { throw new UnsupportedOperationException("Don't support writing to S3 yet"); diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index 3a62f2d7aa1..9e7a5502103 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -46,9 +46,15 @@ static SeekableChannelsProvider getInstance() { @Override public ChannelContext makeContext() { + // No additional context required for local FS return ChannelContext.NULL; } + @Override + public boolean isCompatibleWith(@NotNull ChannelContext context) { + return context == ChannelContext.NULL; + } + @Override public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { 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..d93e6ccc0a4 --- /dev/null +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java @@ -0,0 +1,29 @@ +/** + * 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\\)"}, + {"T get\\(\\)", "OUTPUT_TYPE apply\\(final INPUT_TYPE arg\\)"}, + {"SoftReference", "SoftReference"}, + {"T current", "OUTPUT_TYPE current"}, + {"Supplier", "Function"}, + {"supplier", "function"}, + }; + replaceAll(LAZY_CACHING_SUPPLIER_PATH, LAZY_CACHING_FUNCTION_PATH, null, NO_EXCEPTIONS, pairs); + } +} From 11acf6e1929c705fea1a291f2846f68349df857c Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 12 Jan 2024 17:36:12 +0530 Subject: [PATCH 19/39] Fixed one test failure --- .../parquet/base/util/CachedChannelProviderTest.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java index 851fbc078a9..4599927ed9a 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java @@ -167,6 +167,11 @@ public ChannelContext makeContext() { return ChannelContext.NULL; } + @Override + public boolean isCompatibleWith(@NotNull ChannelContext context) { + return context == ChannelContext.NULL; + } + @Override public SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull String path) { return new TestMockChannel(count.getAndIncrement(), path); From ae5a37101276804a426c9a0618b6b0e1288845d8 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 12 Jan 2024 18:45:39 +0530 Subject: [PATCH 20/39] Resolving Devin's comments --- .../locations/local/URITableLocationKey.java | 3 + .../base/util/SeekableChannelsProvider.java | 4 +- .../deephaven/parquet/table/ParquetTools.java | 2 +- .../table/location/ParquetColumnLocation.java | 4 +- .../table/location/ParquetTableLocation.java | 9 +- .../location/ParquetTableLocationFactory.java | 2 +- .../extensions/s3/S3SeekableByteChannel.java | 37 ++++---- .../TrackedSeekableChannelsProvider.java | 15 +-- py/server/deephaven/experimental/s3.py | 63 ++++++++++++ py/server/deephaven/parquet.py | 95 ++----------------- 10 files changed, 103 insertions(+), 131 deletions(-) create mode 100644 py/server/deephaven/experimental/s3.py 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 index baa388e4c9a..e5852c6dcf2 100644 --- 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 @@ -48,6 +48,9 @@ public class URITableLocationKey extends PartitionedTableLocationKey { 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; } 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 index 624a88f58e0..685bb5a454c 100644 --- 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 @@ -29,11 +29,11 @@ static URI convertToURI(final String source) { uri = new URI(source); } catch (final URISyntaxException e) { // If the URI is invalid, assume it's a file path - return new File(source).getAbsoluteFile().toURI(); + return new File(source).toURI(); } if (uri.getScheme() == null) { // Need to convert to a "file" URI - return new File(source).getAbsoluteFile().toURI(); + return new File(source).toURI(); } return uri; } 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 80b997199f1..6a2b2098e01 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 @@ -681,7 +681,7 @@ private static Table readTableInternal( private static Table readTableInternal( @NotNull final URI source, @NotNull final ParquetInstructions instructions) { - if (!source.getScheme().equals(FILE_URI_SCHEME)) { + if (!FILE_URI_SCHEME.equals(source.getScheme())) { return readSingleFileTable(source, instructions); } return readTableInternal(new File(source), instructions); 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 1e90f6d4033..61bc4ed3d6f 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 @@ -166,8 +166,8 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition if (!hasGroupingTable) { return null; } - final URI parquetFileURI = tl().getParquetFile(); - Assert.assertion(parquetFileURI.getScheme().equals(FILE_URI_SCHEME), + 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 { 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 25f7f0409a3..735bb2f6cd5 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 @@ -26,7 +26,6 @@ import org.jetbrains.annotations.NotNull; import java.io.File; -import java.net.URI; import java.util.*; import java.util.stream.IntStream; @@ -90,10 +89,10 @@ public ParquetTableLocation(@NotNull final TableKey tableKey, columnTypes = tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); version = tableInfo.map(TableInfo::version).orElse(null); - if (!tableLocationKey.getURI().getScheme().equals(FILE_URI_SCHEME)) { + if (!FILE_URI_SCHEME.equals(tableLocationKey.getURI().getScheme())) { handleUpdate(computeIndex(), 0L); // TODO What should I put here? } else { - handleUpdate(computeIndex(), new File(tableLocationKey.getURI().toString()).lastModified()); + handleUpdate(computeIndex(), new File(tableLocationKey.getURI()).lastModified()); } } @@ -105,8 +104,8 @@ public String getImplementationName() { @Override public void refresh() {} - URI getParquetFile() { - return ((ParquetTableLocationKey) getKey()).getURI(); + ParquetTableLocationKey getParquetKey() { + return (ParquetTableLocationKey) getKey(); } ParquetInstructions getReadInstructions() { 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 8222c10b961..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 @@ -34,7 +34,7 @@ public TableLocation makeLocation(@NotNull final TableKey tableKey, @NotNull final ParquetTableLocationKey locationKey, @Nullable final TableDataRefreshService refreshService) { final URI parquetFileURI = locationKey.getURI(); - if (!parquetFileURI.getScheme().equals(FILE_URI_SCHEME) || new File(parquetFileURI).exists()) { + 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/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index d6018cb6b65..4e829466ef3 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -182,26 +182,18 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep return -1; } - // Send async read request the current fragment, if it's not already in the cache + // Send async read requests for current fragment as well as read ahead fragments, if not already in cache final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); - CompletableFuture currFragmentFuture = context.getCachedFuture(currFragmentIndex); - if (currFragmentFuture == null) { - currFragmentFuture = computeFragmentFuture(currFragmentIndex); - context.setFragmentContext(currFragmentIndex, currFragmentFuture); - } - - // Send async requests for read-ahead buffers and store them in the cache - final int numFragmentsToLoad = Math.min(readAheadCount, numFragmentsInObject - currFragmentIndex - 1); - for (int i = 0; i < numFragmentsToLoad; i++) { - final int readAheadFragmentIndex = i + currFragmentIndex + 1; - final CompletableFuture readAheadFragmentFuture = - context.getCachedFuture(readAheadFragmentIndex); - if (readAheadFragmentFuture == null) { - context.setFragmentContext(readAheadFragmentIndex, computeFragmentFuture(readAheadFragmentIndex)); + final int numReadAheadFragments = Math.min(readAheadCount, numFragmentsInObject - currFragmentIndex - 1); + for (int idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { + final CompletableFuture future = context.getCachedFuture(idx); + if (future == null) { + context.setFragmentContext(idx, sendAsyncRequest(idx)); } } // Wait till the current fragment is fetched + final CompletableFuture currFragmentFuture = context.getCachedFuture(currFragmentIndex); final ByteBuffer currentFragment; try { currentFragment = currFragmentFuture.get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); @@ -230,7 +222,10 @@ private int fragmentIndexForByteNumber(final long byteNumber) { return Math.toIntExact(byteNumber / fragmentSize); } - private CompletableFuture computeFragmentFuture(final int fragmentIndex) { + /** + * @return A {@link CompletableFuture} that will be completed with the bytes of the fragment + */ + private CompletableFuture sendAsyncRequest(final int fragmentIndex) { final long readFrom = (long) fragmentIndex * fragmentSize; final long readTo = Math.min(readFrom + fragmentSize, size) - 1; final String range = "bytes=" + readFrom + "-" + readTo; @@ -278,15 +273,17 @@ public SeekableByteChannel position(final long newPosition) throws ClosedChannel @Override public long size() throws ClosedChannelException { checkClosed(position); - if (size < 0) { - populateSize(); - } + populateSize(); return size; } private void populateSize() { + if (size >= 0) { + return; + } if (context.getSize() < 0) { - // Fetch the size of the file on the first read using a blocking HEAD request + // 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(builder -> builder.bucket(bucket).key(key)) diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index 9e7a5502103..92638959cc6 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -25,19 +25,6 @@ */ class TrackedSeekableChannelsProvider implements SeekableChannelsProvider { - private static volatile SeekableChannelsProvider instance; - - static SeekableChannelsProvider getInstance() { - if (instance == null) { - synchronized (TrackedSeekableChannelsProvider.class) { - if (instance == null) { - return instance = new TrackedSeekableChannelsProvider(TrackedFileHandleFactory.getInstance()); - } - } - } - return instance; - } - private final TrackedFileHandleFactory fileHandleFactory; TrackedSeekableChannelsProvider(@NotNull final TrackedFileHandleFactory fileHandleFactory) { @@ -59,7 +46,7 @@ public boolean isCompatibleWith(@NotNull ChannelContext context) { public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) throws IOException { // context is unused here because it is NULL - Assert.assertion(uri.getScheme().equals(FILE_URI_SCHEME), "Expected a file uri, got " + uri); + Assert.assertion(FILE_URI_SCHEME.equals(uri.getScheme()), "Expected a file uri, got " + uri); return new TrackedSeekableByteChannel(fileHandleFactory.readOnlyHandleCreator, new File(uri)); } diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py new file mode 100644 index 00000000000..50cd375e5b5 --- /dev/null +++ b/py/server/deephaven/experimental/s3.py @@ -0,0 +1,63 @@ +# +# Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending +# +""" + This module is useful for reading and writing files stored in S3. Importing this module requires the S3 specific + extensions to be included in the class path. +""" + +import jpy + +_JDuration = jpy.get_type("java.time.Duration") +_JS3Instructions = jpy.get_type("io.deephaven.extensions.s3.S3Instructions") + +def _build_s3_instructions( + aws_region_name: str, + max_concurrent_requests: int = None, + read_ahead_count: int = None, + fragment_size: int = None, + max_cache_size: int = None, + connection_timeout: _JDuration = None, + read_timeout: _JDuration = None, +): + """ + Build specialized instructions for accessing files stored in AWS S3. + + Args: + aws_region_name (str): the AWS region name for reading parquet files stored in AWS S3, by default None + max_concurrent_requests (int): the maximum number of concurrent requests for reading parquet files stored in S3, + by default 50. + read_ahead_count (int): the number of fragments to send asynchronous read requests for while reading the current + fragment, defaults to 1. + fragment_size (int): the maximum size of each fragment to read from S3. The fetched fragment can be smaller than + this in case fewer bytes remaining in the file, defaults to 5 MB. + max_cache_size (int): the maximum number of fragments to cache in memory while reading, defaults to 32. + connection_timeout (Duration): the amount of time to wait when initially establishing a connection before giving + up and timing out, defaults to 2 seconds. + read_timeout (Duration): the amount of time to wait when reading a fragment before giving up and timing out, + defaults to 2 seconds. + """ + + builder = _JS3Instructions.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(connection_timeout) + + if read_timeout is not None: + builder.readTimeout(read_timeout) + + return builder.build() + diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index acb3f5a75f7..945ac7b4591 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -19,42 +19,9 @@ _JFile = jpy.get_type("java.io.File") _JCompressionCodecName = jpy.get_type("org.apache.parquet.hadoop.metadata.CompressionCodecName") _JParquetInstructions = jpy.get_type("io.deephaven.parquet.table.ParquetInstructions") -_JS3Instructions = jpy.get_type("io.deephaven.extensions.s3.S3Instructions") _JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") _JDuration = jpy.get_type("java.time.Duration") -def _build_s3_parquet_instructions( - aws_region_name: str, - max_concurrent_requests: int = None, - read_ahead_count: int = None, - fragment_size: int = None, - max_cache_size: int = None, - connection_timeout: _JDuration = None, - read_timeout: _JDuration = None, -): - builder = _JS3Instructions.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(connection_timeout) - - if read_timeout is not None: - builder.readTimeout(read_timeout) - - return builder.build() - @dataclass class ColumnInstruction: @@ -76,13 +43,7 @@ def _build_parquet_instructions( is_refreshing: bool = False, for_read: bool = True, force_build: bool = False, - aws_region_name: str = None, - max_concurrent_requests: int = None, - read_ahead_count: int = None, - fragment_size: int = None, - max_cache_size: int = None, - connection_timeout: _JDuration = None, - read_timeout: _JDuration = None, + special_instructions: Optional[object] = None, ): if not any( [ @@ -94,13 +55,7 @@ def _build_parquet_instructions( is_legacy_parquet, target_page_size is not None, is_refreshing, - aws_region_name is not None, - max_concurrent_requests is not None, - read_ahead_count is not None, - fragment_size is not None, - max_cache_size is not None, - connection_timeout is not None, - read_timeout is not None, + special_instructions is not None ] ): return None @@ -137,17 +92,8 @@ def _build_parquet_instructions( if is_refreshing: builder.setIsRefreshing(is_refreshing) - if aws_region_name is not None: - s3_parquet_instructions = _build_s3_parquet_instructions( - aws_region_name=aws_region_name, - max_concurrent_requests=max_concurrent_requests, - read_ahead_count=read_ahead_count, - fragment_size=fragment_size, - max_cache_size=max_cache_size, - connection_timeout=connection_timeout, - read_timeout=read_timeout, - ) - builder.setSpecialInstructions(s3_parquet_instructions) + if special_instructions is not None: + builder.setSpecialInstructions(special_instructions) return builder.build() @@ -192,13 +138,7 @@ def read( is_refreshing: bool = False, file_layout: Optional[ParquetFileLayout] = None, table_definition: Union[Dict[str, DType], List[Column], None] = None, - aws_region_name: str = None, - max_concurrent_requests: int = None, - read_ahead_count: int = None, - fragment_size: int = None, - max_cache_size: int = None, - connection_timeout: _JDuration = None, - read_timeout: _JDuration = None, + special_instructions: Optional[object] = None, ) -> Table: """ Reads in a table from a single parquet, metadata file, or directory with recognized layout. @@ -212,22 +152,11 @@ 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. - aws_region_name (str): the AWS region name for reading parquet files stored in AWS S3, by default None - max_concurrent_requests (int): the maximum number of concurrent requests for reading parquet files stored in S3, - by default 50. - read_ahead_count (int): the number of fragments to send asynchronous read requests for while reading the current - fragment, defaults to 1. - fragment_size (int): the maximum size of each fragment to read from S3. The fetched fragment can be smaller than - this in case fewer bytes remaining in the file, defaults to 5 MB. - max_cache_size (int): the maximum number of fragments to cache in memory, defaults to 32. - connection_timeout (Duration): the amount of time to wait when initially establishing a connection before giving - up and timing out, defaults to 2 seconds. - read_timeout (Duration): the amount of time to wait when reading a fragment before giving up and timing out, - defaults to 2 seconds - # TODO Make sure all the defaults are correct + special_instructions (Optional[object]): Special instructions for reading parquet files, useful when reading + files from a non-local file system, like S3. By default, None. When Returns: a table @@ -243,13 +172,7 @@ def read( is_refreshing=is_refreshing, for_read=True, force_build=True, - aws_region_name=aws_region_name, - max_concurrent_requests=max_concurrent_requests, - read_ahead_count=read_ahead_count, - fragment_size=fragment_size, - max_cache_size=max_cache_size, - connection_timeout=connection_timeout, - read_timeout=read_timeout, + special_instructions=special_instructions, ) j_table_definition = _j_table_definition(table_definition) if j_table_definition is not None: From 2dc360e40e9ed931a0b353f3b9ff736bb918964b Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 12 Jan 2024 18:53:40 +0530 Subject: [PATCH 21/39] Cosmetic cleanups --- .../util/datastructures/LazyCachingFunction.java | 2 ++ .../util/datastructures/LazyCachingSupplier.java | 16 +++++++++------- .../replicators/ReplicateCachingSupplier.java | 6 ++---- 3 files changed, 13 insertions(+), 11 deletions(-) diff --git a/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java index da138aa5f40..98e2d47fb79 100644 --- a/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java +++ b/Util/src/main/java/io/deephaven/util/datastructures/LazyCachingFunction.java @@ -16,6 +16,8 @@ /** * {@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 { 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/replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java index d93e6ccc0a4..aeeae50e6d4 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateCachingSupplier.java @@ -16,11 +16,9 @@ public class ReplicateCachingSupplier { public static void main(final String[] args) throws IOException { final String[][] pairs = new String[][] { - {"Supplier", "Function"}, + {"Supplier", "Function"}, {"internalSupplier\\.get\\(\\)", "internalFunction\\.apply\\(arg\\)"}, - {"T get\\(\\)", "OUTPUT_TYPE apply\\(final INPUT_TYPE arg\\)"}, - {"SoftReference", "SoftReference"}, - {"T current", "OUTPUT_TYPE current"}, + {"OUTPUT_TYPE get\\(\\)", "OUTPUT_TYPE apply\\(final INPUT_TYPE arg\\)"}, {"Supplier", "Function"}, {"supplier", "function"}, }; From 2840ec5ced74e078c56624a03973da7009a8a9e7 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Sat, 13 Jan 2024 15:59:28 +0530 Subject: [PATCH 22/39] Resolving Ryan's comments part 2 --- .../parquet/base/ColumnChunkReaderImpl.java | 4 +- .../parquet/base/ParquetFileReader.java | 5 +- .../parquet/base/RowGroupReaderImpl.java | 4 +- .../base/util/CachedChannelProvider.java | 14 +- .../base/util/LocalFSChannelProvider.java | 6 +- .../base/util/SeekableChannelsProvider.java | 12 +- .../base/util/CachedChannelProviderTest.java | 8 +- .../deephaven/parquet/table/ParquetTools.java | 14 +- .../table/location/ParquetTableLocation.java | 4 +- .../table/pagestore/ColumnChunkPageStore.java | 17 +- .../pagestore/topage/ChunkDictionary.java | 8 +- .../ByteBufferAsyncResponseTransformer.java | 6 +- .../extensions/s3/S3SeekableByteChannel.java | 147 +++++++++--------- .../s3/S3SeekableChannelProvider.java | 25 ++- .../TrackedSeekableChannelsProvider.java | 7 +- py/server/deephaven/parquet.py | 56 +++---- 16 files changed, 168 insertions(+), 169 deletions(-) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 812951eff2d..36c9b2c0a89 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 @@ -176,9 +176,9 @@ private Dictionary getDictionary(final SeekableChannelsProvider.ChannelContext c return getDictionaryHelper(channelContext, dictionaryPageOffset); } - private Dictionary getDictionaryHelper(final SeekableChannelsProvider.ChannelContext context, + private Dictionary getDictionaryHelper(final SeekableChannelsProvider.ChannelContext channelContext, final long dictionaryPageOffset) { - try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, getURI())) { + try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, getURI())) { readChannel.position(dictionaryPageOffset); return readDictionary(readChannel); } catch (final IOException e) { 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 67321244255..b43f59cd7ad 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 @@ -64,8 +64,9 @@ public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvide rootURI = parquetFileURI; } final byte[] footer; - try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); - final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, parquetFileURI)) { + try (final SeekableChannelsProvider.ChannelContext 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 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 a495fe4abfd..a46937a17d1 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 @@ -79,8 +79,8 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { OffsetIndex offsetIndex = null; if (columnChunk.isSetOffset_index_offset()) { - try (final SeekableChannelsProvider.ChannelContext context = channelsProvider.makeContext(); - final SeekableByteChannel readChannel = channelsProvider.getReadChannel(context, rootURI)) { + try (final SeekableChannelsProvider.ChannelContext channelContext = channelsProvider.makeContext(); + 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))); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java index 047060c5dff..681e1271209 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java @@ -58,20 +58,20 @@ public ChannelContext makeContext() { } @Override - public boolean isCompatibleWith(@NotNull final ChannelContext context) { - return wrappedProvider.isCompatibleWith(context); + public boolean isCompatibleWith(@NotNull final ChannelContext channelContext) { + return wrappedProvider.isCompatibleWith(channelContext); } @Override - public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) + public SeekableByteChannel getReadChannel(@NotNull final ChannelContext channelContext, @NotNull final URI uri) throws IOException { final String uriString = uri.toString(); final KeyedObjectHashMap channelPool = channelPools.get(ChannelType.Read); final CachedChannel result = tryGetPooledChannel(uriString, channelPool); final CachedChannel channel = result == null - ? new CachedChannel(wrappedProvider.getReadChannel(context, uri), ChannelType.Read, uriString) + ? new CachedChannel(wrappedProvider.getReadChannel(channelContext, uri), ChannelType.Read, uriString) : result.position(0); - channel.setContext(context); + channel.setContext(channelContext); return channel; } @@ -224,9 +224,9 @@ private void dispose() throws IOException { } @Override - public void setContext(ChannelContext context) { + public void setContext(ChannelContext channelContext) { if (wrappedChannel instanceof ContextHolder) { - ((ContextHolder) wrappedChannel).setContext(context); + ((ContextHolder) wrappedChannel).setContext(channelContext); } } } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java index c83237954f9..ca8737f5e6e 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java @@ -21,12 +21,12 @@ public ChannelContext makeContext() { } @Override - public boolean isCompatibleWith(@NotNull final ChannelContext context) { - return context == ChannelContext.NULL; + public boolean isCompatibleWith(@NotNull final ChannelContext channelContext) { + return channelContext == ChannelContext.NULL; } @Override - public SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) + public SeekableByteChannel getReadChannel(@NotNull final ChannelContext channelContext, @NotNull final URI uri) throws IOException { // context is unused here because it is NULL return FileChannel.open(Path.of(uri), StandardOpenOption.READ); 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 index 685bb5a454c..65ec0498a54 100644 --- 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 @@ -55,12 +55,12 @@ default void close() {} /** * 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. + * {@code channelContext} object for creating channels with this provider. */ - boolean isCompatibleWith(@NotNull ChannelContext context); + boolean isCompatibleWith(@NotNull ChannelContext channelContext); interface ContextHolder { - void setContext(ChannelContext context); + void setContext(ChannelContext channelContext); @FinalDefault default void clearContext() { @@ -68,12 +68,12 @@ default void clearContext() { } } - default SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull String uriStr) + default SeekableByteChannel getReadChannel(@NotNull ChannelContext channelContext, @NotNull String uriStr) throws IOException { - return getReadChannel(context, convertToURI(uriStr)); + return getReadChannel(channelContext, convertToURI(uriStr)); } - SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull URI uri) throws IOException; + SeekableByteChannel getReadChannel(@NotNull ChannelContext channelContext, @NotNull URI uri) throws IOException; default SeekableByteChannel getWriteChannel(@NotNull String path, final boolean append) throws IOException { return getWriteChannel(Paths.get(path), append); diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java index 4599927ed9a..a3950f8119e 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java @@ -168,17 +168,17 @@ public ChannelContext makeContext() { } @Override - public boolean isCompatibleWith(@NotNull ChannelContext context) { - return context == ChannelContext.NULL; + public boolean isCompatibleWith(@NotNull ChannelContext channelContext) { + return channelContext == ChannelContext.NULL; } @Override - public SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull String path) { + public SeekableByteChannel getReadChannel(@NotNull ChannelContext channelContext, @NotNull String path) { return new TestMockChannel(count.getAndIncrement(), path); } @Override - public SeekableByteChannel getReadChannel(@NotNull ChannelContext context, @NotNull URI uri) { + public SeekableByteChannel getReadChannel(@NotNull ChannelContext channelContext, @NotNull URI uri) { return new TestMockChannel(count.getAndIncrement(), uri.toString()); } 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 6a2b2098e01..f472209dee5 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 @@ -72,8 +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. The source provided - * can be a local file path or a URI to be resolved via the provided {@link SeekableChannelsProviderPlugin}. + * 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, @@ -100,8 +101,9 @@ public static Table readTable(@NotNull final String source) { } /** - * Reads in a table from a single parquet, 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}. + * 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, @@ -130,7 +132,7 @@ public static Table readTable( } /** - * 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, @@ -156,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, 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 735bb2f6cd5..d06c03feb04 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,6 +4,7 @@ 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.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.ParquetSchemaReader; @@ -90,7 +91,8 @@ public ParquetTableLocation(@NotNull final TableKey tableKey, version = tableInfo.map(TableInfo::version).orElse(null); if (!FILE_URI_SCHEME.equals(tableLocationKey.getURI().getScheme())) { - handleUpdate(computeIndex(), 0L); // TODO What should I put here? + // 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()); } 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 0536125f662..12f35dd9883 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 @@ -174,23 +174,28 @@ public boolean usesDictionaryOnEveryPage() { public void close() {} /** - * Wrapper class which takes a {@link SeekableChannelsProvider.ChannelContext} and implements {@link FillContext}, - * used to update the inner context of {@link PagingContextHolder}. + * Wrapper class for holding a {@link SeekableChannelsProvider.ChannelContext}. */ - private static class ChannelContextWrapper implements FillContext { + private static class ChannelContextWrapper extends PagingContextHolder { @NotNull private final SeekableChannelsProvider.ChannelContext channelContext; - ChannelContextWrapper(@NotNull final SeekableChannelsProvider.ChannelContext context) { - this.channelContext = context; + private ChannelContextWrapper( + final int chunkCapacity, + @Nullable final SharedContext sharedContext, + @NotNull final SeekableChannelsProvider.ChannelContext channelContext) { + super(chunkCapacity, sharedContext); + this.channelContext = channelContext; } + @NotNull SeekableChannelsProvider.ChannelContext getChannelContext() { return channelContext; } @Override public void close() { + super.close(); channelContext.close(); } } @@ -229,6 +234,6 @@ private T fillContextUpdater( } // Create a new context object // noinspection unchecked - return (T) new ChannelContextWrapper(channelsProvider.makeContext()); + return (T) new ChannelContextWrapper(chunkCapacity, sharedContext, channelsProvider.makeContext()); } } 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 13e1fc47874..1d9a1731799 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 @@ -19,8 +19,7 @@ /** * 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 { @@ -36,9 +35,6 @@ public interface Lookup { T lookup(@NotNull final Dictionary dictionary, final int key); } - private final Lookup lookup; - private final Function dictionarySupplier; - private final Supplier> valuesSupplier; private final Supplier> reverseMapSupplier; @@ -51,8 +47,6 @@ public interface Lookup { ChunkDictionary( @NotNull final Lookup lookup, @NotNull final Function dictionarySupplier) { - this.lookup = lookup; - this.dictionarySupplier = dictionarySupplier; this.valuesSupplier = new LazyCachingSupplier<>(() -> { // Dictionary is already materialized at this point, therefore, we can safely use NULL context final Dictionary dictionary = dictionarySupplier.apply(SeekableChannelsProvider.ChannelContext.NULL); 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 index a26d8f55233..005387a7a2e 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java @@ -34,7 +34,7 @@ public void onResponse(final ResponseT response) { @Override public void onStream(final SdkPublisher publisher) { - // TODO Can be improved with a buffer pool + // This could be improved with the addition of a buffer pool or similar resource allocation sharing support publisher.subscribe(new ByteBufferSubscriber(cf, ByteBuffer.allocate(bufferSize))); } @@ -43,7 +43,7 @@ public void exceptionOccurred(final Throwable throwable) { cf.completeExceptionally(throwable); } - final static class ByteBufferSubscriber implements Subscriber { + private static final class ByteBufferSubscriber implements Subscriber { private final CompletableFuture resultFuture; private Subscription subscription; private final ByteBuffer byteBuffer; @@ -79,7 +79,7 @@ public void onError(final Throwable throwable) { @Override public void onComplete() { - resultFuture.complete(byteBuffer.asReadOnlyBuffer()); + resultFuture.complete(byteBuffer.flip().asReadOnlyBuffer()); } } } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index 4e829466ef3..639aa5963b6 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -3,7 +3,6 @@ */ package io.deephaven.extensions.s3; -import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; import java.util.concurrent.CancellationException; import io.deephaven.parquet.base.util.SeekableChannelsProvider; @@ -19,9 +18,6 @@ import java.nio.channels.ClosedChannelException; import java.nio.channels.NonWritableChannelException; import java.nio.channels.SeekableByteChannel; -import java.time.Duration; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -36,15 +32,18 @@ final class S3SeekableByteChannel implements SeekableByteChannel, SeekableChanne private static final long CLOSED_SENTINEL = -1; + private static final long UNINITIALIZED_SIZE = -1; + /** - * Context object used to store read-ahead buffers for efficiently reading from S3. + * Channel context object used to store read-ahead buffers for efficiently reading from S3. */ static final class S3ChannelContext implements SeekableChannelsProvider.ChannelContext { /** * Used to store information related to a single fragment */ - static class FragmentContext { + private static final class FragmentState { + /** * The index of the fragment in the object */ @@ -55,7 +54,7 @@ static class FragmentContext { */ private CompletableFuture future; - private FragmentContext(final int fragmentIndex, final CompletableFuture future) { + private FragmentState(final int fragmentIndex, final CompletableFuture future) { this.fragmentIndex = fragmentIndex; this.future = future; } @@ -64,27 +63,27 @@ private FragmentContext(final int fragmentIndex, final CompletableFuture bufferCache; + 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 ArrayList<>(maxCacheSize); - for (int i = 0; i < maxCacheSize; i++) { - bufferCache.add(null); - } - size = -1; + bufferCache = new FragmentState[maxCacheSize]; + size = UNINITIALIZED_SIZE; } private int getIndex(final int fragmentIndex) { - return fragmentIndex % bufferCache.size(); + return fragmentIndex % bufferCache.length; } - void setFragmentContext(final int fragmentIndex, final CompletableFuture future) { + void setFragmentState(final int fragmentIndex, final CompletableFuture future) { final int cacheIdx = getIndex(fragmentIndex); - final FragmentContext cachedEntry = bufferCache.get(cacheIdx); + final FragmentState cachedEntry = bufferCache[cacheIdx]; if (cachedEntry == null) { - bufferCache.set(cacheIdx, new FragmentContext(fragmentIndex, future)); + bufferCache[cacheIdx] = new FragmentState(fragmentIndex, future); } else { // We should not cache an already cached fragment Assert.neq(cachedEntry.fragmentIndex, "cachedEntry.fragmentIndex", fragmentIndex, "fragmentIdx"); @@ -104,7 +103,7 @@ void setFragmentContext(final int fragmentIndex, final CompletableFuture getCachedFuture(final int fragmentIndex) { - final FragmentContext cachedFragment = bufferCache.get(getIndex(fragmentIndex)); + final FragmentState cachedFragment = bufferCache[getIndex(fragmentIndex)]; if (cachedFragment != null && cachedFragment.fragmentIndex == fragmentIndex) { return cachedFragment.future; } @@ -123,9 +122,7 @@ void setSize(final long size) { private final S3AsyncClient s3AsyncClient; private final String bucket; private final String key; - private final int fragmentSize; - private final int readAheadCount; - private final Duration readTimeout; + private final S3Instructions s3Instructions; /** * The size of the object in bytes, fetched at the time of first read @@ -133,41 +130,38 @@ void setSize(final long size) { private long size; private int numFragmentsInObject; - private S3ChannelContext context; + private S3ChannelContext s3ChannelContext; private long position; - S3SeekableByteChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, @NotNull final URI uri, - @NotNull final S3AsyncClient s3AsyncClient, final int fragmentSize, final int readAheadCount, - final Duration readTimeout) { + S3SeekableByteChannel(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, @NotNull final URI uri, + @NotNull final S3AsyncClient s3AsyncClient, final S3Instructions s3Instructions) { final S3Uri s3Uri = s3AsyncClient.utilities().parseUri(uri); this.bucket = s3Uri.bucket().orElse(null); this.key = s3Uri.key().orElse(null); this.s3AsyncClient = s3AsyncClient; - this.fragmentSize = fragmentSize; - this.readAheadCount = readAheadCount; - this.readTimeout = readTimeout; - Assert.instanceOf(context, "context", S3ChannelContext.class); - this.context = (S3ChannelContext) context; - this.size = -1; + this.s3Instructions = s3Instructions; + Assert.instanceOf(channelContext, "channelContext", S3ChannelContext.class); + this.s3ChannelContext = (S3ChannelContext) channelContext; + this.size = UNINITIALIZED_SIZE; this.position = 0; } @Override - public void setContext(@Nullable final SeekableChannelsProvider.ChannelContext context) { + public void setContext(@Nullable final SeekableChannelsProvider.ChannelContext channelContext) { // null context equivalent to clearing the context - if (context != null && !(context instanceof S3ChannelContext)) { + if (channelContext != null && !(channelContext instanceof S3ChannelContext)) { throw new IllegalArgumentException( - "context must be null or an instance of ChannelContext, provided context" + - " of class " + context.getClass().getName()); + "Context must be null or an instance of S3ChannelContext, provided context of class " + + channelContext.getClass().getName()); } - this.context = (S3ChannelContext) context; + this.s3ChannelContext = (S3ChannelContext) channelContext; } @Override - public int read(@NotNull final ByteBuffer destination) throws ClosedChannelException { - Assert.neqNull(context, "context"); - Assert.neq(context, "context", SeekableChannelsProvider.ChannelContext.NULL, + public int read(@NotNull final ByteBuffer destination) throws IOException { + Assert.neqNull(s3ChannelContext, "s3ChannelContext"); + Assert.neq(s3ChannelContext, "s3ChannelContext", SeekableChannelsProvider.ChannelContext.NULL, "SeekableChannelsProvider.ChannelContext.NULL"); if (!destination.hasRemaining()) { return 0; @@ -184,72 +178,77 @@ public int read(@NotNull final ByteBuffer destination) throws ClosedChannelExcep // Send async read requests for current fragment as well as read ahead fragments, if not already in cache final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); - final int numReadAheadFragments = Math.min(readAheadCount, numFragmentsInObject - currFragmentIndex - 1); + final int numReadAheadFragments = + Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); for (int idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { - final CompletableFuture future = context.getCachedFuture(idx); + final CompletableFuture future = s3ChannelContext.getCachedFuture(idx); if (future == null) { - context.setFragmentContext(idx, sendAsyncRequest(idx)); + s3ChannelContext.setFragmentState(idx, sendAsyncRequest(idx)); } } // Wait till the current fragment is fetched - final CompletableFuture currFragmentFuture = context.getCachedFuture(currFragmentIndex); + final CompletableFuture currFragmentFuture = s3ChannelContext.getCachedFuture(currFragmentIndex); final ByteBuffer currentFragment; try { - currentFragment = currFragmentFuture.get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); + currentFragment = currFragmentFuture.get(s3Instructions.readTimeout().toNanos(), TimeUnit.NANOSECONDS); } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { - final String operation = - "fetching fragment " + currFragmentIndex + " for file " + key + " in S3 bucket " + bucket; - throw handleS3Exception(e, operation); + 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 * fragmentSize)); + final int fragmentOffset = (int) (localPosition - (currFragmentIndex * s3Instructions.fragmentSize())); currentFragment.position(fragmentOffset); - final int limit = Math.min(currentFragment.remaining(), destination.remaining()); + final int sizeToCopy = Math.min(currentFragment.remaining(), destination.remaining()); final int originalBufferLimit = currentFragment.limit(); - currentFragment.limit(currentFragment.position() + limit); + currentFragment.limit(currentFragment.position() + sizeToCopy); destination.put(currentFragment); // Need to reset buffer limit, so we can read from the same buffer again in future currentFragment.limit(originalBufferLimit); - position = localPosition + limit; - return limit; + position = localPosition + sizeToCopy; + return sizeToCopy; } private int fragmentIndexForByteNumber(final long byteNumber) { - return Math.toIntExact(byteNumber / fragmentSize); + return Math.toIntExact(byteNumber / s3Instructions.fragmentSize()); } /** * @return A {@link CompletableFuture} that will be completed with the bytes of the fragment */ private CompletableFuture sendAsyncRequest(final int fragmentIndex) { + final int fragmentSize = s3Instructions.fragmentSize(); final long readFrom = (long) fragmentIndex * fragmentSize; final long readTo = Math.min(readFrom + fragmentSize, size) - 1; final String range = "bytes=" + readFrom + "-" + readTo; return s3AsyncClient.getObject(builder -> builder.bucket(bucket).key(key).range(range), - new ByteBufferAsyncResponseTransformer<>(fragmentSize)); + new ByteBufferAsyncResponseTransformer<>((int) (readTo - readFrom + 1))); } - private UncheckedDeephavenException handleS3Exception(final Exception e, final String operationDescription) { + private IOException handleS3Exception(final Exception e, final String operationDescription) { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); - return new UncheckedDeephavenException("Thread interrupted while " + operationDescription, e); - } else if (e instanceof ExecutionException) { - return new UncheckedDeephavenException("Execution exception occurred while " + operationDescription, e); - } else if (e instanceof TimeoutException) { - return new UncheckedDeephavenException("Operation timeout while " + operationDescription + " after waiting " - + "for duration " + readTimeout, e); - } else if (e instanceof CancellationException) { - return new UncheckedDeephavenException("Cancelled an operation while " + operationDescription, e); + 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); } - throw new UncheckedDeephavenException("Exception caught while " + operationDescription, e); + return new IOException(String.format("Exception caught while %s", operationDescription), e); } @Override - public int write(final ByteBuffer src) throws IOException { + public int write(final ByteBuffer src) { throw new NonWritableChannelException(); } @@ -271,30 +270,30 @@ public SeekableByteChannel position(final long newPosition) throws ClosedChannel } @Override - public long size() throws ClosedChannelException { + public long size() throws IOException { checkClosed(position); populateSize(); return size; } - private void populateSize() { - if (size >= 0) { + private void populateSize() throws IOException { + if (size != UNINITIALIZED_SIZE) { return; } - if (context.getSize() < 0) { + if (s3ChannelContext.getSize() < 0) { // 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(builder -> builder.bucket(bucket).key(key)) - .get(readTimeout.toNanos(), TimeUnit.NANOSECONDS); + .get(s3Instructions.readTimeout().toNanos(), TimeUnit.NANOSECONDS); } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { - final String operation = "fetching HEAD for file " + key + " in S3 bucket " + bucket; - throw handleS3Exception(e, operation); + throw handleS3Exception(e, String.format("fetching HEAD for file %s in S3 bucket %s", key, bucket)); } - context.setSize(headObjectResponse.contentLength()); + s3ChannelContext.setSize(headObjectResponse.contentLength()); } - this.size = context.getSize(); + this.size = s3ChannelContext.getSize(); + final int fragmentSize = s3Instructions.fragmentSize(); this.numFragmentsInObject = (int) ((size + fragmentSize - 1) / fragmentSize); // = ceil(size / fragmentSize) } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index e76e9f159e1..9b8d3a8186f 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -13,7 +13,6 @@ import java.net.URI; import java.nio.channels.SeekableByteChannel; import java.nio.file.Path; -import java.time.Duration; /** * {@link SeekableChannelsProvider} implementation that is used to fetch objects from AWS S3 instances. @@ -21,10 +20,7 @@ final class S3SeekableChannelProvider implements SeekableChannelsProvider { private final S3AsyncClient s3AsyncClient; - private final int fragmentSize; - private final int maxCacheSize; - private final int readAheadCount; - private final Duration readTimeout; + private final S3Instructions s3Instructions; S3SeekableChannelProvider(final S3Instructions s3Instructions) { final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() @@ -35,31 +31,30 @@ final class S3SeekableChannelProvider implements SeekableChannelsProvider { .region(Region.of(s3Instructions.awsRegionName())) .httpClient(asyncHttpClient) .build(); - this.fragmentSize = s3Instructions.fragmentSize(); - this.maxCacheSize = s3Instructions.maxCacheSize(); - this.readAheadCount = s3Instructions.readAheadCount(); - this.readTimeout = s3Instructions.readTimeout(); + this.s3Instructions = s3Instructions; } @Override - public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelsProvider.ChannelContext context, + public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, @NotNull final URI uri) { - return new S3SeekableByteChannel(context, uri, s3AsyncClient, fragmentSize, readAheadCount, readTimeout); + return new S3SeekableByteChannel(channelContext, uri, s3AsyncClient, s3Instructions); } @Override public ChannelContext makeContext() { - return new S3SeekableByteChannel.S3ChannelContext(maxCacheSize); + return new S3SeekableByteChannel.S3ChannelContext(s3Instructions.maxCacheSize()); } @Override - public boolean isCompatibleWith(@NotNull ChannelContext context) { - return context instanceof S3SeekableByteChannel.S3ChannelContext; + public boolean isCompatibleWith(@NotNull final ChannelContext channelContext) { + // A null context implies no caching + return channelContext == ChannelContext.NULL + || channelContext instanceof S3SeekableByteChannel.S3ChannelContext; } @Override public SeekableByteChannel getWriteChannel(@NotNull final Path path, final boolean append) { - throw new UnsupportedOperationException("Don't support writing to S3 yet"); + throw new UnsupportedOperationException("Writing to S3 is currently unsupported"); } public void close() { diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index 92638959cc6..1792ff92513 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -38,12 +38,13 @@ public ChannelContext makeContext() { } @Override - public boolean isCompatibleWith(@NotNull ChannelContext context) { - return context == ChannelContext.NULL; + public boolean isCompatibleWith(@NotNull ChannelContext channelContext) { + return channelContext == ChannelContext.NULL; } @Override - public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext context, @NotNull final URI uri) + public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext channelContext, + @NotNull final URI uri) throws IOException { // context is unused here because it is NULL Assert.assertion(FILE_URI_SCHEME.equals(uri.getScheme()), "Expected a file uri, got " + uri); diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index 945ac7b4591..369fd1fcfca 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -34,29 +34,29 @@ class ColumnInstruction: def _build_parquet_instructions( - col_instructions: List[ColumnInstruction] = None, - compression_codec_name: str = None, - max_dictionary_keys: int = None, - max_dictionary_size: int = None, - is_legacy_parquet: bool = False, - target_page_size: int = None, - is_refreshing: bool = False, - for_read: bool = True, - force_build: bool = False, - special_instructions: Optional[object] = None, + col_instructions: List[ColumnInstruction] = None, + compression_codec_name: str = None, + max_dictionary_keys: int = None, + max_dictionary_size: int = None, + is_legacy_parquet: bool = False, + target_page_size: int = None, + is_refreshing: bool = False, + for_read: bool = True, + force_build: bool = False, + special_instructions: Optional[object] = None, ): if not any( - [ - force_build, - col_instructions, - compression_codec_name, - max_dictionary_keys is not None, - max_dictionary_size is not None, - is_legacy_parquet, - target_page_size is not None, - is_refreshing, - special_instructions is not None - ] + [ + force_build, + col_instructions, + compression_codec_name, + max_dictionary_keys is not None, + max_dictionary_size is not None, + is_legacy_parquet, + target_page_size is not None, + is_refreshing, + special_instructions is not None + ] ): return None @@ -132,13 +132,13 @@ class ParquetFileLayout(Enum): def read( - path: str, - col_instructions: Optional[List[ColumnInstruction]] = None, - is_legacy_parquet: bool = False, - is_refreshing: bool = False, - file_layout: Optional[ParquetFileLayout] = None, - table_definition: Union[Dict[str, DType], List[Column], None] = None, - special_instructions: Optional[object] = None, + path: str, + col_instructions: Optional[List[ColumnInstruction]] = None, + is_legacy_parquet: bool = False, + is_refreshing: bool = False, + file_layout: Optional[ParquetFileLayout] = None, + table_definition: Union[Dict[str, DType], List[Column], None] = None, + special_instructions: Optional[object] = None, ) -> Table: """ Reads in a table from a single parquet, metadata file, or directory with recognized layout. From 27e163826c24e57315497d27c292b8f225482f61 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Sat, 13 Jan 2024 16:41:21 +0530 Subject: [PATCH 23/39] Moved ChannelContext outside of SeekableChannelsProvider --- .../parquet/base/ColumnChunkReader.java | 7 ++--- .../parquet/base/ColumnChunkReaderImpl.java | 17 ++++++----- .../parquet/base/ColumnPageReader.java | 12 ++++---- .../parquet/base/ColumnPageReaderImpl.java | 29 ++++++++++--------- .../parquet/base/ParquetFileReader.java | 3 +- .../parquet/base/RowGroupReaderImpl.java | 3 +- .../base/util/CachedChannelProvider.java | 9 +++--- .../base/util/LocalFSChannelProvider.java | 11 +++---- .../base/util/SeekableChannelContext.java | 16 ++++++++++ .../base/util/SeekableChannelsProvider.java | 25 +++++----------- .../base/util/CachedChannelProviderTest.java | 13 +++++---- .../table/pagestore/ColumnChunkPageStore.java | 20 ++++++------- .../OffsetIndexBasedColumnChunkPageStore.java | 4 +-- .../VariablePageSizeColumnChunkPageStore.java | 11 ++++--- .../pagestore/topage/ChunkDictionary.java | 6 ++-- .../table/pagestore/topage/ToObjectPage.java | 4 +-- .../table/pagestore/topage/ToPage.java | 8 ++--- .../topage/ToPageWithDictionary.java | 6 ++-- .../table/pagestore/topage/ToStringPage.java | 4 +-- .../pagestore/topage/ToStringSetPage.java | 4 +-- .../extensions/s3/S3SeekableByteChannel.java | 13 +++++---- .../s3/S3SeekableChannelProvider.java | 9 +++--- .../TrackedSeekableChannelsProvider.java | 11 +++---- 23 files changed, 130 insertions(+), 115 deletions(-) create mode 100644 extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelContext.java 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 48c7a1bdb10..e8db3729559 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,15 +3,14 @@ */ package io.deephaven.parquet.base; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.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.NotNull; import org.jetbrains.annotations.Nullable; import java.io.IOException; -import java.util.Iterator; import java.util.function.Function; public interface ColumnChunkReader { @@ -51,7 +50,7 @@ interface ColumnPageReaderIterator { * @param channelContext The channel context to use for constructing the reader * @return The next page reader. */ - ColumnPageReader next(SeekableChannelsProvider.ChannelContext channelContext); + ColumnPageReader next(SeekableChannelContext channelContext); } /** @@ -80,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}. */ - Function getDictionarySupplier(); + Function getDictionarySupplier(); Dictionary NULL_DICTIONARY = new NullDictionary(); 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 36c9b2c0a89..1ee125b69c6 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,6 +4,7 @@ package io.deephaven.parquet.base; import io.deephaven.UncheckedDeephavenException; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.compress.CompressorAdapter; import io.deephaven.parquet.compress.DeephavenCompressorAdapterFactory; @@ -46,7 +47,7 @@ public class ColumnChunkReaderImpl implements ColumnChunkReader { private final ColumnDescriptor path; private final OffsetIndex offsetIndex; private final List fieldTypes; - private final Function dictionarySupplier; + private final Function dictionarySupplier; private final PageMaterializer.Factory nullMaterializerFactory; private URI uri; @@ -152,12 +153,12 @@ public boolean usesDictionaryOnEveryPage() { } @Override - public Function getDictionarySupplier() { + public Function getDictionarySupplier() { return dictionarySupplier; } @NotNull - private Dictionary getDictionary(final SeekableChannelsProvider.ChannelContext channelContext) { + private Dictionary getDictionary(final SeekableChannelContext channelContext) { final long dictionaryPageOffset; final ColumnMetaData chunkMeta = columnChunk.getMeta_data(); if (chunkMeta.isSetDictionary_page_offset()) { @@ -176,7 +177,7 @@ private Dictionary getDictionary(final SeekableChannelsProvider.ChannelContext c return getDictionaryHelper(channelContext, dictionaryPageOffset); } - private Dictionary getDictionaryHelper(final SeekableChannelsProvider.ChannelContext channelContext, + private Dictionary getDictionaryHelper(final SeekableChannelContext channelContext, final long dictionaryPageOffset) { try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, getURI())) { readChannel.position(dictionaryPageOffset); @@ -242,7 +243,7 @@ public boolean hasNext() { } @Override - public ColumnPageReader next(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) { + public ColumnPageReader next(@NotNull final SeekableChannelContext channelContext) { if (!hasNext()) { throw new NoSuchElementException("No next element"); } @@ -277,10 +278,10 @@ public ColumnPageReader next(@NotNull final SeekableChannelsProvider.ChannelCont throw new UncheckedDeephavenException( "Unknown parquet data page header type " + pageHeader.type); } - final Function pageDictionarySupplier = + final Function pageDictionarySupplier = (encoding == PLAIN_DICTIONARY || encoding == RLE_DICTIONARY) ? dictionarySupplier - : (SeekableChannelsProvider.ChannelContext context) -> NULL_DICTIONARY; + : (SeekableChannelContext context) -> NULL_DICTIONARY; final ColumnPageReader nextReader = new ColumnPageReaderImpl(channelsProvider, decompressor, pageDictionarySupplier, nullMaterializerFactory, path, getURI(), fieldTypes, readChannel.position(), pageHeader, ColumnPageReaderImpl.NULL_NUM_VALUES); @@ -304,7 +305,7 @@ public boolean hasNext() { } @Override - public ColumnPageReader next(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) { + public ColumnPageReader next(@NotNull final SeekableChannelContext channelContext) { if (!hasNext()) { throw new NoSuchElementException("No next element"); } 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 aca84ac6b29..94555853ae9 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,7 +4,7 @@ package io.deephaven.parquet.base; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import org.apache.parquet.column.Dictionary; import org.jetbrains.annotations.NotNull; @@ -20,7 +20,7 @@ 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(final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + default long numRows(final SeekableChannelContext channelContext) throws IOException { return numValues(channelContext); } @@ -32,7 +32,7 @@ default long numRows(final SeekableChannelsProvider.ChannelContext channelContex * @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, SeekableChannelsProvider.ChannelContext channelContext) throws IOException; + Object materialize(Object nullValue, SeekableChannelContext channelContext) throws IOException; /** * Directly read the key integral values when there's a dictionary. @@ -44,13 +44,13 @@ default long numRows(final SeekableChannelsProvider.ChannelContext channelContex * @return A buffer holding the end of each repeated row. If the column is not repeating, null. */ IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder, - SeekableChannelsProvider.ChannelContext channelContext) throws IOException; + 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(SeekableChannelsProvider.ChannelContext channelContext) throws IOException; + int numValues(SeekableChannelContext channelContext) throws IOException; /** * @param channelContext The channel context to use for reading the parquet file @@ -58,5 +58,5 @@ IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder, * @apiNote The result will never be {@code null}. It will instead be {@link ColumnChunkReader#NULL_DICTIONARY}. */ @NotNull - Dictionary getDictionary(SeekableChannelsProvider.ChannelContext channelContext); + 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 cab06534788..cedc885de08 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,6 +7,7 @@ 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.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.compress.CompressorAdapter; import org.apache.parquet.bytes.ByteBufferInputStream; @@ -49,7 +50,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { private final SeekableChannelsProvider channelsProvider; private final CompressorAdapter compressorAdapter; - private final Function dictionarySupplier; + private final Function dictionarySupplier; private final PageMaterializer.Factory pageMaterializerFactory; private final ColumnDescriptor path; private final URI uri; @@ -83,7 +84,7 @@ public class ColumnPageReaderImpl implements ColumnPageReader { */ ColumnPageReaderImpl(SeekableChannelsProvider channelsProvider, CompressorAdapter compressorAdapter, - Function dictionarySupplier, + Function dictionarySupplier, PageMaterializer.Factory materializerFactory, ColumnDescriptor path, URI uri, @@ -105,14 +106,14 @@ public class ColumnPageReaderImpl implements ColumnPageReader { @Override public Object materialize(@NotNull final Object nullValue, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + @NotNull final SeekableChannelContext channelContext) throws IOException { try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { ensurePageHeader(readChannel); return readDataPage(nullValue, readChannel, channelContext); } } - public int readRowCount(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + public int readRowCount(@NotNull final SeekableChannelContext channelContext) throws IOException { try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { ensurePageHeader(readChannel); return readRowCountFromDataPage(readChannel); @@ -122,7 +123,7 @@ public int readRowCount(@NotNull final SeekableChannelsProvider.ChannelContext c @Override public IntBuffer readKeyValues(IntBuffer keyDest, int nullPlaceholder, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + @NotNull final SeekableChannelContext channelContext) throws IOException { try (final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, uri)) { ensurePageHeader(readChannel); return readKeyFromDataPage(keyDest, nullPlaceholder, readChannel, channelContext); @@ -214,7 +215,7 @@ private int readRowCountFromDataPage(ReadableByteChannel file) throws IOExceptio } private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, - ReadableByteChannel file, @NotNull final SeekableChannelsProvider.ChannelContext channelContext) + ReadableByteChannel file, @NotNull final SeekableChannelContext channelContext) throws IOException { int uncompressedPageSize = pageHeader.getUncompressed_page_size(); int compressedPageSize = pageHeader.getCompressed_page_size(); @@ -267,7 +268,7 @@ private IntBuffer readKeyFromDataPage(IntBuffer keyDest, int nullPlaceholder, } private Object readDataPage(Object nullValue, SeekableByteChannel file, - @NotNull SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + @NotNull SeekableChannelContext channelContext) throws IOException { final int uncompressedPageSize = pageHeader.getUncompressed_page_size(); final int compressedPageSize = pageHeader.getCompressed_page_size(); switch (pageHeader.type) { @@ -334,7 +335,7 @@ private int readRowCountFromPageV1(DataPageV1 page) { } private IntBuffer readKeysFromPageV1(DataPageV1 page, IntBuffer keyDest, int nullPlaceholder, - @NotNull SeekableChannelsProvider.ChannelContext channelContext) { + @NotNull SeekableChannelContext channelContext) { RunLengthBitPackingHybridBufferDecoder rlDecoder = null; RunLengthBitPackingHybridBufferDecoder dlDecoder = null; try { @@ -401,7 +402,7 @@ private int readRepetitionLevels(ByteBuffer byteBuffer) throws IOException { } private Object readPageV1(DataPageV1 page, Object nullValue, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) { + @NotNull final SeekableChannelContext channelContext) { RunLengthBitPackingHybridBufferDecoder dlDecoder = null; try { ByteBuffer bytes = page.getBytes().toByteBuffer(); // TODO - move away from page and use @@ -442,7 +443,7 @@ private Object materialize(PageMaterializer.Factory factory, } private void readKeysFromPageV2(DataPageV2 page, IntBuffer keyDest, int nullPlaceholder, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) + @NotNull final SeekableChannelContext channelContext) throws IOException { if (path.getMaxRepetitionLevel() > 0) { throw new RuntimeException("Repeating levels not supported"); @@ -598,7 +599,7 @@ private static Object materializeNonNull(PageMaterializer.Factory factory, int n } private ValuesReader getDataReader(Encoding dataEncoding, ByteBuffer in, int valueCount, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) { + @NotNull final SeekableChannelContext channelContext) { if (dataEncoding == Encoding.DELTA_BYTE_ARRAY) { throw new RuntimeException("DELTA_BYTE_ARRAY encoding not supported"); } @@ -623,7 +624,7 @@ private ValuesReader getDataReader(Encoding dataEncoding, ByteBuffer in, int val } @Override - public int numValues(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + public int numValues(@NotNull final SeekableChannelContext channelContext) throws IOException { if (numValues >= 0) { return numValues; } @@ -637,7 +638,7 @@ public int numValues(@NotNull final SeekableChannelsProvider.ChannelContext chan @NotNull @Override - public Dictionary getDictionary(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) { + public Dictionary getDictionary(@NotNull final SeekableChannelContext channelContext) { return dictionarySupplier.apply(channelContext); } @@ -647,7 +648,7 @@ public void close() throws Exception { } @Override - public long numRows(@NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + public long numRows(@NotNull final SeekableChannelContext channelContext) throws IOException { if (rowCount == -1) { if (path.getMaxRepetitionLevel() == 0) { rowCount = numValues(channelContext); 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 b43f59cd7ad..a80ef672e0b 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 @@ -5,6 +5,7 @@ import io.deephaven.UncheckedDeephavenException; import io.deephaven.parquet.base.util.Helpers; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.apache.parquet.format.*; import org.apache.parquet.format.ColumnOrder; @@ -64,7 +65,7 @@ public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvide rootURI = parquetFileURI; } final byte[] footer; - try (final SeekableChannelsProvider.ChannelContext channelContext = channelsProvider.makeContext(); + try (final SeekableChannelContext channelContext = channelsProvider.makeContext(); final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, parquetFileURI)) { final long fileLen = readChannel.size(); 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 a46937a17d1..95bdc6cf525 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,6 +3,7 @@ */ package io.deephaven.parquet.base; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.apache.parquet.format.ColumnChunk; import org.apache.parquet.format.RowGroup; @@ -79,7 +80,7 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { OffsetIndex offsetIndex = null; if (columnChunk.isSetOffset_index_offset()) { - try (final SeekableChannelsProvider.ChannelContext channelContext = channelsProvider.makeContext(); + try (final SeekableChannelContext channelContext = channelsProvider.makeContext(); final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, rootURI)) { readChannel.position(columnChunk.getOffset_index_offset()); offsetIndex = ParquetMetadataConverter.fromParquetOffsetIndex(Util.readOffsetIndex( diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java index 681e1271209..b0a9eb70322 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java @@ -53,17 +53,18 @@ public CachedChannelProvider(@NotNull final SeekableChannelsProvider wrappedProv } @Override - public ChannelContext makeContext() { + public SeekableChannelContext makeContext() { return wrappedProvider.makeContext(); } @Override - public boolean isCompatibleWith(@NotNull final ChannelContext channelContext) { + public boolean isCompatibleWith(@NotNull final SeekableChannelContext channelContext) { return wrappedProvider.isCompatibleWith(channelContext); } @Override - public SeekableByteChannel getReadChannel(@NotNull final ChannelContext channelContext, @NotNull final URI uri) + 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); @@ -224,7 +225,7 @@ private void dispose() throws IOException { } @Override - public void setContext(ChannelContext channelContext) { + public void setContext(@NotNull 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/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java index ca8737f5e6e..5e9766d7b4e 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java @@ -15,18 +15,19 @@ public class LocalFSChannelProvider implements SeekableChannelsProvider { @Override - public ChannelContext makeContext() { + public SeekableChannelContext makeContext() { // No additional context required for local FS - return ChannelContext.NULL; + return SeekableChannelContext.NULL; } @Override - public boolean isCompatibleWith(@NotNull final ChannelContext channelContext) { - return channelContext == ChannelContext.NULL; + public boolean isCompatibleWith(@NotNull final SeekableChannelContext channelContext) { + return channelContext == SeekableChannelContext.NULL; } @Override - public SeekableByteChannel getReadChannel(@NotNull final ChannelContext channelContext, @NotNull final URI uri) + public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext, + @NotNull final URI uri) throws IOException { // context is unused here because it is NULL return FileChannel.open(Path.of(uri), StandardOpenOption.READ); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelContext.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelContext.java new file mode 100644 index 00000000000..2ed9f8be3bd --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelContext.java @@ -0,0 +1,16 @@ +package io.deephaven.parquet.base.util; + +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/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 index 65ec0498a54..bec0508f443 100644 --- 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 @@ -38,29 +38,19 @@ static URI convertToURI(final String source) { return uri; } - interface ChannelContext extends SafeCloseable { - - ChannelContext NULL = new ChannelContext() {}; - - /** - * Release any resources associated with this context. The context should not be used afterward. - */ - default void close() {} - } - /** - * Create a new {@link ChannelContext} object for creating read and write channels via this provider. + * Create a new {@link SeekableChannelContext} object for creating read and write channels via this provider. */ - ChannelContext makeContext(); + SeekableChannelContext makeContext(); /** * Check if the given context is compatible with this provider. Useful to test if we can use provided - * {@code channelContext} object for creating channels with this provider. + * {@code context} object for creating channels with this provider. */ - boolean isCompatibleWith(@NotNull ChannelContext channelContext); + boolean isCompatibleWith(@NotNull SeekableChannelContext channelContext); interface ContextHolder { - void setContext(ChannelContext channelContext); + void setContext(SeekableChannelContext channelContext); @FinalDefault default void clearContext() { @@ -68,12 +58,13 @@ default void clearContext() { } } - default SeekableByteChannel getReadChannel(@NotNull ChannelContext channelContext, @NotNull String uriStr) + default SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull String uriStr) throws IOException { return getReadChannel(channelContext, convertToURI(uriStr)); } - SeekableByteChannel getReadChannel(@NotNull ChannelContext channelContext, @NotNull URI uri) throws IOException; + SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull URI uri) + throws IOException; default SeekableByteChannel getWriteChannel(@NotNull String path, final boolean append) throws IOException { return getWriteChannel(Paths.get(path), append); diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java index a3950f8119e..69aae62b5f9 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java @@ -163,22 +163,23 @@ private class TestChannelProvider implements SeekableChannelsProvider { AtomicInteger count = new AtomicInteger(0); @Override - public ChannelContext makeContext() { - return ChannelContext.NULL; + public SeekableChannelContext makeContext() { + return SeekableChannelContext.NULL; } @Override - public boolean isCompatibleWith(@NotNull ChannelContext channelContext) { - return channelContext == ChannelContext.NULL; + public boolean isCompatibleWith(@NotNull SeekableChannelContext channelContext) { + return channelContext == SeekableChannelContext.NULL; } @Override - public SeekableByteChannel getReadChannel(@NotNull ChannelContext channelContext, @NotNull String path) { + public SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, + @NotNull String path) { return new TestMockChannel(count.getAndIncrement(), path); } @Override - public SeekableByteChannel getReadChannel(@NotNull ChannelContext channelContext, @NotNull URI uri) { + public SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull URI uri) { return new TestMockChannel(count.getAndIncrement(), uri.toString()); } 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 12f35dd9883..ca854984827 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,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Context; import io.deephaven.engine.table.SharedContext; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.engine.table.Releasable; @@ -135,7 +136,7 @@ public static CreatorResult create( } ChunkPage toPage(final long offset, @NotNull final ColumnPageReader columnPageReader, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) + @NotNull final SeekableChannelContext channelContext) throws IOException { return toPage.toPage(offset, columnPageReader, channelContext, mask); } @@ -174,22 +175,22 @@ public boolean usesDictionaryOnEveryPage() { public void close() {} /** - * Wrapper class for holding a {@link SeekableChannelsProvider.ChannelContext}. + * Wrapper class for holding a {@link SeekableChannelContext}. */ private static class ChannelContextWrapper extends PagingContextHolder { @NotNull - private final SeekableChannelsProvider.ChannelContext channelContext; + private final SeekableChannelContext channelContext; private ChannelContextWrapper( final int chunkCapacity, @Nullable final SharedContext sharedContext, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) { + @NotNull final SeekableChannelContext channelContext) { super(chunkCapacity, sharedContext); this.channelContext = channelContext; } @NotNull - SeekableChannelsProvider.ChannelContext getChannelContext() { + SeekableChannelContext getChannelContext() { return channelContext; } @@ -205,17 +206,16 @@ public void close() { * {@link #columnChunkReader}, if required. * * @param context The context to populate. - * @return The {@link SeekableChannelsProvider.ChannelContext} to use for reading pages via - * {@link #columnChunkReader}. + * @return The {@link SeekableChannelContext} to use for reading pages via {@link #columnChunkReader}. */ - final SeekableChannelsProvider.ChannelContext innerFillContext(@Nullable final FillContext context) { + 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 SeekableChannelsProvider.ChannelContext.NULL; + return SeekableChannelContext.NULL; } private T fillContextUpdater( @@ -225,7 +225,7 @@ private T fillContextUpdater( final SeekableChannelsProvider channelsProvider = columnChunkReader.getChannelsProvider(); if (currentInnerContext != null) { // Check if we can reuse the context object - final SeekableChannelsProvider.ChannelContext channelContext = + final SeekableChannelContext channelContext = ((ChannelContextWrapper) currentInnerContext).getChannelContext(); if (channelsProvider.isCompatibleWith(channelContext)) { // noinspection unchecked 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 7b7e75acb66..5743240112f 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,7 +7,7 @@ import io.deephaven.base.verify.Require; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.ChunkPage; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; @@ -117,7 +117,7 @@ private ChunkPage getPage(@Nullable final FillContext fillContext, final i // Make sure no one materialized this page as we waited for the lock if ((localRef = pageState.pageRef) == null || (page = localRef.get()) == null) { // Use the latest context while reading the page - final SeekableChannelsProvider.ChannelContext channelContext = innerFillContext(fillContext); + final SeekableChannelContext channelContext = innerFillContext(fillContext); final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(pageNum); try { page = new PageCache.IntrusivePage<>( 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 c8ef551fd0c..328a10176e8 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,7 +7,7 @@ import io.deephaven.base.verify.Require; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.ChunkPage; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; @@ -51,8 +51,7 @@ final class VariablePageSizeColumnChunkPageStore extends Colum pages = (WeakReference>[]) new WeakReference[INIT_ARRAY_SIZE]; } - private void extendOnePage(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, - final int prevNumPages) { + private void extendOnePage(@NotNull final SeekableChannelContext channelContext, final int prevNumPages) { PageCache.IntrusivePage page = null; synchronized (this) { @@ -101,7 +100,7 @@ private void extendOnePage(@NotNull final SeekableChannelsProvider.ChannelContex } } - private int fillToRow(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, int minPageNum, + private int fillToRow(@NotNull final SeekableChannelContext channelContext, int minPageNum, long row) { int localNumPages = numPages; @@ -114,7 +113,7 @@ private int fillToRow(@NotNull final SeekableChannelsProvider.ChannelContext cha return minPageNum; } - private ChunkPage getPage(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, + private ChunkPage getPage(@NotNull final SeekableChannelContext channelContext, final int pageNum) { PageCache.IntrusivePage page = pages[pageNum].get(); @@ -156,7 +155,7 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext } // Use the latest channel context while reading page headers - final SeekableChannelsProvider.ChannelContext channelContext = innerFillContext(fillContext); + final SeekableChannelContext channelContext = innerFillContext(fillContext); if (pageNum >= localNumPages) { final int minPageNum = fillToRow(channelContext, localNumPages, rowKey); 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 1d9a1731799..66113e5fa83 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,7 +6,7 @@ import gnu.trove.map.TObjectIntMap; import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.chunk.attributes.Any; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.stringset.LongBitmapStringSet; import io.deephaven.chunk.ObjectChunk; import io.deephaven.util.datastructures.LazyCachingSupplier; @@ -46,10 +46,10 @@ public interface Lookup { */ ChunkDictionary( @NotNull final Lookup lookup, - @NotNull final Function dictionarySupplier) { + @NotNull final Function dictionarySupplier) { this.valuesSupplier = new LazyCachingSupplier<>(() -> { // Dictionary is already materialized at this point, therefore, we can safely use NULL context - final Dictionary dictionary = dictionarySupplier.apply(SeekableChannelsProvider.ChannelContext.NULL); + 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 0a973753658..5ec46ee5dc6 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,7 +5,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.util.codec.ObjectCodec; import org.apache.parquet.column.Dictionary; import org.apache.parquet.io.api.Binary; @@ -23,7 +23,7 @@ public class ToObjectPage implements ToPage { ToPage create( final Class nativeType, @NotNull final ObjectCodec codec, - final Function 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 9f9b73f2545..d964ce27041 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,7 +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.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.vector.Vector; import io.deephaven.engine.page.ChunkPage; import io.deephaven.vector.VectorFactory; @@ -55,7 +55,7 @@ default Object nullValue() { * @return Gets the result from the columnPageReader. */ default Object getResult(ColumnPageReader columnPageReader, - SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + SeekableChannelContext channelContext) throws IOException { return columnPageReader.materialize(nullValue(), channelContext); } @@ -81,7 +81,7 @@ default Vector makeVector(RESULT result) { @NotNull @FinalDefault default ChunkPage toPage(long offset, ColumnPageReader columnPageReader, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext, long mask) + @NotNull final SeekableChannelContext channelContext, long mask) throws IOException { return ChunkPageFactory.forChunkType(getChunkType()) .pageWrap(offset, convertResult(getResult(columnPageReader, channelContext)), mask); @@ -128,7 +128,7 @@ public Object nullValue() { @NotNull @Override public Object getResult(ColumnPageReader columnPageReader, - SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + SeekableChannelContext channelContext) throws IOException { return toPage.getResult(columnPageReader, channelContext); } 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 ceb39e98ef6..1c4196fbac3 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,7 +3,7 @@ */ package io.deephaven.parquet.table.pagestore.topage; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.stringset.LongBitmapStringSet; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; @@ -53,7 +53,7 @@ public final ChunkType getChunkType() { @Override @NotNull public final Object getResult(@NotNull final ColumnPageReader columnPageReader, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + @NotNull final SeekableChannelContext channelContext) throws IOException { if (columnPageReader.getDictionary(channelContext) == ColumnChunkReader.NULL_DICTIONARY) { return ToPage.super.getResult(columnPageReader, channelContext); } @@ -117,7 +117,7 @@ public Object nullValue() { @Override public Object getResult(@NotNull final ColumnPageReader columnPageReader, - @NotNull final SeekableChannelsProvider.ChannelContext channelContext) + @NotNull final SeekableChannelContext channelContext) throws IOException { return ToPageWithDictionary.this.getResult(columnPageReader, channelContext); } 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 7ce3d819fa2..80cb8fc6430 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,7 +5,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import org.apache.parquet.column.Dictionary; import org.apache.parquet.io.api.Binary; import org.jetbrains.annotations.NotNull; @@ -18,7 +18,7 @@ public class ToStringPage implements ToPage { public static ToPage create( final Class nativeType, - final Function 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 8d3776fd02c..17e411fbc6b 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,7 +4,7 @@ package io.deephaven.parquet.table.pagestore.topage; import io.deephaven.chunk.attributes.Any; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.stringset.HashStringSet; import io.deephaven.stringset.StringSet; import io.deephaven.stringset.LongBitmapStringSet; @@ -100,7 +100,7 @@ public final ChunkType getChunkType() { @Override @NotNull public Object getResult(ColumnPageReader columnPageReader, - SeekableChannelsProvider.ChannelContext channelContext) throws IOException { + SeekableChannelContext channelContext) throws IOException { return toPage.getDictionaryKeysToPage().getResult(columnPageReader, channelContext); } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index 639aa5963b6..86a69fbc426 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -5,6 +5,8 @@ import io.deephaven.base.verify.Assert; import java.util.concurrent.CancellationException; + +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -35,9 +37,9 @@ final class S3SeekableByteChannel implements SeekableByteChannel, SeekableChanne private static final long UNINITIALIZED_SIZE = -1; /** - * Channel context object used to store read-ahead buffers for efficiently reading from S3. + * Context object used to store read-ahead buffers for efficiently reading from S3. */ - static final class S3ChannelContext implements SeekableChannelsProvider.ChannelContext { + static final class S3ChannelContext implements SeekableChannelContext { /** * Used to store information related to a single fragment @@ -134,7 +136,7 @@ void setSize(final long size) { private long position; - S3SeekableByteChannel(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, @NotNull final URI uri, + S3SeekableByteChannel(@NotNull final SeekableChannelContext channelContext, @NotNull final URI uri, @NotNull final S3AsyncClient s3AsyncClient, final S3Instructions s3Instructions) { final S3Uri s3Uri = s3AsyncClient.utilities().parseUri(uri); this.bucket = s3Uri.bucket().orElse(null); @@ -148,7 +150,7 @@ void setSize(final long size) { } @Override - public void setContext(@Nullable final SeekableChannelsProvider.ChannelContext channelContext) { + public void setContext(@Nullable final SeekableChannelContext channelContext) { // null context equivalent to clearing the context if (channelContext != null && !(channelContext instanceof S3ChannelContext)) { throw new IllegalArgumentException( @@ -161,8 +163,7 @@ public void setContext(@Nullable final SeekableChannelsProvider.ChannelContext c @Override public int read(@NotNull final ByteBuffer destination) throws IOException { Assert.neqNull(s3ChannelContext, "s3ChannelContext"); - Assert.neq(s3ChannelContext, "s3ChannelContext", SeekableChannelsProvider.ChannelContext.NULL, - "SeekableChannelsProvider.ChannelContext.NULL"); + Assert.neq(s3ChannelContext, "s3ChannelContext", SeekableChannelContext.NULL, "SeekableChannelContext.NULL"); if (!destination.hasRemaining()) { return 0; } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index 9b8d3a8186f..e5ab39f449a 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -3,6 +3,7 @@ */ package io.deephaven.extensions.s3; +import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; import software.amazon.awssdk.http.async.SdkAsyncHttpClient; @@ -35,20 +36,20 @@ final class S3SeekableChannelProvider implements SeekableChannelsProvider { } @Override - public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelsProvider.ChannelContext channelContext, + public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext, @NotNull final URI uri) { return new S3SeekableByteChannel(channelContext, uri, s3AsyncClient, s3Instructions); } @Override - public ChannelContext makeContext() { + public SeekableChannelContext makeContext() { return new S3SeekableByteChannel.S3ChannelContext(s3Instructions.maxCacheSize()); } @Override - public boolean isCompatibleWith(@NotNull final ChannelContext channelContext) { + public boolean isCompatibleWith(@NotNull final SeekableChannelContext channelContext) { // A null context implies no caching - return channelContext == ChannelContext.NULL + return channelContext == SeekableChannelContext.NULL || channelContext instanceof S3SeekableByteChannel.S3ChannelContext; } diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index 1792ff92513..d7d7946993b 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -8,6 +8,7 @@ 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.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; @@ -32,18 +33,18 @@ class TrackedSeekableChannelsProvider implements SeekableChannelsProvider { } @Override - public ChannelContext makeContext() { + public SeekableChannelContext makeContext() { // No additional context required for local FS - return ChannelContext.NULL; + return SeekableChannelContext.NULL; } @Override - public boolean isCompatibleWith(@NotNull ChannelContext channelContext) { - return channelContext == ChannelContext.NULL; + public boolean isCompatibleWith(@NotNull SeekableChannelContext channelContext) { + return channelContext == SeekableChannelContext.NULL; } @Override - public final SeekableByteChannel getReadChannel(@NotNull final ChannelContext channelContext, + public final SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext, @NotNull final URI uri) throws IOException { // context is unused here because it is NULL From 3b7521e0ee745118aed8a5fa26e208a25209419a Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 16 Jan 2024 01:03:56 +0530 Subject: [PATCH 24/39] Resolving more comments --- .../parquet/base/ColumnChunkReaderImpl.java | 10 +++- .../pagestore/topage/ChunkDictionary.java | 2 +- .../extensions/s3/S3SeekableByteChannel.java | 52 ++++++++++++------- .../s3/S3SeekableChannelProvider.java | 2 +- 4 files changed, 45 insertions(+), 21 deletions(-) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 1ee125b69c6..ed6f54665e6 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 @@ -174,7 +174,15 @@ private Dictionary getDictionary(final SeekableChannelContext channelContext) { } else { return NULL_DICTIONARY; } - return getDictionaryHelper(channelContext, dictionaryPageOffset); + 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, 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 66113e5fa83..fd188f76456 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 @@ -48,7 +48,7 @@ public interface Lookup { @NotNull final Lookup lookup, @NotNull final Function dictionarySupplier) { this.valuesSupplier = new LazyCachingSupplier<>(() -> { - // Dictionary is already materialized at this point, therefore, we can safely use NULL context + // 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) { diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index 86a69fbc426..ae6b425c0a6 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -132,7 +132,7 @@ void setSize(final long size) { private long size; private int numFragmentsInObject; - private S3ChannelContext s3ChannelContext; + private SeekableChannelContext channelContext; private long position; @@ -143,27 +143,25 @@ void setSize(final long size) { this.key = s3Uri.key().orElse(null); this.s3AsyncClient = s3AsyncClient; this.s3Instructions = s3Instructions; - Assert.instanceOf(channelContext, "channelContext", S3ChannelContext.class); - this.s3ChannelContext = (S3ChannelContext) channelContext; + this.channelContext = channelContext; 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) { - // null context equivalent to clearing the context - if (channelContext != null && !(channelContext instanceof S3ChannelContext)) { - throw new IllegalArgumentException( - "Context must be null or an instance of S3ChannelContext, provided context of class " + - channelContext.getClass().getName()); - } - this.s3ChannelContext = (S3ChannelContext) channelContext; + this.channelContext = channelContext; } @Override public int read(@NotNull final ByteBuffer destination) throws IOException { - Assert.neqNull(s3ChannelContext, "s3ChannelContext"); - Assert.neq(s3ChannelContext, "s3ChannelContext", SeekableChannelContext.NULL, "SeekableChannelContext.NULL"); + Assert.neqNull(channelContext, "channelContext"); if (!destination.hasRemaining()) { return 0; } @@ -171,7 +169,8 @@ public int read(@NotNull final ByteBuffer destination) throws IOException { checkClosed(localPosition); // Fetch the file size if this is the first read - populateSize(); + final S3ChannelContext s3ChannelContext = getS3ChannelContextFrom(channelContext); + populateSize(s3ChannelContext); if (localPosition >= size) { // We are finished reading return -1; @@ -179,8 +178,8 @@ public int read(@NotNull final ByteBuffer destination) throws IOException { // Send async read requests for current fragment as well as read ahead fragments, if not already in cache final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); - final int numReadAheadFragments = - Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); + final int numReadAheadFragments = channelContext == SeekableChannelContext.NULL ? 0 + : Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); for (int idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { final CompletableFuture future = s3ChannelContext.getCachedFuture(idx); if (future == null) { @@ -213,6 +212,19 @@ public int read(@NotNull final ByteBuffer destination) throws IOException { return sizeToCopy; } + private static S3ChannelContext getS3ChannelContextFrom(@NotNull final SeekableChannelContext channelContext) { + final S3ChannelContext s3ChannelContext; + if (channelContext == SeekableChannelContext.NULL) { + // Create a new temporary context just for this read with a cache size of 1 just to support the current + // chunk with no read ahead + s3ChannelContext = new S3ChannelContext(1); + } else { + Assert.instanceOf(channelContext, "channelContext", S3ChannelContext.class); + s3ChannelContext = (S3ChannelContext) channelContext; + } + return s3ChannelContext; + } + private int fragmentIndexForByteNumber(final long byteNumber) { return Math.toIntExact(byteNumber / s3Instructions.fragmentSize()); } @@ -273,15 +285,19 @@ public SeekableByteChannel position(final long newPosition) throws ClosedChannel @Override public long size() throws IOException { checkClosed(position); - populateSize(); + populateSize(getS3ChannelContextFrom(channelContext)); return size; } - private void populateSize() throws IOException { + 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() < 0) { + 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; diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index e5ab39f449a..63fca0c621f 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -48,7 +48,7 @@ public SeekableChannelContext makeContext() { @Override public boolean isCompatibleWith(@NotNull final SeekableChannelContext channelContext) { - // A null context implies no caching + // A null context implies no caching or read ahead return channelContext == SeekableChannelContext.NULL || channelContext instanceof S3SeekableByteChannel.S3ChannelContext; } From 36cf3cb47ef6bb76f0d73aae3566c231c12a0259 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 19 Jan 2024 18:02:22 +0530 Subject: [PATCH 25/39] Resolved python comments and updated corresponding descriptions in java code --- .../extensions/s3/S3Instructions.java | 11 +-- py/server/deephaven/experimental/s3.py | 53 +++++++++------ py/server/deephaven/parquet.py | 68 +++++++++---------- 3 files changed, 74 insertions(+), 58 deletions(-) diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index 5b9c699f016..da9a69bc2a0 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -42,7 +42,8 @@ public int maxConcurrentRequests() { /** * The number of fragments to send asynchronous read requests for while reading the current fragment, defaults to - * {@value #DEFAULT_READ_AHEAD_COUNT}. + * {@value #DEFAULT_READ_AHEAD_COUNT}. This means by default, we will fetch {@value #DEFAULT_READ_AHEAD_COUNT} + * fragments in advance when reading current fragment. */ @Value.Default public int readAheadCount() { @@ -50,8 +51,8 @@ public int readAheadCount() { } /** - * The maximum size of each fragment to read from S3. The fetched fragment can be smaller than this in case fewer - * bytes remaining in the file, defaults to {@value #DEFAULT_FRAGMENT_SIZE} bytes. + * The maximum size of each fragment to read from S3, defaults to {@value #DEFAULT_FRAGMENT_SIZE} bytes. If there + * are fewer bytes remaining in the file, the fetched fragment can be smaller. */ @Value.Default public int fragmentSize() { @@ -59,7 +60,9 @@ public int fragmentSize() { } /** - * The maximum number of fragments to cache in memory, defaults to {@value #DEFAULT_MAX_CACHE_SIZE}. + * The maximum number of fragments to cache in memory, defaults to {@value #DEFAULT_MAX_CACHE_SIZE}. This caching is + * done at deephaven layer using a modulo-based logic. For example, if the max cache size is 32,fragment {@code i} + * will be cached in slot {@code i % 32} and will stay there until it is evicted by a more recent fragment. */ @Value.Default public int maxCacheSize() { diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index 50cd375e5b5..12f41037be8 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -5,37 +5,51 @@ This module is useful for reading and writing files stored in S3. Importing this module requires the S3 specific extensions to be included in the class path. """ +import datetime +from typing import Optional, Union import jpy +import numpy as np +import pandas as pd + +from deephaven import time +from deephaven.dtypes import Duration _JDuration = jpy.get_type("java.time.Duration") _JS3Instructions = jpy.get_type("io.deephaven.extensions.s3.S3Instructions") -def _build_s3_instructions( + +def build_s3_instructions( aws_region_name: str, - max_concurrent_requests: int = None, - read_ahead_count: int = None, - fragment_size: int = None, - max_cache_size: int = None, - connection_timeout: _JDuration = None, - read_timeout: _JDuration = None, -): + 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, +) -> Optional[object]: """ Build specialized instructions for accessing files stored in AWS S3. Args: - aws_region_name (str): the AWS region name for reading parquet files stored in AWS S3, by default None + 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, by default 50. read_ahead_count (int): the number of fragments to send asynchronous read requests for while reading the current - fragment, defaults to 1. - fragment_size (int): the maximum size of each fragment to read from S3. The fetched fragment can be smaller than - this in case fewer bytes remaining in the file, defaults to 5 MB. - max_cache_size (int): the maximum number of fragments to cache in memory while reading, defaults to 32. - connection_timeout (Duration): the amount of time to wait when initially establishing a connection before giving - up and timing out, defaults to 2 seconds. - read_timeout (Duration): the amount of time to wait when reading a fragment before giving up and timing out, - defaults to 2 seconds. + fragment, defaults to 1, which means fetch one next fragment in advance when reading 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 deephaven layer using a modulo-based logic. For example, if the max cache size is 32, + fragment i will be cached in slot i % 32 and will stay there until it is evicted by a more recent fragment. + 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. """ builder = _JS3Instructions.builder() @@ -54,10 +68,9 @@ def _build_s3_instructions( builder.maxCacheSize(max_cache_size) if connection_timeout is not None: - builder.connectionTimeout(connection_timeout) + builder.connectionTimeout(time.to_j_duration(connection_timeout)) if read_timeout is not None: - builder.readTimeout(read_timeout) + builder.readTimeout(time.to_j_duration(read_timeout)) return builder.build() - diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index 369fd1fcfca..98af53ad3a1 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -26,20 +26,20 @@ @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, @@ -156,7 +156,7 @@ def read( 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[object]): Special instructions for reading parquet files, useful when reading - files from a non-local file system, like S3. By default, None. When + files from a non-local file system, like S3. By default, None. Returns: a table @@ -228,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. @@ -242,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 @@ -284,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. @@ -304,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 From ecb35a877dde2e7aa371d4a8e7684b541d2030bf Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 19 Jan 2024 22:14:56 +0530 Subject: [PATCH 26/39] Resolved more python comments --- py/server/deephaven/experimental/s3.py | 65 ++++++++++++++------------ py/server/deephaven/parquet.py | 11 +++-- 2 files changed, 42 insertions(+), 34 deletions(-) diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index 12f41037be8..d77da1d21a0 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -13,28 +13,18 @@ import pandas as pd from deephaven import time +from deephaven._wrapper import JObjectWrapper from deephaven.dtypes import Duration -_JDuration = jpy.get_type("java.time.Duration") -_JS3Instructions = jpy.get_type("io.deephaven.extensions.s3.S3Instructions") - -def build_s3_instructions( - 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, -) -> Optional[object]: +class S3Instructions(JObjectWrapper): """ - Build specialized instructions for accessing files stored in AWS S3. + Used to provide specialized instructions for reading from AWS S3. 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, - by default 50. + by default is 50. read_ahead_count (int): the number of fragments to send asynchronous read requests for while reading the current fragment, defaults to 1, which means fetch one next fragment in advance when reading current fragment. fragment_size (int): the maximum size of each fragment to read from S3, defaults to 5 MB. If there are fewer @@ -52,25 +42,42 @@ def build_s3_instructions( types. Default to 2 seconds. """ - builder = _JS3Instructions.builder() - builder.awsRegionName(aws_region_name) + j_object_type = jpy.get_type("io.deephaven.extensions.s3.S3Instructions") + + 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): + + builder = self.j_object_type.builder() + builder.awsRegionName(aws_region_name) + + if max_concurrent_requests is not None: + builder.maxConcurrentRequests(max_concurrent_requests) - 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 read_ahead_count is not None: - builder.readAheadCount(read_ahead_count) + if fragment_size is not None: + builder.fragmentSize(fragment_size) - if fragment_size is not None: - builder.fragmentSize(fragment_size) + if max_cache_size is not None: + builder.maxCacheSize(max_cache_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 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 read_timeout is not None: - builder.readTimeout(time.to_j_duration(read_timeout)) + self._j_object = builder.build() - return builder.build() + @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 98af53ad3a1..b5de555e3cc 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -11,6 +11,7 @@ import jpy from deephaven import DHError +from deephaven._wrapper import JObjectWrapper from deephaven.column import Column from deephaven.dtypes import DType from deephaven.table import Table @@ -43,7 +44,7 @@ def _build_parquet_instructions( is_refreshing: bool = False, for_read: bool = True, force_build: bool = False, - special_instructions: Optional[object] = None, + special_instructions: Optional[JObjectWrapper] = None, ): if not any( [ @@ -93,7 +94,7 @@ def _build_parquet_instructions( builder.setIsRefreshing(is_refreshing) if special_instructions is not None: - builder.setSpecialInstructions(special_instructions) + builder.setSpecialInstructions(special_instructions.j_object) return builder.build() @@ -138,7 +139,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[object] = None, + special_instructions: Optional[JObjectWrapper] = None, ) -> Table: """ Reads in a table from a single parquet, metadata file, or directory with recognized layout. @@ -155,8 +156,8 @@ def read( 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[object]): Special instructions for reading parquet files, useful when reading - files from a non-local file system, like S3. By default, None. + special_instructions (Optional[JObjectWrapper]): Special instructions for reading parquet files, useful when + reading files from a non-local file system, like S3. By default, None. Returns: a table From 36db89cf392a1f0b075ce3c1b5846fa879977260 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Sat, 20 Jan 2024 00:05:59 +0530 Subject: [PATCH 27/39] Resolving Ryan's comments --- .../parquet/base/ColumnChunkReaderImpl.java | 6 +- .../parquet/base/ParquetFileReader.java | 16 +-- .../parquet/base/RowGroupReader.java | 4 +- .../parquet/base/RowGroupReaderImpl.java | 8 +- .../base/util/CachedChannelProvider.java | 2 +- .../base/util/LocalFSChannelProvider.java | 10 +- .../base/util/SeekableChannelsProvider.java | 4 +- .../base/util/CachedChannelProviderTest.java | 41 ++++++- .../table/location/ParquetColumnLocation.java | 22 ++-- .../table/location/ParquetTableLocation.java | 10 +- .../extensions/s3/S3SeekableByteChannel.java | 104 +++++++++++------- .../TrackedSeekableChannelsProvider.java | 10 +- 12 files changed, 151 insertions(+), 86 deletions(-) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index ed6f54665e6..59991285dac 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 @@ -28,10 +28,12 @@ import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; +import java.nio.file.Path; import java.util.List; import java.util.NoSuchElementException; 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; @@ -123,8 +125,8 @@ private URI getURI() { if (uri != null) { return uri; } - if (columnChunk.isSetFile_path()) { - return uri = rootURI.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 uri = rootURI; } 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 a80ef672e0b..c44339f36e5 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 @@ -3,7 +3,6 @@ */ package io.deephaven.parquet.base; -import io.deephaven.UncheckedDeephavenException; import io.deephaven.parquet.base.util.Helpers; import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; @@ -13,14 +12,13 @@ import org.apache.parquet.schema.*; import java.io.ByteArrayInputStream; -import java.io.File; import java.io.IOException; import java.net.URI; -import java.net.URISyntaxException; 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.util.*; import static io.deephaven.parquet.base.util.SeekableChannelsProvider.convertToURI; @@ -52,15 +50,9 @@ public ParquetFileReader(final String source, final SeekableChannelsProvider cha public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvider channelsProvider) throws IOException { this.channelsProvider = channelsProvider; - if (!parquetFileURI.getRawPath().endsWith(".parquet")) { - // Construct a new URI for the parent directory - try { - rootURI = new URI(parquetFileURI.getScheme(), parquetFileURI.getHost(), - new File(parquetFileURI.getPath()).getParent(), parquetFileURI.getRawFragment()); - } catch (final URISyntaxException e) { - throw new UncheckedDeephavenException("Cannot construct URI for parent directory of " + parquetFileURI, - e); - } + 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 { rootURI = parquetFileURI; } 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..3d9a6189a84 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.parquet.base.util.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 95bdc6cf525..87c2ad49803 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 @@ -70,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); @@ -80,12 +81,11 @@ public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { OffsetIndex offsetIndex = null; if (columnChunk.isSetOffset_index_offset()) { - try (final SeekableChannelContext channelContext = channelsProvider.makeContext(); - final SeekableByteChannel readChannel = channelsProvider.getReadChannel(channelContext, rootURI)) { + 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); } } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java index b0a9eb70322..2e9e9286cd5 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java @@ -225,7 +225,7 @@ private void dispose() throws IOException { } @Override - public void setContext(@NotNull final SeekableChannelContext channelContext) { + 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/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java index 5e9766d7b4e..a405eb2c63e 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java @@ -4,6 +4,7 @@ package io.deephaven.parquet.base.util; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.IOException; import java.net.URI; @@ -21,15 +22,16 @@ public SeekableChannelContext makeContext() { } @Override - public boolean isCompatibleWith(@NotNull final SeekableChannelContext channelContext) { - return channelContext == SeekableChannelContext.NULL; + public boolean isCompatibleWith(@Nullable final SeekableChannelContext channelContext) { + // Context is not used, hence always compatible + return true; } @Override - public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext, + public SeekableByteChannel getReadChannel(@Nullable final SeekableChannelContext channelContext, @NotNull final URI uri) throws IOException { - // context is unused here because it is NULL + // context is unused here return FileChannel.open(Path.of(uri), StandardOpenOption.READ); } 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 index bec0508f443..e01c1ed11f3 100644 --- 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 @@ -39,7 +39,7 @@ static URI convertToURI(final String source) { } /** - * Create a new {@link SeekableChannelContext} object for creating read and write channels via this provider. + * Create a new {@link SeekableChannelContext} object for creating read channels via this provider. */ SeekableChannelContext makeContext(); @@ -66,7 +66,7 @@ default SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext chann SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull URI uri) throws IOException; - default SeekableByteChannel getWriteChannel(@NotNull String path, final boolean append) throws IOException { + default SeekableByteChannel getWriteChannel(@NotNull final String path, final boolean append) throws IOException { return getWriteChannel(Paths.get(path), append); } diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java index 69aae62b5f9..17a82b04e07 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java @@ -4,6 +4,7 @@ package io.deephaven.parquet.base.util; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import org.junit.Assert; import org.junit.Test; @@ -34,7 +35,11 @@ public void testSimpleRead() throws IOException { for (int jj = 0; jj < sameFile.length; ++jj) { 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); + Assert.assertEquals(buffer.remaining(), buffer.capacity()); sameFile[jj].close(); } } @@ -64,6 +69,9 @@ 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); @@ -112,8 +120,11 @@ 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(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(); @@ -122,6 +133,8 @@ public void testReuse() throws IOException { for (int ci = 0; ci < someResult.length; ++ci) { Assert.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(); @@ -162,9 +175,12 @@ private class TestChannelProvider implements SeekableChannelsProvider { AtomicInteger count = new AtomicInteger(0); + private final class TestChannelContext implements SeekableChannelContext { + } + @Override public SeekableChannelContext makeContext() { - return SeekableChannelContext.NULL; + return new TestChannelContext(); } @Override @@ -175,12 +191,12 @@ public boolean isCompatibleWith(@NotNull SeekableChannelContext channelContext) @Override public SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull String path) { - return new TestMockChannel(count.getAndIncrement(), 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()); + return new TestMockChannel(count.getAndIncrement(), uri.toString(), channelContext); } @Override @@ -197,23 +213,32 @@ public SeekableByteChannel getWriteChannel(@NotNull Path path, boolean append) { public void close() {} } - private class TestMockChannel implements SeekableByteChannel { + private final class TestMockChannel implements SeekableByteChannel, SeekableChannelsProvider.ContextHolder { private final int id; private final String path; + private SeekableChannelContext channelContext; + + private TestMockChannel(int id, String path, SeekableChannelContext channelContext) { + this(id, path); + this.channelContext = channelContext; + } - public TestMockChannel(int id, String path) { + private TestMockChannel(int id, String path) { this.id = id; this.path = path; + this.channelContext = null; } @Override public int read(ByteBuffer dst) { + Assert.assertTrue(channelContext instanceof TestChannelProvider.TestChannelContext); return 0; } @Override public int write(ByteBuffer src) { + Assert.assertNull(channelContext); return 0; } @@ -245,6 +270,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/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 61bc4ed3d6f..e9ba03a7828 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 @@ -27,6 +27,8 @@ import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.base.RowGroupReader; +import io.deephaven.parquet.base.util.SeekableChannelContext; +import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.apache.parquet.format.converter.ParquetMetadataConverter; import io.deephaven.parquet.table.*; import io.deephaven.parquet.table.metadata.CodecInfo; @@ -174,11 +176,12 @@ public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition 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; @@ -188,7 +191,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 = @@ -196,7 +199,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; @@ -211,12 +214,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 d06c03feb04..85750ed560b 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 @@ -6,6 +6,7 @@ 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.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.ParquetSchemaReader; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; @@ -46,6 +47,7 @@ public class ParquetTableLocation extends AbstractTableLocation { private final Map groupingColumns; private final Map columnTypes; private final String version; + private final SeekableChannelContext channelContext; private volatile RowGroupReader[] rowGroupReaders; @@ -62,6 +64,9 @@ public ParquetTableLocation(@NotNull final TableKey tableKey, rowGroupIndices = tableLocationKey.getRowGroupIndices(); } + // Use a common channelContext for creating all column chunk readers + channelContext = getChannelProvider().makeContext(); + final int rowGroupCount = rowGroupIndices.length; rowGroups = IntStream.of(rowGroupIndices) .mapToObj(rgi -> parquetFileReader.fileMetaData.getRow_groups().get(rgi)) @@ -153,8 +158,9 @@ 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; + 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/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index ae6b425c0a6..a25dbf9b764 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -8,6 +8,7 @@ import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import software.amazon.awssdk.services.s3.S3AsyncClient; @@ -119,6 +120,16 @@ long getSize() { 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.future != null) { + fragmentState.future.cancel(true); + } + } + } } private final S3AsyncClient s3AsyncClient; @@ -168,55 +179,65 @@ public int read(@NotNull final ByteBuffer destination) throws IOException { final long localPosition = position; checkClosed(localPosition); - // Fetch the file size if this is the first read + final int numBytesCopied; final S3ChannelContext s3ChannelContext = getS3ChannelContextFrom(channelContext); - populateSize(s3ChannelContext); - if (localPosition >= size) { - // We are finished reading - return -1; - } + 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, if not already in cache - final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); - final int numReadAheadFragments = channelContext == SeekableChannelContext.NULL ? 0 - : Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); - for (int idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { - final CompletableFuture future = s3ChannelContext.getCachedFuture(idx); - if (future == null) { - s3ChannelContext.setFragmentState(idx, sendAsyncRequest(idx)); + // Send async read requests for current fragment as well as read ahead fragments, if not already in cache + final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); + final int numReadAheadFragments = channelContext != s3ChannelContext + ? 0 // We have a local S3ChannelContext, we don't want to do any read-ahead caching + : Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); + for (int idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { + final CompletableFuture future = s3ChannelContext.getCachedFuture(idx); + if (future == null) { + s3ChannelContext.setFragmentState(idx, sendAsyncRequest(idx)); + } } - } - // Wait till the current fragment is fetched - final CompletableFuture 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)); - } + // Wait till the current fragment is fetched + final CompletableFuture 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); - final int sizeToCopy = Math.min(currentFragment.remaining(), destination.remaining()); - final int originalBufferLimit = currentFragment.limit(); - currentFragment.limit(currentFragment.position() + sizeToCopy); - destination.put(currentFragment); - // Need to reset buffer limit, so we can read from the same buffer again in future - currentFragment.limit(originalBufferLimit); - position = localPosition + sizeToCopy; - return sizeToCopy; + // 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) { - // Create a new temporary context just for this read with a cache size of 1 just to support the current - // chunk with no read ahead s3ChannelContext = new S3ChannelContext(1); } else { Assert.instanceOf(channelContext, "channelContext", S3ChannelContext.class); @@ -285,7 +306,10 @@ public SeekableByteChannel position(final long newPosition) throws ClosedChannel @Override public long size() throws IOException { checkClosed(position); - populateSize(getS3ChannelContextFrom(channelContext)); + final S3ChannelContext s3ChannelContext = getS3ChannelContextFrom(channelContext); + try (final SafeCloseable ignored = s3ChannelContext == channelContext ? null : s3ChannelContext) { + populateSize(s3ChannelContext); + } return size; } diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index d7d7946993b..83afc74cfdd 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -11,6 +11,7 @@ import io.deephaven.parquet.base.util.SeekableChannelContext; import io.deephaven.parquet.base.util.SeekableChannelsProvider; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.File; import java.io.IOException; @@ -39,15 +40,16 @@ public SeekableChannelContext makeContext() { } @Override - public boolean isCompatibleWith(@NotNull SeekableChannelContext channelContext) { - return channelContext == SeekableChannelContext.NULL; + public boolean isCompatibleWith(@Nullable SeekableChannelContext channelContext) { + // Context is not used, hence always compatible + return true; } @Override - public final SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext, + public final SeekableByteChannel getReadChannel(@Nullable final SeekableChannelContext channelContext, @NotNull final URI uri) throws IOException { - // context is unused here because it is NULL + // 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)); } From b191ae5839229669c831be160debaa9492747792 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 22 Jan 2024 15:34:32 +0530 Subject: [PATCH 28/39] Resolving more comments --- .../util/SeekableChannelsProviderLoader.java | 11 +- .../table/ParquetTableReadWriteTest.java | 219 +++--------------- .../extensions/s3/S3SeekableByteChannel.java | 9 +- .../s3/S3SeekableChannelProvider.java | 3 +- 4 files changed, 48 insertions(+), 194 deletions(-) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java index fc3cd787fb5..1dcadbf3f72 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java @@ -30,6 +30,10 @@ public static SeekableChannelsProviderLoader getInstance() { private SeekableChannelsProviderLoader() { providers = new ArrayList<>(); + // Load the plugins + for (final SeekableChannelsProviderPlugin plugin : ServiceLoader.load(SeekableChannelsProviderPlugin.class)) { + providers.add(plugin); + } } /** @@ -42,13 +46,6 @@ private SeekableChannelsProviderLoader() { * @return A {@link SeekableChannelsProvider} for the given URI. */ public SeekableChannelsProvider fromServiceLoader(@NotNull final URI uri, @Nullable final Object object) { - if (providers.isEmpty()) { - // Load the plugins - for (final SeekableChannelsProviderPlugin plugin : ServiceLoader - .load(SeekableChannelsProviderPlugin.class)) { - providers.add(plugin); - } - } for (final SeekableChannelsProviderPlugin plugin : providers) { if (plugin.isCompatible(uri, object)) { return plugin.createProvider(uri, object); 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 4b6655018bc..742f70f1bbf 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; @@ -55,6 +56,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; @@ -115,6 +117,9 @@ public final class ParquetTableReadWriteTest { private static final ParquetInstructions EMPTY = ParquetInstructions.EMPTY; private static final ParquetInstructions REFRESHING = ParquetInstructions.builder().setIsRefreshing(true).build(); + private static final boolean ENABLE_S3_TESTING = + Configuration.getInstance().getBooleanWithDefault("ParquetTest.enableS3Testing", false); + private static File rootFile; @Rule @@ -573,58 +578,45 @@ public void testArrayColumns() { } @Test - public void readRefParquetFileFromS3Test() { + public void readSampleParquetFilesFromS3Test1() { + Assume.assumeTrue("Skipping test because s3 testing disabled.", ENABLE_S3_TESTING); final S3Instructions s3Instructions = S3Instructions.builder() - .awsRegionName("us-east-2") + .awsRegionName("us-east-1") .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")); + 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 fromAws1 = 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).head(5).select(); - final Table fromDisk1 = ParquetTools.readSingleFileTable( - new File( - "/Users/shivammalhotra/Documents/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet"), - ParquetTools.SNAPPY, - tableDefinition).head(5).select(); - assertTableEquals(fromAws1, fromDisk1); - - final Table fromAws2 = 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).head(5).select(); - final Table fromDisk2 = ParquetTools.readSingleFileTable( - new File( - "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), - ParquetTools.SNAPPY, - tableDefinition).head(5).select(); - assertTableEquals(fromAws2, fromDisk2); + 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 readLongParquetFileFromS3Test() { + 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) @@ -637,7 +629,6 @@ public void readLongParquetFileFromS3Test() { final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3Instructions) .build(); - final TableDefinition tableDefinition = TableDefinition.of( ColumnDefinition.ofString("hash"), ColumnDefinition.ofLong("version"), @@ -654,152 +645,14 @@ public void readLongParquetFileFromS3Test() { ColumnDefinition.ofTime("last_modified"), ColumnDefinition.ofDouble("input_value")); - final Table fromAws1 = ParquetTools.readSingleFileTable( - "s3://aws-public-blockchain/v1.0/btc/transactions/date=2023-11-13/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet", + // Reading just the first 5 rows to keep the test brief + 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(); - final Table fromDisk1 = ParquetTools.readSingleFileTable( - new File( - "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), - ParquetTools.SNAPPY, - tableDefinition).select(); - assertTableEquals(fromAws1, fromDisk1); - } - @Test - public void readRefParquetFileLocally() { - 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")); - final Table fromAws1 = - ParquetTools.readSingleFileTable( - new File( - "/Users/shivammalhotra/Documents/part-00000-da3a3c27-700d-496d-9c41-81281388eca8-c000.snappy.parquet"), - ParquetTools.SNAPPY, - tableDefinition).head(5).select(); - } - - @Test - public void profileReadingFromS3() { - final S3Instructions s3Instructions = S3Instructions.builder() - .awsRegionName("us-east-1") - .readAheadCount(1) - .fragmentSize(5 * 1024 * 1024) - .maxConcurrentRequests(50) - .maxCacheSize(32) - .readTimeout(Duration.ofMinutes(5)) - .build(); - final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setSpecialInstructions(s3Instructions) - .build(); - - long totalTime = 0; - long NUM_RUNS = 1; - for (int i = 0; i < NUM_RUNS; i++) { - final long start = System.nanoTime(); - ParquetTools.readTable("s3://dh-s3-parquet-test1/multiColFile.parquet", readInstructions).select(); - final long end = System.nanoTime(); - totalTime += end - start; - System.out.println((i + 1) + ". Execution time AWS is " + (end - start) / 1000_000_000.0 + " sec"); - } - System.out.println("Average execution time AWS is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); - - final S3Instructions s3Instructions2 = S3Instructions.builder() - .awsRegionName("us-east-2") - .readAheadCount(1) - .fragmentSize(5 * 1024 * 1024) - .maxConcurrentRequests(50) - .maxCacheSize(32) - .readTimeout(Duration.ofMinutes(5)) - .build(); - final ParquetInstructions readInstructions2 = new ParquetInstructions.Builder() - .setSpecialInstructions(s3Instructions2) - .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")); - - totalTime = 0; - for (int i = 0; i < NUM_RUNS; i++) { - final long start = System.nanoTime(); - ParquetTools.readSingleFileTable( - "s3://aws-public-blockchain/v1.0/btc/transactions/date=2009-01-03/part-00000-bdd84ab2-82e9-4a79-8212-7accd76815e8-c000.snappy.parquet", - readInstructions2, tableDefinition).head(5).select(); - final long end = System.nanoTime(); - totalTime += end - start; - System.out.println((i + 1) + ". Execution time AWS is " + (end - start) / 1000_000_000.0 + " sec"); - } - System.out.println("Average execution time AWS is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); - - NUM_RUNS = 100; - totalTime = 0; - for (int i = 0; i < NUM_RUNS; i++) { - final long start = System.nanoTime(); - ParquetTools.readTable("/Users/shivammalhotra/documents/multiColFile.parquet").select(); - final long end = System.nanoTime(); - totalTime += end - start; - // System.out.println((i + 1) + ". Execution time local is " + (end - start) / 1000_000_000.0 + " sec"); - } - System.out.println("Average execution time local is " + totalTime / (NUM_RUNS * 1000_000_000.0) + " sec"); - } - - @Test - public void readParquetFileFromS3Test() { - final S3Instructions s3Instructions = S3Instructions.builder() - .awsRegionName("us-east-1") - .readAheadCount(1) - .fragmentSize(5 * 1024 * 1024) - .maxConcurrentRequests(50) - .maxCacheSize(32) - .readTimeout(Duration.ofSeconds(60)) - .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); + 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 diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index a25dbf9b764..0ccfdcc0c2e 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -143,18 +143,21 @@ public void close() { private long size; private int 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 SeekableChannelContext channelContext, @NotNull final URI uri, - @NotNull final S3AsyncClient s3AsyncClient, final S3Instructions s3Instructions) { + S3SeekableByteChannel(@NotNull final URI uri, @NotNull final S3AsyncClient s3AsyncClient, + @NotNull final S3Instructions s3Instructions) { 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.channelContext = channelContext; this.size = UNINITIALIZED_SIZE; this.position = 0; } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index 63fca0c621f..eb8b41bfae8 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -38,7 +38,8 @@ final class S3SeekableChannelProvider implements SeekableChannelsProvider { @Override public SeekableByteChannel getReadChannel(@NotNull final SeekableChannelContext channelContext, @NotNull final URI uri) { - return new S3SeekableByteChannel(channelContext, uri, s3AsyncClient, s3Instructions); + // context is unused here, will be set before reading from the channel + return new S3SeekableByteChannel(uri, s3AsyncClient, s3Instructions); } @Override From c30fcfa3cccf9edfa7e9a41a4497e7719d5d00af Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 23 Jan 2024 18:02:22 +0530 Subject: [PATCH 29/39] Moved files from io.deephaven.parquet.base.util to io.deephaven.util.channel --- .../deephaven/util/channel}/CachedChannelProvider.java | 2 +- .../util/channel}/LocalFSChannelProvider.java | 2 +- .../util/channel}/SeekableChannelContext.java | 2 +- .../util/channel}/SeekableChannelsProvider.java | 2 +- .../util/channel}/SeekableChannelsProviderLoader.java | 2 +- .../util/channel}/SeekableChannelsProviderPlugin.java | 2 +- .../util/channel}/CachedChannelProviderTest.java | 2 +- .../io/deephaven/parquet/base/ColumnChunkReader.java | 4 ++-- .../deephaven/parquet/base/ColumnChunkReaderImpl.java | 4 ++-- .../io/deephaven/parquet/base/ColumnPageReader.java | 2 +- .../deephaven/parquet/base/ColumnPageReaderImpl.java | 4 ++-- .../io/deephaven/parquet/base/ParquetFileReader.java | 6 +++--- .../io/deephaven/parquet/base/ParquetFileWriter.java | 2 +- .../java/io/deephaven/parquet/base/RowGroupReader.java | 2 +- .../io/deephaven/parquet/base/RowGroupReaderImpl.java | 4 ++-- .../io/deephaven/parquet/table/ParquetTableWriter.java | 4 ++-- .../java/io/deephaven/parquet/table/ParquetTools.java | 10 +++++----- .../parquet/table/location/ParquetColumnLocation.java | 5 ++--- .../parquet/table/location/ParquetTableLocation.java | 4 ++-- .../parquet/table/pagestore/ColumnChunkPageStore.java | 4 ++-- .../OffsetIndexBasedColumnChunkPageStore.java | 2 +- .../VariablePageSizeColumnChunkPageStore.java | 2 +- .../table/pagestore/topage/ChunkDictionary.java | 2 +- .../parquet/table/pagestore/topage/ToObjectPage.java | 2 +- .../parquet/table/pagestore/topage/ToPage.java | 2 +- .../table/pagestore/topage/ToPageWithDictionary.java | 2 +- .../parquet/table/pagestore/topage/ToStringPage.java | 2 +- .../table/pagestore/topage/ToStringSetPage.java | 2 +- extensions/s3/build.gradle | 2 -- .../deephaven/extensions/s3/S3SeekableByteChannel.java | 4 ++-- .../extensions/s3/S3SeekableChannelProvider.java | 4 ++-- .../extensions/s3/S3SeekableChannelProviderPlugin.java | 4 ++-- extensions/trackedfile/build.gradle | 2 -- .../trackedfile/TrackedSeekableChannelsProvider.java | 4 ++-- .../TrackedSeekableChannelsProviderPlugin.java | 4 ++-- 35 files changed, 52 insertions(+), 57 deletions(-) rename {extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util => Util/src/main/java/io/deephaven/util/channel}/CachedChannelProvider.java (99%) rename {extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util => Util/src/main/java/io/deephaven/util/channel}/LocalFSChannelProvider.java (97%) rename {extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util => Util/src/main/java/io/deephaven/util/channel}/SeekableChannelContext.java (91%) rename {extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util => Util/src/main/java/io/deephaven/util/channel}/SeekableChannelsProvider.java (98%) rename {extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util => Util/src/main/java/io/deephaven/util/channel}/SeekableChannelsProviderLoader.java (98%) rename {extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util => Util/src/main/java/io/deephaven/util/channel}/SeekableChannelsProviderPlugin.java (94%) rename {extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util => Util/src/test/java/io/deephaven/util/channel}/CachedChannelProviderTest.java (99%) diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java b/Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java similarity index 99% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java rename to Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java index 2e9e9286cd5..baea7fa10c8 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/CachedChannelProvider.java +++ b/Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java @@ -1,7 +1,7 @@ /** * 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; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java b/Util/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java similarity index 97% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java rename to Util/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java index a405eb2c63e..59ddcbd3d4a 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/LocalFSChannelProvider.java +++ b/Util/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java @@ -1,7 +1,7 @@ /** * 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; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelContext.java b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java similarity index 91% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelContext.java rename to Util/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java index 2ed9f8be3bd..c5e7e155068 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelContext.java +++ b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java @@ -1,4 +1,4 @@ -package io.deephaven.parquet.base.util; +package io.deephaven.util.channel; import io.deephaven.util.SafeCloseable; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProvider.java b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java similarity index 98% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProvider.java rename to Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java index e01c1ed11f3..8184943a48a 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProvider.java +++ b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java @@ -1,7 +1,7 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.parquet.base.util; +package io.deephaven.util.channel; import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.FinalDefault; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java similarity index 98% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java rename to Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java index 1dcadbf3f72..47ba411d70d 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderLoader.java +++ b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java @@ -1,7 +1,7 @@ /** * 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; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java similarity index 94% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java rename to Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java index 92990fb94f5..0ea6fa3527d 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/util/SeekableChannelsProviderPlugin.java +++ b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java @@ -1,4 +1,4 @@ -package io.deephaven.parquet.base.util; +package io.deephaven.util.channel; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; diff --git a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java b/Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java similarity index 99% rename from extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java rename to Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java index 17a82b04e07..60dae356ecf 100644 --- a/extensions/parquet/base/src/test/java/io/deephaven/parquet/base/util/CachedChannelProviderTest.java +++ b/Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java @@ -1,7 +1,7 @@ /** * 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; 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 e8db3729559..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,8 +3,8 @@ */ package io.deephaven.parquet.base; -import io.deephaven.parquet.base.util.SeekableChannelContext; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +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; 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 59991285dac..841891c2d3a 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,8 +4,8 @@ package io.deephaven.parquet.base; import io.deephaven.UncheckedDeephavenException; -import io.deephaven.parquet.base.util.SeekableChannelContext; -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.LazyCachingFunction; 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 94555853ae9..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,7 +4,7 @@ package io.deephaven.parquet.base; -import io.deephaven.parquet.base.util.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.column.Dictionary; import org.jetbrains.annotations.NotNull; 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 cedc885de08..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,8 +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.SeekableChannelContext; -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; 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 c44339f36e5..1c869d706fb 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,8 +4,8 @@ package io.deephaven.parquet.base; import io.deephaven.parquet.base.util.Helpers; -import io.deephaven.parquet.base.util.SeekableChannelContext; -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; @@ -21,7 +21,7 @@ import java.nio.file.Path; import java.util.*; -import static io.deephaven.parquet.base.util.SeekableChannelsProvider.convertToURI; +import static io.deephaven.util.channel.SeekableChannelsProvider.convertToURI; /** * Top level accessor for a parquet file which can read both from a file path string or a CLI style file URI, 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 3d9a6189a84..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,7 +3,7 @@ */ package io.deephaven.parquet.base; -import io.deephaven.parquet.base.util.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.format.RowGroup; import org.jetbrains.annotations.NotNull; 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 87c2ad49803..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,8 +3,8 @@ */ package io.deephaven.parquet.base; -import io.deephaven.parquet.base.util.SeekableChannelContext; -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; 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 5b1b2f2310f..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,7 +20,7 @@ import io.deephaven.parquet.base.ColumnWriter; import io.deephaven.parquet.base.ParquetFileWriter; import io.deephaven.parquet.base.RowGroupWriter; -import io.deephaven.parquet.base.util.SeekableChannelsProviderLoader; +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; @@ -45,7 +45,7 @@ import java.nio.file.Paths; import java.util.*; -import static io.deephaven.parquet.base.util.SeekableChannelsProvider.convertToURI; +import static io.deephaven.util.channel.SeekableChannelsProvider.convertToURI; /** * API for writing DH tables in parquet format 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 f472209dee5..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,9 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; -import io.deephaven.parquet.base.util.SeekableChannelsProviderLoader; -import io.deephaven.parquet.base.util.SeekableChannelsProviderPlugin; +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.table.impl.PartitionAwareSourceTable; @@ -38,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; @@ -55,7 +55,7 @@ import java.util.*; import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; -import static io.deephaven.parquet.base.util.SeekableChannelsProvider.convertToURI; +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; 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 e9ba03a7828..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 @@ -3,7 +3,6 @@ */ package io.deephaven.parquet.table.location; -import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.chunk.*; @@ -27,8 +26,8 @@ import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.base.RowGroupReader; -import io.deephaven.parquet.base.util.SeekableChannelContext; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +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; 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 85750ed560b..204ee511297 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 @@ -6,7 +6,7 @@ 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.parquet.base.util.SeekableChannelContext; +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; @@ -21,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; 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 ca854984827..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,8 +8,8 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Context; import io.deephaven.engine.table.SharedContext; -import io.deephaven.parquet.base.util.SeekableChannelContext; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +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; 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 5743240112f..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,7 +7,7 @@ import io.deephaven.base.verify.Require; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.ChunkPage; -import io.deephaven.parquet.base.util.SeekableChannelContext; +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; 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 328a10176e8..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,7 +7,7 @@ import io.deephaven.base.verify.Require; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.ChunkPage; -import io.deephaven.parquet.base.util.SeekableChannelContext; +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; 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 fd188f76456..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,7 +6,7 @@ import gnu.trove.map.TObjectIntMap; import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.chunk.attributes.Any; -import io.deephaven.parquet.base.util.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.stringset.LongBitmapStringSet; import io.deephaven.chunk.ObjectChunk; import io.deephaven.util.datastructures.LazyCachingSupplier; 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 5ec46ee5dc6..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,7 +5,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; -import io.deephaven.parquet.base.util.SeekableChannelContext; +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; 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 d964ce27041..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,7 +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.parquet.base.util.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.vector.Vector; import io.deephaven.engine.page.ChunkPage; import io.deephaven.vector.VectorFactory; 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 1c4196fbac3..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,7 +3,7 @@ */ package io.deephaven.parquet.table.pagestore.topage; -import io.deephaven.parquet.base.util.SeekableChannelContext; +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; 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 80cb8fc6430..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,7 +5,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; -import io.deephaven.parquet.base.util.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.column.Dictionary; import org.apache.parquet.io.api.Binary; import org.jetbrains.annotations.NotNull; 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 17e411fbc6b..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,7 +4,7 @@ package io.deephaven.parquet.table.pagestore.topage; import io.deephaven.chunk.attributes.Any; -import io.deephaven.parquet.base.util.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.stringset.HashStringSet; import io.deephaven.stringset.StringSet; import io.deephaven.stringset.LongBitmapStringSet; diff --git a/extensions/s3/build.gradle b/extensions/s3/build.gradle index ac2921e3397..6ce57a1cd75 100644 --- a/extensions/s3/build.gradle +++ b/extensions/s3/build.gradle @@ -6,8 +6,6 @@ plugins { description 'Used to create a channel provider plugin for reading and writing files stored in S3.' dependencies { - api project(':extensions-parquet-base') - implementation project(':Base') implementation project(':Util') diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index 0ccfdcc0c2e..e0ec78876a2 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -6,8 +6,8 @@ import io.deephaven.base.verify.Assert; import java.util.concurrent.CancellationException; -import io.deephaven.parquet.base.util.SeekableChannelContext; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index eb8b41bfae8..00cb897cf97 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -3,8 +3,8 @@ */ package io.deephaven.extensions.s3; -import io.deephaven.parquet.base.util.SeekableChannelContext; -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 software.amazon.awssdk.http.async.SdkAsyncHttpClient; import software.amazon.awssdk.regions.Region; 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 index 67de3592c40..69144f5062e 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProviderPlugin.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProviderPlugin.java @@ -4,8 +4,8 @@ package io.deephaven.extensions.s3; import com.google.auto.service.AutoService; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; -import io.deephaven.parquet.base.util.SeekableChannelsProviderPlugin; +import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProviderPlugin; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; diff --git a/extensions/trackedfile/build.gradle b/extensions/trackedfile/build.gradle index 9d3f32cfd01..5bb1334b0a8 100644 --- a/extensions/trackedfile/build.gradle +++ b/extensions/trackedfile/build.gradle @@ -6,8 +6,6 @@ plugins { description 'Used to create a channel provider plugin for reading and writing local files.' dependencies { - api project(':extensions-parquet-base') - implementation project(':Base') implementation project(':Util') implementation project(':engine-table') diff --git a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java index 83afc74cfdd..7b6f0c55fbe 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProvider.java @@ -8,8 +8,8 @@ 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.SeekableChannelContext; -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; 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 index 5f6a3c449b0..5e4a8a60004 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java @@ -5,8 +5,8 @@ import com.google.auto.service.AutoService; import io.deephaven.engine.util.file.TrackedFileHandleFactory; -import io.deephaven.parquet.base.util.SeekableChannelsProvider; -import io.deephaven.parquet.base.util.SeekableChannelsProviderPlugin; +import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProviderPlugin; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; From 6bc9f54285bed6f9f244b477e7ca6295ad40024b Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 23 Jan 2024 18:26:02 +0530 Subject: [PATCH 30/39] Resolving more comments --- .../deephaven/util/channel/CachedChannelProvider.java | 10 ++++++++++ .../util/channel/SeekableChannelsProvider.java | 9 --------- .../util/channel/SeekableChannelsProviderPlugin.java | 3 +++ .../util/channel/CachedChannelProviderTest.java | 2 +- .../parquet/table/location/ParquetTableLocation.java | 10 ++++------ .../table/location/ParquetTableLocationKey.java | 1 - .../s3/ByteBufferAsyncResponseTransformer.java | 2 +- .../io/deephaven/extensions/s3/S3Instructions.java | 3 +-- .../deephaven/extensions/s3/S3SeekableByteChannel.java | 4 ++-- .../TrackedSeekableChannelsProviderPlugin.java | 1 - py/server/deephaven/experimental/s3.py | 3 +-- 11 files changed, 23 insertions(+), 25 deletions(-) diff --git a/Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java b/Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java index baea7fa10c8..2abc89c1357 100644 --- a/Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java +++ b/Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java @@ -8,6 +8,7 @@ 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; @@ -23,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; diff --git a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java index 8184943a48a..b8539997618 100644 --- a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java +++ b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java @@ -49,15 +49,6 @@ static URI convertToURI(final String source) { */ boolean isCompatibleWith(@NotNull SeekableChannelContext channelContext); - interface ContextHolder { - void setContext(SeekableChannelContext channelContext); - - @FinalDefault - default void clearContext() { - setContext(null); - } - } - default SeekableByteChannel getReadChannel(@NotNull SeekableChannelContext channelContext, @NotNull String uriStr) throws IOException { return getReadChannel(channelContext, convertToURI(uriStr)); diff --git a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java index 0ea6fa3527d..efb2e4892c0 100644 --- a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java +++ b/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java @@ -1,3 +1,6 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ package io.deephaven.util.channel; import org.jetbrains.annotations.NotNull; diff --git a/Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java b/Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java index 60dae356ecf..9339e7a82a8 100644 --- a/Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java +++ b/Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java @@ -213,7 +213,7 @@ public SeekableByteChannel getWriteChannel(@NotNull Path path, boolean append) { public void close() {} } - private final class TestMockChannel implements SeekableByteChannel, SeekableChannelsProvider.ContextHolder { + private final class TestMockChannel implements SeekableByteChannel, CachedChannelProvider.ContextHolder { private final int id; private final String path; 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 204ee511297..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 @@ -47,7 +47,6 @@ public class ParquetTableLocation extends AbstractTableLocation { private final Map groupingColumns; private final Map columnTypes; private final String version; - private final SeekableChannelContext channelContext; private volatile RowGroupReader[] rowGroupReaders; @@ -64,9 +63,6 @@ public ParquetTableLocation(@NotNull final TableKey tableKey, rowGroupIndices = tableLocationKey.getRowGroupIndices(); } - // Use a common channelContext for creating all column chunk readers - channelContext = getChannelProvider().makeContext(); - final int rowGroupCount = rowGroupIndices.length; rowGroups = IntStream.of(rowGroupIndices) .mapToObj(rgi -> parquetFileReader.fileMetaData.getRow_groups().get(rgi)) @@ -159,8 +155,10 @@ protected ParquetColumnLocation makeColumnLocation(@NotNull final String final List nameList = columnPath == null ? Collections.singletonList(parquetColumnName) : Arrays.asList(columnPath); final ColumnChunkReader[] columnChunkReaders; - columnChunkReaders = Arrays.stream(getRowGroupReaders()) - .map(rgr -> rgr.getColumnChunk(nameList, channelContext)).toArray(ColumnChunkReader[]::new); + 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/ParquetTableLocationKey.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java index 4bec4f8438c..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,7 +3,6 @@ */ package io.deephaven.parquet.table.location; -import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.impl.locations.local.URITableLocationKey; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.ParquetTools; 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 index 005387a7a2e..7fc3847f0f8 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java @@ -48,7 +48,7 @@ private static final class ByteBufferSubscriber implements Subscriber resultFuture, ByteBuffer byteBuffer) { + ByteBufferSubscriber(final CompletableFuture resultFuture, final ByteBuffer byteBuffer) { this.resultFuture = resultFuture; this.byteBuffer = byteBuffer; } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index da9a69bc2a0..b701c9595e4 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -61,8 +61,7 @@ public int fragmentSize() { /** * The maximum number of fragments to cache in memory, defaults to {@value #DEFAULT_MAX_CACHE_SIZE}. This caching is - * done at deephaven layer using a modulo-based logic. For example, if the max cache size is 32,fragment {@code i} - * will be cached in slot {@code i % 32} and will stay there until it is evicted by a more recent fragment. + * done at deephaven layer for faster access of recently read fragments. */ @Value.Default public int maxCacheSize() { diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index e0ec78876a2..c83d248b345 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -6,8 +6,8 @@ import io.deephaven.base.verify.Assert; import java.util.concurrent.CancellationException; +import io.deephaven.util.channel.CachedChannelProvider; import io.deephaven.util.channel.SeekableChannelContext; -import io.deephaven.util.channel.SeekableChannelsProvider; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -31,7 +31,7 @@ * {@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, SeekableChannelsProvider.ContextHolder { +final class S3SeekableByteChannel implements SeekableByteChannel, CachedChannelProvider.ContextHolder { private static final long CLOSED_SENTINEL = -1; 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 index 5e4a8a60004..e33529147f3 100644 --- a/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java +++ b/extensions/trackedfile/src/main/java/io/deephaven/extensions/trackedfile/TrackedSeekableChannelsProviderPlugin.java @@ -33,7 +33,6 @@ public SeekableChannelsProvider createProvider(@NotNull final URI uri, @Nullable } throw new IllegalArgumentException("Arguments not compatible, provided uri " + uri); } - // TODO I am confused which constructor should I use, this or getInstance()?? return new TrackedSeekableChannelsProvider(TrackedFileHandleFactory.getInstance()); } } diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index d77da1d21a0..29095cb071e 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -30,8 +30,7 @@ class S3Instructions(JObjectWrapper): 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 deephaven layer using a modulo-based logic. For example, if the max cache size is 32, - fragment i will be cached in slot i % 32 and will stay there until it is evicted by a more recent fragment. + caching is done at deephaven layer for faster access of 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 From 10172e38d07b29d873c0872725f7ede5a7908a04 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 23 Jan 2024 21:39:47 +0530 Subject: [PATCH 31/39] Added buffer pool support and TODOs for future projects --- .../parquet/base/ColumnChunkReaderImpl.java | 1 + .../parquet/base/ParquetFileReader.java | 1 + .../table/ParquetTableReadWriteTest.java | 2 +- .../deephaven/extensions/s3/BufferPool.java | 27 ++++++++ .../ByteBufferAsyncResponseTransformer.java | 65 ++++++++++++++----- .../extensions/s3/S3SeekableByteChannel.java | 54 +++++++++++---- .../s3/S3SeekableChannelProvider.java | 8 ++- .../extensions/s3/SegmentedBufferPool.java | 61 +++++++++++++++++ 8 files changed, 185 insertions(+), 34 deletions(-) create mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java create mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/SegmentedBufferPool.java 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 841891c2d3a..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 @@ -128,6 +128,7 @@ private URI getURI() { if (columnChunk.isSetFile_path() && FILE_URI_SCHEME.equals(uri.getScheme())) { return uri = Path.of(rootURI).resolve(columnChunk.getFile_path()).toUri(); } else { + // TODO(deephaven-core#5066): Add support for reading metadata files from non-file URIs return uri = rootURI; } } 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 1c869d706fb..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 @@ -54,6 +54,7 @@ public ParquetFileReader(final URI parquetFileURI, final SeekableChannelsProvide // 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; 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 742f70f1bbf..245dc88ac0c 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 @@ -117,6 +117,7 @@ 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); @@ -645,7 +646,6 @@ public void readSampleParquetFilesFromS3Test2() { ColumnDefinition.ofTime("last_modified"), ColumnDefinition.ofDouble("input_value")); - // Reading just the first 5 rows to keep the test brief 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(); 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..85afed69b7d --- /dev/null +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java @@ -0,0 +1,27 @@ +package io.deephaven.extensions.s3; + +import org.jetbrains.annotations.Nullable; + +import java.nio.ByteBuffer; + +public interface BufferPool { + + interface BufferHolder extends AutoCloseable { + + /** + * @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 index 7fc3847f0f8..a347154f70c 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java @@ -4,6 +4,7 @@ package io.deephaven.extensions.s3; import io.deephaven.base.verify.Assert; +import org.jetbrains.annotations.NotNull; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; import software.amazon.awssdk.core.async.AsyncResponseTransformer; @@ -14,17 +15,24 @@ final class ByteBufferAsyncResponseTransformer implements AsyncResponseTransformer { - private final int bufferSize; - private volatile CompletableFuture cf; + private final BufferPool.BufferHolder bufferHolder; + private final ByteBuffer byteBuffer; - ByteBufferAsyncResponseTransformer(final int bufferSize) { - this.bufferSize = bufferSize; + private volatile boolean released; + private volatile CompletableFuture currentFuture; + + /** + * @param bufferHolder A {@link BufferPool.BufferHolder} that will provide a buffer to store the response bytes. + * This will be {@link BufferPool.BufferHolder#close}d when {@link #release()} is called. + */ + ByteBufferAsyncResponseTransformer(@NotNull final BufferPool.BufferHolder bufferHolder) { + this.bufferHolder = bufferHolder; + this.byteBuffer = bufferHolder.get(); } @Override public CompletableFuture prepare() { - cf = new CompletableFuture<>(); - return cf; + return currentFuture = new CompletableFuture<>(); } @Override @@ -34,28 +42,43 @@ public void onResponse(final ResponseT response) { @Override public void onStream(final SdkPublisher publisher) { - // This could be improved with the addition of a buffer pool or similar resource allocation sharing support - publisher.subscribe(new ByteBufferSubscriber(cf, ByteBuffer.allocate(bufferSize))); + publisher.subscribe(new ByteBufferSubscriber(currentFuture)); } @Override public void exceptionOccurred(final Throwable throwable) { - cf.completeExceptionally(throwable); + currentFuture.completeExceptionally(throwable); + } + + /** + * Prevent further mutation of the underlying buffer by this ByteBufferAsyncResponseTransformer and any of its + * Subscribers. + */ + public synchronized void release() { + released = true; + bufferHolder.close(); } - private static final class ByteBufferSubscriber implements Subscriber { + 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. + */ + private final ByteBuffer duplicate; + private Subscription subscription; - private final ByteBuffer byteBuffer; - ByteBufferSubscriber(final CompletableFuture resultFuture, final ByteBuffer byteBuffer) { + ByteBufferSubscriber(CompletableFuture resultFuture) { this.resultFuture = resultFuture; - this.byteBuffer = byteBuffer; + this.duplicate = byteBuffer.duplicate(); } @Override public void onSubscribe(final Subscription s) { if (subscription != null) { + // Only maintain the first successful subscription s.cancel(); return; } @@ -66,9 +89,17 @@ public void onSubscribe(final Subscription s) { @Override public void onNext(final ByteBuffer responseBytes) { // Assuming responseBytes will fit in the buffer - Assert.assertion(responseBytes.remaining() <= byteBuffer.remaining(), - "responseBytes.remaining() <= byteBuffer.remaining()"); - byteBuffer.put(responseBytes); + 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); } @@ -79,7 +110,7 @@ public void onError(final Throwable throwable) { @Override public void onComplete() { - resultFuture.complete(byteBuffer.flip().asReadOnlyBuffer()); + resultFuture.complete(byteBuffer.asReadOnlyBuffer().limit(duplicate.position())); } } } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index c83d248b345..8df0bba4e05 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -4,6 +4,7 @@ package io.deephaven.extensions.s3; import io.deephaven.base.verify.Assert; + import java.util.concurrent.CancellationException; import io.deephaven.util.channel.CachedChannelProvider; @@ -55,11 +56,20 @@ private static final class FragmentState { /** * The future that will be completed with the fragment's bytes */ + @NotNull private CompletableFuture future; - private FragmentState(final int fragmentIndex, final CompletableFuture future) { + /** + * The {@link ByteBufferAsyncResponseTransformer} that will be used to complete the future + */ + @NotNull + private ByteBufferAsyncResponseTransformer asyncResponseTransformer; + + private FragmentState(final int fragmentIndex, @NotNull final CompletableFuture future, + @NotNull final ByteBufferAsyncResponseTransformer asyncResponseTransformer) { this.fragmentIndex = fragmentIndex; this.future = future; + this.asyncResponseTransformer = asyncResponseTransformer; } } @@ -79,24 +89,28 @@ private FragmentState(final int fragmentIndex, final CompletableFuture future) { + void setFragmentState(final int fragmentIndex, @NotNull final CompletableFuture future, + @NotNull final ByteBufferAsyncResponseTransformer asyncResponseTransformer) { final int cacheIdx = getIndex(fragmentIndex); final FragmentState cachedEntry = bufferCache[cacheIdx]; if (cachedEntry == null) { - bufferCache[cacheIdx] = new FragmentState(fragmentIndex, future); + bufferCache[cacheIdx] = new FragmentState(fragmentIndex, future, asyncResponseTransformer); } else { // We should not cache an already cached fragment Assert.neq(cachedEntry.fragmentIndex, "cachedEntry.fragmentIndex", fragmentIndex, "fragmentIdx"); - // Cancel any outstanding requests for this cached fragment + // Cancel any outstanding requests for this cached fragment, and release the buffer for reuse cachedEntry.future.cancel(true); + cachedEntry.asyncResponseTransformer.release(); // Reuse the existing entry cachedEntry.fragmentIndex = fragmentIndex; cachedEntry.future = future; + cachedEntry.asyncResponseTransformer = asyncResponseTransformer; } } @@ -127,6 +141,7 @@ public void close() { for (final FragmentState fragmentState : bufferCache) { if (fragmentState != null && fragmentState.future != null) { fragmentState.future.cancel(true); + fragmentState.asyncResponseTransformer.release(); } } } @@ -136,6 +151,7 @@ public void close() { 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 @@ -152,12 +168,13 @@ public void close() { private long position; S3SeekableByteChannel(@NotNull final URI uri, @NotNull final S3AsyncClient s3AsyncClient, - @NotNull final S3Instructions s3Instructions) { + @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; } @@ -192,16 +209,13 @@ public int read(@NotNull final ByteBuffer destination) throws IOException { return -1; } - // Send async read requests for current fragment as well as read ahead fragments, if not already in cache + // Send async read requests for current fragment as well as read ahead fragments final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); final int numReadAheadFragments = channelContext != s3ChannelContext ? 0 // We have a local S3ChannelContext, we don't want to do any read-ahead caching : Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); for (int idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { - final CompletableFuture future = s3ChannelContext.getCachedFuture(idx); - if (future == null) { - s3ChannelContext.setFragmentState(idx, sendAsyncRequest(idx)); - } + sendAsyncRequest(idx, s3ChannelContext); } // Wait till the current fragment is fetched @@ -254,15 +268,27 @@ private int fragmentIndexForByteNumber(final long byteNumber) { } /** - * @return A {@link CompletableFuture} that will be completed with the bytes of the fragment + * 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 CompletableFuture sendAsyncRequest(final int fragmentIndex) { + private void sendAsyncRequest(final int fragmentIndex, + @NotNull final S3ChannelContext s3ChannelContext) { + if (s3ChannelContext.getCachedFuture(fragmentIndex) != null) { + // We have a pending request for this fragment + return; + } final int fragmentSize = s3Instructions.fragmentSize(); final long readFrom = (long) fragmentIndex * fragmentSize; final long readTo = Math.min(readFrom + fragmentSize, size) - 1; final String range = "bytes=" + readFrom + "-" + readTo; - return s3AsyncClient.getObject(builder -> builder.bucket(bucket).key(key).range(range), - new ByteBufferAsyncResponseTransformer<>((int) (readTo - readFrom + 1))); + + final int numBytes = (int) (readTo - readFrom + 1); + final ByteBufferAsyncResponseTransformer asyncResponseTransformer = + new ByteBufferAsyncResponseTransformer(bufferPool.take(numBytes)); + final CompletableFuture future = s3AsyncClient.getObject( + builder -> builder.bucket(bucket).key(key).range(range), + asyncResponseTransformer); + s3ChannelContext.setFragmentState(fragmentIndex, future, asyncResponseTransformer); } private IOException handleS3Exception(final Exception e, final String operationDescription) { diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index 00cb897cf97..17e2e0d0914 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -22,24 +22,28 @@ final class S3SeekableChannelProvider implements SeekableChannelsProvider { private final S3AsyncClient s3AsyncClient; private final S3Instructions s3Instructions; + private final BufferPool bufferPool; - S3SeekableChannelProvider(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 this.s3AsyncClient = S3AsyncClient.builder() .region(Region.of(s3Instructions.awsRegionName())) .httpClient(asyncHttpClient) .build(); this.s3Instructions = s3Instructions; + this.bufferPool = new SegmentedBufferPool(s3Instructions.fragmentSize()); } @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); + return new S3SeekableByteChannel(uri, s3AsyncClient, s3Instructions, bufferPool); } @Override 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); + } + } + } +} From 7264be1fab037015718ba5ee50bbc3563a0d61fd Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 23 Jan 2024 22:43:37 +0530 Subject: [PATCH 32/39] Added a new module Util/channel --- Util/channel/build.gradle | 22 +++++++++ Util/channel/gradle.properties | 1 + .../util/channel/CachedChannelProvider.java | 0 .../util/channel/LocalFSChannelProvider.java | 0 .../util/channel/SeekableChannelContext.java | 0 .../channel/SeekableChannelsProvider.java | 1 - .../SeekableChannelsProviderLoader.java | 0 .../SeekableChannelsProviderPlugin.java | 0 .../channel/CachedChannelProviderTest.java | 47 +++++++++---------- extensions/parquet/base/build.gradle | 2 + extensions/s3/build.gradle | 2 + settings.gradle | 3 ++ 12 files changed, 53 insertions(+), 25 deletions(-) create mode 100644 Util/channel/build.gradle create mode 100644 Util/channel/gradle.properties rename Util/{ => channel}/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java (100%) rename Util/{ => channel}/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java (100%) rename Util/{ => channel}/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java (100%) rename Util/{ => channel}/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java (97%) rename Util/{ => channel}/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java (100%) rename Util/{ => channel}/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java (100%) rename Util/{ => channel}/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java (88%) 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/Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java similarity index 100% rename from Util/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java rename to Util/channel/src/main/java/io/deephaven/util/channel/CachedChannelProvider.java diff --git a/Util/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java similarity index 100% rename from Util/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java rename to Util/channel/src/main/java/io/deephaven/util/channel/LocalFSChannelProvider.java diff --git a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java similarity index 100% rename from Util/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java rename to Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelContext.java diff --git a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java similarity index 97% rename from Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java rename to Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java index b8539997618..75be19f4592 100644 --- a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProvider.java @@ -4,7 +4,6 @@ package io.deephaven.util.channel; import io.deephaven.util.SafeCloseable; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.io.File; diff --git a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java similarity index 100% rename from Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java rename to Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderLoader.java diff --git a/Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java b/Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java similarity index 100% rename from Util/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java rename to Util/channel/src/main/java/io/deephaven/util/channel/SeekableChannelsProviderPlugin.java diff --git a/Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java b/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java similarity index 88% rename from Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java rename to Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java index 9339e7a82a8..64ee9c667e9 100644 --- a/Util/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java +++ b/Util/channel/src/test/java/io/deephaven/util/channel/CachedChannelProviderTest.java @@ -5,8 +5,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.net.URI; @@ -17,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(); @@ -39,13 +38,13 @@ public void testSimpleRead() throws IOException { 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); - Assert.assertEquals(buffer.remaining(), buffer.capacity()); + 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)); } } @@ -59,8 +58,8 @@ public void testSimpleReadWrite() throws IOException { : 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 @@ -74,9 +73,9 @@ public void testSimpleWrite() throws IOException { 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))); } } @@ -88,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))); } } @@ -107,10 +106,10 @@ public void testCloseOrder() throws IOException { 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))); } } } @@ -131,7 +130,7 @@ public void testReuse() throws IOException { } for (int step = 0; step < 10; ++step) { for (int ci = 0; ci < someResult.length; ++ci) { - Assert.assertSame(someResult[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); @@ -140,7 +139,7 @@ public void testReuse() throws IOException { someResult[someResult.length - ci - 1].close(); } } - Assert.assertEquals(closed.size(), 0); + assertEquals(0, closed.size()); } @Test @@ -160,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()); } @@ -232,13 +231,13 @@ private TestMockChannel(int id, String path) { @Override public int read(ByteBuffer dst) { - Assert.assertTrue(channelContext instanceof TestChannelProvider.TestChannelContext); + assertTrue(channelContext instanceof TestChannelProvider.TestChannelContext); return 0; } @Override public int write(ByteBuffer src) { - Assert.assertNull(channelContext); + assertNull(channelContext); return 0; } 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/s3/build.gradle b/extensions/s3/build.gradle index 6ce57a1cd75..b817b0c98c2 100644 --- a/extensions/s3/build.gradle +++ b/extensions/s3/build.gradle @@ -6,6 +6,8 @@ plugins { 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') diff --git a/settings.gradle b/settings.gradle index 0188fca02ff..f619d3e5923 100644 --- a/settings.gradle +++ b/settings.gradle @@ -320,6 +320,9 @@ include(':application-mode') include(':util-immutables') project(':util-immutables').projectDir = file('Util/util-immutables') +include(':util-channel') +project(':util-channel').projectDir = file('Util/channel') + include(':deephaven-jpy-config') project(':deephaven-jpy-config').projectDir = file('py/jpy-config') From e28b993feabeaaf3f594070b4826a4e9b51f4e4d Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Tue, 23 Jan 2024 22:45:57 +0530 Subject: [PATCH 33/39] Minor fix --- extensions/trackedfile/build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/extensions/trackedfile/build.gradle b/extensions/trackedfile/build.gradle index 5bb1334b0a8..e5df2b81c16 100644 --- a/extensions/trackedfile/build.gradle +++ b/extensions/trackedfile/build.gradle @@ -6,6 +6,8 @@ plugins { 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') From 3b3dbbda8913d13551eb7bbbf3e6d70f27aaefc8 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 25 Jan 2024 23:13:12 +0530 Subject: [PATCH 34/39] Resolved more comments --- extensions/s3/build.gradle | 1 + .../deephaven/extensions/s3/BufferPool.java | 5 +- .../ByteBufferAsyncResponseTransformer.java | 19 ++- .../extensions/s3/S3Instructions.java | 26 ++-- .../extensions/s3/S3SeekableByteChannel.java | 114 ++++++++++-------- .../s3/S3SeekableChannelProvider.java | 10 +- py/server/deephaven/experimental/s3.py | 50 ++++---- py/server/deephaven/parquet.py | 9 +- py/server/tests/test_parquet.py | 18 +++ 9 files changed, 155 insertions(+), 97 deletions(-) diff --git a/extensions/s3/build.gradle b/extensions/s3/build.gradle index b817b0c98c2..45b356bb34c 100644 --- a/extensions/s3/build.gradle +++ b/extensions/s3/build.gradle @@ -10,6 +10,7 @@ dependencies { implementation project(':Base') implementation project(':Util') + implementation project(':Configuration') implementation platform('software.amazon.awssdk:bom:2.21.43') implementation 'software.amazon.awssdk:s3' diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java index 85afed69b7d..8bcc56995ee 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BufferPool.java @@ -1,12 +1,13 @@ package io.deephaven.extensions.s3; +import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.Nullable; import java.nio.ByteBuffer; -public interface BufferPool { +interface BufferPool { - interface BufferHolder extends AutoCloseable { + interface BufferHolder extends SafeCloseable { /** * @return The buffer if available, else {@code null} 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 index a347154f70c..6aaff7f6ea2 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/ByteBufferAsyncResponseTransformer.java @@ -4,6 +4,7 @@ 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; @@ -13,21 +14,19 @@ import java.nio.ByteBuffer; import java.util.concurrent.CompletableFuture; -final class ByteBufferAsyncResponseTransformer implements AsyncResponseTransformer { +final class ByteBufferAsyncResponseTransformer + implements AsyncResponseTransformer, SafeCloseable { - private final BufferPool.BufferHolder bufferHolder; private final ByteBuffer byteBuffer; private volatile boolean released; private volatile CompletableFuture currentFuture; /** - * @param bufferHolder A {@link BufferPool.BufferHolder} that will provide a buffer to store the response bytes. - * This will be {@link BufferPool.BufferHolder#close}d when {@link #release()} is called. + * @param byteBuffer A {@link ByteBuffer} to store the response bytes. */ - ByteBufferAsyncResponseTransformer(@NotNull final BufferPool.BufferHolder bufferHolder) { - this.bufferHolder = bufferHolder; - this.byteBuffer = bufferHolder.get(); + ByteBufferAsyncResponseTransformer(@NotNull final ByteBuffer byteBuffer) { + this.byteBuffer = byteBuffer; } @Override @@ -54,9 +53,9 @@ public void exceptionOccurred(final Throwable throwable) { * Prevent further mutation of the underlying buffer by this ByteBufferAsyncResponseTransformer and any of its * Subscribers. */ - public synchronized void release() { + @Override + public synchronized void close() { released = true; - bufferHolder.close(); } private final class ByteBufferSubscriber implements Subscriber { @@ -64,7 +63,7 @@ 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. + * buffer's position, limit, or mark. */ private final ByteBuffer duplicate; diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index b701c9595e4..a0f03f9e103 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -4,6 +4,7 @@ package io.deephaven.extensions.s3; import io.deephaven.annotations.BuildableStyle; +import io.deephaven.configuration.Configuration; import org.immutables.value.Value; import java.time.Duration; @@ -17,7 +18,12 @@ 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 int DEFAULT_FRAGMENT_SIZE = 512 << 20; // 5 MB + + 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); @@ -41,8 +47,8 @@ public int maxConcurrentRequests() { } /** - * The number of fragments to send asynchronous read requests for while reading the current fragment, defaults to - * {@value #DEFAULT_READ_AHEAD_COUNT}. This means by default, we will fetch {@value #DEFAULT_READ_AHEAD_COUNT} + * 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 @@ -51,8 +57,9 @@ public int readAheadCount() { } /** - * The maximum size of each fragment to read from S3, defaults to {@value #DEFAULT_FRAGMENT_SIZE} bytes. If there - * are fewer bytes remaining in the file, the fetched fragment can be smaller. + * 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() { @@ -61,7 +68,7 @@ public int fragmentSize() { /** * The maximum number of fragments to cache in memory, defaults to {@value #DEFAULT_MAX_CACHE_SIZE}. This caching is - * done at deephaven layer for faster access of recently read fragments. + * done at the deephaven layer for faster access to recently read fragments. */ @Value.Default public int maxCacheSize() { @@ -103,7 +110,12 @@ final void boundsCheckReadAheadCount() { @Value.Check final void boundsCheckMaxFragmentSize() { if (fragmentSize() < MIN_FRAGMENT_SIZE) { - throw new IllegalArgumentException("fragmentSize(=" + fragmentSize() + ") must be >= 8*1024 or 8 KB"); + 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"); } } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java index 8df0bba4e05..adf17bd5f50 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -5,7 +5,7 @@ import io.deephaven.base.verify.Assert; -import java.util.concurrent.CancellationException; +import java.util.Objects; import io.deephaven.util.channel.CachedChannelProvider; import io.deephaven.util.channel.SeekableChannelContext; @@ -14,6 +14,9 @@ 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; @@ -22,8 +25,10 @@ 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; @@ -38,6 +43,8 @@ final class S3SeekableByteChannel implements SeekableByteChannel, CachedChannelP 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. */ @@ -49,27 +56,40 @@ static final class S3ChannelContext implements SeekableChannelContext { private static final class FragmentState { /** - * The index of the fragment in the object + * The index of the fragment in the object. */ - private int fragmentIndex; + private long fragmentIndex = UNINITIALIZED_FRAGMENT_INDEX; /** - * The future that will be completed with the fragment's bytes + * The future that will be completed with the fragment's bytes. */ - @NotNull - private CompletableFuture future; + private Future future; /** - * The {@link ByteBufferAsyncResponseTransformer} that will be used to complete the future + * The {@link SafeCloseable} that will be used to release outstanding resources post-cancellation. */ - @NotNull - private ByteBufferAsyncResponseTransformer asyncResponseTransformer; + private SafeCloseable bufferRelease; + + private boolean matches(final int fragmentIndex) { + return this.fragmentIndex == fragmentIndex; + } - private FragmentState(final int fragmentIndex, @NotNull final CompletableFuture future, - @NotNull final ByteBufferAsyncResponseTransformer asyncResponseTransformer) { + 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.asyncResponseTransformer = asyncResponseTransformer; + this.bufferRelease = bufferRelease; } } @@ -93,25 +113,13 @@ private int getIndex(final int fragmentIndex) { return fragmentIndex % bufferCache.length; } - void setFragmentState(final int fragmentIndex, @NotNull final CompletableFuture future, - @NotNull final ByteBufferAsyncResponseTransformer asyncResponseTransformer) { + private FragmentState getFragmentState(final int fragmentIndex) { final int cacheIdx = getIndex(fragmentIndex); - final FragmentState cachedEntry = bufferCache[cacheIdx]; + FragmentState cachedEntry = bufferCache[cacheIdx]; if (cachedEntry == null) { - bufferCache[cacheIdx] = new FragmentState(fragmentIndex, future, asyncResponseTransformer); - } else { - // We should not cache an already cached fragment - Assert.neq(cachedEntry.fragmentIndex, "cachedEntry.fragmentIndex", fragmentIndex, "fragmentIdx"); - - // Cancel any outstanding requests for this cached fragment, and release the buffer for reuse - cachedEntry.future.cancel(true); - cachedEntry.asyncResponseTransformer.release(); - - // Reuse the existing entry - cachedEntry.fragmentIndex = fragmentIndex; - cachedEntry.future = future; - cachedEntry.asyncResponseTransformer = asyncResponseTransformer; + bufferCache[cacheIdx] = cachedEntry = new FragmentState(); } + return cachedEntry; } /** @@ -119,19 +127,19 @@ void setFragmentState(final int fragmentIndex, @NotNull final CompletableFuture< * else will return {@code null} */ @Nullable - CompletableFuture getCachedFuture(final int fragmentIndex) { + private Future getCachedFuture(final int fragmentIndex) { final FragmentState cachedFragment = bufferCache[getIndex(fragmentIndex)]; - if (cachedFragment != null && cachedFragment.fragmentIndex == fragmentIndex) { + if (cachedFragment != null && cachedFragment.matches(fragmentIndex)) { return cachedFragment.future; } return null; } - long getSize() { + private long getSize() { return size; } - void setSize(final long size) { + private void setSize(final long size) { this.size = size; } @@ -139,9 +147,8 @@ void setSize(final long size) { public void close() { // Cancel all outstanding requests for (final FragmentState fragmentState : bufferCache) { - if (fragmentState != null && fragmentState.future != null) { - fragmentState.future.cancel(true); - fragmentState.asyncResponseTransformer.release(); + if (fragmentState != null) { + fragmentState.cancelAndRelease(); } } } @@ -219,8 +226,7 @@ public int read(@NotNull final ByteBuffer destination) throws IOException { } // Wait till the current fragment is fetched - final CompletableFuture currFragmentFuture = - s3ChannelContext.getCachedFuture(currFragmentIndex); + final Future currFragmentFuture = s3ChannelContext.getCachedFuture(currFragmentIndex); final ByteBuffer currentFragment; try { currentFragment = currFragmentFuture.get(s3Instructions.readTimeout().toNanos(), TimeUnit.NANOSECONDS); @@ -271,24 +277,32 @@ private int fragmentIndexForByteNumber(final long byteNumber) { * 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 int fragmentIndex, - @NotNull final S3ChannelContext s3ChannelContext) { - if (s3ChannelContext.getCachedFuture(fragmentIndex) != null) { - // We have a pending request for this fragment + private void sendAsyncRequest(final int 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 = (long) 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 ByteBufferAsyncResponseTransformer asyncResponseTransformer = - new ByteBufferAsyncResponseTransformer(bufferPool.take(numBytes)); - final CompletableFuture future = s3AsyncClient.getObject( - builder -> builder.bucket(bucket).key(key).range(range), - asyncResponseTransformer); - s3ChannelContext.setFragmentState(fragmentIndex, future, asyncResponseTransformer); + 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) { @@ -355,7 +369,11 @@ private void populateSize(final S3ChannelContext s3ChannelContext) throws IOExce // for future use final HeadObjectResponse headObjectResponse; try { - headObjectResponse = s3AsyncClient.headObject(builder -> builder.bucket(bucket).key(key)) + 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)); diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index 17e2e0d0914..b21c14bb258 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -15,16 +15,23 @@ 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 { + private static final BufferPool bufferPool = new SegmentedBufferPool(MAX_FRAGMENT_SIZE); + private final S3AsyncClient s3AsyncClient; private final S3Instructions s3Instructions; - private final BufferPool bufferPool; S3SeekableChannelProvider(@NotNull final S3Instructions s3Instructions) { + if (s3Instructions.fragmentSize() > MAX_FRAGMENT_SIZE) { + throw new IllegalArgumentException("Fragment size " + s3Instructions.fragmentSize() + " is larger than " + + " maximum allowed " + MAX_FRAGMENT_SIZE); + } final SdkAsyncHttpClient asyncHttpClient = AwsCrtAsyncHttpClient.builder() .maxConcurrency(s3Instructions.maxConcurrentRequests()) .connectionTimeout(s3Instructions.connectionTimeout()) @@ -36,7 +43,6 @@ final class S3SeekableChannelProvider implements SeekableChannelsProvider { .httpClient(asyncHttpClient) .build(); this.s3Instructions = s3Instructions; - this.bufferPool = new SegmentedBufferPool(s3Instructions.fragmentSize()); } @Override diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index 29095cb071e..e33f77db1de 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -2,8 +2,8 @@ # Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending # """ - This module is useful for reading and writing files stored in S3. Importing this module requires the S3 specific - extensions to be included in the class path. + This module is useful for reading files stored in S3. Importing this module requires the S3 specific extensions to + be included in the class path. """ import datetime from typing import Optional, Union @@ -12,25 +12,25 @@ import numpy as np import pandas as pd -from deephaven import time +from deephaven import time, DHError from deephaven._wrapper import JObjectWrapper from deephaven.dtypes import Duration class S3Instructions(JObjectWrapper): """ - Used to provide specialized instructions for reading from AWS S3. + S3Instructions provides specialized instructions for reading from AWS S3. 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, - by default is 50. + 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, defaults to 1, which means fetch one next fragment in advance when reading current fragment. + fragment. Defaults to 1, which means fetch one 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 deephaven layer for faster access of recently read fragments. + 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 @@ -41,6 +41,7 @@ class S3Instructions(JObjectWrapper): types. Default to 2 seconds. """ + # TODO(deephaven-core#5064): Add support for local S3 testing j_object_type = jpy.get_type("io.deephaven.extensions.s3.S3Instructions") def __init__(self, @@ -54,28 +55,31 @@ def __init__(self, read_timeout: Union[ Duration, int, str, datetime.timedelta, np.timedelta64, pd.Timedelta, None] = None): - builder = self.j_object_type.builder() - builder.awsRegionName(aws_region_name) + 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 max_concurrent_requests is not None: + builder.maxConcurrentRequests(max_concurrent_requests) - if read_ahead_count is not None: - builder.readAheadCount(read_ahead_count) + if read_ahead_count is not None: + builder.readAheadCount(read_ahead_count) - if fragment_size is not None: - builder.fragmentSize(fragment_size) + if fragment_size is not None: + builder.fragmentSize(fragment_size) - if max_cache_size is not None: - builder.maxCacheSize(max_cache_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 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 read_timeout is not None: + builder.readTimeout(time.to_j_duration(read_timeout)) - self._j_object = builder.build() + 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: diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index b5de555e3cc..053da2b1c76 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -11,17 +11,16 @@ import jpy from deephaven import DHError -from deephaven._wrapper import JObjectWrapper 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") _JCompressionCodecName = jpy.get_type("org.apache.parquet.hadoop.metadata.CompressionCodecName") _JParquetInstructions = jpy.get_type("io.deephaven.parquet.table.ParquetInstructions") _JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") -_JDuration = jpy.get_type("java.time.Duration") @dataclass @@ -44,7 +43,7 @@ def _build_parquet_instructions( is_refreshing: bool = False, for_read: bool = True, force_build: bool = False, - special_instructions: Optional[JObjectWrapper] = None, + special_instructions: Optional[s3.S3Instructions] = None, ): if not any( [ @@ -139,7 +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[JObjectWrapper] = None, + special_instructions: Optional[s3.S3Instructions] = None, ) -> Table: """ Reads in a table from a single parquet, metadata file, or directory with recognized layout. @@ -156,7 +155,7 @@ def read( 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[JObjectWrapper]): Special instructions for reading parquet files, useful when + 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: diff --git a/py/server/tests/test_parquet.py b/py/server/tests/test_parquet.py index 56cce45957a..4f88f2cb05c 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,23 @@ 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, + ) + + 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() From cf2b130459461e25df068ceecfbb48772b74d91f Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 26 Jan 2024 19:24:33 +0530 Subject: [PATCH 35/39] Added support to pass AWS credentials --- .../table/ParquetTableReadWriteTest.java | 2 ++ .../extensions/s3/AwsCredentials.java | 15 ++++++++ .../extensions/s3/AwsCredentialsImpl.java | 11 ++++++ .../extensions/s3/DefaultCredentials.java | 20 +++++++++++ .../extensions/s3/S3Instructions.java | 9 +++++ .../extensions/s3/S3SeekableByteChannel.java | 28 +++++++-------- .../s3/S3SeekableChannelProvider.java | 12 ++++--- .../extensions/s3/StaticCredentials.java | 35 +++++++++++++++++++ py/server/deephaven/experimental/s3.py | 34 +++++++++++++----- py/server/tests/test_parquet.py | 6 ++++ 10 files changed, 145 insertions(+), 27 deletions(-) create mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentials.java create mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentialsImpl.java create mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/DefaultCredentials.java create mode 100644 extensions/s3/src/main/java/io/deephaven/extensions/s3/StaticCredentials.java 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 245dc88ac0c..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 @@ -27,6 +27,7 @@ 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; @@ -588,6 +589,7 @@ public void readSampleParquetFilesFromS3Test1() { .maxConcurrentRequests(50) .maxCacheSize(32) .readTimeout(Duration.ofSeconds(60)) + .credentials(AwsCredentials.defaultCredentials()) .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3Instructions) 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..6e526d86a4d --- /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 staticCredentials(final String awsAccessKeyId, final String awsSecretAccessKey) { + return StaticCredentials.of(awsAccessKeyId, awsSecretAccessKey); + } +} diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentialsImpl.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentialsImpl.java new file mode 100644 index 00000000000..0ba0ed6f91e --- /dev/null +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentialsImpl.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 AwsCredentialsImpl extends AwsCredentials { + + AwsCredentialsProvider awsCredentialsProvider(); +} 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..327a46ab68a --- /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 AwsCredentialsImpl { + 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 index a0f03f9e103..5921759d8ea 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -92,6 +92,13 @@ public Duration readTimeout() { return DEFAULT_READ_TIMEOUT; } + /** + * The credentials to use when reading or writing to S3. + */ + @Value.Default + public AwsCredentials credentials() { + return AwsCredentials.defaultCredentials(); + } @Value.Check final void boundsCheckMaxConcurrentRequests() { @@ -142,6 +149,8 @@ public interface Builder { 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 index adf17bd5f50..13cb4b457a3 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableByteChannel.java @@ -70,7 +70,7 @@ private static final class FragmentState { */ private SafeCloseable bufferRelease; - private boolean matches(final int fragmentIndex) { + private boolean matches(final long fragmentIndex) { return this.fragmentIndex == fragmentIndex; } @@ -108,12 +108,12 @@ private void set( size = UNINITIALIZED_SIZE; } - private int getIndex(final int fragmentIndex) { + private int getIndex(final long fragmentIndex) { // TODO(deephaven-core#5061): Experiment with LRU caching - return fragmentIndex % bufferCache.length; + return (int) (fragmentIndex % bufferCache.length); } - private FragmentState getFragmentState(final int fragmentIndex) { + private FragmentState getFragmentState(final long fragmentIndex) { final int cacheIdx = getIndex(fragmentIndex); FragmentState cachedEntry = bufferCache[cacheIdx]; if (cachedEntry == null) { @@ -127,7 +127,7 @@ private FragmentState getFragmentState(final int fragmentIndex) { * else will return {@code null} */ @Nullable - private Future getCachedFuture(final int fragmentIndex) { + private Future getCachedFuture(final long fragmentIndex) { final FragmentState cachedFragment = bufferCache[getIndex(fragmentIndex)]; if (cachedFragment != null && cachedFragment.matches(fragmentIndex)) { return cachedFragment.future; @@ -164,7 +164,7 @@ public void close() { * The size of the object in bytes, fetched at the time of first read */ private long size; - private int numFragmentsInObject; + private long numFragmentsInObject; /** * The {@link SeekableChannelContext} object used to cache read-ahead buffers for efficiently reading from S3. This @@ -217,11 +217,11 @@ public int read(@NotNull final ByteBuffer destination) throws IOException { } // Send async read requests for current fragment as well as read ahead fragments - final int currFragmentIndex = fragmentIndexForByteNumber(localPosition); + 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 - : Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); - for (int idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { + : (int) Math.min(s3Instructions.readAheadCount(), numFragmentsInObject - currFragmentIndex - 1); + for (long idx = currFragmentIndex; idx <= currFragmentIndex + numReadAheadFragments; idx++) { sendAsyncRequest(idx, s3ChannelContext); } @@ -269,15 +269,15 @@ private static S3ChannelContext getS3ChannelContextFrom(@NotNull final SeekableC return s3ChannelContext; } - private int fragmentIndexForByteNumber(final long byteNumber) { - return Math.toIntExact(byteNumber / s3Instructions.fragmentSize()); + 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 int fragmentIndex, @NotNull final S3ChannelContext s3ChannelContext) { + 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 @@ -287,7 +287,7 @@ private void sendAsyncRequest(final int fragmentIndex, @NotNull final S3ChannelC fragmentState.cancelAndRelease(); final int fragmentSize = s3Instructions.fragmentSize(); - final long readFrom = (long) fragmentIndex * fragmentSize; + final long readFrom = fragmentIndex * fragmentSize; final long readTo = Math.min(readFrom + fragmentSize, size) - 1; final String range = "bytes=" + readFrom + "-" + readTo; @@ -382,7 +382,7 @@ private void populateSize(final S3ChannelContext s3ChannelContext) throws IOExce } this.size = s3ChannelContext.getSize(); final int fragmentSize = s3Instructions.fragmentSize(); - this.numFragmentsInObject = (int) ((size + fragmentSize - 1) / fragmentSize); // = ceil(size / fragmentSize) + this.numFragmentsInObject = (size + fragmentSize - 1) / fragmentSize; // = ceil(size / fragmentSize) } @Override diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index b21c14bb258..711e3e1445c 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -3,6 +3,7 @@ */ 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; @@ -22,25 +23,26 @@ */ final class S3SeekableChannelProvider implements SeekableChannelsProvider { - private static final BufferPool bufferPool = new SegmentedBufferPool(MAX_FRAGMENT_SIZE); + // 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) { - if (s3Instructions.fragmentSize() > MAX_FRAGMENT_SIZE) { - throw new IllegalArgumentException("Fragment size " + s3Instructions.fragmentSize() + " is larger than " + - " maximum allowed " + MAX_FRAGMENT_SIZE); - } 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", AwsCredentialsImpl.class); this.s3AsyncClient = S3AsyncClient.builder() .region(Region.of(s3Instructions.awsRegionName())) .httpClient(asyncHttpClient) + .credentialsProvider(((AwsCredentialsImpl) s3Instructions.credentials()).awsCredentialsProvider()) .build(); this.s3Instructions = s3Instructions; } diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/StaticCredentials.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/StaticCredentials.java new file mode 100644 index 00000000000..5773adaf2b1 --- /dev/null +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/StaticCredentials.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 static access key and secret key. + */ +@Immutable +@SimpleStyle +abstract class StaticCredentials implements AwsCredentialsImpl { + + static StaticCredentials of(final String awsAccessKeyId, final String awsSecretAccessKey) { + return ImmutableStaticCredentials.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/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index e33f77db1de..4a20b04ef44 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -1,10 +1,6 @@ # # Copyright (c) 2016-2023 Deephaven Data Labs and Patent Pending # -""" - This module is useful for reading files stored in S3. Importing this module requires the S3 specific extensions to - be included in the class path. -""" import datetime from typing import Optional, Union @@ -16,7 +12,15 @@ 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. @@ -26,7 +30,7 @@ class S3Instructions(JObjectWrapper): 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. Defaults to 1, which means fetch one next fragment in advance when reading the current fragment. + fragment. Defaults 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 @@ -39,10 +43,15 @@ class S3Instructions(JObjectWrapper): 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. """ - # TODO(deephaven-core#5064): Add support for local S3 testing - j_object_type = jpy.get_type("io.deephaven.extensions.s3.S3Instructions") + j_object_type = _JS3Instructions def __init__(self, aws_region_name: str, @@ -53,7 +62,9 @@ def __init__(self, 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): + 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): try: builder = self.j_object_type.builder() @@ -77,6 +88,13 @@ def __init__(self, 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.staticCredentials(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 diff --git a/py/server/tests/test_parquet.py b/py/server/tests/test_parquet.py index 4f88f2cb05c..688f0e237a9 100644 --- a/py/server/tests/test_parquet.py +++ b/py/server/tests/test_parquet.py @@ -562,6 +562,12 @@ def test_read_parquet_from_s3(self): 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, ) From 661424a1bc289bbc00f17d194db569bdc87e67e2 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 26 Jan 2024 22:42:18 +0530 Subject: [PATCH 36/39] Improved comments --- .../deephaven/extensions/s3/S3SeekableChannelProvider.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index 711e3e1445c..27864d17155 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -23,8 +23,10 @@ */ 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. + /** + * 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); From 5695f2ac918c53d85d9dcbf83df97fd7353e09fe Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Sat, 27 Jan 2024 00:11:07 +0530 Subject: [PATCH 37/39] Renamed static credentials to basic credentials --- .../java/io/deephaven/extensions/s3/AwsCredentials.java | 4 ++-- .../s3/{StaticCredentials.java => BasicCredentials.java} | 8 ++++---- .../java/io/deephaven/extensions/s3/S3Instructions.java | 3 ++- py/server/deephaven/experimental/s3.py | 2 +- 4 files changed, 9 insertions(+), 8 deletions(-) rename extensions/s3/src/main/java/io/deephaven/extensions/s3/{StaticCredentials.java => BasicCredentials.java} (72%) 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 index 6e526d86a4d..5318ec540af 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentials.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentials.java @@ -9,7 +9,7 @@ static AwsCredentials defaultCredentials() { return DefaultCredentials.DEFAULT_CREDENTIALS; } - static AwsCredentials staticCredentials(final String awsAccessKeyId, final String awsSecretAccessKey) { - return StaticCredentials.of(awsAccessKeyId, awsSecretAccessKey); + 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/StaticCredentials.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BasicCredentials.java similarity index 72% rename from extensions/s3/src/main/java/io/deephaven/extensions/s3/StaticCredentials.java rename to extensions/s3/src/main/java/io/deephaven/extensions/s3/BasicCredentials.java index 5773adaf2b1..c152a9a5c59 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/StaticCredentials.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BasicCredentials.java @@ -11,14 +11,14 @@ import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; /** - * AWS credentials provider that uses static access key and secret key. + * AWS credentials provider that uses access key and secret key provided at construction. */ @Immutable @SimpleStyle -abstract class StaticCredentials implements AwsCredentialsImpl { +abstract class BasicCredentials implements AwsCredentialsImpl { - static StaticCredentials of(final String awsAccessKeyId, final String awsSecretAccessKey) { - return ImmutableStaticCredentials.of(awsAccessKeyId, awsSecretAccessKey); + static BasicCredentials of(final String awsAccessKeyId, final String awsSecretAccessKey) { + return ImmutableBasicCredentials.of(awsAccessKeyId, awsSecretAccessKey); } @Value.Parameter diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index 5921759d8ea..1c2239045d6 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -6,6 +6,7 @@ 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; @@ -93,7 +94,7 @@ public Duration readTimeout() { } /** - * The credentials to use when reading or writing to S3. + * The credentials to use when reading or writing to S3. By default, uses {@link DefaultCredentialsProvider}. */ @Value.Default public AwsCredentials credentials() { diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index 4a20b04ef44..fc872e0f1ca 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -93,7 +93,7 @@ def __init__(self, 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.staticCredentials(aws_access_key_id, aws_secret_access_key)) + builder.credentials(_JAwsCredentials.basicCredentials(aws_access_key_id, aws_secret_access_key)) self._j_object = builder.build() except Exception as e: From 9847b59db3fa2f644342313ff3104f6d9fac1190 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Sat, 27 Jan 2024 00:37:53 +0530 Subject: [PATCH 38/39] Renamed AwsCredentialsImpl to AwsSdkV2Credentials --- .../s3/{AwsCredentialsImpl.java => AwsSdkV2Credentials.java} | 2 +- .../java/io/deephaven/extensions/s3/BasicCredentials.java | 2 +- .../java/io/deephaven/extensions/s3/DefaultCredentials.java | 2 +- .../io/deephaven/extensions/s3/S3SeekableChannelProvider.java | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) rename extensions/s3/src/main/java/io/deephaven/extensions/s3/{AwsCredentialsImpl.java => AwsSdkV2Credentials.java} (81%) diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentialsImpl.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsSdkV2Credentials.java similarity index 81% rename from extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentialsImpl.java rename to extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsSdkV2Credentials.java index 0ba0ed6f91e..04f58eff1df 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsCredentialsImpl.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/AwsSdkV2Credentials.java @@ -5,7 +5,7 @@ import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -interface AwsCredentialsImpl extends AwsCredentials { +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 index c152a9a5c59..3459ef9ea6b 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/BasicCredentials.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/BasicCredentials.java @@ -15,7 +15,7 @@ */ @Immutable @SimpleStyle -abstract class BasicCredentials implements AwsCredentialsImpl { +abstract class BasicCredentials implements AwsSdkV2Credentials { static BasicCredentials of(final String awsAccessKeyId, final String awsSecretAccessKey) { return ImmutableBasicCredentials.of(awsAccessKeyId, awsSecretAccessKey); 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 index 327a46ab68a..5f69f50653f 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/DefaultCredentials.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/DefaultCredentials.java @@ -10,7 +10,7 @@ * Default AWS credentials provider that looks for credentials at a number of locations as described in * {@link DefaultCredentialsProvider} */ -enum DefaultCredentials implements AwsCredentialsImpl { +enum DefaultCredentials implements AwsSdkV2Credentials { DEFAULT_CREDENTIALS; @Override diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java index 27864d17155..5f146619e18 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3SeekableChannelProvider.java @@ -40,11 +40,11 @@ final class S3SeekableChannelProvider implements SeekableChannelsProvider { .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", AwsCredentialsImpl.class); + Assert.instanceOf(s3Instructions.credentials(), "credentials", AwsSdkV2Credentials.class); this.s3AsyncClient = S3AsyncClient.builder() .region(Region.of(s3Instructions.awsRegionName())) .httpClient(asyncHttpClient) - .credentialsProvider(((AwsCredentialsImpl) s3Instructions.credentials()).awsCredentialsProvider()) + .credentialsProvider(((AwsSdkV2Credentials) s3Instructions.credentials()).awsCredentialsProvider()) .build(); this.s3Instructions = s3Instructions; } From 0750e56f50c39fb2bd0466996e4e6e91f7528514 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Mon, 29 Jan 2024 21:30:51 +0530 Subject: [PATCH 39/39] Minor refactoring around python comments --- py/server/deephaven/experimental/s3.py | 57 +++++++++++++++----------- 1 file changed, 32 insertions(+), 25 deletions(-) diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index fc872e0f1ca..c4c34b3d9ab 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -24,31 +24,6 @@ class S3Instructions(JObjectWrapper): """ S3Instructions provides specialized instructions for reading from AWS S3. - - 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. Defaults 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. """ j_object_type = _JS3Instructions @@ -66,6 +41,38 @@ def __init__(self, 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)