diff --git a/Base/src/main/java/io/deephaven/base/FileUtils.java b/Base/src/main/java/io/deephaven/base/FileUtils.java index 8f13f724859..bd6f9e907fd 100644 --- a/Base/src/main/java/io/deephaven/base/FileUtils.java +++ b/Base/src/main/java/io/deephaven/base/FileUtils.java @@ -30,7 +30,13 @@ public boolean accept(File dir, String name) { }; private final static String[] EMPTY_STRING_ARRAY = new String[0]; - public static final Pattern DUPLICATE_SLASH_PATTERN = Pattern.compile("//+"); + public static final char URI_SEPARATOR_CHAR = '/'; + + public static final String URI_SEPARATOR = "" + URI_SEPARATOR_CHAR; + + public static final String REPEATED_URI_SEPARATOR = URI_SEPARATOR + URI_SEPARATOR; + + public static final Pattern REPEATED_URI_SEPARATOR_PATTERN = Pattern.compile("//+"); /** * Cleans the specified path. All files and subdirectories in the path will be deleted. (ie you'll be left with an @@ -258,7 +264,7 @@ public boolean accept(File pathname) { /** * Take the file source path or URI string and convert it to a URI object. Any unnecessary path separators will be - * removed. + * removed. The URI object will always be {@link URI#isAbsolute() absolute}, i.e., will always have a scheme. * * @param source The file source path or URI * @param isDirectory Whether the source is a directory @@ -273,8 +279,8 @@ public static URI convertToURI(final String source, final boolean isDirectory) { uri = new URI(source); // Replace two or more consecutive slashes in the path with a single slash final String path = uri.getPath(); - if (path.contains("//")) { - final String canonicalizedPath = DUPLICATE_SLASH_PATTERN.matcher(path).replaceAll("/"); + if (path.contains(REPEATED_URI_SEPARATOR)) { + final String canonicalizedPath = REPEATED_URI_SEPARATOR_PATTERN.matcher(path).replaceAll(URI_SEPARATOR); uri = new URI(uri.getScheme(), uri.getUserInfo(), uri.getHost(), uri.getPort(), canonicalizedPath, uri.getQuery(), uri.getFragment()); } @@ -300,17 +306,17 @@ public static URI convertToURI(final String source, final boolean isDirectory) { */ public static URI convertToURI(final File file, final boolean isDirectory) { String absPath = file.getAbsolutePath(); - if (File.separatorChar != '/') { - absPath = absPath.replace(File.separatorChar, '/'); + if (File.separatorChar != URI_SEPARATOR_CHAR) { + absPath = absPath.replace(File.separatorChar, URI_SEPARATOR_CHAR); } - if (absPath.charAt(0) != '/') { - absPath = "/" + absPath; + if (absPath.charAt(0) != URI_SEPARATOR_CHAR) { + absPath = URI_SEPARATOR_CHAR + absPath; } - if (isDirectory && absPath.charAt(absPath.length() - 1) != '/') { - absPath = absPath + "/"; + if (isDirectory && absPath.charAt(absPath.length() - 1) != URI_SEPARATOR_CHAR) { + absPath = absPath + URI_SEPARATOR_CHAR; } - if (absPath.startsWith("//")) { - absPath = "//" + absPath; + if (absPath.startsWith(REPEATED_URI_SEPARATOR)) { + absPath = REPEATED_URI_SEPARATOR + absPath; } try { return new URI("file", null, absPath, null); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/DataIndexer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/DataIndexer.java index 09065809080..98c85a7c6cf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/DataIndexer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/DataIndexer.java @@ -92,8 +92,8 @@ private int priorityOf(@NotNull final ColumnSource keyColumn) { @NotNull private static Collection> getColumnSources( @NotNull final Table table, - @NotNull final String... keyColumnNames) { - return Arrays.stream(keyColumnNames) + @NotNull final Collection keyColumnNames) { + return keyColumnNames.stream() .map(table::getColumnSource) .collect(Collectors.toList()); } @@ -107,16 +107,29 @@ private static Collection> getColumnSources( * @param keyColumnNames The key column names to check * @return Whether {@code table} has a DataIndexer with a {@link DataIndex} for the given key columns */ - public static boolean hasDataIndex(@NotNull Table table, @NotNull final String... keyColumnNames) { - if (keyColumnNames.length == 0) { + public static boolean hasDataIndex(@NotNull final Table table, @NotNull final String... keyColumnNames) { + return hasDataIndex(table, Arrays.asList(keyColumnNames)); + } + + /** + * Test whether {@code table} has a DataIndexer with a usable {@link DataIndex} for the given key columns. Note that + * a result from this method is a snapshot of current state, and does not guarantee anything about future calls to + * {@link #hasDataIndex}, {@link #getDataIndex}, or {@link #getOrCreateDataIndex(Table, String...)}. + * + * @param table The {@link Table} to check + * @param keyColumnNames The key column names to check + * @return Whether {@code table} has a DataIndexer with a {@link DataIndex} for the given key columns + */ + public static boolean hasDataIndex(@NotNull final Table table, @NotNull final Collection keyColumnNames) { + if (keyColumnNames.isEmpty()) { return false; } - table = table.coalesce(); - final DataIndexer indexer = DataIndexer.existingOf(table.getRowSet()); + final Table tableToUse = table.coalesce(); + final DataIndexer indexer = DataIndexer.existingOf(tableToUse.getRowSet()); if (indexer == null) { return false; } - return indexer.hasDataIndex(getColumnSources(table, keyColumnNames)); + return indexer.hasDataIndex(getColumnSources(tableToUse, keyColumnNames)); } /** @@ -152,19 +165,34 @@ public boolean hasDataIndex(@NotNull final Collection> keyColumn * index is no longer live. * * @param table The {@link Table} to check - * @param keyColumnNames The key column for which to retrieve a DataIndex + * @param keyColumnNames The key columns for which to retrieve a DataIndex * @return The {@link DataIndex}, or {@code null} if one does not exist */ - public static DataIndex getDataIndex(@NotNull Table table, final String... keyColumnNames) { - if (keyColumnNames.length == 0) { + @Nullable + public static DataIndex getDataIndex(@NotNull final Table table, final String... keyColumnNames) { + return getDataIndex(table, Arrays.asList(keyColumnNames)); + } + + /** + * If {@code table} has a DataIndexer, return a {@link DataIndex} for the given key columns, or {@code null} if no + * such index exists, if the cached index is invalid, or if the {@link DataIndex#isRefreshing() refreshing} cached + * index is no longer live. + * + * @param table The {@link Table} to check + * @param keyColumnNames The key columns for which to retrieve a DataIndex + * @return The {@link DataIndex}, or {@code null} if one does not exist + */ + @Nullable + public static DataIndex getDataIndex(@NotNull final Table table, final Collection keyColumnNames) { + if (keyColumnNames.isEmpty()) { return null; } - table = table.coalesce(); - final DataIndexer indexer = DataIndexer.existingOf(table.getRowSet()); + final Table tableToUse = table.coalesce(); + final DataIndexer indexer = DataIndexer.existingOf(tableToUse.getRowSet()); if (indexer == null) { return null; } - return indexer.getDataIndex(getColumnSources(table, keyColumnNames)); + return indexer.getDataIndex(getColumnSources(tableToUse, keyColumnNames)); } /** @@ -239,13 +267,28 @@ public static DataIndex getOptimalPartialIndex(Table table, final String... keyC public static DataIndex getOrCreateDataIndex( @NotNull final Table table, @NotNull final String... keyColumnNames) { - if (keyColumnNames.length == 0) { + return getOrCreateDataIndex(table, Arrays.asList(keyColumnNames)); + } + + /** + * Create a {@link DataIndex} for {@code table} indexing {@code keyColumns}, if no valid, live data index already + * exists for these inputs. + * + * @param table The {@link Table} to index + * @param keyColumnNames The key column names to include + * @return The existing or newly created {@link DataIndex} + * @apiNote This method causes the returned {@link DataIndex} to be managed by the enclosing liveness manager. + */ + public static DataIndex getOrCreateDataIndex( + @NotNull final Table table, + @NotNull final Collection keyColumnNames) { + if (keyColumnNames.isEmpty()) { throw new IllegalArgumentException("Cannot create a DataIndex without any key columns"); } final QueryTable tableToUse = (QueryTable) table.coalesce(); final DataIndexer dataIndexer = DataIndexer.of(tableToUse.getRowSet()); return dataIndexer.rootCache.computeIfAbsent(dataIndexer.pathFor(getColumnSources(tableToUse, keyColumnNames)), - () -> new TableBackedDataIndex(tableToUse, keyColumnNames)); + () -> new TableBackedDataIndex(tableToUse, keyColumnNames.toArray(String[]::new))); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URIStreamKeyValuePartitionLayout.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URIStreamKeyValuePartitionLayout.java index d70055dcd04..2defb0a762b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URIStreamKeyValuePartitionLayout.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/local/URIStreamKeyValuePartitionLayout.java @@ -25,14 +25,14 @@ import java.util.function.Supplier; import java.util.stream.Stream; +import static io.deephaven.base.FileUtils.URI_SEPARATOR; + /** * Extracts a key-value partitioned table layout from a stream of URIs. */ public abstract class URIStreamKeyValuePartitionLayout extends KeyValuePartitionLayout { - private static final String URI_SEPARATOR = "/"; - protected final URI tableRootDirectory; private final Supplier locationTableBuilderFactory; private final int maxPartitioningLevels; @@ -96,7 +96,7 @@ private void getPartitions(@NotNull final URI relativePath, @NotNull final TIntObjectMap partitionColInfo, final boolean registered) { final String relativePathString = relativePath.getPath(); - // The following assumes that there is exactly one URI_SEPARATOR between each subdirectory in the path + // The following assumes that there is exactly one separator between each subdirectory in the path final String[] subDirs = relativePathString.split(URI_SEPARATOR); final int numPartitioningCol = subDirs.length - 1; if (registered) { 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 e3fd4cb145f..dae54bc536d 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 @@ -4,7 +4,6 @@ package io.deephaven.engine.table.impl.sources.regioned; import io.deephaven.base.FileUtils; -import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.*; import io.deephaven.stringset.ArrayStringSet; @@ -226,36 +225,34 @@ public void setUp() throws Exception { final String tableName = "TestTable"; final PartitionedTable partitionedInputData = inputData.partitionBy("PC"); - final File[] partitionedInputDestinations; + final String[] partitionedInputDestinations; try (final Stream partitionNames = partitionedInputData.table() .objectColumnIterator("PC").stream()) { partitionedInputDestinations = partitionNames.map(pcv -> new File(dataDirectory, "IP" + File.separator + "P" + pcv + File.separator + tableName + File.separator - + PARQUET_FILE_NAME)) - .toArray(File[]::new); + + PARQUET_FILE_NAME) + .getPath()) + .toArray(String[]::new); } - ParquetTools.writeParquetTables( + ParquetTools.writeTables( partitionedInputData.constituents(), - partitionedDataDefinition.getWritable(), - parquetInstructions, partitionedInputDestinations, - CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY); + parquetInstructions.withTableDefinition(partitionedDataDefinition.getWritable())); final PartitionedTable partitionedInputMissingData = inputMissingData.view("PC", "II").partitionBy("PC"); - final File[] partitionedInputMissingDestinations; + final String[] partitionedInputMissingDestinations; try (final Stream partitionNames = partitionedInputMissingData.table() .objectColumnIterator("PC").stream()) { partitionedInputMissingDestinations = partitionNames.map(pcv -> new File(dataDirectory, "IP" + File.separator + "P" + pcv + File.separator + tableName + File.separator - + PARQUET_FILE_NAME)) - .toArray(File[]::new); + + PARQUET_FILE_NAME) + .getPath()) + .toArray(String[]::new); } - ParquetTools.writeParquetTables( + ParquetTools.writeTables( partitionedInputMissingData.constituents(), - partitionedMissingDataDefinition.getWritable(), - parquetInstructions, partitionedInputMissingDestinations, - CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY); + parquetInstructions.withTableDefinition(partitionedMissingDataDefinition.getWritable())); expected = TableTools .merge( 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 90943e4d34f..3b423e9cff4 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,12 +3,17 @@ // package io.deephaven.parquet.base; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.util.channel.CachedChannelProvider; import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProviderLoader; import org.apache.parquet.format.*; import org.apache.parquet.format.ColumnOrder; import org.apache.parquet.format.Type; import org.apache.parquet.schema.*; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.File; import java.io.IOException; @@ -37,6 +42,76 @@ public class ParquetFileReader { private final URI rootURI; private final MessageType type; + /** + * Make a {@link ParquetFileReader} for the supplied {@link File}. Wraps {@link IOException} as + * {@link UncheckedDeephavenException}. + * + * @param parquetFile The parquet file or the parquet metadata file + * @param specialInstructions Optional read instructions to pass to {@link SeekableChannelsProvider} while creating + * channels + * @return The new {@link ParquetFileReader} + */ + public static ParquetFileReader create( + @NotNull final File parquetFile, + @Nullable final Object specialInstructions) { + try { + return createChecked(parquetFile, specialInstructions); + } catch (IOException e) { + throw new UncheckedDeephavenException("Failed to create Parquet file reader: " + parquetFile, e); + } + } + + /** + * Make a {@link ParquetFileReader} for the supplied {@link URI}. Wraps {@link IOException} as + * {@link UncheckedDeephavenException}. + * + * @param parquetFileURI The URI for the parquet file or the parquet metadata file + * @param specialInstructions Optional read instructions to pass to {@link SeekableChannelsProvider} while creating + * channels + * @return The new {@link ParquetFileReader} + */ + public static ParquetFileReader create( + @NotNull final URI parquetFileURI, + @Nullable final Object specialInstructions) { + try { + return createChecked(parquetFileURI, specialInstructions); + } catch (IOException e) { + throw new UncheckedDeephavenException("Failed to create Parquet file reader: " + parquetFileURI, e); + } + } + + /** + * Make a {@link ParquetFileReader} for the supplied {@link File}. + * + * @param parquetFile The parquet file or the parquet metadata file + * @param specialInstructions Optional read instructions to pass to {@link SeekableChannelsProvider} while creating + * channels + * @return The new {@link ParquetFileReader} + * @throws IOException if an IO exception occurs + */ + public static ParquetFileReader createChecked( + @NotNull final File parquetFile, + @Nullable final Object specialInstructions) throws IOException { + return createChecked(convertToURI(parquetFile, false), specialInstructions); + } + + /** + * Make a {@link ParquetFileReader} for the supplied {@link URI}. + * + * @param parquetFileURI The URI for the parquet file or the parquet metadata file + * @param specialInstructions Optional read instructions to pass to {@link SeekableChannelsProvider} while creating + * channels + * @return The new {@link ParquetFileReader} + * @throws IOException if an IO exception occurs + */ + public static ParquetFileReader createChecked( + @NotNull final URI parquetFileURI, + @Nullable final Object specialInstructions) throws IOException { + final SeekableChannelsProvider provider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader( + parquetFileURI, specialInstructions); + return new ParquetFileReader(parquetFileURI, new CachedChannelProvider(provider, 1 << 7)); + } + /** * Create a new ParquetFileReader for the provided source. * diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetUtils.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetUtils.java index 12f7f8e9ce1..5af9e80e98b 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetUtils.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetUtils.java @@ -9,11 +9,17 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Path; +import static io.deephaven.base.FileUtils.URI_SEPARATOR_CHAR; + public final class ParquetUtils { public static final String METADATA_FILE_NAME = "_metadata"; public static final String COMMON_METADATA_FILE_NAME = "_common_metadata"; public static final String PARQUET_FILE_EXTENSION = ".parquet"; + public static final String METADATA_FILE_URI_SUFFIX = URI_SEPARATOR_CHAR + METADATA_FILE_NAME; + public static final String COMMON_METADATA_FILE_URI_SUFFIX = URI_SEPARATOR_CHAR + COMMON_METADATA_FILE_NAME; + public static final String METADATA_FILE_SUFFIX = File.separatorChar + METADATA_FILE_NAME; + public static final String COMMON_METADATA_FILE_SUFFIX = File.separatorChar + COMMON_METADATA_FILE_NAME; private static final String MAGIC_STR = "PAR1"; public static final byte[] MAGIC = MAGIC_STR.getBytes(StandardCharsets.US_ASCII); @@ -35,6 +41,20 @@ public static String getPerFileMetadataKey(final String filePath) { return "deephaven_per_file_" + filePath.replace(File.separatorChar, '_'); } + /** + * This method verifies if the source points to a parquet file or a metadata file. Provided source can be a local + * file path or a URI. Also, it can point to a parquet file, metadata file or a directory. + */ + public static boolean isParquetFile(@NotNull final String source) { + boolean ret = source.endsWith(PARQUET_FILE_EXTENSION) + || source.endsWith(METADATA_FILE_URI_SUFFIX) + || source.endsWith(COMMON_METADATA_FILE_URI_SUFFIX); + if (File.separatorChar != URI_SEPARATOR_CHAR) { + ret = ret || source.endsWith(METADATA_FILE_SUFFIX) || source.endsWith(COMMON_METADATA_FILE_SUFFIX); + } + return ret; + } + /** * Check if the provided path points to a non-hidden parquet file, and that none of its parents (till rootDir) are * hidden. 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 9e0cc929229..f67e11bd650 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 @@ -5,6 +5,7 @@ import io.deephaven.base.verify.Require; import io.deephaven.configuration.Configuration; +import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.ColumnToCodecMappings; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; @@ -14,11 +15,16 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.function.Function; import java.util.function.Predicate; +import java.util.stream.Collectors; /** * This class provides instructions intended for read and write parquet operations (which take it as an optional @@ -116,6 +122,34 @@ public static int getDefaultTargetPageSize() { return defaultTargetPageSize; } + public enum ParquetFileLayout { + /** + * A single parquet file. + */ + SINGLE_FILE, + + /** + * A single directory of parquet files. + */ + FLAT_PARTITIONED, + + /** + * A key-value directory partitioning of parquet files. + */ + KV_PARTITIONED, + + /** + * Layout can be used to describe: + *
    + *
  • A directory containing a {@value ParquetUtils#METADATA_FILE_NAME} parquet file and an optional + * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} parquet file + *
  • A single parquet {@value ParquetUtils#METADATA_FILE_NAME} file + *
  • A single parquet {@value ParquetUtils#COMMON_METADATA_FILE_NAME} file + *
+ */ + METADATA_PARTITIONED; + } + private static final boolean DEFAULT_GENERATE_METADATA_FILES = false; static final String UUID_TOKEN = "{uuid}"; @@ -178,6 +212,31 @@ public final String getColumnNameFromParquetColumnNameOrDefault(final String par */ public abstract boolean generateMetadataFiles(); + public abstract Optional getFileLayout(); + + public abstract Optional getTableDefinition(); + + public abstract Optional>> getIndexColumns(); + + /** + * Creates a new {@link ParquetInstructions} object with the same properties as the current object but definition + * set as the provided {@link TableDefinition}. + */ + public abstract ParquetInstructions withTableDefinition(final TableDefinition tableDefinition); + + /** + * Creates a new {@link ParquetInstructions} object with the same properties as the current object but definition + * and layout set as the provided values. + */ + public abstract ParquetInstructions withTableDefinitionAndLayout(final TableDefinition tableDefinition, + final ParquetFileLayout fileLayout); + + /** + * Creates a new {@link ParquetInstructions} object with the same properties as the current object but index columns + * set as the provided values. + */ + @VisibleForTesting + abstract ParquetInstructions withIndexColumns(final Collection> indexColumns); /** * @return the base name for partitioned parquet data. Check @@ -207,16 +266,19 @@ public String getParquetColumnNameFromColumnNameOrDefault(final String columnNam } @Override + @Nullable public String getColumnNameFromParquetColumnName(final String parquetColumnName) { return null; } @Override + @Nullable public String getCodecName(final String columnName) { return null; } @Override + @Nullable public String getCodecArgs(final String columnName) { return null; } @@ -227,7 +289,8 @@ public boolean useDictionary(final String columnName) { } @Override - public @Nullable String getSpecialInstructions() { + @Nullable + public Object getSpecialInstructions() { return null; } @@ -270,6 +333,44 @@ public boolean generateMetadataFiles() { public String baseNameForPartitionedParquetData() { return DEFAULT_BASE_NAME_FOR_PARTITIONED_PARQUET_DATA; } + + @Override + public Optional getFileLayout() { + return Optional.empty(); + } + + @Override + public Optional getTableDefinition() { + return Optional.empty(); + } + + @Override + public Optional>> getIndexColumns() { + return Optional.empty(); + } + + @Override + public ParquetInstructions withTableDefinition(@Nullable final TableDefinition tableDefinition) { + return withTableDefinitionAndLayout(tableDefinition, null); + } + + @Override + public ParquetInstructions withTableDefinitionAndLayout( + @Nullable final TableDefinition tableDefinition, + @Nullable final ParquetFileLayout fileLayout) { + return new ReadOnly(null, null, getCompressionCodecName(), getMaximumDictionaryKeys(), + getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), + getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), + fileLayout, tableDefinition, null); + } + + @Override + ParquetInstructions withIndexColumns(final Collection> indexColumns) { + return new ReadOnly(null, null, getCompressionCodecName(), getMaximumDictionaryKeys(), + getMaximumDictionarySize(), isLegacyParquet(), getTargetPageSize(), isRefreshing(), + getSpecialInstructions(), generateMetadataFiles(), baseNameForPartitionedParquetData(), + null, null, indexColumns); + } }; private static class ColumnInstructions { @@ -340,6 +441,9 @@ private static final class ReadOnly extends ParquetInstructions { private final Object specialInstructions; private final boolean generateMetadataFiles; private final String baseNameForPartitionedParquetData; + private final ParquetFileLayout fileLayout; + private final TableDefinition tableDefinition; + private final Collection> indexColumns; private ReadOnly( final KeyedObjectHashMap columnNameToInstructions, @@ -352,7 +456,10 @@ private ReadOnly( final boolean isRefreshing, final Object specialInstructions, final boolean generateMetadataFiles, - final String baseNameForPartitionedParquetData) { + final String baseNameForPartitionedParquetData, + final ParquetFileLayout fileLayout, + final TableDefinition tableDefinition, + final Collection> indexColumns) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; this.compressionCodecName = compressionCodecName; @@ -364,6 +471,12 @@ private ReadOnly( this.specialInstructions = specialInstructions; this.generateMetadataFiles = generateMetadataFiles; this.baseNameForPartitionedParquetData = baseNameForPartitionedParquetData; + this.fileLayout = fileLayout; + this.tableDefinition = tableDefinition; + this.indexColumns = indexColumns == null ? null + : indexColumns.stream() + .map(List::copyOf) + .collect(Collectors.toUnmodifiableList()); } private String getOrDefault(final String columnName, final String defaultValue, @@ -453,7 +566,8 @@ public boolean isRefreshing() { } @Override - public @Nullable Object getSpecialInstructions() { + @Nullable + public Object getSpecialInstructions() { return specialInstructions; } @@ -467,6 +581,46 @@ public String baseNameForPartitionedParquetData() { return baseNameForPartitionedParquetData; } + @Override + public Optional getFileLayout() { + return Optional.ofNullable(fileLayout); + } + + @Override + public Optional getTableDefinition() { + return Optional.ofNullable(tableDefinition); + } + + @Override + public Optional>> getIndexColumns() { + return Optional.ofNullable(indexColumns); + } + + @Override + public ParquetInstructions withTableDefinition(@Nullable final TableDefinition useDefinition) { + return withTableDefinitionAndLayout(useDefinition, getFileLayout().orElse(null)); + } + + @Override + public ParquetInstructions withTableDefinitionAndLayout( + @Nullable final TableDefinition useDefinition, + @Nullable final ParquetFileLayout useLayout) { + return new ReadOnly(columnNameToInstructions, parquetColumnNameToInstructions, + getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), + isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), + generateMetadataFiles(), baseNameForPartitionedParquetData(), useLayout, useDefinition, + indexColumns); + } + + @Override + ParquetInstructions withIndexColumns(final Collection> useIndexColumns) { + return new ReadOnly(columnNameToInstructions, parquetColumnNameToInstructions, + getCompressionCodecName(), getMaximumDictionaryKeys(), getMaximumDictionarySize(), + isLegacyParquet(), getTargetPageSize(), isRefreshing(), getSpecialInstructions(), + generateMetadataFiles(), baseNameForPartitionedParquetData(), fileLayout, + tableDefinition, useIndexColumns); + } + KeyedObjectHashMap copyColumnNameToInstructions() { // noinspection unchecked return (columnNameToInstructions == null) @@ -520,9 +674,17 @@ public static class Builder { private Object specialInstructions; private boolean generateMetadataFiles = DEFAULT_GENERATE_METADATA_FILES; private String baseNameForPartitionedParquetData = DEFAULT_BASE_NAME_FOR_PARTITIONED_PARQUET_DATA; + private ParquetFileLayout fileLayout; + private TableDefinition tableDefinition; + private Collection> indexColumns; public Builder() {} + /** + * Creates a new {@link ParquetInstructions} object by only copying the column name to instructions mapping and + * parquet column name to instructions mapping from the given {@link ParquetInstructions} object. For copying + * all properties, use something like {@link ParquetInstructions#withTableDefinition}. + */ public Builder(final ParquetInstructions parquetInstructions) { if (parquetInstructions == EMPTY) { return; @@ -737,6 +899,66 @@ public Builder setBaseNameForPartitionedParquetData(final String baseNameForPart return this; } + /** + * Set the expected file layout when reading a parquet file or a directory. This info can be used to skip some + * computations to deduce the file layout from the source directory structure. + */ + public Builder setFileLayout(final ParquetFileLayout fileLayout) { + this.fileLayout = fileLayout; + return this; + } + + /** + *
    + *
  • When reading a parquet file, this corresponds to the table definition to use instead of the one implied + * by the parquet file being read. Providing a definition can help save additional computations to deduce the + * table definition from the parquet files as well as from the directory layouts when reading partitioned + * data.
  • + *
  • When writing a parquet file, this corresponds to the table definition to use instead of the one implied + * by the table being written
  • + *
+ * This definition can be used to skip some columns or add additional columns with {@code null} values. + */ + public Builder setTableDefinition(final TableDefinition tableDefinition) { + this.tableDefinition = tableDefinition; + return this; + } + + private void initIndexColumns() { + if (indexColumns == null) { + indexColumns = new ArrayList<>(); + } + } + + /** + * Add a list of columns to persist together as indexes. The write operation will store the index info as + * sidecar tables. This argument is used to narrow the set of indexes to write, or to be explicit about the + * expected set of indexes present on all sources. Indexes that are specified but missing will be computed on + * demand. + */ + public Builder addIndexColumns(final String... indexColumns) { + initIndexColumns(); + this.indexColumns.add(List.of(indexColumns)); + return this; + } + + /** + * Adds provided lists of columns to persist together as indexes. This method accepts an {@link Iterable} of + * lists, where each list represents a group of columns to be indexed together. + *

+ * The write operation will store the index info as sidecar tables. This argument is used to narrow the set of + * indexes to write, or to be explicit about the expected set of indexes present on all sources. Indexes that + * are specified but missing will be computed on demand. To prevent the generation of index files, provide an + * empty iterable. + */ + public Builder addAllIndexColumns(final Iterable> indexColumns) { + initIndexColumns(); + for (final List indexColumnList : indexColumns) { + this.indexColumns.add(List.copyOf(indexColumnList)); + } + return this; + } + public ParquetInstructions build() { final KeyedObjectHashMap columnNameToInstructionsOut = columnNameToInstructions; columnNameToInstructions = null; @@ -745,7 +967,8 @@ public ParquetInstructions build() { parquetColumnNameToInstructions = null; return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, - specialInstructions, generateMetadataFiles, baseNameForPartitionedParquetData); + specialInstructions, generateMetadataFiles, baseNameForPartitionedParquetData, fileLayout, + tableDefinition, indexColumns); } } 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 4ff3615b2e9..a8e599b04e0 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 @@ -4,12 +4,25 @@ package io.deephaven.parquet.table; import io.deephaven.UncheckedDeephavenException; +import io.deephaven.api.util.NameValidator; +import io.deephaven.base.ClassUtil; +import io.deephaven.base.Pair; +import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.stringset.StringSet; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.parquet.table.metadata.CodecInfo; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; import io.deephaven.parquet.table.metadata.TableInfo; import io.deephaven.parquet.base.ParquetFileReader; +import io.deephaven.util.SimpleTypeMap; +import io.deephaven.vector.ByteVector; +import io.deephaven.vector.CharVector; +import io.deephaven.vector.DoubleVector; +import io.deephaven.vector.FloatVector; +import io.deephaven.vector.IntVector; +import io.deephaven.vector.LongVector; +import io.deephaven.vector.ObjectVector; +import io.deephaven.vector.ShortVector; import org.apache.parquet.format.converter.ParquetMetadataConverter; import io.deephaven.util.codec.SimpleByteArrayCodec; import io.deephaven.util.codec.UTF8StringAsByteArrayCodec; @@ -99,7 +112,7 @@ public static ParquetInstructions readParquetSchema( @NotNull final ColumnDefinitionConsumer consumer, @NotNull final BiFunction, String> legalizeColumnNameFunc) throws IOException { final ParquetFileReader parquetFileReader = - ParquetTools.getParquetFileReaderChecked(new File(filePath), readInstructions); + ParquetFileReader.createChecked(new File(filePath), readInstructions.getSpecialInstructions()); final ParquetMetadata parquetMetadata = new ParquetMetadataConverter().fromParquetMetadata(parquetFileReader.fileMetaData); return readParquetSchema(parquetFileReader.getSchema(), parquetMetadata.getFileMetaData().getKeyValueMetaData(), @@ -282,6 +295,92 @@ public static ParquetInstructions readParquetSchema( : instructionsBuilder.getValue().build(); } + /** + * Convert schema information from a {@link ParquetMetadata} into {@link ColumnDefinition ColumnDefinitions}. + * + * @param schema Parquet schema. DO NOT RELY ON {@link ParquetMetadataConverter} FOR THIS! USE + * {@link ParquetFileReader}! + * @param keyValueMetadata Parquet key-value metadata map + * @param readInstructionsIn Input conversion {@link ParquetInstructions} + * @return A {@link Pair} with {@link ColumnDefinition ColumnDefinitions} and adjusted {@link ParquetInstructions} + */ + public static Pair>, ParquetInstructions> convertSchema( + @NotNull final MessageType schema, + @NotNull final Map keyValueMetadata, + @NotNull final ParquetInstructions readInstructionsIn) { + final ArrayList> cols = new ArrayList<>(); + final ParquetSchemaReader.ColumnDefinitionConsumer colConsumer = makeSchemaReaderConsumer(cols); + return new Pair<>(cols, ParquetSchemaReader.readParquetSchema( + schema, + keyValueMetadata, + readInstructionsIn, + colConsumer, + (final String colName, final Set takenNames) -> NameValidator.legalizeColumnName(colName, + s -> s.replace(" ", "_"), takenNames))); + } + + private static ParquetSchemaReader.ColumnDefinitionConsumer makeSchemaReaderConsumer( + final ArrayList> colsOut) { + return (final ParquetSchemaReader.ParquetMessageDefinition parquetColDef) -> { + Class baseType; + if (parquetColDef.baseType == boolean.class) { + baseType = Boolean.class; + } else { + baseType = parquetColDef.baseType; + } + ColumnDefinition colDef; + if (parquetColDef.codecType != null && !parquetColDef.codecType.isEmpty()) { + final Class componentType = + (parquetColDef.codecComponentType != null && !parquetColDef.codecComponentType.isEmpty()) + ? loadClass(parquetColDef.name, "codecComponentType", parquetColDef.codecComponentType) + : null; + final Class dataType = loadClass(parquetColDef.name, "codecType", parquetColDef.codecType); + colDef = ColumnDefinition.fromGenericType(parquetColDef.name, dataType, componentType); + } else if (parquetColDef.dhSpecialType != null) { + if (parquetColDef.dhSpecialType == ColumnTypeInfo.SpecialType.StringSet) { + colDef = ColumnDefinition.fromGenericType(parquetColDef.name, StringSet.class, null); + } else if (parquetColDef.dhSpecialType == ColumnTypeInfo.SpecialType.Vector) { + final Class vectorType = VECTOR_TYPE_MAP.get(baseType); + if (vectorType != null) { + colDef = ColumnDefinition.fromGenericType(parquetColDef.name, vectorType, baseType); + } else { + colDef = ColumnDefinition.fromGenericType(parquetColDef.name, ObjectVector.class, baseType); + } + } else { + throw new UncheckedDeephavenException("Unhandled dbSpecialType=" + parquetColDef.dhSpecialType); + } + } else { + if (parquetColDef.isArray) { + if (baseType == byte.class && parquetColDef.noLogicalType) { + colDef = ColumnDefinition.fromGenericType(parquetColDef.name, byte[].class, byte.class); + } else { + // TODO: ParquetInstruction.loadAsVector + final Class componentType = baseType; + // On Java 12, replace by: dataType = componentType.arrayType(); + final Class dataType = java.lang.reflect.Array.newInstance(componentType, 0).getClass(); + colDef = ColumnDefinition.fromGenericType(parquetColDef.name, dataType, componentType); + } + } else { + colDef = ColumnDefinition.fromGenericType(parquetColDef.name, baseType, null); + } + } + colsOut.add(colDef); + }; + } + + private static final SimpleTypeMap> VECTOR_TYPE_MAP = SimpleTypeMap.create( + null, CharVector.class, ByteVector.class, ShortVector.class, IntVector.class, LongVector.class, + FloatVector.class, DoubleVector.class, ObjectVector.class); + + private static Class loadClass(final String colName, final String desc, final String className) { + try { + return ClassUtil.lookupClass(className); + } catch (ClassNotFoundException e) { + throw new UncheckedDeephavenException( + "Column " + colName + " with " + desc + "=" + className + " that can't be found in classloader"); + } + } + private static LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> getVisitor( final Map nonDefaultTypeColumns, final MutableObject errorString, 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 eed80903ae0..8b69a67cd7e 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 @@ -68,7 +68,7 @@ static class IndexWritingInfo { /** * Names of the indexing key columns */ - final String[] indexColumnNames; + final List indexColumnNames; /** * Parquet names of the indexing key columns */ @@ -85,7 +85,7 @@ static class IndexWritingInfo { final File destFile; IndexWritingInfo( - final String[] indexColumnNames, + final List indexColumnNames, final String[] parquetColumnNames, final File destFileForMetadata, final File destFile) { 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 cc9fbd61745..e5b0d96f3a5 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 @@ -4,8 +4,6 @@ package io.deephaven.parquet.table; import io.deephaven.UncheckedDeephavenException; -import io.deephaven.api.util.NameValidator; -import io.deephaven.base.ClassUtil; import io.deephaven.base.FileUtils; import io.deephaven.base.Pair; import io.deephaven.base.verify.Require; @@ -24,11 +22,8 @@ import io.deephaven.parquet.base.ParquetMetadataFileWriter; import io.deephaven.parquet.base.NullParquetMetadataFileWriter; import io.deephaven.util.SafeCloseable; -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.*; import io.deephaven.engine.table.impl.PartitionAwareSourceTable; import io.deephaven.engine.table.impl.SimpleSourceTable; @@ -49,13 +44,9 @@ 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.metadata.ColumnTypeInfo; -import io.deephaven.util.SimpleTypeMap; +import io.deephaven.parquet.table.ParquetInstructions.ParquetFileLayout; import io.deephaven.util.annotations.VisibleForTesting; -import io.deephaven.util.channel.CachedChannelProvider; import org.apache.commons.lang3.mutable.MutableObject; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -74,6 +65,9 @@ import static io.deephaven.base.FileUtils.convertToURI; import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; +import static io.deephaven.parquet.base.ParquetUtils.COMMON_METADATA_FILE_URI_SUFFIX; +import static io.deephaven.parquet.base.ParquetUtils.METADATA_FILE_URI_SUFFIX; +import static io.deephaven.parquet.base.ParquetUtils.isParquetFile; import static io.deephaven.parquet.table.ParquetInstructions.FILE_INDEX_TOKEN; import static io.deephaven.parquet.table.ParquetInstructions.PARTITIONS_TOKEN; import static io.deephaven.parquet.table.ParquetInstructions.UUID_TOKEN; @@ -90,7 +84,7 @@ public class ParquetTools { private static final int MAX_PARTITIONING_LEVELS_INFERENCE = 32; - private static final String[][] EMPTY_INDEXES = new String[0][]; + private static final Collection> EMPTY_INDEXES = Collections.emptyList(); private ParquetTools() {} @@ -107,13 +101,6 @@ private ParquetTools() {} * metadata file is supplied or discovered in the directory, the highest (by {@link ParquetTableLocationKey location * key} order) location found will be used to infer schema. * - *

- * 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 source The path or URI of file or directory to examine * @return table * @see ParquetSingleFileLayout @@ -122,7 +109,7 @@ private ParquetTools() {} * @see ParquetFlatPartitionedLayout */ public static Table readTable(@NotNull final String source) { - return readTableInternal(source, ParquetInstructions.EMPTY); + return readTable(source, ParquetInstructions.EMPTY); } /** @@ -131,16 +118,11 @@ public static Table readTable(@NotNull final String source) { * {@link SeekableChannelsProviderPlugin}. * *

- * This method attempts to "do the right thing." It examines the source to determine if it's a single parquet file, - * a metadata file, or a directory. If it's a directory, it additionally tries to guess the layout to use. Unless a - * metadata file is supplied or discovered in the directory, the highest (by {@link ParquetTableLocationKey location - * key} order) location found will be used to infer schema. - * - *

- * Delegates to one of {@link #readSingleFileTable(File, ParquetInstructions)}, - * {@link #readPartitionedTableWithMetadata(File, ParquetInstructions)}, - * {@link #readFlatPartitionedTable(File, ParquetInstructions)}, or - * {@link #readKeyValuePartitionedTable(File, ParquetInstructions)}. + * If the {@link ParquetFileLayout} is not provided in the {@link ParquetInstructions instructions}, this method + * attempts to "do the right thing." It examines the source to determine if it's a single parquet file, a metadata + * file, or a directory. If it's a directory, it additionally tries to guess the layout to use. Unless a metadata + * file is supplied or discovered in the directory, the highest (by {@link ParquetTableLocationKey location key} + * order) location found will be used to infer schema. * * @param source The path or URI of file or directory to examine * @param readInstructions Instructions for customizations while reading @@ -153,7 +135,32 @@ public static Table readTable(@NotNull final String source) { public static Table readTable( @NotNull final String source, @NotNull final ParquetInstructions readInstructions) { - return readTableInternal(source, readInstructions); + final boolean isDirectory = !isParquetFile(source); + final URI sourceURI = convertToURI(source, isDirectory); + if (readInstructions.getFileLayout().isPresent()) { + switch (readInstructions.getFileLayout().get()) { + case SINGLE_FILE: + return readSingleFileTable(sourceURI, readInstructions); + case FLAT_PARTITIONED: + return readFlatPartitionedTable(sourceURI, readInstructions); + case KV_PARTITIONED: + return readKeyValuePartitionedTable(sourceURI, readInstructions); + case METADATA_PARTITIONED: + return readPartitionedTableWithMetadata(sourceURI, readInstructions); + } + } + if (FILE_URI_SCHEME.equals(sourceURI.getScheme())) { + return readTableFromFileUri(sourceURI, readInstructions); + } + if (source.endsWith(METADATA_FILE_URI_SUFFIX) || source.endsWith(COMMON_METADATA_FILE_URI_SUFFIX)) { + throw new UncheckedDeephavenException("We currently do not support reading parquet metadata files " + + "from non local storage"); + } + if (!isDirectory) { + return readSingleFileTable(sourceURI, readInstructions); + } + // Both flat partitioned and key-value partitioned data can be read under key-value partitioned layout + return readKeyValuePartitionedTable(sourceURI, readInstructions); } /** @@ -165,21 +172,18 @@ public static Table readTable( * metadata file is supplied or discovered in the directory, the highest (by {@link ParquetTableLocationKey location * key} order) location found will be used to infer schema. * - *

- * Delegates to one of {@link #readSingleFileTable(File, ParquetInstructions)}, - * {@link #readPartitionedTableWithMetadata(File, ParquetInstructions)}, - * {@link #readFlatPartitionedTable(File, ParquetInstructions)}, or - * {@link #readKeyValuePartitionedTable(File, ParquetInstructions)}. - * * @param sourceFile The file or directory to examine * @return table * @see ParquetSingleFileLayout * @see ParquetMetadataFileLayout * @see ParquetKeyValuePartitionedLayout * @see ParquetFlatPartitionedLayout + * + * @deprecated Use {@link #readTable(String)} instead. */ + @Deprecated public static Table readTable(@NotNull final File sourceFile) { - return readTableInternal(sourceFile, ParquetInstructions.EMPTY); + return readTable(sourceFile.getPath()); } /** @@ -204,25 +208,30 @@ public static Table readTable(@NotNull final File sourceFile) { * @see ParquetMetadataFileLayout * @see ParquetKeyValuePartitionedLayout * @see ParquetFlatPartitionedLayout + * + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead. */ + // TODO(deephaven-core#5362): Delete this and all deprecated APIs in this class + @Deprecated public static Table readTable( @NotNull final File sourceFile, @NotNull final ParquetInstructions readInstructions) { - return readTableInternal(sourceFile, readInstructions); + return readTable(sourceFile.getPath(), readInstructions); } /** * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. * * @param sourceTable source table - * @param destPath destination file path; the file name should end in ".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 + * @param destination destination path or URI; the file name should end in ".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 */ public static void writeTable( @NotNull final Table sourceTable, - @NotNull final String destPath) { - writeTable(sourceTable, new File(destPath), sourceTable.getDefinition(), ParquetInstructions.EMPTY); + @NotNull final String destination) { + writeTables(new Table[] {sourceTable}, new String[] {destination}, + ParquetInstructions.EMPTY.withTableDefinition(sourceTable.getDefinition())); } /** @@ -231,11 +240,14 @@ public static void writeTable( * @param sourceTable source table * @param destFile destination file; the file name should end in ".parquet" extension If the path includes * non-existing directories they are created + * + * @deprecated Use {@link #writeTable(Table, String)} instead. */ + @Deprecated public static void writeTable( @NotNull final Table sourceTable, @NotNull final File destFile) { - writeTable(sourceTable, destFile, sourceTable.getDefinition(), ParquetInstructions.EMPTY); + writeTable(sourceTable, destFile.getPath()); } /** @@ -246,12 +258,15 @@ public static void writeTable( * created If there is an error any intermediate directories previously created are removed; note this makes * this method unsafe for concurrent use * @param definition table definition to use (instead of the one implied by the table itself) + * @deprecated Use {@link #writeTable(Table, String, ParquetInstructions)} instead with {@link TableDefinition} + * provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ + @Deprecated public static void writeTable( @NotNull final Table sourceTable, @NotNull final File destFile, @NotNull final TableDefinition definition) { - writeTable(sourceTable, destFile, definition, ParquetInstructions.EMPTY); + writeTable(sourceTable, destFile.getPath(), ParquetInstructions.EMPTY.withTableDefinition(definition)); } /** @@ -262,30 +277,66 @@ public static void writeTable( * created If there is an error any intermediate directories previously created are removed; note this makes * this method unsafe for concurrent use * @param writeInstructions instructions for customizations while writing + * @deprecated Use {@link #writeTable(Table, String, ParquetInstructions)} instead. */ + @Deprecated public static void writeTable( @NotNull final Table sourceTable, @NotNull final File destFile, @NotNull final ParquetInstructions writeInstructions) { - writeTable(sourceTable, destFile, sourceTable.getDefinition(), writeInstructions); + writeTable(sourceTable, destFile.getPath(), + ensureTableDefinition(writeInstructions, sourceTable.getDefinition(), false)); + } + + /** + * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. + * + * @param sourceTable source table + * @param destination destination path or URI; the file name should end in ".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 + * @param writeInstructions instructions for customizations while writing + */ + public static void writeTable( + @NotNull final Table sourceTable, + @NotNull final String destination, + @NotNull final ParquetInstructions writeInstructions) { + writeTables(new Table[] {sourceTable}, new String[] {destination}, + ensureTableDefinition(writeInstructions, sourceTable.getDefinition(), false)); } /** * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. * * @param sourceTable source table - * @param destPath destination path; it must end in ".parquet". Any non existing directories in the path are created - * If there is an error any intermediate directories previously created are removed; note this makes this - * method unsafe for concurrent use + * @param destination destination path or URI; the file name should end in ".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 * @param definition table definition to use (instead of the one implied by the table itself) * @param writeInstructions instructions for customizations while writing + * @deprecated Use {@link #writeTable(Table, String, ParquetInstructions)} instead with {@link TableDefinition} + * provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ + @Deprecated public static void writeTable( @NotNull final Table sourceTable, - @NotNull final String destPath, + @NotNull final String destination, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions) { - writeTable(sourceTable, new File(destPath), definition, writeInstructions); + writeTable(sourceTable, destination, ensureTableDefinition(writeInstructions, definition, true)); + } + + private static ParquetInstructions ensureTableDefinition( + @NotNull final ParquetInstructions instructions, + @NotNull final TableDefinition definition, + final boolean validateExisting) { + if (instructions.getTableDefinition().isEmpty()) { + return instructions.withTableDefinition(definition); + } else if (validateExisting && !instructions.getTableDefinition().get().equals(definition)) { + throw new IllegalArgumentException( + "Table definition provided in instructions does not match the one provided in the method call"); + } + return instructions; } /** @@ -297,13 +348,16 @@ public static void writeTable( * this method unsafe for concurrent use * @param definition table definition to use (instead of the one implied by the table itself) * @param writeInstructions instructions for customizations while writing + * @deprecated Use {@link #writeTable(Table, String, ParquetInstructions)} instead with {@link TableDefinition} + * provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ + @Deprecated public static void writeTable( @NotNull final Table sourceTable, @NotNull final File destFile, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions) { - writeTables(new Table[] {sourceTable}, definition, new File[] {destFile}, writeInstructions); + writeTable(sourceTable, destFile.getPath(), ensureTableDefinition(writeInstructions, definition, true)); } private static File getShadowFile(final File destFile) { @@ -332,7 +386,7 @@ private static String minusParquetSuffix(@NotNull final String s) { * indexing column {@code "IndexingColName"}, the method will return * {@code ".dh_metadata/indexes/IndexingColName/index_IndexingColName_table.parquet"} on unix systems. */ - public static String getRelativeIndexFilePath(@NotNull final File tableDest, @NotNull final String[] columnNames) { + private static String getRelativeIndexFilePath(@NotNull final File tableDest, @NotNull final String[] columnNames) { final String columns = String.join(",", columnNames); return String.format(".dh_metadata%sindexes%s%s%sindex_%s_%s", File.separator, File.separator, columns, File.separator, columns, tableDest.getName()); @@ -349,6 +403,7 @@ public static String getRelativeIndexFilePath(@NotNull final File tableDest, @No * grouping column {@code "GroupingColName"}, the method will return * {@code "table_GroupingColName_grouping.parquet"} */ + @VisibleForTesting public static String legacyGroupingFileName(@NotNull final File tableDest, @NotNull final String columnName) { final String prefix = minusParquetSuffix(tableDest.getName()); return prefix + "_" + columnName + "_grouping.parquet"; @@ -460,19 +515,20 @@ private static File prepareDestinationFileLocation(@NotNull File destination) { /** * Helper function for building index column info for writing and deleting any backup index column files * - * @param indexColumnNameArr Names of index columns, stored as String[] for each index + * @param indexColumns Names of index columns, stored as String list for each index * @param parquetColumnNameArr Names of index columns for the parquet file, stored as String[] for each index * @param destFile The destination path for the main table containing these index columns */ private static List indexInfoBuilderHelper( - @NotNull final String[][] indexColumnNameArr, + @NotNull final Collection> indexColumns, @NotNull final String[][] parquetColumnNameArr, @NotNull final File destFile) { - Require.eq(indexColumnNameArr.length, "indexColumnNameArr.length", parquetColumnNameArr.length, + Require.eq(indexColumns.size(), "indexColumns.size", parquetColumnNameArr.length, "parquetColumnNameArr.length"); - final List indexInfoList = new ArrayList<>(); - for (int gci = 0; gci < indexColumnNameArr.length; gci++) { - final String[] indexColumnNames = indexColumnNameArr[gci]; + final int numIndexes = indexColumns.size(); + final List indexInfoList = new ArrayList<>(numIndexes); + int gci = 0; + for (final List indexColumnNames : indexColumns) { final String[] parquetColumnNames = parquetColumnNameArr[gci]; final String indexFileRelativePath = getRelativeIndexFilePath(destFile, parquetColumnNames); final File indexFile = new File(destFile.getParent(), indexFileRelativePath); @@ -486,8 +542,8 @@ private static List indexInfoBuilderHelper( parquetColumnNames, indexFile, shadowIndexFile); - indexInfoList.add(info); + gci++; } return indexInfoList; } @@ -497,92 +553,22 @@ private static List indexInfoBuilderHelper( * written as "key=value" format in a nested directory structure. To generate these individual partitions, this * method will call {@link Table#partitionBy(String...) partitionBy} on all the partitioning columns of provided * table. The generated parquet files will have names of the format provided by - * {@link ParquetInstructions#baseNameForPartitionedParquetData()}. Any indexing columns present on the source table - * will be written as sidecar tables. To write only a subset of the indexes or add additional indexes while writing, - * use {@link #writeKeyValuePartitionedTable(Table, String, ParquetInstructions, String[][])}. - * - * @param sourceTable The table to partition and write - * @param destinationDir The path to destination root directory to store partitioned data in nested format. - * Non-existing directories are created. - * @param writeInstructions Write instructions for customizations while writing - */ - public static void writeKeyValuePartitionedTable(@NotNull final Table sourceTable, - @NotNull final String destinationDir, - @NotNull final ParquetInstructions writeInstructions) { - writeKeyValuePartitionedTable(sourceTable, sourceTable.getDefinition(), destinationDir, - writeInstructions, indexedColumnNames(sourceTable)); - } - - /** - * Write table to disk in parquet format with {@link TableDefinition#getPartitioningColumns() partitioning columns} - * written as "key=value" format in a nested directory structure. To generate these individual partitions, this - * method will call {@link Table#partitionBy(String...) partitionBy} on all the partitioning columns of provided - * table. The generated parquet files will have names of the format provided by - * {@link ParquetInstructions#baseNameForPartitionedParquetData()}. + * {@link ParquetInstructions#baseNameForPartitionedParquetData()}. By default, any indexing columns present on the + * source table will be written as sidecar tables. To write only a subset of the indexes or add additional indexes + * while writing, use {@link ParquetInstructions.Builder#addIndexColumns}. * * @param sourceTable The table to partition and write * @param destinationDir The path to destination root directory to store partitioned data in nested format. * Non-existing directories are created. * @param writeInstructions Write instructions for customizations while writing - * @param indexColumnArr Arrays containing the column names for indexes to persist. The write operation will store - * the index info as sidecar tables. This argument is used to narrow the set of indexes to write, or to be - * explicit about the expected set of indexes present on all sources. Indexes that are specified but missing - * will be computed on demand. - */ - public static void writeKeyValuePartitionedTable(@NotNull final Table sourceTable, - @NotNull final String destinationDir, - @NotNull final ParquetInstructions writeInstructions, - @Nullable final String[][] indexColumnArr) { - writeKeyValuePartitionedTable(sourceTable, sourceTable.getDefinition(), destinationDir, - writeInstructions, indexColumnArr); - } - - /** - * Write table to disk in parquet format with {@link TableDefinition#getPartitioningColumns() partitioning columns} - * written as "key=value" format in a nested directory structure. To generate these individual partitions, this - * method will call {@link Table#partitionBy(String...) partitionBy} on all the partitioning columns in the provided - * table definition. The generated parquet files will have names of the format provided by - * {@link ParquetInstructions#baseNameForPartitionedParquetData()}. Any indexing columns present on the source table - * will be written as sidecar tables. To write only a subset of the indexes or add additional indexes while writing, - * use {@link #writeKeyValuePartitionedTable(Table, TableDefinition, String, ParquetInstructions, String[][])}. - * - * @param sourceTable The table to partition and write - * @param definition table definition to use (instead of the one implied by the table itself) - * @param destinationDir The path to destination root directory to store partitioned data in nested format. - * Non-existing directories are created. - * @param writeInstructions Write instructions for customizations while writing */ - public static void writeKeyValuePartitionedTable(@NotNull final Table sourceTable, - @NotNull final TableDefinition definition, + public static void writeKeyValuePartitionedTable( + @NotNull final Table sourceTable, @NotNull final String destinationDir, @NotNull final ParquetInstructions writeInstructions) { - writeKeyValuePartitionedTable(sourceTable, definition, destinationDir, writeInstructions, - indexedColumnNames(sourceTable)); - - } - - /** - * Write table to disk in parquet format with {@link TableDefinition#getPartitioningColumns() partitioning columns} - * written as "key=value" format in a nested directory structure. To generate these individual partitions, this - * method will call {@link Table#partitionBy(String...) partitionBy} on all the partitioning columns in the provided - * table definition. The generated parquet files will have names of the format provided by - * {@link ParquetInstructions#baseNameForPartitionedParquetData()}. - * - * @param sourceTable The table to partition and write - * @param definition table definition to use (instead of the one implied by the table itself) - * @param destinationDir The path to destination root directory to store partitioned data in nested format. - * Non-existing directories are created. - * @param writeInstructions Write instructions for customizations while writing - * @param indexColumnArr Arrays containing the column names for indexes to persist. The write operation will store - * the index info as sidecar tables. This argument is used to narrow the set of indexes to write, or to be - * explicit about the expected set of indexes present on all sources. Indexes that are specified but missing - * will be computed on demand. - */ - public static void writeKeyValuePartitionedTable(@NotNull final Table sourceTable, - @NotNull final TableDefinition definition, - @NotNull final String destinationDir, - @NotNull final ParquetInstructions writeInstructions, - @Nullable final String[][] indexColumnArr) { + final Collection> indexColumns = + writeInstructions.getIndexColumns().orElseGet(() -> indexedColumnNames(sourceTable)); + final TableDefinition definition = writeInstructions.getTableDefinition().orElse(sourceTable.getDefinition()); final List> partitioningColumns = definition.getPartitioningColumns(); if (partitioningColumns.isEmpty()) { throw new IllegalArgumentException("Table must have partitioning columns to write partitioned data"); @@ -595,101 +581,40 @@ public static void writeKeyValuePartitionedTable(@NotNull final Table sourceTabl final TableDefinition leafDefinition = getNonKeyTableDefinition(new HashSet<>(Arrays.asList(partitioningColNames)), definition); writeKeyValuePartitionedTableImpl(partitionedTable, keyTableDefinition, leafDefinition, destinationDir, - writeInstructions, indexColumnArr, Optional.of(sourceTable)); - } - - /** - * Write a partitioned table to disk in parquet format with all the {@link PartitionedTable#keyColumnNames() key - * columns} as "key=value" format in a nested directory structure. To generate the partitioned table, users can call - * {@link Table#partitionBy(String...) partitionBy} on the required columns. The generated parquet files will have - * names of the format provided by {@link ParquetInstructions#baseNameForPartitionedParquetData()}. This method does - * not write any indexes as sidecar tables to disk. To write indexes, use - * {@link #writeKeyValuePartitionedTable(PartitionedTable, String, ParquetInstructions, String[][])}. - * - * @param partitionedTable The partitioned table to write - * @param destinationDir The path to destination root directory to store partitioned data in nested format. - * Non-existing directories are created. - * @param writeInstructions Write instructions for customizations while writing - */ - public static void writeKeyValuePartitionedTable(@NotNull final PartitionedTable partitionedTable, - @NotNull final String destinationDir, - @NotNull final ParquetInstructions writeInstructions) { - writeKeyValuePartitionedTable(partitionedTable, destinationDir, writeInstructions, EMPTY_INDEXES); - } - - /** - * Write a partitioned table to disk in parquet format with all the {@link PartitionedTable#keyColumnNames() key - * columns} as "key=value" format in a nested directory structure. To generate the partitioned table, users can call - * {@link Table#partitionBy(String...) partitionBy} on the required columns. The generated parquet files will have - * names of the format provided by {@link ParquetInstructions#baseNameForPartitionedParquetData()}. - * - * @param partitionedTable The partitioned table to write - * @param destinationDir The path to destination root directory to store partitioned data in nested format. - * Non-existing directories are created. - * @param writeInstructions Write instructions for customizations while writing - * @param indexColumnArr Arrays containing the column names for indexes to persist. The write operation will store - * the index info as sidecar tables. This argument is used to narrow the set of indexes to write, or to be - * explicit about the expected set of indexes present on all sources. Indexes that are specified but missing - * will be computed on demand. - */ - public static void writeKeyValuePartitionedTable(@NotNull final PartitionedTable partitionedTable, - @NotNull final String destinationDir, - @NotNull final ParquetInstructions writeInstructions, - @Nullable final String[][] indexColumnArr) { - final TableDefinition keyTableDefinition = getKeyTableDefinition(partitionedTable.keyColumnNames(), - partitionedTable.table().getDefinition()); - final TableDefinition leafDefinition = getNonKeyTableDefinition(partitionedTable.keyColumnNames(), - partitionedTable.constituentDefinition()); - writeKeyValuePartitionedTableImpl(partitionedTable, keyTableDefinition, leafDefinition, destinationDir, - writeInstructions, indexColumnArr, Optional.empty()); + writeInstructions, indexColumns, Optional.of(sourceTable)); } /** * Write a partitioned table to disk in parquet format with all the {@link PartitionedTable#keyColumnNames() key * columns} as "key=value" format in a nested directory structure. To generate the partitioned table, users can call * {@link Table#partitionBy(String...) partitionBy} on the required columns. The generated parquet files will have - * names of the format provided by {@link ParquetInstructions#baseNameForPartitionedParquetData()}. This method does - * not write any indexes as sidecar tables to disk. To write indexes, use - * {@link #writeKeyValuePartitionedTable(PartitionedTable, TableDefinition, String, ParquetInstructions, String[][])}. + * names of the format provided by {@link ParquetInstructions#baseNameForPartitionedParquetData()}. By default, this + * method does not write any indexes as sidecar tables to disk. To write such indexes, use + * {@link ParquetInstructions.Builder#addIndexColumns}. * * @param partitionedTable The partitioned table to write - * @param definition table definition to use (instead of the one implied by the table itself) * @param destinationDir The path to destination root directory to store partitioned data in nested format. * Non-existing directories are created. * @param writeInstructions Write instructions for customizations while writing */ - public static void writeKeyValuePartitionedTable(@NotNull final PartitionedTable partitionedTable, - @NotNull final TableDefinition definition, + public static void writeKeyValuePartitionedTable( + @NotNull final PartitionedTable partitionedTable, @NotNull final String destinationDir, @NotNull final ParquetInstructions writeInstructions) { - writeKeyValuePartitionedTable(partitionedTable, definition, destinationDir, writeInstructions, EMPTY_INDEXES); - } - - /** - * Write a partitioned table to disk in parquet format with all the {@link PartitionedTable#keyColumnNames() key - * columns} as "key=value" format in a nested directory structure. To generate the partitioned table, users can call - * {@link Table#partitionBy(String...) partitionBy} on the required columns. The generated parquet files will have - * names of the format provided by {@link ParquetInstructions#baseNameForPartitionedParquetData()}. - * - * @param partitionedTable The partitioned table to write - * @param definition table definition to use (instead of the one implied by the table itself) - * @param destinationDir The path to destination root directory to store partitioned data in nested format. - * Non-existing directories are created. - * @param writeInstructions Write instructions for customizations while writing - * @param indexColumnArr Arrays containing the column names for indexes to persist. The write operation will store - * the index info as sidecar tables. This argument is used to narrow the set of indexes to write, or to be - * explicit about the expected set of indexes present on all sources. Indexes that are specified but missing - * will be computed on demand. - */ - public static void writeKeyValuePartitionedTable(@NotNull final PartitionedTable partitionedTable, - @NotNull final TableDefinition definition, - @NotNull final String destinationDir, - @NotNull final ParquetInstructions writeInstructions, - @NotNull final String[][] indexColumnArr) { - final TableDefinition keyTableDefinition = getKeyTableDefinition(partitionedTable.keyColumnNames(), definition); - final TableDefinition leafDefinition = getNonKeyTableDefinition(partitionedTable.keyColumnNames(), definition); + final Collection> indexColumns = writeInstructions.getIndexColumns().orElse(EMPTY_INDEXES); + final TableDefinition keyTableDefinition, leafDefinition; + if (writeInstructions.getTableDefinition().isEmpty()) { + keyTableDefinition = getKeyTableDefinition(partitionedTable.keyColumnNames(), + partitionedTable.table().getDefinition()); + leafDefinition = getNonKeyTableDefinition(partitionedTable.keyColumnNames(), + partitionedTable.constituentDefinition()); + } else { + final TableDefinition definition = writeInstructions.getTableDefinition().get(); + keyTableDefinition = getKeyTableDefinition(partitionedTable.keyColumnNames(), definition); + leafDefinition = getNonKeyTableDefinition(partitionedTable.keyColumnNames(), definition); + } writeKeyValuePartitionedTableImpl(partitionedTable, keyTableDefinition, leafDefinition, destinationDir, - writeInstructions, indexColumnArr, Optional.empty()); + writeInstructions, indexColumns, Optional.empty()); } /** @@ -701,20 +626,20 @@ public static void writeKeyValuePartitionedTable(@NotNull final PartitionedTable * @param leafDefinition The definition for leaf parquet files to be written * @param destinationRoot The path to destination root directory to store partitioned data in nested format * @param writeInstructions Write instructions for customizations while writing - * @param indexColumnArr Arrays containing the column names for indexes to persist. The write operation will store + * @param indexColumns Collection containing the column names for indexes to persist. The write operation will store * the index info as sidecar tables. This argument is used to narrow the set of indexes to write, or to be * explicit about the expected set of indexes present on all sources. Indexes that are specified but missing * will be computed on demand. * @param sourceTable The optional source table, provided when user provides a merged source table to write, like in - * {@link #writeKeyValuePartitionedTable(Table, String, ParquetInstructions)} and - * {@link #writeKeyValuePartitionedTable(Table, TableDefinition, String, ParquetInstructions)} + * {@link #writeKeyValuePartitionedTable(Table, String, ParquetInstructions)} */ - private static void writeKeyValuePartitionedTableImpl(@NotNull final PartitionedTable partitionedTable, + private static void writeKeyValuePartitionedTableImpl( + @NotNull final PartitionedTable partitionedTable, @NotNull final TableDefinition keyTableDefinition, @NotNull final TableDefinition leafDefinition, @NotNull final String destinationRoot, @NotNull final ParquetInstructions writeInstructions, - @Nullable final String[][] indexColumnArr, + @NotNull final Collection> indexColumns, @NotNull final Optional sourceTable) { if (leafDefinition.numColumns() == 0) { throw new IllegalArgumentException("Cannot write a partitioned parquet table without any non-partitioning " @@ -796,9 +721,9 @@ private static void writeKeyValuePartitionedTableImpl(@NotNull final Partitioned buildComputedCache(() -> sourceTable.orElseGet(partitionedTable::merge), leafDefinition); // TODO(deephaven-core#5292): Optimize creating index on constituent tables // Store hard reference to prevent indexes from being garbage collected - final List dataIndexes = addIndexesToTables(partitionedDataArray, indexColumnArr); - writeParquetTablesImpl(partitionedDataArray, leafDefinition, writeInstructions, - destinations.toArray(File[]::new), indexColumnArr, partitioningColumnsSchema, + final List dataIndexes = addIndexesToTables(partitionedDataArray, indexColumns); + writeTablesImpl(partitionedDataArray, leafDefinition, writeInstructions, + destinations.toArray(File[]::new), indexColumns, partitioningColumnsSchema, new File(destinationRoot), computedCache); if (dataIndexes != null) { dataIndexes.clear(); @@ -810,14 +735,15 @@ private static void writeKeyValuePartitionedTableImpl(@NotNull final Partitioned * Add data indexes to provided tables, if not present, and return a list of hard references to the indexes. */ @Nullable - private static List addIndexesToTables(@NotNull final Table[] tables, - @Nullable final String[][] indexColumnArr) { - if (indexColumnArr == null || indexColumnArr.length == 0) { + private static List addIndexesToTables( + @NotNull final Table[] tables, + @NotNull final Collection> indexColumns) { + if (indexColumns.isEmpty()) { return null; } - final List dataIndexes = new ArrayList<>(indexColumnArr.length * tables.length); + final List dataIndexes = new ArrayList<>(indexColumns.size() * tables.length); for (final Table table : tables) { - for (final String[] indexCols : indexColumnArr) { + for (final List indexCols : indexColumns) { dataIndexes.add(DataIndexer.getOrCreateDataIndex(table, indexCols)); } } @@ -828,7 +754,8 @@ private static List addIndexesToTables(@NotNull final Table[] tables, * Using the provided definition and key column names, create a sub table definition for the key columns that are * present in the definition. */ - private static TableDefinition getKeyTableDefinition(@NotNull final Collection keyColumnNames, + private static TableDefinition getKeyTableDefinition( + @NotNull final Collection keyColumnNames, @NotNull final TableDefinition definition) { final Collection> keyColumnDefinitions = new ArrayList<>(keyColumnNames.size()); for (final String keyColumnName : keyColumnNames) { @@ -843,7 +770,8 @@ private static TableDefinition getKeyTableDefinition(@NotNull final Collection keyColumnNames, + private static TableDefinition getNonKeyTableDefinition( + @NotNull final Collection keyColumnNames, @NotNull final TableDefinition definition) { final Collection> nonKeyColumnDefinition = definition.getColumns().stream() .filter(columnDefinition -> !keyColumnNames.contains(columnDefinition.getName())) @@ -878,62 +806,22 @@ private static Map> buildComputedCache( } /** - * Writes tables to disk in parquet format to a supplied set of destinations. - * - * @param sources The tables to write - * @param definition The common definition for all the tables to write - * @param writeInstructions Write instructions for customizations while writing - * @param destinations The destination paths. Any non-existing directories in the paths provided are created. If - * there is an error, any intermediate directories previously created are removed; note this makes this - * method unsafe for concurrent use. - * @param indexColumnArr Arrays containing the column names for indexes to persist. The write operation will store - * the index info as sidecar tables. This argument is used to narrow the set of indexes to write, or to be - * explicit about the expected set of indexes present on all sources. Indexes that are specified but missing - * will be computed on demand. + * Refer to {@link #writeTables(Table[], String[], ParquetInstructions)} for more details. */ - public static void writeParquetTables( + private static void writeTablesImpl( @NotNull final Table[] sources, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions, @NotNull final File[] destinations, - @Nullable final String[][] indexColumnArr) { - final File metadataRootDir; - if (writeInstructions.generateMetadataFiles()) { - // We insist on writing the metadata file in the same directory as the destination files, thus all - // destination files should be in the same directory. - final String firstDestinationDir = destinations[0].getAbsoluteFile().getParentFile().getAbsolutePath(); - for (int i = 1; i < destinations.length; i++) { - if (!firstDestinationDir.equals(destinations[i].getParentFile().getAbsolutePath())) { - throw new IllegalArgumentException("All destination files must be in the same directory for " + - " generating metadata files"); - } - } - metadataRootDir = new File(firstDestinationDir); - } else { - metadataRootDir = null; - } - - final Map> computedCache = - buildComputedCache(() -> PartitionedTableFactory.ofTables(definition, sources).merge(), definition); - // We do not have any additional schema for partitioning columns in this case. Schema for all columns will be - // generated at the time of writing the parquet files and merged to generate the metadata files. - writeParquetTablesImpl(sources, definition, writeInstructions, destinations, indexColumnArr, - null, metadataRootDir, computedCache); - } - - /** - * Refer to {@link #writeParquetTables(Table[], TableDefinition, ParquetInstructions, File[], String[][])} for more - * details. - */ - private static void writeParquetTablesImpl(@NotNull final Table[] sources, - @NotNull final TableDefinition definition, - @NotNull final ParquetInstructions writeInstructions, - @NotNull final File[] destinations, - @Nullable final String[][] indexColumnArr, + @NotNull final Collection> indexColumns, @Nullable final MessageType partitioningColumnsSchema, @Nullable final File metadataRootDir, @NotNull final Map> computedCache) { Require.eq(sources.length, "sources.length", destinations.length, "destinations.length"); + if (writeInstructions.getFileLayout().isPresent()) { + throw new UnsupportedOperationException("File layout is not supported for writing parquet files, use the " + + "appropriate API"); + } if (definition.numColumns() == 0) { throw new TableDataException("Cannot write a parquet table with zero columns"); } @@ -963,7 +851,7 @@ private static void writeParquetTablesImpl(@NotNull final Table[] sources, final List destFiles = new ArrayList<>(); try { final List> indexInfoLists; - if (indexColumnArr == null || indexColumnArr.length == 0) { + if (indexColumns.isEmpty()) { // Write the tables without any index info indexInfoLists = null; for (int tableIdx = 0; tableIdx < sources.length; tableIdx++) { @@ -979,8 +867,8 @@ private static void writeParquetTablesImpl(@NotNull final Table[] sources, indexInfoLists = new ArrayList<>(sources.length); // Shared parquet column names across all tables - final String[][] parquetColumnNameArr = Arrays.stream(indexColumnArr) - .map((String[] columns) -> Arrays.stream(columns) + final String[][] parquetColumnNameArr = indexColumns.stream() + .map((Collection columns) -> columns.stream() .map(writeInstructions::getParquetColumnNameFromColumnNameOrDefault) .toArray(String[]::new)) .toArray(String[][]::new); @@ -988,7 +876,7 @@ private static void writeParquetTablesImpl(@NotNull final Table[] sources, for (int tableIdx = 0; tableIdx < sources.length; tableIdx++) { final File tableDestination = destinations[tableIdx]; final List indexInfoList = - indexInfoBuilderHelper(indexColumnArr, parquetColumnNameArr, tableDestination); + indexInfoBuilderHelper(indexColumns, parquetColumnNameArr, tableDestination); indexInfoLists.add(indexInfoList); shadowFiles.add(shadowDestFiles[tableIdx]); @@ -1058,15 +946,15 @@ private static void writeParquetTablesImpl(@NotNull final Table[] sources, } /** - * Examine the source tables to retrieve the list of indexes as String[] arrays. + * Examine the source tables to retrieve the list of indexes as String lists. * * @param sources The tables from which to retrieve the indexes - * @return An array containing the indexes as String[] arrays + * @return A {@link Collection} containing the indexes as String lists * @implNote This only examines the first source table. The writing code will compute missing indexes for the other * source tables. */ @NotNull - private static String[][] indexedColumnNames(@NotNull final Table @NotNull [] sources) { + private static Collection> indexedColumnNames(@NotNull final Table @NotNull [] sources) { if (sources.length == 0) { return EMPTY_INDEXES; } @@ -1075,13 +963,13 @@ private static String[][] indexedColumnNames(@NotNull final Table @NotNull [] so } /** - * Examine the source table to retrieve the list of indexes as String[] arrays. + * Examine the source table to retrieve the list of indexes as String lists. * * @param source The table from which to retrieve the indexes - * @return An array containing the indexes as String[] arrays. + * @return A {@link Collection} containing the indexes as String lists. */ @NotNull - private static String[][] indexedColumnNames(@NotNull final Table source) { + private static Collection> indexedColumnNames(@NotNull final Table source) { final DataIndexer dataIndexer = DataIndexer.existingOf(source.getRowSet()); if (dataIndexer == null) { return EMPTY_INDEXES; @@ -1095,24 +983,24 @@ private static String[][] indexedColumnNames(@NotNull final Table source) { final Map, String> columnToName = nameToColumn.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); - final List indexesToWrite = new ArrayList<>(); + final Collection> indexesToWrite = new ArrayList<>(); // Build the list of indexes to write dataIndexes.forEach(di -> { final Map, String> keyColumnNamesByIndexedColumn = di.keyColumnNamesByIndexedColumn(); // Re-map the index columns to their names in this table - final String[] keyColumnNames = keyColumnNamesByIndexedColumn.keySet().stream() + final List keyColumnNames = keyColumnNamesByIndexedColumn.keySet().stream() .map(columnToName::get) .filter(Objects::nonNull) - .toArray(String[]::new); + .collect(Collectors.toUnmodifiableList()); // Make sure all the columns actually exist in the table - if (keyColumnNames.length == keyColumnNamesByIndexedColumn.size()) { + if (keyColumnNames.size() == keyColumnNamesByIndexedColumn.size()) { indexesToWrite.add(keyColumnNames); } }); - return indexesToWrite.toArray(String[][]::new); + return Collections.unmodifiableCollection(indexesToWrite); } /** @@ -1121,12 +1009,69 @@ private static String[][] indexedColumnNames(@NotNull final Table source) { * @param sources source tables * @param definition table definition * @param destinations destinations + * @deprecated Use {@link #writeTables(Table[], String[], ParquetInstructions)} instead with {@link TableDefinition} + * provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ + @Deprecated public static void writeTables( @NotNull final Table[] sources, @NotNull final TableDefinition definition, @NotNull final File[] destinations) { - writeParquetTables(sources, definition, ParquetInstructions.EMPTY, destinations, indexedColumnNames(sources)); + final String[] destinationPaths = Arrays.stream(destinations).map(File::getPath).toArray(String[]::new); + writeTables(sources, destinationPaths, ParquetInstructions.EMPTY.withTableDefinition(definition)); + } + + /** + * Write out tables to disk. Data indexes to write are determined by those already present on the first source or + * those provided through {@link ParquetInstructions.Builder#addIndexColumns}. The {@link TableDefinition} to use + * for writing must be provided as part of {@link ParquetInstructions}. + * + * @param sources The tables to write + * @param destinations The destination paths or URIs. Any non-existing directories in the paths provided are + * created. If there is an error, any intermediate directories previously created are removed; note this + * makes this method unsafe for concurrent use. + * @param writeInstructions Write instructions for customizations while writing + */ + public static void writeTables( + @NotNull final Table[] sources, + @NotNull final String[] destinations, + @NotNull final ParquetInstructions writeInstructions) { + final Collection> indexColumns = + writeInstructions.getIndexColumns().orElseGet(() -> indexedColumnNames(sources)); + final TableDefinition definition = writeInstructions.getTableDefinition().orElseThrow( + () -> new IllegalArgumentException("Table definition must be provided")); + final File[] destinationFiles = new File[destinations.length]; + for (int i = 0; i < destinations.length; i++) { + final URI destinationURI = convertToURI(destinations[i], false); + if (!FILE_URI_SCHEME.equals(destinationURI.getScheme())) { + throw new IllegalArgumentException( + "Only file URI scheme is supported for writing parquet files, found" + + "non-file URI: " + destinations[i]); + } + destinationFiles[i] = new File(destinationURI); + } + final File metadataRootDir; + if (writeInstructions.generateMetadataFiles()) { + // We insist on writing the metadata file in the same directory as the destination files, thus all + // destination files should be in the same directory. + final String firstDestinationDir = destinationFiles[0].getAbsoluteFile().getParentFile().getAbsolutePath(); + for (int i = 1; i < destinations.length; i++) { + if (!firstDestinationDir.equals(destinationFiles[i].getParentFile().getAbsolutePath())) { + throw new IllegalArgumentException("All destination files must be in the same directory for " + + " generating metadata files"); + } + } + metadataRootDir = new File(firstDestinationDir); + } else { + metadataRootDir = null; + } + + final Map> computedCache = + buildComputedCache(() -> PartitionedTableFactory.ofTables(definition, sources).merge(), definition); + // We do not have any additional schema for partitioning columns in this case. Schema for all columns will be + // generated at the time of writing the parquet files and merged to generate the metadata files. + writeTablesImpl(sources, definition, writeInstructions, destinationFiles, indexColumns, null, metadataRootDir, + computedCache); } /** @@ -1136,13 +1081,17 @@ public static void writeTables( * @param definition table definition * @param destinations destinations * @param writeInstructions instructions for customizations while writing + * @deprecated Use {@link #writeTables(Table[], String[], ParquetInstructions)} instead with {@link TableDefinition} + * provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ + @Deprecated public static void writeTables( @NotNull final Table[] sources, @NotNull final TableDefinition definition, @NotNull final File[] destinations, @NotNull final ParquetInstructions writeInstructions) { - writeParquetTables(sources, definition, writeInstructions, destinations, indexedColumnNames(sources)); + final String[] destinationPaths = Arrays.stream(destinations).map(File::getPath).toArray(String[]::new); + writeTables(sources, destinationPaths, ensureTableDefinition(writeInstructions, definition, true)); } /** @@ -1156,48 +1105,42 @@ public static void deleteTable(File path) { } /** - * This method attempts to "do the right thing." It examines the source to determine if it's a single parquet file, - * a metadata file, or a directory. If it's a directory, it additionally tries to guess the layout to use. Unless a - * metadata file is supplied or discovered in the directory, the highest (by {@link ParquetTableLocationKey location - * key} order) location found will be used to infer schema. + * This method attempts to "do the right thing." It examines the source file URI to determine if it's a single + * parquet file, a metadata file, or a directory. If it's a directory, it additionally tries to guess the layout to + * use. Unless a metadata file is supplied or discovered in the directory, the highest (by + * {@link ParquetTableLocationKey location key} order) location found will be used to infer schema. * - *

- * Delegates to one of {@link #readSingleFileTable(File, ParquetInstructions)}, - * {@link #readPartitionedTableWithMetadata(File, ParquetInstructions)}, - * {@link #readFlatPartitionedTable(File, ParquetInstructions)}, or - * {@link #readKeyValuePartitionedTable(File, ParquetInstructions)}. - * - * @param source The source file or directory + * @param source The source URI with {@value ParquetFileReader#FILE_URI_SCHEME} scheme * @param instructions Instructions for reading * @return A {@link Table} */ - private static Table readTableInternal( - @NotNull final File source, + private static Table readTableFromFileUri( + @NotNull final URI source, @NotNull final ParquetInstructions instructions) { - final Path sourcePath = source.toPath(); + final Path sourcePath = Path.of(source); 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); } + final URI parentDirURI = convertToURI(sourcePath.getParent(), true); if (sourceFileName.equals(METADATA_FILE_NAME)) { - return readPartitionedTableWithMetadata(sourceFile.getParentFile(), instructions); + return readPartitionedTableWithMetadata(parentDirURI, instructions); } if (sourceFileName.equals(COMMON_METADATA_FILE_NAME)) { - return readPartitionedTableWithMetadata(sourceFile.getParentFile(), instructions); + return readPartitionedTableWithMetadata(parentDirURI, instructions); } throw new TableDataException( - "Source file " + sourceFile + " does not appear to be a parquet file or metadata file"); + "Source file " + source + " does not appear to be a parquet file or metadata file"); } if (sourceAttr.isDirectory()) { final Path metadataPath = sourcePath.resolve(METADATA_FILE_NAME); if (Files.exists(metadataPath)) { - return readPartitionedTableWithMetadata(sourceFile, instructions); + return readPartitionedTableWithMetadata(source, instructions); } final Path firstEntryPath; // Ignore dot files while looking for the first entry @@ -1214,43 +1157,16 @@ private static Table readTableInternal( final String firstEntryFileName = firstEntryPath.getFileName().toString(); final BasicFileAttributes firstEntryAttr = readAttributes(firstEntryPath); if (firstEntryAttr.isDirectory() && firstEntryFileName.contains("=")) { - return readKeyValuePartitionedTable(sourceFile, instructions); + return readKeyValuePartitionedTable(source, instructions); } if (firstEntryAttr.isRegularFile() && firstEntryFileName.endsWith(PARQUET_FILE_EXTENSION)) { - return readFlatPartitionedTable(sourceFile, instructions); + return readFlatPartitionedTable(source, instructions); } throw new TableDataException("No recognized Parquet table layout found in " + source); } throw new TableDataException("Source " + source + " is neither a directory nor a regular file"); } - /** - * Similar to {@link #readTableInternal(File, ParquetInstructions)} but with a string source. - * - * @param source The source path or URI - * @param instructions Instructions for reading - * @return A {@link Table} - */ - private static Table readTableInternal( - @NotNull final String source, - @NotNull final ParquetInstructions instructions) { - final boolean isDirectory = !source.endsWith(PARQUET_FILE_EXTENSION); - final URI sourceURI = convertToURI(source, isDirectory); - if (FILE_URI_SCHEME.equals(sourceURI.getScheme())) { - return readTableInternal(new File(sourceURI), instructions); - } - if (!isDirectory) { - return readSingleFileTable(sourceURI, instructions); - } - if (source.endsWith(METADATA_FILE_NAME) || source.endsWith(COMMON_METADATA_FILE_NAME)) { - throw new UncheckedDeephavenException("We currently do not support reading parquet metadata files " + - "from non local storage"); - } - // Both flat partitioned and key-value partitioned data can be read under key-value partitioned layout - return readPartitionedTable(new ParquetKeyValuePartitionedLayout(sourceURI, MAX_PARTITIONING_LEVELS_INFERENCE, - instructions), instructions); - } - private static boolean ignoreDotFiles(Path path) { final String filename = path.getFileName().toString(); return !filename.isEmpty() && filename.charAt(0) != '.'; @@ -1265,24 +1181,27 @@ private static BasicFileAttributes readAttributes(@NotNull final Path path) { } /** - * Reads in a table from a single parquet file using the provided table definition. + * Reads in a table from a single parquet file using the table definition provided through the + * {@link ParquetInstructions}. * *

- * Callers may prefer the simpler methods {@link #readSingleFileTable(File, ParquetInstructions)} or - * {@link #readSingleFileTable(File, ParquetInstructions, TableDefinition)}. + * Callers may prefer the simpler methods {@link #readTable(String, ParquetInstructions)} with layout provided as + * {@link ParquetFileLayout#SINGLE_FILE} using {@link ParquetInstructions.Builder#setFileLayout} and + * {@link TableDefinition} provided through {@link ParquetInstructions.Builder#setTableDefinition}. * * @param tableLocationKey The {@link ParquetTableLocationKey location keys} to include * @param readInstructions Instructions for customizations while reading - * @param tableDefinition The table's {@link TableDefinition definition} * @return The table */ - public static Table readSingleFileTable( + private static Table readTable( @NotNull final ParquetTableLocationKey tableLocationKey, - @NotNull final ParquetInstructions readInstructions, - @NotNull final TableDefinition tableDefinition) { + @NotNull final ParquetInstructions readInstructions) { if (readInstructions.isRefreshing()) { throw new IllegalArgumentException("Unable to have a refreshing single parquet file"); } + final TableDefinition tableDefinition = readInstructions.getTableDefinition().orElseThrow( + () -> new IllegalArgumentException("Table definition must be provided")); + verifyFileLayout(readInstructions, ParquetFileLayout.SINGLE_FILE); final TableLocationProvider locationProvider = new PollingTableLocationProvider<>( StandaloneTableKey.getInstance(), new KnownLocationKeyFinder<>(tableLocationKey), @@ -1294,91 +1213,179 @@ public static Table readSingleFileTable( } /** - * Reads in a table from files discovered with {@code locationKeyFinder} using a definition built from the highest - * (by {@link ParquetTableLocationKey location key} order) location found, which must have non-null partition values - * for all partition keys. + * Reads in a table from a single parquet file using the table definition provided through the + * {@link ParquetInstructions}. * - * @param locationKeyFinder The source of {@link ParquetTableLocationKey location keys} to include + * @param tableLocationKey The {@link ParquetTableLocationKey location keys} to include * @param readInstructions Instructions for customizations while reading * @return The table + * + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#SINGLE_FILE} using {@link ParquetInstructions.Builder#setFileLayout}. */ - public static Table readPartitionedTable( - @NotNull final TableLocationKeyFinder locationKeyFinder, + @Deprecated + public static Table readSingleFileTable( + @NotNull final ParquetTableLocationKey tableLocationKey, @NotNull final ParquetInstructions readInstructions) { - final KnownLocationKeyFinder inferenceKeys = toKnownKeys(locationKeyFinder); - final Pair inference = infer(inferenceKeys, readInstructions); - return readPartitionedTable( - // In the case of a static output table, we can re-use the already fetched inference keys - readInstructions.isRefreshing() ? locationKeyFinder : inferenceKeys, - inference.getSecond(), - inference.getFirst()); + return readTable(tableLocationKey, readInstructions); } /** - * Reads in a table from files discovered with {@code locationKeyFinder} using the provided table definition. + * Reads in a table from a single parquet file using the provided table definition. * - * @param locationKeyFinder The source of {@link ParquetTableLocationKey location keys} to include + * @param tableLocationKey The {@link ParquetTableLocationKey location keys} to include * @param readInstructions Instructions for customizations while reading * @param tableDefinition The table's {@link TableDefinition definition} * @return The table + * + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#SINGLE_FILE} using {@link ParquetInstructions.Builder#setFileLayout} and + * {@link TableDefinition} provided through {@link ParquetInstructions.Builder#setTableDefinition} */ - public static Table readPartitionedTable( - @NotNull final TableLocationKeyFinder locationKeyFinder, + @Deprecated + public static Table readSingleFileTable( + @NotNull final ParquetTableLocationKey tableLocationKey, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { + return readTable(tableLocationKey, ensureTableDefinition(readInstructions, tableDefinition, true)); + } + + /** + * Reads in a table from files discovered with {@code locationKeyFinder} using a definition either provided using + * {@link ParquetInstructions} or built from the highest (by {@link ParquetTableLocationKey location key} order) + * location found, which must have non-null partition values for all partition keys. + * + *

+ * Callers may prefer the simpler methods {@link #readTable(String, ParquetInstructions)} with layout provided using + * {@link ParquetInstructions.Builder#setFileLayout}. + * + * @param locationKeyFinder The source of {@link ParquetTableLocationKey location keys} to include + * @param readInstructions Instructions for customizations while reading + * @return The table + */ + public static Table readTable( + @NotNull final TableLocationKeyFinder locationKeyFinder, + @NotNull final ParquetInstructions readInstructions) { + final TableDefinition definition; + final ParquetInstructions useInstructions; + final TableLocationKeyFinder useLocationKeyFinder; + if (readInstructions.getTableDefinition().isEmpty()) { + // Infer the definition + final KnownLocationKeyFinder inferenceKeys = toKnownKeys(locationKeyFinder); + final Pair inference = infer(inferenceKeys, readInstructions); + // In the case of a static output table, we can re-use the already fetched inference keys + useLocationKeyFinder = readInstructions.isRefreshing() ? locationKeyFinder : inferenceKeys; + definition = inference.getFirst(); + useInstructions = inference.getSecond(); + } else { + definition = readInstructions.getTableDefinition().get(); + useInstructions = readInstructions; + useLocationKeyFinder = locationKeyFinder; + } final String description; final TableLocationKeyFinder keyFinder; final TableDataRefreshService refreshService; final UpdateSourceRegistrar updateSourceRegistrar; - if (readInstructions.isRefreshing()) { - keyFinder = locationKeyFinder; + if (useInstructions.isRefreshing()) { + keyFinder = useLocationKeyFinder; description = "Read refreshing parquet files with " + keyFinder; refreshService = TableDataRefreshService.getSharedRefreshService(); updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); } else { - keyFinder = toKnownKeys(locationKeyFinder); + keyFinder = toKnownKeys(useLocationKeyFinder); description = "Read multiple parquet files with " + keyFinder; refreshService = null; updateSourceRegistrar = null; } return new PartitionAwareSourceTable( - tableDefinition, + definition, description, RegionedTableComponentFactoryImpl.INSTANCE, new PollingTableLocationProvider<>( StandaloneTableKey.getInstance(), keyFinder, - new ParquetTableLocationFactory(readInstructions), + new ParquetTableLocationFactory(useInstructions), refreshService), updateSourceRegistrar); } + /** + * Reads in a table from files discovered with {@code locationKeyFinder} using a definition either provided using + * {@link ParquetInstructions} or built from the highest (by {@link ParquetTableLocationKey location key} order) + * location found, which must have non-null partition values for all partition keys. + * + *

+ * Callers may prefer the simpler methods {@link #readTable(String, ParquetInstructions)} with layout provided using + * {@link ParquetInstructions.Builder#setFileLayout}. + * + * @param locationKeyFinder The source of {@link ParquetTableLocationKey location keys} to include + * @param readInstructions Instructions for customizations while reading + * @return The table + * + * @deprecated use {@link #readTable(TableLocationKeyFinder, ParquetInstructions)} instead + */ + @Deprecated + public static Table readPartitionedTable( + @NotNull final TableLocationKeyFinder locationKeyFinder, + @NotNull final ParquetInstructions readInstructions) { + return readTable(locationKeyFinder, readInstructions); + } + + /** + * Reads in a table from files discovered with {@code locationKeyFinder} using the provided table definition. + * + *

+ * Callers may prefer the simpler methods {@link #readTable(String, ParquetInstructions)} with layout provided using + * {@link ParquetInstructions.Builder#setFileLayout} and {@link TableDefinition} provided through + * {@link ParquetInstructions.Builder#setTableDefinition}. + * + * @param locationKeyFinder The source of {@link ParquetTableLocationKey location keys} to include + * @param readInstructions Instructions for customizations while reading + * @param tableDefinition The table's {@link TableDefinition definition} + * @return The table + * + * @deprecated use {@link #readTable(TableLocationKeyFinder, ParquetInstructions)} instead with the table definition + * provided through {@link ParquetInstructions.Builder#setTableDefinition}. + */ + @Deprecated + public static Table readPartitionedTable( + @NotNull final TableLocationKeyFinder locationKeyFinder, + @NotNull final ParquetInstructions readInstructions, + @NotNull final TableDefinition tableDefinition) { + return readTable(locationKeyFinder, ensureTableDefinition(readInstructions, tableDefinition, true)); + } + /** * Reads in a table from files discovered with {@code locationKeyFinder} using a definition built from the highest * (by {@link ParquetTableLocationKey location key} order) location found, which must have non-null partition values * for all partition keys. * + *

+ * Callers may prefer the simpler methods {@link #readTable(String, ParquetInstructions)} with layout provided using + * {@link ParquetInstructions.Builder#setFileLayout}. + * * @param locationKeyFinder The source of {@link ParquetTableLocationKey location keys} to include * @param readInstructions Instructions for customizations while reading * @return The table - * @deprecated use {@link #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions)} + * @deprecated use {@link #readTable(TableLocationKeyFinder, ParquetInstructions)} */ @Deprecated public static Table readPartitionedTableInferSchema( @NotNull final TableLocationKeyFinder locationKeyFinder, @NotNull final ParquetInstructions readInstructions) { - return readPartitionedTable(locationKeyFinder, readInstructions); + return readTable(locationKeyFinder, readInstructions); } private static Pair infer( - KnownLocationKeyFinder inferenceKeys, ParquetInstructions readInstructions) { + KnownLocationKeyFinder inferenceKeys, + ParquetInstructions readInstructions) { // TODO(deephaven-core#877): Support schema merge when discovering multiple parquet files final ParquetTableLocationKey lastKey = inferenceKeys.getLastKey().orElse(null); if (lastKey == null) { throw new IllegalArgumentException( "Unable to infer schema for a partitioned parquet table when there are no initial parquet files"); } - final Pair>, ParquetInstructions> schemaInfo = convertSchema( + final Pair>, ParquetInstructions> schemaInfo = ParquetSchemaReader.convertSchema( lastKey.getFileReader().getSchema(), lastKey.getMetadata().getFileMetaData().getKeyValueMetaData(), readInstructions); @@ -1417,15 +1424,68 @@ private static KnownLocationKeyFinder toKnownKeys( /** * Reads in a table using metadata files found in the supplied directory. * - * @param directory The source of {@link ParquetTableLocationKey location keys} to include + * @param directory the path for the root directory to search for .parquet files * @param readInstructions Instructions for customizations while reading * @return The table + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#METADATA_PARTITIONED} using + * {@link ParquetInstructions.Builder#setFileLayout}. */ + @Deprecated public static Table readPartitionedTableWithMetadata( @NotNull final File directory, @NotNull final ParquetInstructions readInstructions) { + return readPartitionedTableWithMetadata(convertToURI(directory, true), readInstructions); + } + + /** + * Reads in a table using metadata files found in the supplied directory. + * + * @param directory the path or URI for the root directory to search for .parquet files + * @param readInstructions Instructions for customizations while reading + * @return The table + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#METADATA_PARTITIONED} using + * {@link ParquetInstructions.Builder#setFileLayout}. + */ + @Deprecated + public static Table readPartitionedTableWithMetadata( + @NotNull final String directory, + @NotNull final ParquetInstructions readInstructions) { + return readPartitionedTableWithMetadata(convertToURI(directory, true), readInstructions); + } + + private static Table readPartitionedTableWithMetadata( + @NotNull final URI sourceURI, + @NotNull final ParquetInstructions readInstructions) { + if (!FILE_URI_SCHEME.equals(sourceURI.getScheme())) { + throw new UnsupportedOperationException("Reading metadata files from non local storage is not supported"); + } + verifyFileLayout(readInstructions, ParquetFileLayout.METADATA_PARTITIONED); + if (readInstructions.getTableDefinition().isPresent()) { + throw new UnsupportedOperationException("Detected table definition inside read instructions, reading " + + "metadata files with custom table definition is currently not supported"); + } + final File sourceFile = new File(sourceURI); + final String fileName = sourceFile.getName(); + final File directory; + if (fileName.equals(METADATA_FILE_NAME) || fileName.equals(COMMON_METADATA_FILE_NAME)) { + directory = sourceFile.getParentFile(); + } else { + directory = sourceFile; + } final ParquetMetadataFileLayout layout = new ParquetMetadataFileLayout(directory, readInstructions); - return readPartitionedTable(layout, layout.getInstructions(), layout.getTableDefinition()); + return readTable(layout, + ensureTableDefinition(layout.getInstructions(), layout.getTableDefinition(), true)); + } + + private static void verifyFileLayout( + @NotNull final ParquetInstructions readInstructions, + @NotNull final ParquetFileLayout expectedLayout) { + if (readInstructions.getFileLayout().isPresent() && readInstructions.getFileLayout().get() != expectedLayout) { + throw new IllegalArgumentException("File layout provided in read instructions (=" + + readInstructions.getFileLayout() + ") does not match with " + expectedLayout); + } } /** @@ -1439,82 +1499,62 @@ public static Table readPartitionedTableWithMetadata( * @param directory the root directory to search for .parquet files * @param readInstructions the instructions for customizations while reading * @return the table - * @see ParquetKeyValuePartitionedLayout#ParquetKeyValuePartitionedLayout(File, int, ParquetInstructions) - * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions) + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#KV_PARTITIONED} using {@link ParquetInstructions.Builder#setFileLayout}. */ + @Deprecated public static Table readKeyValuePartitionedTable( @NotNull final File directory, @NotNull final ParquetInstructions readInstructions) { - return readPartitionedTable(new ParquetKeyValuePartitionedLayout(directory, MAX_PARTITIONING_LEVELS_INFERENCE, - readInstructions), readInstructions); + return readKeyValuePartitionedTable(convertToURI(directory, true), readInstructions); } /** * Creates a partitioned table via the key-value partitioned parquet files from the root {@code directory}, * inferring the table definition from those files. - * *

- * Callers wishing to be more explicit and skip the inference step may prefer to call - * {@link #readKeyValuePartitionedTable(String, ParquetInstructions, TableDefinition)}. + * Callers wishing to be more explicit and skip the inference step should provide a {@link TableDefinition} as part + * of read instructions using {@link ParquetInstructions.Builder#setTableDefinition}. * - * @param directory the path or URI for the root directory to search for .parquet files + * @param directoryUri the URI for the root directory to search for .parquet files * @param readInstructions the instructions for customizations while reading * @return the table - * @see ParquetKeyValuePartitionedLayout#ParquetKeyValuePartitionedLayout(URI, int, ParquetInstructions) - * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions) */ - @Deprecated - public static Table readKeyValuePartitionedTable( - @NotNull final String directory, + private static Table readKeyValuePartitionedTable( + @NotNull final URI directoryUri, @NotNull final ParquetInstructions readInstructions) { - return readPartitionedTable(new ParquetKeyValuePartitionedLayout(convertToURI(directory, true), - MAX_PARTITIONING_LEVELS_INFERENCE, readInstructions), readInstructions); - } - - /** - * Creates a partitioned table via the key-value partitioned parquet files from the root {@code directory} using the - * provided {@code tableDefinition}. - * - * @param directory the root directory to search for .parquet files - * @param readInstructions the instructions for customizations while reading - * @param tableDefinition the table definition - * @return the table - * @see ParquetKeyValuePartitionedLayout#ParquetKeyValuePartitionedLayout(File, TableDefinition, - * ParquetInstructions) - * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions, TableDefinition) - */ - public static Table readKeyValuePartitionedTable( - @NotNull final File directory, - @NotNull final ParquetInstructions readInstructions, - @NotNull final TableDefinition tableDefinition) { + verifyFileLayout(readInstructions, ParquetFileLayout.KV_PARTITIONED); + if (readInstructions.getTableDefinition().isEmpty()) { + return readTable(new ParquetKeyValuePartitionedLayout(directoryUri, + MAX_PARTITIONING_LEVELS_INFERENCE, readInstructions), readInstructions); + } + final TableDefinition tableDefinition = readInstructions.getTableDefinition().get(); if (tableDefinition.getColumnStream().noneMatch(ColumnDefinition::isPartitioning)) { throw new IllegalArgumentException("No partitioning columns"); } - return readPartitionedTable(new ParquetKeyValuePartitionedLayout(directory, tableDefinition, readInstructions), - readInstructions, tableDefinition); + return readTable(new ParquetKeyValuePartitionedLayout(directoryUri, tableDefinition, + readInstructions), readInstructions); } /** * Creates a partitioned table via the key-value partitioned parquet files from the root {@code directory} using the * provided {@code tableDefinition}. * - * @param directory the path or URI for the root directory to search for .parquet files + * @param directory the root directory to search for .parquet files * @param readInstructions the instructions for customizations while reading * @param tableDefinition the table definition * @return the table - * @see ParquetKeyValuePartitionedLayout#ParquetKeyValuePartitionedLayout(URI, TableDefinition, ParquetInstructions) - * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions, TableDefinition) + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#KV_PARTITIONED} using {@link ParquetInstructions.Builder#setFileLayout} and + * {@link TableDefinition} provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ @Deprecated public static Table readKeyValuePartitionedTable( - @NotNull final String directory, + @NotNull final File directory, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { - if (tableDefinition.getColumnStream().noneMatch(ColumnDefinition::isPartitioning)) { - throw new IllegalArgumentException("No partitioning columns"); - } - return readPartitionedTable(new ParquetKeyValuePartitionedLayout(convertToURI(directory, true), tableDefinition, - readInstructions), readInstructions, tableDefinition); + return readKeyValuePartitionedTable(convertToURI(directory, true), + ensureTableDefinition(readInstructions, tableDefinition, true)); } /** @@ -1528,35 +1568,31 @@ public static Table readKeyValuePartitionedTable( * @param directory the directory to search for .parquet files * @param readInstructions the instructions for customizations while reading * @return the table - * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions) - * @see ParquetFlatPartitionedLayout#ParquetFlatPartitionedLayout(File, ParquetInstructions) + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#FLAT_PARTITIONED} using {@link ParquetInstructions.Builder#setFileLayout}. */ + @Deprecated public static Table readFlatPartitionedTable( @NotNull final File directory, @NotNull final ParquetInstructions readInstructions) { - return readPartitionedTable(new ParquetFlatPartitionedLayout(directory, readInstructions), readInstructions); + return readFlatPartitionedTable(convertToURI(directory, true), readInstructions); } /** * Creates a partitioned table via the flat parquet files from the root {@code directory}, inferring the table * definition from those files. * - *

- * Callers wishing to be more explicit and skip the inference step may prefer to call - * {@link #readFlatPartitionedTable(String, ParquetInstructions, TableDefinition)}. - * - * @param directory the path or URI for the directory to search for .parquet files + * @param sourceURI the path or URI for the directory to search for .parquet files * @param readInstructions the instructions for customizations while reading * @return the table - * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions) + * @see #readTable(TableLocationKeyFinder, ParquetInstructions) * @see ParquetFlatPartitionedLayout#ParquetFlatPartitionedLayout(URI, ParquetInstructions) */ - @Deprecated - public static Table readFlatPartitionedTable( - @NotNull final String directory, + private static Table readFlatPartitionedTable( + @NotNull final URI sourceURI, @NotNull final ParquetInstructions readInstructions) { - return readPartitionedTable(new ParquetFlatPartitionedLayout(convertToURI(directory, true), readInstructions), - readInstructions); + verifyFileLayout(readInstructions, ParquetFileLayout.FLAT_PARTITIONED); + return readTable(new ParquetFlatPartitionedLayout(sourceURI, readInstructions), readInstructions); } /** @@ -1567,35 +1603,17 @@ public static Table readFlatPartitionedTable( * @param readInstructions the instructions for customizations while reading * @param tableDefinition the table definition * @return the table - * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions, TableDefinition) - * @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), readInstructions, - tableDefinition); - } - - /** - * Creates a partitioned table via the flat parquet files from the root {@code directory} using the provided - * {@code tableDefinition}. - * - * @param directory the path or URI for the directory to search for .parquet files - * @param readInstructions the instructions for customizations while reading - * @param tableDefinition the table definition - * @return the table - * @see #readPartitionedTable(TableLocationKeyFinder, ParquetInstructions, TableDefinition) - * @see ParquetFlatPartitionedLayout#ParquetFlatPartitionedLayout(URI, ParquetInstructions) + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#FLAT_PARTITIONED} using {@link ParquetInstructions.Builder#setFileLayout} + * and {@link TableDefinition} provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ @Deprecated public static Table readFlatPartitionedTable( - @NotNull final String directory, + @NotNull final File directory, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { - return readPartitionedTable(new ParquetFlatPartitionedLayout(convertToURI(directory, true), readInstructions), - readInstructions, tableDefinition); + return readFlatPartitionedTable(convertToURI(directory, true), + ensureTableDefinition(readInstructions, tableDefinition, true)); } /** @@ -1608,9 +1626,10 @@ public static Table readFlatPartitionedTable( * @param file the parquet file * @param readInstructions the instructions for customizations while reading * @return the table - * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) - * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#SINGLE_FILE} using {@link ParquetInstructions.Builder#setFileLayout}. */ + @Deprecated public static Table readSingleFileTable( @NotNull final File file, @NotNull final ParquetInstructions readInstructions) { @@ -1627,9 +1646,10 @@ public static Table readSingleFileTable( * @param source the path or URI for the parquet file * @param readInstructions the instructions for customizations while reading * @return the table - * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) - * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#SINGLE_FILE} using {@link ParquetInstructions.Builder#setFileLayout}. */ + @Deprecated public static Table readSingleFileTable( @NotNull final String source, @NotNull final ParquetInstructions readInstructions) { @@ -1639,11 +1659,20 @@ public static Table readSingleFileTable( private static Table readSingleFileTable( @NotNull final URI parquetFileURI, @NotNull final ParquetInstructions readInstructions) { - final ParquetSingleFileLayout keyFinder = new ParquetSingleFileLayout(parquetFileURI, readInstructions); - final KnownLocationKeyFinder inferenceKeys = toKnownKeys(keyFinder); + verifyFileLayout(readInstructions, ParquetFileLayout.SINGLE_FILE); + if (readInstructions.getTableDefinition().isPresent()) { + return readTable(new ParquetTableLocationKey(parquetFileURI, 0, null, readInstructions), + readInstructions); + } + // Infer the table definition + final TableLocationKeyFinder singleFileLayout = + new ParquetSingleFileLayout(parquetFileURI, readInstructions); + final KnownLocationKeyFinder inferenceKeys = toKnownKeys(singleFileLayout); final Pair inference = infer(inferenceKeys, readInstructions); - return readSingleFileTable(inferenceKeys.getFirstKey().orElseThrow(), inference.getSecond(), - inference.getFirst()); + final TableDefinition inferredTableDefinition = inference.getFirst(); + final ParquetInstructions inferredInstructions = inference.getSecond(); + return readTable(inferenceKeys.getFirstKey().orElseThrow(), + ensureTableDefinition(inferredInstructions, inferredTableDefinition, true)); } /** @@ -1653,14 +1682,17 @@ private static Table readSingleFileTable( * @param readInstructions the instructions for customizations while reading * @param tableDefinition the table definition * @return the table - * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) - * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#SINGLE_FILE} using {@link ParquetInstructions.Builder#setFileLayout} and + * {@link TableDefinition} provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ + @Deprecated public static Table readSingleFileTable( @NotNull final File file, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { - return readSingleFileTable(convertToURI(file, false), readInstructions, tableDefinition); + return readSingleFileTable(convertToURI(file, false), + ensureTableDefinition(readInstructions, tableDefinition, true)); } /** @@ -1671,187 +1703,36 @@ public static Table readSingleFileTable( * @param readInstructions the instructions for customizations while reading * @param tableDefinition the table definition * @return the table - * @see ParquetTableLocationKey#ParquetTableLocationKey(URI, int, Map, ParquetInstructions) - * @see #readSingleFileTable(ParquetTableLocationKey, ParquetInstructions, TableDefinition) + * @deprecated Use {@link #readTable(String, ParquetInstructions)} instead with layout provided as + * {@link ParquetFileLayout#SINGLE_FILE} using {@link ParquetInstructions.Builder#setFileLayout} and + * {@link TableDefinition} provided through {@link ParquetInstructions.Builder#setTableDefinition}. */ + @Deprecated public static Table readSingleFileTable( @NotNull final String source, @NotNull final ParquetInstructions readInstructions, @NotNull final TableDefinition tableDefinition) { - return readSingleFileTable(convertToURI(source, false), 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( - null, CharVector.class, ByteVector.class, ShortVector.class, IntVector.class, LongVector.class, - FloatVector.class, DoubleVector.class, ObjectVector.class); - - private static Class loadClass(final String colName, final String desc, final String className) { - try { - return ClassUtil.lookupClass(className); - } catch (ClassNotFoundException e) { - throw new UncheckedDeephavenException( - "Column " + colName + " with " + desc + "=" + className + " that can't be found in classloader"); - } - } - - private static ParquetSchemaReader.ColumnDefinitionConsumer makeSchemaReaderConsumer( - final ArrayList> colsOut) { - return (final ParquetSchemaReader.ParquetMessageDefinition parquetColDef) -> { - Class baseType; - if (parquetColDef.baseType == boolean.class) { - baseType = Boolean.class; - } else { - baseType = parquetColDef.baseType; - } - ColumnDefinition colDef; - if (parquetColDef.codecType != null && !parquetColDef.codecType.isEmpty()) { - final Class componentType = - (parquetColDef.codecComponentType != null && !parquetColDef.codecComponentType.isEmpty()) - ? loadClass(parquetColDef.name, "codecComponentType", parquetColDef.codecComponentType) - : null; - final Class dataType = loadClass(parquetColDef.name, "codecType", parquetColDef.codecType); - colDef = ColumnDefinition.fromGenericType(parquetColDef.name, dataType, componentType); - } else if (parquetColDef.dhSpecialType != null) { - if (parquetColDef.dhSpecialType == ColumnTypeInfo.SpecialType.StringSet) { - colDef = ColumnDefinition.fromGenericType(parquetColDef.name, StringSet.class, null); - } else if (parquetColDef.dhSpecialType == ColumnTypeInfo.SpecialType.Vector) { - final Class vectorType = VECTOR_TYPE_MAP.get(baseType); - if (vectorType != null) { - colDef = ColumnDefinition.fromGenericType(parquetColDef.name, vectorType, baseType); - } else { - colDef = ColumnDefinition.fromGenericType(parquetColDef.name, ObjectVector.class, baseType); - } - } else { - throw new UncheckedDeephavenException("Unhandled dbSpecialType=" + parquetColDef.dhSpecialType); - } - } else { - if (parquetColDef.isArray) { - if (baseType == byte.class && parquetColDef.noLogicalType) { - colDef = ColumnDefinition.fromGenericType(parquetColDef.name, byte[].class, byte.class); - } else { - // TODO: ParquetInstruction.loadAsVector - final Class componentType = baseType; - // On Java 12, replace by: dataType = componentType.arrayType(); - final Class dataType = java.lang.reflect.Array.newInstance(componentType, 0).getClass(); - colDef = ColumnDefinition.fromGenericType(parquetColDef.name, dataType, componentType); - } - } else { - colDef = ColumnDefinition.fromGenericType(parquetColDef.name, baseType, null); - } - } - colsOut.add(colDef); - }; - } - - /** - * Make a {@link ParquetFileReader} for the supplied {@link File}. Wraps {@link IOException} as - * {@link TableDataException}. - * - * @param parquetFile The parquet file or the parquet metadata file - * @param readInstructions the instructions for customizations while reading - * @return The new {@link ParquetFileReader} - */ - public static ParquetFileReader getParquetFileReader(@NotNull final File parquetFile, - @NotNull final ParquetInstructions readInstructions) { - try { - return getParquetFileReaderChecked(parquetFile, readInstructions); - } catch (IOException e) { - throw new TableDataException("Failed to create Parquet file reader: " + parquetFile, e); - } - } - - /** - * Make a {@link ParquetFileReader} for the supplied {@link URI}. Wraps {@link IOException} as - * {@link TableDataException}. - * - * @param parquetFileURI The URI for the parquet file or the parquet metadata file - * @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}. - * - * @param parquetFile The parquet file or the parquet metadata file - * @return The new {@link ParquetFileReader} - * @throws IOException if an IO exception occurs - */ - public static ParquetFileReader getParquetFileReaderChecked( - @NotNull final File parquetFile, - @NotNull final ParquetInstructions readInstructions) throws IOException { - return getParquetFileReaderChecked(convertToURI(parquetFile, false), readInstructions); - } - - /** - * Make a {@link ParquetFileReader} for the supplied {@link URI}. - * - * @param parquetFileURI The URI for the parquet file or the parquet metadata file - * @return The new {@link ParquetFileReader} - * @throws IOException if an IO exception occurs - */ - public static ParquetFileReader getParquetFileReaderChecked( - @NotNull final URI parquetFileURI, - @NotNull final ParquetInstructions readInstructions) throws IOException { - final SeekableChannelsProvider provider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader( - parquetFileURI, readInstructions.getSpecialInstructions()); - return new ParquetFileReader(parquetFileURI, new CachedChannelProvider(provider, 1 << 7)); + return readSingleFileTable(convertToURI(source, false), + ensureTableDefinition(readInstructions, tableDefinition, true)); } @VisibleForTesting public static Table readParquetSchemaAndTable( - @NotNull final File source, @NotNull final ParquetInstructions readInstructionsIn, - MutableObject instructionsOut) { + @NotNull final File source, + @NotNull final ParquetInstructions readInstructionsIn, + @Nullable final MutableObject mutableInstructionsOut) { final ParquetTableLocationKey tableLocationKey = new ParquetTableLocationKey(source, 0, null, readInstructionsIn); - final Pair>, ParquetInstructions> schemaInfo = convertSchema( + final Pair>, ParquetInstructions> schemaInfo = ParquetSchemaReader.convertSchema( tableLocationKey.getFileReader().getSchema(), tableLocationKey.getMetadata().getFileMetaData().getKeyValueMetaData(), readInstructionsIn); final TableDefinition def = TableDefinition.of(schemaInfo.getFirst()); - if (instructionsOut != null) { - instructionsOut.setValue(schemaInfo.getSecond()); + final ParquetInstructions instructionsOut = ensureTableDefinition(schemaInfo.getSecond(), def, true); + if (mutableInstructionsOut != null) { + mutableInstructionsOut.setValue(instructionsOut); } - return readSingleFileTable(tableLocationKey, schemaInfo.getSecond(), def); - } - - /** - * Convert schema information from a {@link ParquetMetadata} into {@link ColumnDefinition ColumnDefinitions}. - * - * @param schema Parquet schema. DO NOT RELY ON {@link ParquetMetadataConverter} FOR THIS! USE - * {@link ParquetFileReader}! - * @param keyValueMetadata Parquet key-value metadata map - * @param readInstructionsIn Input conversion {@link ParquetInstructions} - * @return A {@link Pair} with {@link ColumnDefinition ColumnDefinitions} and adjusted {@link ParquetInstructions} - */ - public static Pair>, ParquetInstructions> convertSchema( - @NotNull final MessageType schema, - @NotNull final Map keyValueMetadata, - @NotNull final ParquetInstructions readInstructionsIn) { - final ArrayList> cols = new ArrayList<>(); - final ParquetSchemaReader.ColumnDefinitionConsumer colConsumer = makeSchemaReaderConsumer(cols); - return new Pair<>(cols, ParquetSchemaReader.readParquetSchema( - schema, - keyValueMetadata, - readInstructionsIn, - colConsumer, - (final String colName, final Set takenNames) -> NameValidator.legalizeColumnName(colName, - s -> s.replace(" ", "_"), takenNames))); + return readTable(tableLocationKey, instructionsOut); } public static final ParquetInstructions UNCOMPRESSED = @@ -1880,7 +1761,7 @@ public static Pair>, ParquetInstructions> convertSchema /** * @deprecated Do not use this method, instead pass the above codecs as arguments to - * {@link #writeTable(Table, File, ParquetInstructions)} method + * {@link #writeTable(Table, String, ParquetInstructions)} method */ @Deprecated public static void setDefaultCompressionCodecName(final String compressionCodecName) { 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 6f279f4651c..82eee8e2353 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 @@ -9,7 +9,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.util.PartitionParser; -import io.deephaven.parquet.table.ParquetTools; +import io.deephaven.parquet.table.ParquetSchemaReader; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.parquet.base.ParquetUtils; @@ -99,23 +99,25 @@ public ParquetMetadataFileLayout( if (!metadataFile.exists()) { throw new TableDataException(String.format("Parquet metadata file %s does not exist", metadataFile)); } - final ParquetFileReader metadataFileReader = ParquetTools.getParquetFileReader(metadataFile, inputInstructions); + final ParquetFileReader metadataFileReader = + ParquetFileReader.create(metadataFile, inputInstructions.getSpecialInstructions()); final ParquetMetadataConverter converter = new ParquetMetadataConverter(); final ParquetMetadata metadataFileMetadata = convertMetadata(metadataFile, metadataFileReader, converter); - final Pair>, ParquetInstructions> leafSchemaInfo = ParquetTools.convertSchema( + final Pair>, ParquetInstructions> leafSchemaInfo = ParquetSchemaReader.convertSchema( metadataFileReader.getSchema(), metadataFileMetadata.getFileMetaData().getKeyValueMetaData(), inputInstructions); if (commonMetadataFile != null && commonMetadataFile.exists()) { final ParquetFileReader commonMetadataFileReader = - ParquetTools.getParquetFileReader(commonMetadataFile, inputInstructions); - final Pair>, ParquetInstructions> fullSchemaInfo = ParquetTools.convertSchema( - commonMetadataFileReader.getSchema(), - convertMetadata(commonMetadataFile, commonMetadataFileReader, converter).getFileMetaData() - .getKeyValueMetaData(), - leafSchemaInfo.getSecond()); + ParquetFileReader.create(commonMetadataFile, inputInstructions.getSpecialInstructions()); + final Pair>, ParquetInstructions> fullSchemaInfo = + ParquetSchemaReader.convertSchema( + commonMetadataFileReader.getSchema(), + convertMetadata(commonMetadataFile, commonMetadataFileReader, converter).getFileMetaData() + .getKeyValueMetaData(), + leafSchemaInfo.getSecond()); final List> adjustedColumnDefinitions = new ArrayList<>(); final Map> leafDefinitionsMap = leafSchemaInfo.getFirst().stream().collect(toMap(ColumnDefinition::getName, Function.identity())); 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 99c321eeb34..1cb61e20872 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 @@ -344,8 +344,9 @@ private static Table readDataIndexTable( @NotNull final URI parentFileURI, @NotNull final ParquetTableLocation.IndexFileMetadata indexFileMetaData, @NotNull final ParquetInstructions parquetInstructions) { - final Table indexTable = - ParquetTools.readSingleFileTable(indexFileMetaData.fileURI.toString(), parquetInstructions); + final Table indexTable = ParquetTools.readTable(indexFileMetaData.fileURI.toString(), + parquetInstructions.withTableDefinitionAndLayout(null, + ParquetInstructions.ParquetFileLayout.SINGLE_FILE)); if (indexFileMetaData.dataIndexInfo != null) { return indexTable; } 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 a044c309cea..a33cd48f609 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 @@ -5,7 +5,6 @@ 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.parquet.base.ParquetFileReader; @@ -102,7 +101,7 @@ public String getImplementationName() { * * * Callers wishing to handle these cases more explicit may call - * {@link ParquetTools#getParquetFileReaderChecked(URI, ParquetInstructions)}. + * {@link ParquetFileReader#createChecked(URI, Object)}. * * @return true if the file reader exists or was successfully created */ @@ -111,7 +110,7 @@ public synchronized boolean verifyFileReader() { return true; } try { - fileReader = ParquetTools.getParquetFileReaderChecked(uri, readInstructions); + fileReader = ParquetFileReader.createChecked(uri, readInstructions.getSpecialInstructions()); } catch (IOException e) { return false; } @@ -128,7 +127,7 @@ public synchronized ParquetFileReader getFileReader() { if (fileReader != null) { return fileReader; } - return fileReader = ParquetTools.getParquetFileReader(uri, readInstructions); + return fileReader = ParquetFileReader.create(uri, readInstructions.getSpecialInstructions()); } /** 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 8483818dc87..d8005ef598e 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 @@ -78,6 +78,7 @@ import java.time.LocalTime; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Collection; @@ -109,9 +110,9 @@ import static io.deephaven.parquet.table.ParquetTools.readKeyValuePartitionedTable; import static io.deephaven.parquet.table.ParquetTools.readSingleFileTable; import static io.deephaven.parquet.table.ParquetTools.readTable; -import static io.deephaven.parquet.table.ParquetTools.writeParquetTables; import static io.deephaven.parquet.table.ParquetTools.writeKeyValuePartitionedTable; import static io.deephaven.parquet.table.ParquetTools.writeTable; +import static io.deephaven.parquet.table.ParquetTools.writeTables; import static io.deephaven.util.QueryConstants.*; import static org.junit.Assert.*; @@ -151,7 +152,7 @@ public void tearDown() { private static Table getTableFlat(int size, boolean includeSerializable, boolean includeBigDecimal) { ExecutionContext.getContext().getQueryLibrary().importClass(SomeSillyTest.class); - ArrayList columns = + final Collection columns = new ArrayList<>(Arrays.asList("someStringColumn = i % 10 == 0?null:(`` + (i % 101))", "nonNullString = `` + (i % 60)", "nonNullPolyString = `` + (i % 600)", @@ -549,13 +550,70 @@ public void basicParquetWithMetadataTest() { final File metadataFile = new File(rootFile, "_metadata"); assertTrue(metadataFile.exists()); - assertTrue(new File(rootFile, "_common_metadata").exists()); + final File commonMetadataFile = new File(rootFile, "_common_metadata"); + assertTrue(commonMetadataFile.exists()); final Table fromDisk = readTable(destFile); assertTableEquals(table, fromDisk); - final Table fromDiskWithMetadata = readTable(metadataFile); + Table fromDiskWithMetadata = readTable(metadataFile); + assertTableEquals(table, fromDiskWithMetadata); + Table fromDiskWithCommonMetadata = readTable(commonMetadataFile); + assertTableEquals(table, fromDiskWithCommonMetadata); + + final ParquetInstructions readInstructions = ParquetInstructions.builder() + .setFileLayout(ParquetInstructions.ParquetFileLayout.METADATA_PARTITIONED) + .build(); + fromDiskWithMetadata = readTable(metadataFile, readInstructions); assertTableEquals(table, fromDiskWithMetadata); + fromDiskWithCommonMetadata = readTable(commonMetadataFile, readInstructions); + assertTableEquals(table, fromDiskWithCommonMetadata); + + + } + + @Test + public void parquetIndexingBuilderTest() { + final Table source = TableTools.emptyTable(1_000_000).updateView( + "A = (int)(ii%3)", + "B = (double)(ii%2)", + "C = ii"); + DataIndexer.getOrCreateDataIndex(source, "A", "B"); + final File destFile = new File(rootFile, "parquetIndexingBuilderTest.parquet"); + writeTable(source, destFile.getPath()); + Table fromDisk = readTable(destFile.getPath()); + assertTableEquals(source, fromDisk); + verifyIndexingInfoExists(fromDisk, "A", "B"); + + // Set a single column for indexing + ParquetInstructions writeInstructions = ParquetInstructions.builder() + .addIndexColumns("A") + .build(); + writeTable(source, destFile.getPath(), writeInstructions); + fromDisk = readTable(destFile.getPath()); + assertTableEquals(source, fromDisk); + verifyIndexingInfoExists(fromDisk, "A"); + assertFalse(DataIndexer.hasDataIndex(fromDisk, "A", "B")); + assertNull(DataIndexer.getDataIndex(fromDisk, "A", "B")); + + // Clear all indexing columns + writeInstructions = ParquetInstructions.builder() + .addAllIndexColumns(Collections.emptyList()) + .build(); + writeTable(source, destFile.getPath(), writeInstructions); + fromDisk = readTable(destFile.getPath()); + assertFalse(DataIndexer.hasDataIndex(fromDisk, "A", "B")); + + // Set multiple columns for indexing + final Collection> indexColumns = List.of(List.of("A", "C"), List.of("C")); + writeInstructions = ParquetInstructions.builder() + .addAllIndexColumns(indexColumns) + .build(); + writeTable(source, destFile.getPath(), writeInstructions); + fromDisk = readTable(destFile.getPath()); + assertTableEquals(source, fromDisk); + verifyIndexingInfoExists(fromDisk, "A", "C"); + verifyIndexingInfoExists(fromDisk, "C"); } @Test @@ -596,8 +654,9 @@ public void flatPartitionedParquetWithMetadataTest() throws IOException { final File secondDataFile = new File(parentDir, "data2.parquet"); // Write without any metadata files - writeParquetTables(new Table[] {someTable, someTable}, someTable.getDefinition(), ParquetInstructions.EMPTY, - new File[] {firstDataFile, secondDataFile}, null); + writeTables(new Table[] {someTable, someTable}, + new String[] {firstDataFile.getPath(), secondDataFile.getPath()}, + ParquetInstructions.EMPTY.withTableDefinition(someTable.getDefinition())); final Table source = readTable(parentDir).select(); // Now write with metadata files @@ -606,8 +665,9 @@ public void flatPartitionedParquetWithMetadataTest() throws IOException { final ParquetInstructions writeInstructions = ParquetInstructions.builder() .setGenerateMetadataFiles(true) .build(); - writeParquetTables(new Table[] {someTable, someTable}, someTable.getDefinition(), writeInstructions, - new File[] {firstDataFile, secondDataFile}, null); + writeTables(new Table[] {someTable, someTable}, + new String[] {firstDataFile.getPath(), secondDataFile.getPath()}, + writeInstructions.withTableDefinition(someTable.getDefinition())); final Table fromDisk = readTable(parentDir); assertTableEquals(source, fromDisk); @@ -629,8 +689,9 @@ public void flatPartitionedParquetWithMetadataTest() throws IOException { parentDir.delete(); final File updatedSecondDataFile = new File(rootFile, "testDir/data2.parquet"); try { - writeParquetTables(new Table[] {someTable, someTable}, someTable.getDefinition(), writeInstructions, - new File[] {firstDataFile, updatedSecondDataFile}, null); + writeTables(new Table[] {someTable, someTable}, + new String[] {firstDataFile.getPath(), updatedSecondDataFile.getPath()}, + writeInstructions.withTableDefinition(someTable.getDefinition())); fail("Expected exception when writing the metadata files for tables with different parent directories"); } catch (final RuntimeException expected) { } @@ -651,8 +712,8 @@ public void flatPartitionedParquetWithBigDecimalMetadataTest() throws IOExceptio .setGenerateMetadataFiles(true) .build(); final Table[] sources = new Table[] {firstTable, secondTable}; - writeParquetTables(sources, firstTable.getDefinition(), writeInstructions, - new File[] {firstDataFile, secondDataFile}, null); + writeTables(sources, new String[] {firstDataFile.getPath(), secondDataFile.getPath()}, + writeInstructions.withTableDefinition(firstTable.getDefinition())); // Merge the tables and compute the precision and scale as per the union of the two tables final Table expected = @@ -891,7 +952,7 @@ public void writeKeyValuePartitionedDataWithMixedPartitionsTest() { "I = i")) .withDefinitionUnsafe(definition); - // We skip one partitioning and one non partitioing column in the definition, and add some more partitioning + // We skip one partitioning and one non-partitioning column in the definition, and add some more partitioning // and non-partitioning columns final TableDefinition tableDefinitionToWrite = TableDefinition.of( ColumnDefinition.ofInt("PC1").withPartitioning(), @@ -903,8 +964,9 @@ public void writeKeyValuePartitionedDataWithMixedPartitionsTest() { final File parentDir = new File(rootFile, "writeKeyValuePartitionedDataTest"); final ParquetInstructions writeInstructions = ParquetInstructions.builder() .setGenerateMetadataFiles(true) + .setTableDefinition(tableDefinitionToWrite) .build(); - writeKeyValuePartitionedTable(inputData, tableDefinitionToWrite, parentDir.getPath(), writeInstructions); + writeKeyValuePartitionedTable(inputData, parentDir.getPath(), writeInstructions); // Verify that the partitioned data exists for (int PC1 = 0; PC1 <= 2; PC1++) { @@ -1047,7 +1109,7 @@ public void testAllNonPartitioningColumnTypes() { ColumnDefinition.of("NPC13", Type.find(LocalDate.class)), ColumnDefinition.of("NPC14", Type.find(LocalTime.class))); - Table inputData = ((QueryTable) TableTools.emptyTable(10).updateView( + final Table inputData = ((QueryTable) TableTools.emptyTable(10).updateView( "PC1 = (ii%2 == 0) ? `AA` : `BB`", "PC2 = (int)(ii%3)", "NPC1 = (ii%2 == 0) ? `AA` : `BB`", @@ -1071,7 +1133,7 @@ public void testAllNonPartitioningColumnTypes() { .setGenerateMetadataFiles(true) .build(); - // The First API we test is passing the table directly without any table definition + // First we test passing the table directly without any table definition writeKeyValuePartitionedTable(inputData, parentDir.getPath(), writeInstructions); // Store the big decimal with the precision and scale consistent with what we write to parquet @@ -1088,11 +1150,12 @@ public void testAllNonPartitioningColumnTypes() { FileUtils.deleteRecursively(parentDir); } - // Next API we test is to pass additional indexing columns + // Next test passing additional indexing columns final String indexColumn = "NPC5"; - final String[][] indexColumns = new String[][] {{indexColumn}}; + final Collection> indexColumns = Collections.singleton(List.of(indexColumn)); + final ParquetInstructions withIndexColumns = writeInstructions.withIndexColumns(indexColumns); { - writeKeyValuePartitionedTable(inputData, parentDir.getPath(), writeInstructions, indexColumns); + writeKeyValuePartitionedTable(inputData, parentDir.getPath(), withIndexColumns); assertFalse(DataIndexer.hasDataIndex(inputData, indexColumn)); final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); verifyIndexingInfoExists(fromDisk, indexColumn); @@ -1103,7 +1166,7 @@ public void testAllNonPartitioningColumnTypes() { FileUtils.deleteRecursively(parentDir); } - // Next API we test is passing the partitioned table without any table definition + // Next test passing the partitioned table without any table definition final PartitionedTable partitionedTable = inputData.partitionBy("PC1"); { writeKeyValuePartitionedTable(partitionedTable, parentDir.getPath(), writeInstructions); @@ -1115,9 +1178,9 @@ public void testAllNonPartitioningColumnTypes() { FileUtils.deleteRecursively(parentDir); } - // Next API we test is to pass additional indexing columns with partitioned table and no definition + // Next test passing additional indexing columns with partitioned table and no definition { - writeKeyValuePartitionedTable(partitionedTable, parentDir.getPath(), writeInstructions, indexColumns); + writeKeyValuePartitionedTable(partitionedTable, parentDir.getPath(), withIndexColumns); final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); verifyIndexingInfoExists(fromDisk, "NPC5"); assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), fromDisk.sort(partitioningColumns)); @@ -1127,7 +1190,7 @@ public void testAllNonPartitioningColumnTypes() { FileUtils.deleteRecursively(parentDir); } - // Next API we test is passing the regular table with an updated table definition where we drop + // Next test passing the regular table with an updated table definition where we drop // some partitioning columns and non-partitioning columns and add some new non-partitioning columns final List> oldColumns = definition.getColumns(); final List> newColumns = oldColumns.stream() @@ -1135,9 +1198,10 @@ public void testAllNonPartitioningColumnTypes() { .collect(Collectors.toList()); newColumns.add(ColumnDefinition.ofInt("NPC15")); final TableDefinition newDefinition = TableDefinition.of(newColumns); + final ParquetInstructions withDefinition = writeInstructions.withTableDefinition(newDefinition); final Table expected = bigDecimalFixedInputData.dropColumns("PC2", "NPC6").updateView("NPC15 = (int)null"); { - writeKeyValuePartitionedTable(inputData, newDefinition, parentDir.getAbsolutePath(), writeInstructions); + writeKeyValuePartitionedTable(inputData, parentDir.getAbsolutePath(), withDefinition); final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY).select(); assertTableEquals(expected.sort("PC1"), fromDisk.sort("PC1")); final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); @@ -1145,10 +1209,10 @@ public void testAllNonPartitioningColumnTypes() { FileUtils.deleteRecursively(parentDir); } - // Next API to test takes table with updated definition and additional indexing columns + // Next test passing table with updated definition and additional indexing columns + final ParquetInstructions withDefinitionAndIndexColumns = withDefinition.withIndexColumns(indexColumns); { - writeKeyValuePartitionedTable(inputData, newDefinition, parentDir.getAbsolutePath(), writeInstructions, - indexColumns); + writeKeyValuePartitionedTable(inputData, parentDir.getAbsolutePath(), withDefinitionAndIndexColumns); assertFalse(DataIndexer.hasDataIndex(inputData, indexColumn)); final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); verifyIndexingInfoExists(fromDisk, indexColumn); @@ -1158,9 +1222,9 @@ public void testAllNonPartitioningColumnTypes() { FileUtils.deleteRecursively(parentDir); } - // Next API we test is passing the partitioned table with an updated table definition + // Next test passing the partitioned table with an updated table definition { - writeKeyValuePartitionedTable(partitionedTable, newDefinition, parentDir.getPath(), writeInstructions); + writeKeyValuePartitionedTable(partitionedTable, parentDir.getPath(), withDefinition); final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY).select(); assertTableEquals(expected.sort("PC1"), fromDisk.sort("PC1")); final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); @@ -1168,10 +1232,9 @@ public void testAllNonPartitioningColumnTypes() { FileUtils.deleteRecursively(parentDir); } - // Next API we test is passing the indexing columns with partitioned table and an updated table definition + // Next test passing the indexing columns with partitioned table and an updated table definition { - writeKeyValuePartitionedTable(partitionedTable, newDefinition, parentDir.getPath(), writeInstructions, - indexColumns); + writeKeyValuePartitionedTable(partitionedTable, parentDir.getPath(), withDefinitionAndIndexColumns); final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); verifyIndexingInfoExists(fromDisk, "NPC5"); assertTableEquals(expected.sort("PC1"), fromDisk.sort("PC1")); @@ -1346,9 +1409,6 @@ public void readSampleParquetFilesFromPublicS3() { .readTimeout(Duration.ofSeconds(60)) .credentials(Credentials.anonymous()) .build(); - final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setSpecialInstructions(s3Instructions) - .build(); final TableDefinition tableDefinition = TableDefinition.of( ColumnDefinition.ofString("hash"), ColumnDefinition.ofLong("version"), @@ -1364,14 +1424,17 @@ public void readSampleParquetFilesFromPublicS3() { ColumnDefinition.ofDouble("output_value"), ColumnDefinition.ofTime("last_modified"), ColumnDefinition.ofDouble("input_value")); - - ParquetTools.readSingleFileTable( + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setSpecialInstructions(s3Instructions) + .setTableDefinition(tableDefinition) + .build(); + 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(10).select(); + readInstructions).head(10).select(); - ParquetTools.readSingleFileTable( + 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(10).select(); + readInstructions).head(10).select(); } @Test @@ -1388,8 +1451,9 @@ public void readFlatPartitionedParquetFromS3() { .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3Instructions) + .setFileLayout(ParquetInstructions.ParquetFileLayout.FLAT_PARTITIONED) .build(); - final Table table = ParquetTools.readFlatPartitionedTable("s3://dh-s3-parquet-test1/flatPartitionedParquet/", + final Table table = ParquetTools.readTable("s3://dh-s3-parquet-test1/flatPartitionedParquet/", readInstructions); final Table expected = emptyTable(30).update("A = (int)i % 10"); assertTableEquals(expected, table); @@ -1409,10 +1473,10 @@ public void readFlatPartitionedDataAsKeyValuePartitionedParquetFromS3() { .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3Instructions) + .setFileLayout(ParquetInstructions.ParquetFileLayout.KV_PARTITIONED) .build(); - final Table table = - ParquetTools.readKeyValuePartitionedTable("s3://dh-s3-parquet-test1/flatPartitionedParquet3/", - readInstructions); + final Table table = ParquetTools.readTable("s3://dh-s3-parquet-test1/flatPartitionedParquet3/", + readInstructions); final Table expected = emptyTable(30).update("A = (int)i % 10"); assertTableEquals(expected, table); } @@ -1431,10 +1495,10 @@ public void readKeyValuePartitionedParquetFromS3() { .build(); final ParquetInstructions readInstructions = new ParquetInstructions.Builder() .setSpecialInstructions(s3Instructions) + .setFileLayout(ParquetInstructions.ParquetFileLayout.KV_PARTITIONED) .build(); - final Table table = - ParquetTools.readKeyValuePartitionedTable("s3://dh-s3-parquet-test1/KeyValuePartitionedData/", - readInstructions); + final Table table = ParquetTools.readTable("s3://dh-s3-parquet-test1/KeyValuePartitionedData/", + readInstructions); final List> partitioningColumns = table.getDefinition().getPartitioningColumns(); assertEquals(3, partitioningColumns.size()); assertEquals("PC1", partitioningColumns.get(0).getName()); @@ -1460,14 +1524,16 @@ public void readKeyValuePartitionedParquetFromPublicS3() { .readTimeout(Duration.ofSeconds(60)) .credentials(Credentials.anonymous()) .build(); - final ParquetInstructions readInstructions = new ParquetInstructions.Builder() - .setSpecialInstructions(s3Instructions) - .build(); final TableDefinition ookla_table_definition = TableDefinition.of( ColumnDefinition.ofInt("quarter").withPartitioning(), ColumnDefinition.ofString("quadkey")); - ParquetTools.readKeyValuePartitionedTable("s3://ookla-open-data/parquet/performance/type=mobile/year=2023", - readInstructions, ookla_table_definition).head(10).select(); + final ParquetInstructions readInstructions = new ParquetInstructions.Builder() + .setSpecialInstructions(s3Instructions) + .setTableDefinition(ookla_table_definition) + .build(); + final Table table = ParquetTools.readTable("s3://ookla-open-data/parquet/performance/type=mobile/year=2023", + readInstructions).head(10).select(); + assertEquals(2, table.numColumns()); } @Test @@ -1648,8 +1714,8 @@ private interface TestParquetTableWriter { } private static final TestParquetTableWriter SINGLE_WRITER = ParquetTools::writeTable; - private static final TestParquetTableWriter MULTI_WRITER = (table, destFile) -> ParquetTools - .writeTables(new Table[] {table}, table.getDefinition(), new File[] {destFile}); + private static final TestParquetTableWriter MULTI_WRITER = + (table, destFile) -> writeTables(new Table[] {table}, table.getDefinition(), new File[] {destFile}); /** * Verify that the parent directory contains the expected parquet files and index files in the right directory @@ -1838,7 +1904,7 @@ public void writeMultiTableBasicTest() { Table[] tablesToSave = new Table[] {firstTable, secondTable}; File[] destFiles = new File[] {firstDestFile, secondDestFile}; - ParquetTools.writeTables(tablesToSave, firstTable.getDefinition(), destFiles); + writeTables(tablesToSave, firstTable.getDefinition(), destFiles); verifyFilesInDir(parentDir, new String[] {firstFilename, secondFilename}, null); checkSingleTable(firstTable, firstDestFile); @@ -1868,7 +1934,7 @@ public void writeMultiTableExceptionTest() { // This write should fail try { - ParquetTools.writeTables(tablesToSave, firstTable.getDefinition(), destFiles); + writeTables(tablesToSave, firstTable.getDefinition(), destFiles); TestCase.fail("Exception expected for invalid formula"); } catch (UncheckedDeephavenException e) { assertTrue(e.getCause() instanceof FormulaEvaluationException); @@ -2145,7 +2211,7 @@ public void writeMultiTableIndexTest() { Table[] tablesToSave = new Table[] {firstTable, secondTable}; File[] destFiles = new File[] {firstDestFile, secondDestFile}; - ParquetTools.writeTables(tablesToSave, firstTable.getDefinition(), destFiles); + writeTables(tablesToSave, firstTable.getDefinition(), destFiles); String firstIndexFilePath = ".dh_metadata/indexes/vvv/index_vvv_firstTable.parquet"; String secondIndexFilePath = ".dh_metadata/indexes/vvv/index_vvv_secondTable.parquet"; 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 bd62aefc062..5d773ea484b 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,9 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; -import static io.deephaven.base.FileUtils.DUPLICATE_SLASH_PATTERN; +import static io.deephaven.base.FileUtils.REPEATED_URI_SEPARATOR; +import static io.deephaven.base.FileUtils.REPEATED_URI_SEPARATOR_PATTERN; +import static io.deephaven.base.FileUtils.URI_SEPARATOR; import static io.deephaven.extensions.s3.S3ChannelContext.handleS3Exception; import static io.deephaven.extensions.s3.S3SeekableChannelProviderPlugin.S3_URI_SCHEME; @@ -208,8 +210,8 @@ private void fetchNextBatch() throws IOException { .filter(s3Object -> !s3Object.key().equals(directoryKey)) .map(s3Object -> { String path = "/" + s3Object.key(); - if (path.contains("//")) { - path = DUPLICATE_SLASH_PATTERN.matcher(path).replaceAll("/"); + if (path.contains(REPEATED_URI_SEPARATOR)) { + path = REPEATED_URI_SEPARATOR_PATTERN.matcher(path).replaceAll(URI_SEPARATOR); } try { return new URI(S3_URI_SCHEME, directory.getUserInfo(), directory.getHost(), diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index dcf89f10d1d..75794a151c0 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -4,6 +4,7 @@ """ This module supports reading an external Parquet files into Deephaven tables and writing Deephaven tables out as Parquet files. """ +from warnings import warn from dataclasses import dataclass from enum import Enum from typing import List, Optional, Union, Dict, Sequence @@ -13,6 +14,7 @@ from deephaven import DHError from deephaven.column import Column from deephaven.dtypes import DType +from deephaven.jcompat import j_array_list from deephaven.table import Table, PartitionedTable from deephaven.experimental import s3 @@ -20,6 +22,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") +_JParquetFileLayout = jpy.get_type("io.deephaven.parquet.table.ParquetInstructions$ParquetFileLayout") _JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") @@ -33,6 +36,28 @@ class ColumnInstruction: use_dictionary: bool = False +class ParquetFileLayout(Enum): + """ The parquet file layout. """ + + SINGLE_FILE = 1 + """ A single parquet file. """ + + FLAT_PARTITIONED = 2 + """ A single directory of parquet files. """ + + KV_PARTITIONED = 3 + """ A hierarchically partitioned directory layout of parquet files. Directory names are of the format "key=value" + with keys derived from the partitioning columns. """ + + METADATA_PARTITIONED = 4 + """ + Layout can be used to describe: + - A directory containing a METADATA_FILE_NAME parquet file and an optional COMMON_METADATA_FILE_NAME parquet file + - A single parquet METADATA_FILE_NAME file + - A single parquet COMMON_METADATA_FILE_NAME file + """ + + def _build_parquet_instructions( col_instructions: Optional[List[ColumnInstruction]] = None, compression_codec_name: Optional[str] = None, @@ -45,6 +70,10 @@ def _build_parquet_instructions( force_build: bool = False, generate_metadata_files: Optional[bool] = None, base_name: Optional[str] = None, + file_layout: Optional[ParquetFileLayout] = None, + table_definition: Optional[Union[Dict[str, DType], List[Column]]] = None, + col_definitions: Optional[List[Column]] = None, + index_columns: Optional[Sequence[Sequence[str]]] = None, special_instructions: Optional[s3.S3Instructions] = None, ): if not any( @@ -59,6 +88,10 @@ def _build_parquet_instructions( is_refreshing, generate_metadata_files is not None, base_name is not None, + file_layout is not None, + table_definition is not None, + col_definitions is not None, + index_columns is not None, special_instructions is not None ] ): @@ -102,6 +135,21 @@ def _build_parquet_instructions( if base_name: builder.setBaseNameForPartitionedParquetData(base_name) + if file_layout is not None: + builder.setFileLayout(_j_file_layout(file_layout)) + + if table_definition is not None and col_definitions is not None: + raise ValueError("table_definition and col_definitions cannot both be specified.") + + if table_definition is not None: + builder.setTableDefinition(_j_table_definition(table_definition)) + + if col_definitions is not None: + builder.setTableDefinition(_JTableDefinition.of([col.j_column_definition for col in col_definitions])) + + if index_columns: + builder.addAllIndexColumns(_j_list_of_list_of_string(index_columns)) + if special_instructions is not None: builder.setSpecialInstructions(special_instructions.j_object) @@ -125,20 +173,18 @@ def _j_table_definition(table_definition: Union[Dict[str, DType], List[Column], raise DHError(f"Unexpected table_definition type: {type(table_definition)}") -class ParquetFileLayout(Enum): - """ The parquet file layout. """ - - SINGLE_FILE = 1 - """ A single parquet file. """ - - FLAT_PARTITIONED = 2 - """ A single directory of parquet files. """ - - KV_PARTITIONED = 3 - """ A key-value directory partitioning of parquet files. """ - - METADATA_PARTITIONED = 4 - """ A directory containing a _metadata parquet file and an optional _common_metadata parquet file. """ +def _j_file_layout(file_layout: Optional[ParquetFileLayout]) -> Optional[jpy.JType]: + if file_layout is None: + return None + if file_layout == ParquetFileLayout.SINGLE_FILE: + return _JParquetFileLayout.SINGLE_FILE + if file_layout == ParquetFileLayout.FLAT_PARTITIONED: + return _JParquetFileLayout.FLAT_PARTITIONED + if file_layout == ParquetFileLayout.KV_PARTITIONED: + return _JParquetFileLayout.KV_PARTITIONED + if file_layout == ParquetFileLayout.METADATA_PARTITIONED: + return _JParquetFileLayout.METADATA_PARTITIONED + raise DHError(f"Invalid parquet file_layout '{file_layout}'") def read( @@ -163,8 +209,7 @@ def read( 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 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. + empty and is_refreshing=True. It is also useful for specifying a subset of the parquet definition. 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. @@ -183,46 +228,18 @@ def read( for_read=True, force_build=True, special_instructions=special_instructions, + file_layout=file_layout, + table_definition=table_definition, ) - 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(path, read_instructions, j_table_definition) - elif file_layout == ParquetFileLayout.FLAT_PARTITIONED: - j_table = _JParquetTools.readFlatPartitionedTable(path, read_instructions, j_table_definition) - elif file_layout == ParquetFileLayout.KV_PARTITIONED: - j_table = _JParquetTools.readKeyValuePartitionedTable(path, read_instructions, j_table_definition) - elif file_layout == ParquetFileLayout.METADATA_PARTITIONED: - raise DHError(f"file_layout={ParquetFileLayout.METADATA_PARTITIONED} with table_definition not currently supported") - else: - raise DHError(f"Invalid parquet file_layout '{file_layout}'") - else: - if not file_layout: - j_table = _JParquetTools.readTable(path, read_instructions) - elif file_layout == ParquetFileLayout.SINGLE_FILE: - j_table = _JParquetTools.readSingleFileTable(path, read_instructions) - elif file_layout == ParquetFileLayout.FLAT_PARTITIONED: - j_table = _JParquetTools.readFlatPartitionedTable(path, read_instructions) - elif file_layout == ParquetFileLayout.KV_PARTITIONED: - j_table = _JParquetTools.readKeyValuePartitionedTable(path, read_instructions) - elif file_layout == ParquetFileLayout.METADATA_PARTITIONED: - j_table = _JParquetTools.readPartitionedTableWithMetadata(_JFile(path), read_instructions) - else: - raise DHError(f"Invalid parquet file_layout '{file_layout}'") - return Table(j_table=j_table) + return Table(_JParquetTools.readTable(path, read_instructions)) except Exception as e: raise DHError(e, "failed to read parquet data.") from e +def _j_string_array(str_seq: Sequence[str]): + return jpy.array("java.lang.String", str_seq) -def _j_file_array(paths: List[str]): - return jpy.array("java.io.File", [_JFile(el) for el in paths]) - - -def _j_array_of_array_of_string(index_columns: Sequence[Sequence[str]]): - return jpy.array("[Ljava.lang.String;", [jpy.array("java.lang.String", index_cols) for index_cols in index_columns]) - +def _j_list_of_list_of_string(str_seq_seq: Sequence[Sequence[str]]): + return j_array_list([j_array_list(str_seq) for str_seq in str_seq_seq]) def delete(path: str) -> None: """ Deletes a Parquet table on disk. @@ -242,6 +259,7 @@ def delete(path: str) -> None: def write( table: Table, path: str, + table_definition: Optional[Union[Dict[str, DType], List[Column]]] = None, col_definitions: Optional[List[Column]] = None, col_instructions: Optional[List[ColumnInstruction]] = None, compression_codec_name: Optional[str] = None, @@ -258,8 +276,13 @@ def write( path (str): the destination file path; the file name should end in a ".parquet" extension. If the path includes any 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 (Optional[List[Column]]): the column definitions to use for writing, instead of the definitions - implied by the table. Default is None, which means use the column definitions implied by the table + table_definition (Optional[Union[Dict[str, DType], List[Column]]): the table definition to use for writing, + instead of the definitions implied by the table. Default is None, which means use the column definitions + implied by the table. This definition can be used to skip some columns or add additional columns with + null values. Both table_definition and col_definitions cannot be specified at the same time. + col_definitions (Optional[List[Column]]): the column definitions to use for writing, instead of the + definitions implied by the table. Default is None, which means use the column definitions implied by the + table. This argument is deprecated and will be removed in a future release. Use table_definition instead. col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while writing particular columns, default is None, which means no specialization for any column compression_codec_name (Optional[str]): the compression codec to use. Allowed values include "UNCOMPRESSED", @@ -283,6 +306,9 @@ def write( Raises: DHError """ + if col_definitions is not None: + warn("col_definitions is deprecated and will be removed in a future release. Use table_definition " + "instead.", DeprecationWarning, stacklevel=2) try: write_instructions = _build_parquet_instructions( col_instructions=col_instructions, @@ -292,19 +318,11 @@ def write( target_page_size=target_page_size, for_read=False, generate_metadata_files=generate_metadata_files, + table_definition=table_definition, + col_definitions=col_definitions, + index_columns=index_columns, ) - if col_definitions is not None: - table_definition = _JTableDefinition.of([col.j_column_definition for col in col_definitions]) - else: - table_definition = table._definition - - if index_columns: - table_array = jpy.array("io.deephaven.engine.table.Table", [table.j_table]) - index_columns_array = _j_array_of_array_of_string(index_columns) - _JParquetTools.writeParquetTables(table_array, table_definition, write_instructions, - _j_file_array([path]), index_columns_array) - else: - _JParquetTools.writeTable(table.j_table, path, table_definition, write_instructions) + _JParquetTools.writeTable(table.j_table, path, write_instructions) except Exception as e: raise DHError(e, "failed to write to parquet data.") from e @@ -312,6 +330,7 @@ def write( def write_partitioned( table: Union[Table, PartitionedTable], destination_dir: str, + table_definition: Optional[Union[Dict[str, DType], List[Column]]] = None, col_definitions: Optional[List[Column]] = None, col_instructions: Optional[List[ColumnInstruction]] = None, compression_codec_name: Optional[str] = None, @@ -332,8 +351,13 @@ def write_partitioned( table (Table): the source table or partitioned table destination_dir (str): The path to destination root directory in which the partitioned parquet data will be stored in a nested directory structure format. Non-existing directories in the provided path will be created. + table_definition (Optional[Union[Dict[str, DType], List[Column]]): the table definition to use for writing, + instead of the definitions implied by the table. Default is None, which means use the column definitions + implied by the table. This definition can be used to skip some columns or add additional columns with + null values. Both table_definition and col_definitions cannot be specified at the same time. col_definitions (Optional[List[Column]]): the column definitions to use for writing, instead of the definitions - implied by the table. Default is None, which means use the column definitions implied by the table + implied by the table. Default is None, which means use the column definitions implied by the table. This + argument is deprecated and will be removed in a future release. Use table_definition instead. col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while writing particular columns, default is None, which means no specialization for any column compression_codec_name (Optional[str]): the compression codec to use. Allowed values include "UNCOMPRESSED", @@ -370,6 +394,9 @@ def write_partitioned( Raises: DHError """ + if col_definitions is not None: + warn("col_definitions is deprecated and will be removed in a future release. Use table_definition " + "instead.", DeprecationWarning, stacklevel=2) try: write_instructions = _build_parquet_instructions( col_instructions=col_instructions, @@ -380,26 +407,11 @@ def write_partitioned( for_read=False, generate_metadata_files=generate_metadata_files, base_name=base_name, + table_definition=table_definition, + col_definitions=col_definitions, + index_columns=index_columns, ) - - table_definition = None - if col_definitions is not None: - table_definition = _JTableDefinition.of([col.j_column_definition for col in col_definitions]) - - if index_columns: - index_columns_array = _j_array_of_array_of_string(index_columns) - if table_definition: - _JParquetTools.writeKeyValuePartitionedTable(table.j_object, table_definition, destination_dir, - write_instructions, index_columns_array) - else: - _JParquetTools.writeKeyValuePartitionedTable(table.j_object, destination_dir, write_instructions, - index_columns_array) - else: - if table_definition: - _JParquetTools.writeKeyValuePartitionedTable(table.j_object, table_definition, destination_dir, - write_instructions) - else: - _JParquetTools.writeKeyValuePartitionedTable(table.j_object, destination_dir, write_instructions) + _JParquetTools.writeKeyValuePartitionedTable(table.j_object, destination_dir, write_instructions) except Exception as e: raise DHError(e, "failed to write to parquet data.") from e @@ -407,7 +419,8 @@ def write_partitioned( def batch_write( tables: List[Table], paths: List[str], - col_definitions: List[Column], + table_definition: Optional[Union[Dict[str, DType], List[Column]]] = 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, @@ -425,7 +438,12 @@ def batch_write( paths (List[str]): the destination paths. Any non-existing directories in the paths provided 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 for writing. + table_definition (Optional[Union[Dict[str, DType], List[Column]]): the table definition to use for writing, + instead of the definitions implied by the table. Default is None, which means use the column definitions + implied by the table. This definition can be used to skip some columns or add additional columns with + null values. Both table_definition and col_definitions cannot be specified at the same time. + col_definitions (List[Column]): the column definitions to use for writing. This argument is deprecated and will + be removed in a future release. Use table_definition instead. col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while writing compression_codec_name (Optional[str]): the compression codec to use. Allowed values include "UNCOMPRESSED", "SNAPPY", "GZIP", "LZO", "LZ4", "LZ4_RAW", "ZSTD", etc. If not specified, defaults to "SNAPPY". @@ -449,6 +467,12 @@ def batch_write( Raises: DHError """ + if col_definitions is not None: + warn("col_definitions is deprecated and will be removed in a future release. Use table_definition " + "instead.", DeprecationWarning, stacklevel=2) + #TODO(deephaven-core#5362): Remove col_definitions parameter + elif table_definition is None: + raise ValueError("Either table_definition or col_definitions must be specified.") try: write_instructions = _build_parquet_instructions( col_instructions=col_instructions, @@ -458,16 +482,10 @@ def batch_write( target_page_size=target_page_size, for_read=False, generate_metadata_files=generate_metadata_files, + table_definition=table_definition, + col_definitions=col_definitions, + index_columns=index_columns, ) - - table_definition = _JTableDefinition.of([col.j_column_definition for col in col_definitions]) - - if index_columns: - index_columns_array = _j_array_of_array_of_string(index_columns) - _JParquetTools.writeParquetTables([t.j_table for t in tables], table_definition, write_instructions, - _j_file_array(paths), index_columns_array) - else: - _JParquetTools.writeTables([t.j_table for t in tables], table_definition, - _j_file_array(paths)) + _JParquetTools.writeTables([t.j_table for t in tables], _j_string_array(paths), write_instructions) except Exception as e: raise DHError(e, "write multiple tables to parquet data failed.") from e diff --git a/py/server/tests/test_parquet.py b/py/server/tests/test_parquet.py index 04627e1c392..4887cf7a78b 100644 --- a/py/server/tests/test_parquet.py +++ b/py/server/tests/test_parquet.py @@ -547,22 +547,14 @@ def test_read_kv_partitioned(self): ) self.assert_table_equals(actual, table) - def test_read_with_table_definition_no_type(self): - # no need to write actual file, shouldn't be reading it - fake_parquet = os.path.join(self.temp_dir.name, "fake.parquet") - with self.subTest(msg="read definition no type"): - with self.assertRaises(DHError) as cm: - read( - fake_parquet, - table_definition={ - "x": dtypes.int32, - "y": dtypes.double, - "z": dtypes.double, - }, - ) - self.assertIn( - "Must provide file_layout when table_definition is set", str(cm.exception) - ) + def test_read_with_table_definition_no_layout(self): + table = empty_table(3).update( + formulas=["x=i", "y=(double)(i/10.0)", "z=(double)(i*i)"] + ) + single_parquet = os.path.join(self.temp_dir.name, "single.parquet") + write(table, single_parquet) + from_disk = read(single_parquet, table_definition={"x": dtypes.int32, "y": dtypes.double}) + self.assert_table_equals(from_disk, table.select(["x", "y"])) def test_read_parquet_from_s3(self): """ Test that we can read parquet files from s3 """ @@ -696,6 +688,26 @@ def test_write_with_index_columns(self): self.verify_index_files(".dh_metadata/indexes/x", expected_num_index_files=2) self.verify_index_files(".dh_metadata/indexes/y,z", expected_num_index_files=2) + def test_write_with_definition(self): + table = empty_table(3).update( + formulas=["a=i", "b=(double)(i/10.0)", "c=(double)(i*i)", "d=ii"] + ) + table_definition = { + "a": dtypes.int32, + "b": dtypes.double, + "c": dtypes.double, + } + write(table, "data_from_dh.parquet", table_definition=table_definition) + from_disk = read("data_from_dh.parquet") + self.assert_table_equals(from_disk, table.select(["a", "b", "c"])) + + col_definitions = from_disk.columns + write(table, "data_from_dh.parquet", col_definitions=col_definitions) + from_disk = read("data_from_dh.parquet") + self.assert_table_equals(from_disk, table.select(["a", "b", "c"])) + + with self.assertRaises(Exception): + write(table, "data_from_dh.parquet", table_definition=table_definition, col_definitions=col_definitions) if __name__ == '__main__': unittest.main()