diff --git a/BenchmarkSupport/BenchmarkSupport.gradle b/BenchmarkSupport/BenchmarkSupport.gradle index 70118d168a0..8c8f9c0cf77 100644 --- a/BenchmarkSupport/BenchmarkSupport.gradle +++ b/BenchmarkSupport/BenchmarkSupport.gradle @@ -30,6 +30,7 @@ dependencies { implementation project(':Base') implementation project(':engine-table') implementation project(':extensions-parquet-table') + implementation project(':extensions-parquet-base') implementation project(':Configuration') implementation 'org.openjdk.jmh:jmh-core:1.20' diff --git a/BenchmarkSupport/src/main/java/io/deephaven/benchmarking/runner/TableBenchmarkState.java b/BenchmarkSupport/src/main/java/io/deephaven/benchmarking/runner/TableBenchmarkState.java index 482e9b568dc..19986b3c28e 100644 --- a/BenchmarkSupport/src/main/java/io/deephaven/benchmarking/runner/TableBenchmarkState.java +++ b/BenchmarkSupport/src/main/java/io/deephaven/benchmarking/runner/TableBenchmarkState.java @@ -8,11 +8,12 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.parquet.table.ParquetTools; import io.deephaven.engine.util.TableTools; -import io.deephaven.parquet.table.ParquetTableWriter; import io.deephaven.engine.table.impl.util.TableBuilder; import io.deephaven.benchmarking.BenchmarkTools; import org.openjdk.jmh.infra.BenchmarkParams; +import static io.deephaven.parquet.base.ParquetUtils.PARQUET_FILE_EXTENSION; + import java.io.File; import java.io.IOException; import java.nio.file.Path; @@ -44,7 +45,7 @@ public void init() { public void logOutput() throws IOException { final Path outputPath = BenchmarkTools.dataDir() - .resolve(BenchmarkTools.getDetailOutputPath(benchmarkName) + ParquetTableWriter.PARQUET_FILE_EXTENSION); + .resolve(BenchmarkTools.getDetailOutputPath(benchmarkName) + PARQUET_FILE_EXTENSION); final Table output = outputBuilder.build(); ParquetTools.writeTable(output, outputPath.toFile(), RESULT_DEF); diff --git a/Util/channel/src/main/java/io/deephaven/util/channel/Channels.java b/Util/channel/src/main/java/io/deephaven/util/channel/Channels.java index 41982cb16fc..97a8b0c1ae2 100644 --- a/Util/channel/src/main/java/io/deephaven/util/channel/Channels.java +++ b/Util/channel/src/main/java/io/deephaven/util/channel/Channels.java @@ -5,8 +5,6 @@ import java.io.InputStream; import java.nio.channels.ReadableByteChannel; -import java.time.Instant; -import java.time.temporal.ChronoUnit; public final class Channels { diff --git a/engine/table/build.gradle b/engine/table/build.gradle index c00709d35c0..510230d3530 100644 --- a/engine/table/build.gradle +++ b/engine/table/build.gradle @@ -66,6 +66,7 @@ dependencies { testImplementation project(':BenchmarkSupport') testImplementation project(':extensions-csv') testImplementation project(':extensions-parquet-table') + testImplementation project(':extensions-parquet-base') testImplementation project(':extensions-source-support') testImplementation project(':Numerics') testImplementation project(':extensions-suanshu') diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/PartitionFormatter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/PartitionFormatter.java new file mode 100644 index 00000000000..780c9f999f6 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/PartitionFormatter.java @@ -0,0 +1,157 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.locations.util; + +import io.deephaven.time.DateTimeUtils; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalTime; +import java.util.HashMap; +import java.util.Map; + +/** + * This class takes a partition value object and formats it to a {@link String}. Useful when generating partitioning + * paths for table. Complementary to {@link PartitionParser}. + */ +public enum PartitionFormatter { + ForString { + @Override + public String formatObject(@NotNull final Object value) { + return (String) value; + } + }, + ForBoolean { + @Override + public String formatObject(@NotNull final Object value) { + return ((Boolean) value).toString(); + } + }, + ForChar { + @Override + public String formatObject(@NotNull final Object value) { + return ((Character) value).toString(); + } + }, + ForByte { + @Override + public String formatObject(@NotNull final Object value) { + return ((Byte) value).toString(); + } + }, + ForShort { + @Override + public String formatObject(@NotNull final Object value) { + return ((Short) value).toString(); + } + }, + ForInt { + @Override + public String formatObject(@NotNull final Object value) { + return ((Integer) value).toString(); + } + }, + ForLong { + @Override + public String formatObject(@NotNull final Object value) { + return ((Long) value).toString(); + } + }, + ForFloat { + @Override + public String formatObject(@NotNull final Object value) { + return ((Float) value).toString(); + } + }, + ForDouble { + @Override + public String formatObject(@NotNull final Object value) { + return ((Double) value).toString(); + } + }, + ForBigInteger { + @Override + public String formatObject(@NotNull final Object value) { + return ((BigInteger) value).toString(); + } + }, + ForBigDecimal { + @Override + public String formatObject(@NotNull final Object value) { + return ((BigDecimal) value).toString(); + } + }, + ForInstant { + @Override + public String formatObject(@NotNull final Object value) { + return ((Instant) value).toString(); + } + }, + ForLocalDate { + @Override + public String formatObject(@NotNull final Object value) { + return DateTimeUtils.formatDate((LocalDate) value); + } + }, + ForLocalTime { + @Override + public String formatObject(@NotNull final Object value) { + return ((LocalTime) value).toString(); + } + }; + + private static final Map, PartitionFormatter> typeMap = new HashMap<>(); + static { + typeMap.put(String.class, ForString); + typeMap.put(Boolean.class, ForBoolean); + typeMap.put(boolean.class, ForBoolean); + typeMap.put(Character.class, ForChar); + typeMap.put(char.class, ForChar); + typeMap.put(Byte.class, ForByte); + typeMap.put(byte.class, ForByte); + typeMap.put(Short.class, ForShort); + typeMap.put(short.class, ForShort); + typeMap.put(Integer.class, ForInt); + typeMap.put(int.class, ForInt); + typeMap.put(Long.class, ForLong); + typeMap.put(long.class, ForLong); + typeMap.put(Float.class, ForFloat); + typeMap.put(float.class, ForFloat); + typeMap.put(Double.class, ForDouble); + typeMap.put(double.class, ForDouble); + typeMap.put(BigInteger.class, ForBigInteger); + typeMap.put(BigDecimal.class, ForBigDecimal); + typeMap.put(Instant.class, ForInstant); + typeMap.put(LocalDate.class, ForLocalDate); + typeMap.put(LocalTime.class, ForLocalTime); + } + + abstract String formatObject(@NotNull final Object value); + + /** + * Takes a partition value object and returns a formatted string. Returns an empty string if the object is null. + */ + public String format(@Nullable final Object value) { + if (value == null) { + return ""; + } + return formatObject(value); + } + + /** + * Takes a partitioning column type and returns the corresponding formatter. + */ + public static PartitionFormatter getFormatterForType(@NotNull final Class clazz) { + final PartitionFormatter formatter = typeMap.get(clazz); + if (formatter != null) { + return formatter; + } else { + throw new UnsupportedOperationException("Unsupported type: " + clazz.getSimpleName()); + } + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index 99aedfdb176..0456fb78dcc 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -37,7 +37,6 @@ import io.deephaven.engine.util.TableTools; import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker; import io.deephaven.parquet.table.ParquetInstructions; -import io.deephaven.parquet.table.ParquetTableWriter; import io.deephaven.parquet.table.ParquetTools; import io.deephaven.parquet.table.layout.ParquetKeyValuePartitionedLayout; import io.deephaven.test.types.OutOfBandTest; @@ -75,6 +74,7 @@ import static io.deephaven.util.QueryConstants.*; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.*; +import static io.deephaven.parquet.base.ParquetUtils.PARQUET_FILE_EXTENSION; @Category(OutOfBandTest.class) public class QueryTableAggregationTest { @@ -3951,7 +3951,7 @@ private Table makeDiskTable(File directory) throws IOException { final TableDefaults result = testTable(stringCol("Symbol", syms), intCol("Value", values)); - final File outputFile = new File(directory, "disk_table" + ParquetTableWriter.PARQUET_FILE_EXTENSION); + final File outputFile = new File(directory, "disk_table" + PARQUET_FILE_EXTENSION); ParquetTools.writeTable(result, outputFile, result.getDefinition()); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReader.java index 1310eed4f75..9cca3e31972 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReader.java @@ -31,10 +31,16 @@ public interface ColumnChunkReader { int getMaxRl(); /** - * @return The offset index for this column chunk, or null if it not found in the metadata. + * @return Whether the column chunk has offset index information set in the metadata or not. */ - @Nullable - OffsetIndex getOffsetIndex(); + boolean hasOffsetIndex(); + + /** + * @param context The channel context to use for reading the offset index. + * @return Get the offset index for a column chunk. + * @throws UnsupportedOperationException If the column chunk does not have an offset index. + */ + OffsetIndex getOffsetIndex(final SeekableChannelContext context); /** * Used to iterate over column page readers for each page with the capability to set channel context to for reading @@ -69,9 +75,9 @@ interface ColumnPageDirectAccessor { } /** - * @return An accessor for individual parquet pages. + * @return An accessor for individual parquet pages which uses the provided offset index. */ - ColumnPageDirectAccessor getPageAccessor(); + ColumnPageDirectAccessor getPageAccessor(OffsetIndex offsetIndex); /** * @return Whether this column chunk uses a dictionary-based encoding on every page. diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java index 1166b3956b4..ce4030b895a 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ColumnChunkReaderImpl.java @@ -10,6 +10,7 @@ import io.deephaven.parquet.compress.DeephavenCompressorAdapterFactory; import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import io.deephaven.util.datastructures.LazyCachingFunction; +import org.apache.commons.io.FilenameUtils; import org.apache.parquet.bytes.BytesInput; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; @@ -41,18 +42,13 @@ final class ColumnChunkReaderImpl implements ColumnChunkReader { private final ColumnChunk columnChunk; private final SeekableChannelsProvider channelsProvider; - /** - * If reading a single parquet file, root URI is the URI of the file, else the parent directory for a metadata file - */ - private final URI rootURI; private final CompressorAdapter decompressor; private final ColumnDescriptor path; - private final OffsetIndex offsetIndex; + private OffsetIndexReader offsetIndexReader; private final List fieldTypes; private final Function dictionarySupplier; private final PageMaterializerFactory nullMaterializerFactory; - - private URI uri; + private final URI columnChunkURI; /** * Number of rows in the row group of this column chunk. */ @@ -63,11 +59,9 @@ final class ColumnChunkReaderImpl implements ColumnChunkReader { private final String version; ColumnChunkReaderImpl(ColumnChunk columnChunk, SeekableChannelsProvider channelsProvider, URI rootURI, - MessageType type, OffsetIndex offsetIndex, List fieldTypes, final long numRows, - final String version) { + MessageType type, List fieldTypes, final long numRows, final String version) { this.channelsProvider = channelsProvider; this.columnChunk = columnChunk; - this.rootURI = rootURI; this.path = type .getColumnDescription(columnChunk.meta_data.getPath_in_schema().toArray(new String[0])); if (columnChunk.getMeta_data().isSetCodec()) { @@ -76,12 +70,22 @@ final class ColumnChunkReaderImpl implements ColumnChunkReader { } else { decompressor = CompressorAdapter.PASSTHRU; } - this.offsetIndex = offsetIndex; this.fieldTypes = fieldTypes; this.dictionarySupplier = new LazyCachingFunction<>(this::getDictionary); this.nullMaterializerFactory = PageMaterializer.factoryForType(path.getPrimitiveType().getPrimitiveTypeName()); this.numRows = numRows; this.version = version; + if (columnChunk.isSetFile_path() && FILE_URI_SCHEME.equals(rootURI.getScheme())) { + final String relativePath = FilenameUtils.separatorsToSystem(columnChunk.getFile_path()); + this.columnChunkURI = convertToURI(Path.of(rootURI).resolve(relativePath), false); + } else { + // TODO(deephaven-core#5066): Add support for reading metadata files from non-file URIs + this.columnChunkURI = rootURI; + } + // Construct the reader object but don't read the offset index yet + this.offsetIndexReader = (columnChunk.isSetOffset_index_offset()) + ? new OffsetIndexReaderImpl(channelsProvider, columnChunk, columnChunkURI) + : OffsetIndexReader.NULL; } @Override @@ -99,8 +103,16 @@ public int getMaxRl() { return path.getMaxRepetitionLevel(); } - public OffsetIndex getOffsetIndex() { - return offsetIndex; + @Override + public boolean hasOffsetIndex() { + return columnChunk.isSetOffset_index_offset(); + } + + @Override + public OffsetIndex getOffsetIndex(final SeekableChannelContext context) { + // Reads and caches the offset index if it hasn't been read yet. Throws an exception if the offset index cannot + // be read from this source + return offsetIndexReader.getOffsetIndex(context); } @Override @@ -109,23 +121,15 @@ public ColumnPageReaderIterator getPageIterator() { } @Override - public final ColumnPageDirectAccessor getPageAccessor() { + public ColumnPageDirectAccessor getPageAccessor(final OffsetIndex offsetIndex) { if (offsetIndex == null) { throw new UnsupportedOperationException("Cannot use direct accessor without offset index"); } - return new ColumnPageDirectAccessorImpl(); + return new ColumnPageDirectAccessorImpl(offsetIndex); } private URI getURI() { - if (uri != null) { - return uri; - } - if (columnChunk.isSetFile_path() && FILE_URI_SCHEME.equals(rootURI.getScheme())) { - return uri = convertToURI(Path.of(rootURI).resolve(columnChunk.getFile_path()), false); - } else { - // TODO(deephaven-core#5066): Add support for reading metadata files from non-file URIs - return uri = rootURI; - } + return columnChunkURI; } @Override @@ -308,7 +312,11 @@ private static int getNumValues(PageHeader pageHeader) { private final class ColumnPageDirectAccessorImpl implements ColumnPageDirectAccessor { - ColumnPageDirectAccessorImpl() {} + private final OffsetIndex offsetIndex; + + ColumnPageDirectAccessorImpl(final OffsetIndex offsetIndex) { + this.offsetIndex = offsetIndex; + } @Override public ColumnPageReader getPageReader(final int pageNum, final SeekableChannelContext channelContext) { diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/NullParquetMetadataFileWriter.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/NullParquetMetadataFileWriter.java new file mode 100644 index 00000000000..bb9be1350e6 --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/NullParquetMetadataFileWriter.java @@ -0,0 +1,23 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.base; + +import org.apache.parquet.hadoop.metadata.ParquetMetadata; + +/** + * A no-op implementation of MetadataFileWriterBase when we don't want to write metadata files for Parquet files. + */ +public enum NullParquetMetadataFileWriter implements ParquetMetadataFileWriter { + + INSTANCE; + + @Override + public void addParquetFileMetadata(final String parquetFilePath, final ParquetMetadata metadata) {} + + @Override + public void writeMetadataFiles(final String metadataFilePath, final String commonMetadataFilePath) {} + + @Override + public void clear() {} +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/OffsetIndexReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/OffsetIndexReader.java new file mode 100644 index 00000000000..00b57cf1bb2 --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/OffsetIndexReader.java @@ -0,0 +1,27 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.base; + +import io.deephaven.util.channel.SeekableChannelContext; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; + +/** + * Interface for reading the offset index for a column chunk. + */ +public interface OffsetIndexReader { + + /** + * @param context The channel context to use for reading the offset index. + * @return Reads, caches, and returns the offset index for a column chunk. + * @throws UnsupportedOperationException If the offset index cannot be read from this source. + */ + OffsetIndex getOffsetIndex(SeekableChannelContext context); + + /** + * A null implementation of the offset index reader which always throws an exception when called. + */ + OffsetIndexReader NULL = context -> { + throw new UnsupportedOperationException("Cannot read offset index from this source."); + }; +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/OffsetIndexReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/OffsetIndexReaderImpl.java new file mode 100644 index 00000000000..8758883bd39 --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/OffsetIndexReaderImpl.java @@ -0,0 +1,64 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.base; + +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.channel.SeekableChannelsProvider; +import org.apache.parquet.format.ColumnChunk; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.format.Util; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; +import java.net.URI; +import java.nio.channels.SeekableByteChannel; + +/** + * Implementation of {@link OffsetIndexReader}, which reads the offset index for a column chunk on demand, and caches it + * for future requests. + */ +final class OffsetIndexReaderImpl implements OffsetIndexReader { + + private final SeekableChannelsProvider channelsProvider; + private final ColumnChunk columnChunk; + private final URI columnChunkURI; + private OffsetIndex offsetIndex; + + OffsetIndexReaderImpl(final SeekableChannelsProvider channelsProvider, final ColumnChunk columnChunk, + final URI columnChunkURI) { + this.channelsProvider = channelsProvider; + this.columnChunk = columnChunk; + this.columnChunkURI = columnChunkURI; + this.offsetIndex = null; + } + + @Override + @Nullable + public OffsetIndex getOffsetIndex(@NotNull final SeekableChannelContext context) { + if (offsetIndex != null) { + return offsetIndex; + } + if (!columnChunk.isSetOffset_index_offset()) { + throw new UnsupportedOperationException("Cannot read offset index from this source."); + } + return readOffsetIndex(context); + } + + private OffsetIndex readOffsetIndex(@NotNull final SeekableChannelContext channelContext) { + try ( + final SeekableChannelContext.ContextHolder holder = + SeekableChannelContext.ensureContext(channelsProvider, channelContext); + final SeekableByteChannel readChannel = channelsProvider.getReadChannel(holder.get(), columnChunkURI); + final InputStream in = + channelsProvider.getInputStream(readChannel.position(columnChunk.getOffset_index_offset()))) { + return (offsetIndex = ParquetMetadataConverter.fromParquetOffsetIndex(Util.readOffsetIndex(in))); + } catch (final IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileReader.java index e97576f95a7..90943e4d34f 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 @@ -15,9 +15,9 @@ import java.io.InputStream; import java.net.URI; import java.nio.channels.SeekableByteChannel; -import java.nio.charset.StandardCharsets; import java.util.*; +import static io.deephaven.parquet.base.ParquetUtils.MAGIC; import static io.deephaven.base.FileUtils.convertToURI; /** @@ -26,8 +26,6 @@ */ public class ParquetFileReader { private static final int FOOTER_LENGTH_SIZE = 4; - private static final String MAGIC_STR = "PAR1"; - static final byte[] MAGIC = MAGIC_STR.getBytes(StandardCharsets.US_ASCII); public static final String FILE_URI_SCHEME = "file"; public final FileMetaData fileMetaData; diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java index 0ef81d8a2fd..f050c119dce 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetFileWriter.java @@ -16,19 +16,22 @@ import org.apache.parquet.internal.column.columnindex.OffsetIndex; import org.apache.parquet.internal.hadoop.metadata.IndexReference; import org.apache.parquet.schema.MessageType; +import org.jetbrains.annotations.NotNull; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import static io.deephaven.parquet.base.ParquetUtils.MAGIC; +import static io.deephaven.parquet.base.ParquetUtils.PARQUET_OUTPUT_BUFFER_SIZE; import static org.apache.parquet.format.Util.writeFileMetaData; public final class ParquetFileWriter { private static final ParquetMetadataConverter metadataConverter = new ParquetMetadataConverter(); private static final int VERSION = 1; - private static final int OUTPUT_BUFFER_SIZE = 1 << 18; private final PositionedBufferedOutputStream bufferedOutput; private final MessageType type; @@ -38,23 +41,29 @@ public final class ParquetFileWriter { private final Map extraMetaData; private final List blocks = new ArrayList<>(); private final List> offsetIndexes = new ArrayList<>(); + private final String destFilePathForMetadata; + private final ParquetMetadataFileWriter metadataFileWriter; public ParquetFileWriter( - final String filePath, + final String destFilePath, + final String destFilePathForMetadata, final SeekableChannelsProvider channelsProvider, final int targetPageSize, final ByteBufferAllocator allocator, final MessageType type, final String codecName, - final Map extraMetaData) throws IOException { + final Map extraMetaData, + @NotNull final ParquetMetadataFileWriter metadataFileWriter) throws IOException { this.targetPageSize = targetPageSize; this.allocator = allocator; this.extraMetaData = new HashMap<>(extraMetaData); - bufferedOutput = new PositionedBufferedOutputStream(channelsProvider.getWriteChannel(filePath, false), - OUTPUT_BUFFER_SIZE); - bufferedOutput.write(ParquetFileReader.MAGIC); + bufferedOutput = new PositionedBufferedOutputStream(channelsProvider.getWriteChannel(destFilePath, false), + PARQUET_OUTPUT_BUFFER_SIZE); + bufferedOutput.write(MAGIC); this.type = type; this.compressorAdapter = DeephavenCompressorAdapterFactory.getInstance().getByName(codecName); + this.destFilePathForMetadata = destFilePathForMetadata; + this.metadataFileWriter = metadataFileWriter; } public RowGroupWriter addRowGroup(final long size) { @@ -70,19 +79,22 @@ public void close() throws IOException { serializeOffsetIndexes(); final ParquetMetadata footer = new ParquetMetadata(new FileMetaData(type, extraMetaData, Version.FULL_VERSION), blocks); - serializeFooter(footer); + serializeFooter(footer, bufferedOutput); + metadataFileWriter.addParquetFileMetadata(destFilePathForMetadata, footer); // Flush any buffered data and close the channel bufferedOutput.close(); compressorAdapter.close(); } - private void serializeFooter(final ParquetMetadata footer) throws IOException { + public static void serializeFooter(final ParquetMetadata footer, + final PositionedBufferedOutputStream bufferedOutput) + throws IOException { final long footerIndex = bufferedOutput.position(); final org.apache.parquet.format.FileMetaData parquetMetadata = metadataConverter.toParquetMetadata(VERSION, footer); writeFileMetaData(parquetMetadata, bufferedOutput); BytesUtils.writeIntLittleEndian(bufferedOutput, (int) (bufferedOutput.position() - footerIndex)); - bufferedOutput.write(ParquetFileReader.MAGIC); + bufferedOutput.write(MAGIC); } private void serializeOffsetIndexes() throws IOException { diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetMetadataFileWriter.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetMetadataFileWriter.java new file mode 100644 index 00000000000..42a93dc24e1 --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetMetadataFileWriter.java @@ -0,0 +1,37 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.base; + +import org.apache.parquet.hadoop.metadata.ParquetMetadata; + +import java.io.IOException; + +/** + * Used to write {@value ParquetUtils#METADATA_FILE_NAME} and {@value ParquetUtils#COMMON_METADATA_FILE_NAME} files for + * Parquet. + */ +public interface ParquetMetadataFileWriter { + + /** + * Add the parquet metadata for the provided parquet file to the list of metadata to be written to combined metadata + * files. + * + * @param parquetFilePath The parquet file destination path + * @param metadata The parquet metadata corresponding to the parquet file + */ + void addParquetFileMetadata(String parquetFilePath, ParquetMetadata metadata); + + /** + * Write the combined metadata files for all metadata accumulated so far and clear the list. + * + * @param metadataFilePath The destination path for the {@value ParquetUtils#METADATA_FILE_NAME} file + * @param commonMetadataFilePath The destination path for the {@value ParquetUtils#COMMON_METADATA_FILE_NAME} file + */ + void writeMetadataFiles(String metadataFilePath, String commonMetadataFilePath) throws IOException; + + /** + * Clear the list of metadata accumulated so far. + */ + void clear(); +} 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 new file mode 100644 index 00000000000..604444ae8be --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/ParquetUtils.java @@ -0,0 +1,43 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.base; + +import java.io.File; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; + +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"; + private static final String MAGIC_STR = "PAR1"; + public static final byte[] MAGIC = MAGIC_STR.getBytes(StandardCharsets.US_ASCII); + + /** + * The number of bytes to buffer before flushing while writing parquet files and metadata files. + */ + public static final int PARQUET_OUTPUT_BUFFER_SIZE = 1 << 18; + + /** + * Used as a key for storing deephaven specific metadata in the key-value metadata of parquet files. + */ + public static final String METADATA_KEY = "deephaven"; + + /** + * Used as a filter to select relevant parquet files while reading all files in a directory. + */ + public static boolean fileNameMatches(final Path path) { + final String fileName = path.getFileName().toString(); + return fileName.endsWith(PARQUET_FILE_EXTENSION) && fileName.charAt(0) != '.'; + } + + /** + * @return the key value derived from the file path, used for storing each file's metadata in the combined + * {@value #METADATA_FILE_NAME} and {@value #COMMON_METADATA_FILE_NAME} files. + */ + public static String getPerFileMetadataKey(final String filePath) { + return "deephaven_per_file_" + filePath.replace(File.separatorChar, '_'); + } +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/PositionedBufferedOutputStream.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/PositionedBufferedOutputStream.java index 719bd8f7194..3d26162f806 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/PositionedBufferedOutputStream.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/PositionedBufferedOutputStream.java @@ -3,22 +3,24 @@ // package io.deephaven.parquet.base; +import org.jetbrains.annotations.NotNull; + import java.io.BufferedOutputStream; import java.io.IOException; import java.nio.channels.Channels; import java.nio.channels.SeekableByteChannel; -final class PositionedBufferedOutputStream extends BufferedOutputStream { +public final class PositionedBufferedOutputStream extends BufferedOutputStream { private final SeekableByteChannel writeChannel; - PositionedBufferedOutputStream(final SeekableByteChannel writeChannel, final int size) { + public PositionedBufferedOutputStream(@NotNull final SeekableByteChannel writeChannel, final int size) { super(Channels.newOutputStream(writeChannel), size); this.writeChannel = writeChannel; } /** - * Get total number of bytes written to this stream + * Get the total number of bytes written to this stream */ long position() throws IOException { // Number of bytes buffered in the stream + bytes written to the underlying channel diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReader.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReader.java index 2be283eb575..c49966bc3b4 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReader.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReader.java @@ -3,9 +3,9 @@ // package io.deephaven.parquet.base; -import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.format.RowGroup; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.List; @@ -17,10 +17,10 @@ public interface RowGroupReader { * Returns the accessor to a given Column Chunk * * @param path the full column path - * @param channelContext the channel context to use while reading the parquet file - * @return the accessor to a given Column Chunk + * @return the accessor to a given Column Chunk, or null if the column is not present in this Row Group */ - ColumnChunkReader getColumnChunk(@NotNull List path, @NotNull final SeekableChannelContext channelContext); + @Nullable + ColumnChunkReader getColumnChunk(@NotNull List path); long numRows(); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java index f9d81a6f607..e50f17f2dae 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/RowGroupReaderImpl.java @@ -4,31 +4,20 @@ package io.deephaven.parquet.base; import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; -import io.deephaven.util.channel.SeekableChannelContext; import org.apache.parquet.format.ColumnChunk; import org.apache.parquet.format.RowGroup; -import org.apache.parquet.format.Util; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.internal.column.columnindex.OffsetIndex; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.io.IOException; -import java.io.InputStream; -import java.io.UncheckedIOException; import java.net.URI; -import java.nio.channels.SeekableByteChannel; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; final class RowGroupReaderImpl implements RowGroupReader { - - private static final int BUFFER_SIZE = 65536; private final RowGroup rowGroup; private final SeekableChannelsProvider channelsProvider; private final MessageType type; @@ -70,37 +59,15 @@ final class RowGroupReaderImpl implements RowGroupReader { } @Override - public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path, - @NotNull final SeekableChannelContext channelContext) { - String key = path.toString(); - ColumnChunk columnChunk = chunkMap.get(key); - List fieldTypes = schemaMap.get(key); + @Nullable + public ColumnChunkReaderImpl getColumnChunk(@NotNull final List path) { + final String key = path.toString(); + final ColumnChunk columnChunk = chunkMap.get(key); + final List fieldTypes = schemaMap.get(key); if (columnChunk == null) { return null; } - final OffsetIndex offsetIndex = offsetIndex(columnChunk, channelContext); - return new ColumnChunkReaderImpl(columnChunk, channelsProvider, rootURI, type, offsetIndex, fieldTypes, - numRows(), version); - } - - private OffsetIndex offsetIndex(ColumnChunk chunk, @NotNull SeekableChannelContext context) { - if (!chunk.isSetOffset_index_offset()) { - return null; - } - return ParquetMetadataConverter.fromParquetOffsetIndex(readOffsetIndex(chunk, context)); - } - - private org.apache.parquet.format.OffsetIndex readOffsetIndex(ColumnChunk chunk, - @NotNull SeekableChannelContext channelContext) { - try ( - final ContextHolder holder = SeekableChannelContext.ensureContext(channelsProvider, channelContext); - final SeekableByteChannel readChannel = channelsProvider.getReadChannel(holder.get(), rootURI); - final InputStream in = - channelsProvider.getInputStream(readChannel.position(chunk.getOffset_index_offset()))) { - return Util.readOffsetIndex(in); - } catch (IOException e) { - throw new UncheckedIOException(e); - } + return new ColumnChunkReaderImpl(columnChunk, channelsProvider, rootURI, type, fieldTypes, numRows(), version); } @Override 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 ba2e77e3b52..9e0cc929229 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 @@ -8,6 +8,7 @@ import io.deephaven.engine.table.impl.ColumnToCodecMappings; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; +import io.deephaven.parquet.base.ParquetUtils; import io.deephaven.util.annotations.VisibleForTesting; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.jetbrains.annotations.NotNull; @@ -115,6 +116,13 @@ public static int getDefaultTargetPageSize() { return defaultTargetPageSize; } + private static final boolean DEFAULT_GENERATE_METADATA_FILES = false; + + static final String UUID_TOKEN = "{uuid}"; + static final String PARTITIONS_TOKEN = "{partitions}"; + static final String FILE_INDEX_TOKEN = "{i}"; + private static final String DEFAULT_BASE_NAME_FOR_PARTITIONED_PARQUET_DATA = UUID_TOKEN; + public ParquetInstructions() {} public final String getColumnNameFromParquetColumnNameOrDefault(final String parquetColumnName) { @@ -164,6 +172,20 @@ public final String getColumnNameFromParquetColumnNameOrDefault(final String par */ public abstract boolean isRefreshing(); + /** + * @return should we generate {@value ParquetUtils#METADATA_FILE_NAME} and + * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} files while writing parquet files? + */ + public abstract boolean generateMetadataFiles(); + + + /** + * @return the base name for partitioned parquet data. Check + * {@link Builder#setBaseNameForPartitionedParquetData(String) setBaseNameForPartitionedParquetData} for + * more details about different tokens that can be used in the base name. + */ + public abstract String baseNameForPartitionedParquetData(); + @VisibleForTesting public static boolean sameColumnNamesAndCodecMappings(final ParquetInstructions i1, final ParquetInstructions i2) { if (i1 == EMPTY) { @@ -238,6 +260,16 @@ public int getTargetPageSize() { public boolean isRefreshing() { return DEFAULT_IS_REFRESHING; } + + @Override + public boolean generateMetadataFiles() { + return DEFAULT_GENERATE_METADATA_FILES; + } + + @Override + public String baseNameForPartitionedParquetData() { + return DEFAULT_BASE_NAME_FOR_PARTITIONED_PARQUET_DATA; + } }; private static class ColumnInstructions { @@ -306,6 +338,8 @@ private static final class ReadOnly extends ParquetInstructions { private final int targetPageSize; private final boolean isRefreshing; private final Object specialInstructions; + private final boolean generateMetadataFiles; + private final String baseNameForPartitionedParquetData; private ReadOnly( final KeyedObjectHashMap columnNameToInstructions, @@ -316,7 +350,9 @@ private ReadOnly( final boolean isLegacyParquet, final int targetPageSize, final boolean isRefreshing, - final Object specialInstructions) { + final Object specialInstructions, + final boolean generateMetadataFiles, + final String baseNameForPartitionedParquetData) { this.columnNameToInstructions = columnNameToInstructions; this.parquetColumnNameToInstructions = parquetColumnNameToColumnName; this.compressionCodecName = compressionCodecName; @@ -326,6 +362,8 @@ private ReadOnly( this.targetPageSize = targetPageSize; this.isRefreshing = isRefreshing; this.specialInstructions = specialInstructions; + this.generateMetadataFiles = generateMetadataFiles; + this.baseNameForPartitionedParquetData = baseNameForPartitionedParquetData; } private String getOrDefault(final String columnName, final String defaultValue, @@ -419,6 +457,15 @@ public boolean isRefreshing() { return specialInstructions; } + @Override + public boolean generateMetadataFiles() { + return generateMetadataFiles; + } + + @Override + public String baseNameForPartitionedParquetData() { + return baseNameForPartitionedParquetData; + } KeyedObjectHashMap copyColumnNameToInstructions() { // noinspection unchecked @@ -471,6 +518,8 @@ public static class Builder { private int targetPageSize = defaultTargetPageSize; private boolean isRefreshing = DEFAULT_IS_REFRESHING; private Object specialInstructions; + private boolean generateMetadataFiles = DEFAULT_GENERATE_METADATA_FILES; + private String baseNameForPartitionedParquetData = DEFAULT_BASE_NAME_FOR_PARTITIONED_PARQUET_DATA; public Builder() {} @@ -647,6 +696,47 @@ public Builder setSpecialInstructions(final Object specialInstructions) { return this; } + /** + * Set whether to generate {@value ParquetUtils#METADATA_FILE_NAME} and + * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} files while writing parquet files. On setting this parameter, + *
    + *
  • When writing a single parquet file, metadata files will be generated in the same parent directory as the + * parquet file. + *
  • When writing multiple parquet files in a single write call, the writing code insists that all parquet + * files should be written to the same parent directory, and only then metadata files will be generated in the + * same parent directory. + *
  • When writing key-value partitioned parquet data, metadata files are generated in the root directory of + * the partitioned parquet files. + *
+ */ + public Builder setGenerateMetadataFiles(final boolean generateMetadataFiles) { + this.generateMetadataFiles = generateMetadataFiles; + return this; + } + + /** + * Set the base name for partitioned parquet data. This is used to generate the file name for partitioned + * parquet files, and therefore, this parameter is only used when writing partitioned parquet data. Users can + * provide the following tokens to be replaced in the base name: + *
    + *
  • The token {@value #FILE_INDEX_TOKEN} will be replaced with an automatically incremented integer for files + * in a directory. For example, a base name of "table-{i}" will result in files named like + * "PC=partition1/table-0.parquet", "PC=partition1/table-1.parquet", etc., where PC is a partitioning + * column.
  • + *
  • The token {@value #UUID_TOKEN} will be replaced with a random UUID. For example, a base name of + * "table-{uuid}" will result in files named like "table-8e8ab6b2-62f2-40d1-8191-1c5b70c5f330.parquet".
  • + *
  • The token {@value #PARTITIONS_TOKEN} will be replaced with an underscore-delimited, concatenated string + * of partition values. For example, a base name of "{partitions}-table" will result in files like + * "PC1=partition1/PC2=partitionA/PC1=partition1_PC2=partitionA-table.parquet", where "PC1" and "PC2" are + * partitioning columns.
  • + *
+ * The default value of this parameter is {@value #DEFAULT_BASE_NAME_FOR_PARTITIONED_PARQUET_DATA}. + */ + public Builder setBaseNameForPartitionedParquetData(final String baseNameForPartitionedParquetData) { + this.baseNameForPartitionedParquetData = baseNameForPartitionedParquetData; + return this; + } + public ParquetInstructions build() { final KeyedObjectHashMap columnNameToInstructionsOut = columnNameToInstructions; columnNameToInstructions = null; @@ -655,7 +745,7 @@ public ParquetInstructions build() { parquetColumnNameToInstructions = null; return new ReadOnly(columnNameToInstructionsOut, parquetColumnNameToColumnNameOut, compressionCodecName, maximumDictionaryKeys, maximumDictionarySize, isLegacyParquet, targetPageSize, isRefreshing, - specialInstructions); + specialInstructions, generateMetadataFiles, baseNameForPartitionedParquetData); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetMetadataFileWriterImpl.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetMetadataFileWriterImpl.java new file mode 100644 index 00000000000..90da731eb0c --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetMetadataFileWriterImpl.java @@ -0,0 +1,278 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.table; + +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.parquet.base.ParquetFileWriter; +import io.deephaven.parquet.base.ParquetMetadataFileWriter; +import io.deephaven.parquet.base.ParquetUtils; +import io.deephaven.parquet.base.PositionedBufferedOutputStream; +import io.deephaven.parquet.table.metadata.ColumnTypeInfo; +import io.deephaven.parquet.table.metadata.TableInfo; +import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProviderLoader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.FileMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import static io.deephaven.base.FileUtils.convertToURI; +import static io.deephaven.parquet.base.ParquetUtils.MAGIC; +import static io.deephaven.parquet.base.ParquetUtils.METADATA_KEY; +import static io.deephaven.parquet.base.ParquetUtils.getPerFileMetadataKey; + +/** + * Used to generate a combined {@value ParquetUtils#METADATA_FILE_NAME} and + * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} file for provided Parquet files. This class is stateful and therefore + * should not be used by multiple threads concurrently. + */ +final class ParquetMetadataFileWriterImpl implements ParquetMetadataFileWriter { + + /** + * A class to hold the parquet file and its metadata. + */ + private static class ParquetFileMetadata { + final String filePath; + final ParquetMetadata metadata; + + ParquetFileMetadata(final String filePath, final ParquetMetadata metadata) { + this.filePath = filePath; + this.metadata = metadata; + } + } + + private final Path metadataRootDirAbsPath; + private final List parquetFileMetadataList; + private final SeekableChannelsProvider channelsProvider; + private final MessageType partitioningColumnsSchema; + + // The following fields are used to accumulate metadata for all parquet files + private MessageType mergedSchema; + private String mergedCreatedByString; + private final Map mergedKeyValueMetaData; + private final List mergedBlocks; + /** + * Per-column type information stored in key-value metadata + */ + private List mergedColumnTypes; + private String mergedVersion; + + /** + * @param metadataRootDir The root directory for the metadata files + * @param destinations The individual parquet file destinations, all of which must be contained in the metadata root + * @param partitioningColumnsSchema The common schema for partitioning columns to be included in the + * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} file, can be null if there are no partitioning columns. + */ + ParquetMetadataFileWriterImpl(@NotNull final File metadataRootDir, @NotNull final File[] destinations, + @Nullable final MessageType partitioningColumnsSchema) { + if (destinations.length == 0) { + throw new IllegalArgumentException("No destinations provided"); + } + this.metadataRootDirAbsPath = metadataRootDir.getAbsoluteFile().toPath(); + final String metadataRootDirAbsPathString = metadataRootDirAbsPath.toString(); + for (final File destination : destinations) { + if (!destination.getAbsolutePath().startsWith(metadataRootDirAbsPathString)) { + throw new UncheckedDeephavenException("All destinations must be nested under the provided metadata root" + + " directory, provided destination " + destination.getAbsolutePath() + " is not under " + + metadataRootDirAbsPathString); + } + } + this.parquetFileMetadataList = new ArrayList<>(destinations.length); + this.channelsProvider = SeekableChannelsProviderLoader.getInstance().fromServiceLoader( + convertToURI(metadataRootDirAbsPathString, true), null); + this.partitioningColumnsSchema = partitioningColumnsSchema; + + this.mergedSchema = null; + this.mergedCreatedByString = null; + this.mergedKeyValueMetaData = new HashMap<>(); + this.mergedBlocks = new ArrayList<>(); + this.mergedColumnTypes = null; + this.mergedVersion = null; + } + + /** + * Add parquet metadata for the provided parquet file to the combined metadata file. + * + * @param parquetFilePath The parquet file destination path + * @param metadata The parquet metadata + */ + public void addParquetFileMetadata(final String parquetFilePath, final ParquetMetadata metadata) { + parquetFileMetadataList.add(new ParquetFileMetadata(parquetFilePath, metadata)); + } + + /** + * Write the accumulated metadata to the provided files and clear the metadata accumulated so far. + * + * @param metadataFilePath The destination path for the {@value ParquetUtils#METADATA_FILE_NAME} file + * @param commonMetadataFilePath The destination path for the {@value ParquetUtils#COMMON_METADATA_FILE_NAME} file + */ + public void writeMetadataFiles(final String metadataFilePath, final String commonMetadataFilePath) + throws IOException { + if (parquetFileMetadataList.isEmpty()) { + throw new UncheckedDeephavenException("No parquet files to write metadata for"); + } + mergeMetadata(); + final ParquetMetadata metadataFooter = new ParquetMetadata(new FileMetaData(mergedSchema, + mergedKeyValueMetaData, mergedCreatedByString), mergedBlocks); + writeMetadataFile(metadataFooter, metadataFilePath); + + // Skip the blocks data and merge schema with partitioning columns' schema to write the common metadata file. + // The ordering of arguments in method call is important because we want to keep partitioning columns in the + // beginning. + mergedSchema = mergeSchemaInto(mergedSchema, partitioningColumnsSchema); + final ParquetMetadata commonMetadataFooter = + new ParquetMetadata(new FileMetaData(mergedSchema, mergedKeyValueMetaData, mergedCreatedByString), + new ArrayList<>()); + writeMetadataFile(commonMetadataFooter, commonMetadataFilePath); + + // Clear the accumulated metadata + clear(); + } + + /** + * Merge all the accumulated metadata for the parquet files. + */ + private void mergeMetadata() throws IOException { + final Collection mergedCreatedBy = new HashSet<>(); + for (final ParquetFileMetadata parquetFileMetadata : parquetFileMetadataList) { + final FileMetaData fileMetaData = parquetFileMetadata.metadata.getFileMetaData(); + mergedSchema = mergeSchemaInto(fileMetaData.getSchema(), mergedSchema); + final String relativePath = getRelativePath(parquetFileMetadata.filePath, metadataRootDirAbsPath); + mergeKeyValueMetaData(parquetFileMetadata, relativePath); + mergeBlocksInto(parquetFileMetadata, relativePath, mergedBlocks); + mergedCreatedBy.add(fileMetaData.getCreatedBy()); + } + if (mergedKeyValueMetaData.size() != parquetFileMetadataList.size()) { + throw new IllegalStateException("We should have one entry for each file in the merged key-value metadata, " + + "but we have " + mergedKeyValueMetaData.size() + " entries for " + parquetFileMetadataList.size() + + " files."); + } + // Add table info to the merged key-value metadata + final TableInfo.Builder tableInfoBuilder = TableInfo.builder().addAllColumnTypes(mergedColumnTypes); + if (mergedVersion != null) { + tableInfoBuilder.version(mergedVersion); + } + mergedKeyValueMetaData.put(METADATA_KEY, tableInfoBuilder.build().serializeToJSON()); + mergedCreatedByString = + mergedCreatedBy.size() == 1 ? mergedCreatedBy.iterator().next() : mergedCreatedBy.toString(); + } + + /** + * Merge the provided schema into the merged schema. Note that if there are common fields between the two schemas, + * the output schema will have the fields in the order they appear in the merged schema. + */ + private static MessageType mergeSchemaInto(final MessageType schema, final MessageType mergedSchema) { + if (mergedSchema == null) { + return schema; + } + if (mergedSchema.equals(schema)) { + return mergedSchema; + } + return mergedSchema.union(schema, true); + } + + /** + * This method processes both deephaven specific and non-deephaven key-value metadata for each file. + *
    + *
  • For non-deephaven specific key-value metadata, we accumulate it directly and enforce that there is only one + * value for each key
  • + *
  • For deephaven specific key-value metadata, we copy each file's metadata directly into the merged metadata as + * well as accumulate the required fields to generate a common table info later once all files are processed.
  • + *
+ */ + private void mergeKeyValueMetaData(@NotNull final ParquetFileMetadata parquetFileMetadata, + @NotNull final String relativePath) throws IOException { + final Map keyValueMetaData = + parquetFileMetadata.metadata.getFileMetaData().getKeyValueMetaData(); + for (final Map.Entry entry : keyValueMetaData.entrySet()) { + if (!entry.getKey().equals(METADATA_KEY)) { + // Make sure we have unique value for each key. + mergedKeyValueMetaData.compute(entry.getKey(), (k, v) -> { + if (v == null) { + // No existing value for this key, so put the new value + return entry.getValue(); + } else if (!v.equals(entry.getValue())) { + // Existing value does not match the new value + throw new UncheckedDeephavenException("Could not merge metadata for key " + entry.getKey() + + ", has conflicting values: " + entry.getValue() + " and " + v); + } + // Existing value matches the new value, no action needed + return v; + }); + } else { + // Add a separate entry for each file + final String fileKey = getPerFileMetadataKey(relativePath); + // Assuming the keys are unique for each file because file names are unique, verified in the constructor + if (mergedKeyValueMetaData.containsKey(fileKey)) { + throw new IllegalStateException("Could not merge metadata for file " + + parquetFileMetadata.filePath + " because it has conflicting file key: " + fileKey); + } + mergedKeyValueMetaData.put(fileKey, entry.getValue()); + + // Also, process and accumulate the relevant fields: + // - groupingColumns, dataIndexes are skipped + // - columnTypes must be the same for all partitions + // - version is set as non-null if all the files have the same version + final TableInfo tableInfo = TableInfo.deserializeFromJSON(entry.getValue()); + if (mergedColumnTypes == null) { + // The First file for which we've seen deephaven specific metadata, so just copy the relevant fields + mergedColumnTypes = tableInfo.columnTypes(); + mergedVersion = tableInfo.version(); + } else { + if (!mergedColumnTypes.equals(tableInfo.columnTypes())) { + throw new UncheckedDeephavenException("Could not merge metadata for key " + METADATA_KEY + + ", has conflicting values for columnTypes: " + tableInfo.columnTypes() + " and " + + mergedColumnTypes); + } + if (!tableInfo.version().equals(mergedVersion)) { + mergedVersion = null; + } + } + } + } + } + + private static void mergeBlocksInto(final ParquetFileMetadata parquetFileMetadata, + final String fileRelativePathString, final Collection mergedBlocks) { + for (final BlockMetaData block : parquetFileMetadata.metadata.getBlocks()) { + block.setPath(fileRelativePathString); + mergedBlocks.add(block); + } + } + + private static String getRelativePath(final String parquetFilePath, final Path metadataRootDirAbsPath) { + final Path parquetFileAbsPath = new File(parquetFilePath).getAbsoluteFile().toPath(); + return metadataRootDirAbsPath.relativize(parquetFileAbsPath).toString(); + } + + private void writeMetadataFile(final ParquetMetadata metadataFooter, final String outputPath) throws IOException { + final PositionedBufferedOutputStream metadataOutputStream = + new PositionedBufferedOutputStream(channelsProvider.getWriteChannel(outputPath, false), + ParquetUtils.PARQUET_OUTPUT_BUFFER_SIZE); + metadataOutputStream.write(MAGIC); + ParquetFileWriter.serializeFooter(metadataFooter, metadataOutputStream); + metadataOutputStream.close(); + } + + public void clear() { + parquetFileMetadataList.clear(); + mergedKeyValueMetaData.clear(); + mergedBlocks.clear(); + mergedColumnTypes = null; + mergedSchema = null; + mergedCreatedByString = null; + } +} 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 28c073433a5..4ff3615b2e9 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 @@ -32,6 +32,8 @@ import java.util.function.BiFunction; import java.util.function.Supplier; +import static io.deephaven.parquet.base.ParquetUtils.METADATA_KEY; + public class ParquetSchemaReader { @FunctionalInterface public interface ColumnDefinitionConsumer { @@ -105,14 +107,14 @@ public static ParquetInstructions readParquetSchema( } public static Optional parseMetadata(@NotNull final Map keyValueMetadata) { - final String tableInfoRaw = keyValueMetadata.get(ParquetTableWriter.METADATA_KEY); + final String tableInfoRaw = keyValueMetadata.get(METADATA_KEY); if (tableInfoRaw == null) { return Optional.empty(); } try { return Optional.of(TableInfo.deserializeFromJSON(tableInfoRaw)); } catch (IOException e) { - throw new TableDataException("Failed to parse " + ParquetTableWriter.METADATA_KEY + " metadata", e); + throw new TableDataException("Failed to parse " + METADATA_KEY + " metadata", e); } } 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 e6b2291221d..eed80903ae0 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 @@ -13,7 +13,10 @@ import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.engine.table.impl.select.SourceColumn; import io.deephaven.parquet.base.ColumnWriter; +import io.deephaven.parquet.base.NullParquetMetadataFileWriter; +import io.deephaven.parquet.base.ParquetMetadataFileWriter; import io.deephaven.parquet.base.ParquetFileWriter; +import io.deephaven.parquet.base.ParquetUtils; import io.deephaven.parquet.base.RowGroupWriter; import io.deephaven.parquet.table.metadata.CodecInfo; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; @@ -31,6 +34,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.parquet.bytes.HeapByteBufferAllocator; import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Types; import org.jetbrains.annotations.NotNull; @@ -43,22 +47,19 @@ import java.nio.file.Paths; import java.util.*; +import static io.deephaven.parquet.base.ParquetUtils.METADATA_KEY; import static io.deephaven.base.FileUtils.convertToURI; /** * API for writing DH tables in parquet format */ public class ParquetTableWriter { - - public static final String METADATA_KEY = "deephaven"; - public static final String GROUPING_KEY_COLUMN_NAME = "dh_key"; public static final String GROUPING_BEGIN_POS_COLUMN_NAME = "dh_begin_pos"; public static final String GROUPING_END_POS_COLUMN_NAME = "dh_end_pos"; public static final String INDEX_ROW_SET_COLUMN_NAME = "dh_row_set"; - public static final String PARQUET_FILE_EXTENSION = ".parquet"; /** * Helper struct used to pass information about where to write the index files @@ -73,10 +74,9 @@ static class IndexWritingInfo { */ final String[] parquetColumnNames; /** - * File path to be added in the index metadata of main parquet file + * File path to be added in the index metadata of the main parquet file */ - final File metadataFilePath; - + final File destFileForMetadata; /** * Destination path for writing the index file. The two filenames can differ because we write index files to * shadow file paths first and then place them at the final path once the write is complete. The metadata should @@ -87,12 +87,12 @@ static class IndexWritingInfo { IndexWritingInfo( final String[] indexColumnNames, final String[] parquetColumnNames, - final File metadataFilePath, + final File destFileForMetadata, final File destFile) { this.indexColumnNames = indexColumnNames; this.parquetColumnNames = parquetColumnNames; - this.metadataFilePath = metadataFilePath; - this.destFile = destFile; + this.destFileForMetadata = destFileForMetadata.getAbsoluteFile(); + this.destFile = destFile.getAbsoluteFile(); } } @@ -102,22 +102,32 @@ static class IndexWritingInfo { * @param t The table to write * @param definition Table definition * @param writeInstructions Write instructions for customizations while writing - * @param destPathName The destination path + * @param destFilePath The destination path + * @param destFilePathForMetadata The destination path to store in the metadata files. This can be different from + * {@code destFilePath} if we are writing the parquet file to a shadow location first since the metadata + * should always hold the accurate path. * @param incomingMeta A map of metadata values to be stores in the file footer * @param indexInfoList Arrays containing the column names for indexes to persist as sidecar tables. Indexes that * are specified but missing will be computed on demand. - * @throws SchemaMappingException Error creating a parquet table schema for the given table (likely due to - * unsupported types) + * @param metadataFileWriter The writer for the {@value ParquetUtils#METADATA_FILE_NAME} and + * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} files + * @param computedCache When we need to perform some computation depending on column data to make a decision + * impacting both schema and written data, we store results in computedCache to avoid having to calculate + * twice. An example is the necessary precision and scale for a BigDecimal column written as a decimal + * logical type. + * * @throws IOException For file writing related errors */ static void write( @NotNull final Table t, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions, - @NotNull final String destPathName, + @NotNull final String destFilePath, + @NotNull final String destFilePathForMetadata, @NotNull final Map incomingMeta, - @Nullable final List indexInfoList) - throws SchemaMappingException, IOException { + @Nullable final List indexInfoList, + @NotNull final ParquetMetadataFileWriter metadataFileWriter, + @NotNull final Map> computedCache) throws IOException { if (t.isRefreshing()) { /* * We mustn't write inconsistent tables or data indexes. This check is "basic". Snapshotting logic here @@ -133,7 +143,7 @@ static void write( try { if (indexInfoList != null) { cleanupFiles = new ArrayList<>(indexInfoList.size()); - final Path destDirPath = Paths.get(destPathName).getParent(); + final Path destDirPath = new File(destFilePath).getAbsoluteFile().getParentFile().toPath(); for (final ParquetTableWriter.IndexWritingInfo info : indexInfoList) { try (final SafeCloseable ignored = t.isRefreshing() ? LivenessScopeStack.open() : null) { // This will retrieve an existing index if one exists, or create a new one if not @@ -146,7 +156,7 @@ static void write( cleanupFiles.add(info.destFile); tableInfoBuilder.addDataIndexes(DataIndexInfo.of( - destDirPath.relativize(info.metadataFilePath.toPath()).toString(), + destDirPath.relativize(info.destFileForMetadata.toPath()).toString(), info.parquetColumnNames)); final ParquetInstructions writeInstructionsToUse; if (INDEX_ROW_SET_COLUMN_NAME.equals(dataIndex.rowSetColumnName())) { @@ -157,11 +167,14 @@ static void write( .build(); } write(indexTable, indexTable.getDefinition(), writeInstructionsToUse, - info.destFile.getAbsolutePath(), Collections.emptyMap(), TableInfo.builder()); + info.destFile.getAbsolutePath(), info.destFileForMetadata.getAbsolutePath(), + Collections.emptyMap(), TableInfo.builder(), NullParquetMetadataFileWriter.INSTANCE, + computedCache); } } } - write(t, definition, writeInstructions, destPathName, incomingMeta, tableInfoBuilder); + write(t, definition, writeInstructions, destFilePath, destFilePathForMetadata, incomingMeta, + tableInfoBuilder, metadataFileWriter, computedCache); } catch (Exception e) { if (cleanupFiles != null) { for (final File cleanupFile : cleanupFiles) { @@ -182,30 +195,34 @@ static void write( * @param table The table to write * @param definition The table definition * @param writeInstructions Write instructions for customizations while writing - * @param path The destination path + * @param destFilePath The destination path + * @param destFilePathForMetadata The destination path to store in the metadata files. This can be different from + * {@code destFilePath} if we are writing the parquet file to a shadow location first since the metadata + * should always hold the accurate path. * @param tableMeta A map of metadata values to be stores in the file footer - * @param tableInfoBuilder A partially-constructed builder for the metadata object - * @throws SchemaMappingException Error creating a parquet table schema for the given table (likely due to - * unsupported types) + * @param tableInfoBuilder A partially constructed builder for the metadata object + * @param metadataFileWriter The writer for the {@value ParquetUtils#METADATA_FILE_NAME} and + * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} files + * @param computedCache Per column cache tags * @throws IOException For file writing related errors */ - public static void write( + static void write( @NotNull final Table table, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions, - @NotNull final String path, + @NotNull final String destFilePath, + @NotNull final String destFilePathForMetadata, @NotNull final Map tableMeta, - @NotNull final TableInfo.Builder tableInfoBuilder) throws SchemaMappingException, IOException { + @NotNull final TableInfo.Builder tableInfoBuilder, + @NotNull final ParquetMetadataFileWriter metadataFileWriter, + @NotNull final Map> computedCache) throws IOException { try (final SafeCloseable ignored = LivenessScopeStack.open()) { final Table t = pretransformTable(table, definition); final TrackingRowSet tableRowSet = t.getRowSet(); final Map> columnSourceMap = t.getColumnSourceMap(); - // When we need to perform some computation depending on column data to make a decision impacting both - // schema and written data, we store results in computedCache to avoid having to calculate twice. - // An example is the necessary precision and scale for a BigDecimal column writen as decimal logical type. - final Map> computedCache = new HashMap<>(); final ParquetFileWriter parquetFileWriter = getParquetFileWriter(computedCache, definition, tableRowSet, - columnSourceMap, path, writeInstructions, tableMeta, tableInfoBuilder); + columnSourceMap, destFilePath, destFilePathForMetadata, writeInstructions, tableMeta, + tableInfoBuilder, metadataFileWriter); // Given the transformation, do not use the original table's "definition" for writing write(t, writeInstructions, parquetFileWriter, computedCache); } @@ -246,6 +263,25 @@ private static void write( parquetFileWriter.close(); } + /** + * Get the parquet schema for a table + * + * @param table the input table + * @param definition the definition to use for creating the schema + * @param instructions write instructions for the file + * @return the parquet schema + */ + static MessageType getSchemaForTable(@NotNull final Table table, + @NotNull final TableDefinition definition, + @NotNull final ParquetInstructions instructions) { + if (definition.numColumns() == 0) { + throw new IllegalArgumentException("Table definition must have at least one column"); + } + final Table pretransformTable = pretransformTable(table, definition); + return MappedSchema.create(new HashMap<>(), definition, pretransformTable.getRowSet(), + pretransformTable.getColumnSourceMap(), instructions).getParquetSchema(); + } + /** * Detect any missing or StringSet columns and convert them to arrays / null values as appropriate to prepare the * input table to be written to the parquet file. @@ -293,10 +329,15 @@ private static Table pretransformTable(@NotNull final Table table, @NotNull fina * @param definition the writable definition * @param tableRowSet the row set being written * @param columnSourceMap the columns of the table - * @param path the destination to write to + * @param destFilePath the destination to write to + * @param destFilePathForMetadata The destination path to store in the metadata files. This can be different from + * {@code destFilePath} if we are writing the parquet file to a shadow location first since the metadata + * should always hold the accurate path. * @param writeInstructions write instructions for the file * @param tableMeta metadata to include in the parquet metadata * @param tableInfoBuilder a builder for accumulating per-column information to construct the deephaven metadata + * @param metadataFileWriter The writer for the {@value ParquetUtils#METADATA_FILE_NAME} and + * {@value ParquetUtils#COMMON_METADATA_FILE_NAME} files * * @return a new file writer */ @@ -306,10 +347,12 @@ private static ParquetFileWriter getParquetFileWriter( @NotNull final TableDefinition definition, @NotNull final RowSet tableRowSet, @NotNull final Map> columnSourceMap, - @NotNull final String path, + @NotNull final String destFilePath, + @NotNull final String destFilePathForMetadata, @NotNull final ParquetInstructions writeInstructions, @NotNull final Map tableMeta, - @NotNull final TableInfo.Builder tableInfoBuilder) throws IOException { + @NotNull final TableInfo.Builder tableInfoBuilder, + @NotNull final ParquetMetadataFileWriter metadataFileWriter) throws IOException { // First, map the TableDefinition to a parquet Schema final MappedSchema mappedSchema = @@ -351,11 +394,11 @@ private static ParquetFileWriter getParquetFileWriter( final Map extraMetaData = new HashMap<>(tableMeta); extraMetaData.put(METADATA_KEY, tableInfoBuilder.build().serializeToJSON()); - return new ParquetFileWriter(path, - SeekableChannelsProviderLoader.getInstance().fromServiceLoader(convertToURI(path, false), null), + return new ParquetFileWriter(destFilePath, destFilePathForMetadata, + SeekableChannelsProviderLoader.getInstance().fromServiceLoader(convertToURI(destFilePath, false), null), writeInstructions.getTargetPageSize(), new HeapByteBufferAllocator(), mappedSchema.getParquetSchema(), - writeInstructions.getCompressionCodecName(), extraMetaData); + writeInstructions.getCompressionCodecName(), extraMetaData, metadataFileWriter); } @VisibleForTesting 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 be4bcad53c2..f76af02d108 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 @@ -10,6 +10,25 @@ import io.deephaven.base.Pair; import io.deephaven.base.verify.Require; import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.primitive.iterator.CloseableIterator; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.PartitionedTable; +import io.deephaven.engine.table.PartitionedTableFactory; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.locations.util.PartitionFormatter; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +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; @@ -20,9 +39,7 @@ import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; -import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; -import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; import io.deephaven.parquet.base.ParquetFileReader; @@ -33,14 +50,9 @@ import io.deephaven.parquet.table.location.ParquetTableLocationFactory; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; -import io.deephaven.stringset.StringSet; import io.deephaven.util.SimpleTypeMap; import io.deephaven.util.annotations.VisibleForTesting; import io.deephaven.util.channel.CachedChannelProvider; -import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.util.channel.SeekableChannelsProviderLoader; -import io.deephaven.util.channel.SeekableChannelsProviderPlugin; -import io.deephaven.vector.*; import org.apache.commons.lang3.mutable.MutableObject; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.metadata.ParquetMetadata; @@ -50,17 +62,25 @@ import java.io.File; import java.io.IOException; +import java.math.BigDecimal; import java.net.URI; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributes; import java.util.*; +import java.util.function.Supplier; import java.util.stream.Collectors; import static io.deephaven.base.FileUtils.convertToURI; import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; -import static io.deephaven.parquet.table.ParquetTableWriter.*; +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; +import static io.deephaven.parquet.base.ParquetUtils.PARQUET_FILE_EXTENSION; +import static io.deephaven.parquet.base.ParquetUtils.COMMON_METADATA_FILE_NAME; +import static io.deephaven.parquet.base.ParquetUtils.METADATA_FILE_NAME; +import static io.deephaven.parquet.table.ParquetTableWriter.getSchemaForTable; import static io.deephaven.util.type.TypeUtils.getUnboxedTypeIfBoxed; /** @@ -70,6 +90,7 @@ public class ParquetTools { private static final int MAX_PARTITIONING_LEVELS_INFERENCE = 32; + private static final String[][] EMPTY_INDEXES = new String[0][]; private ParquetTools() {} @@ -298,12 +319,12 @@ public static void writeTable( writeTables(new Table[] {sourceTable}, definition, new File[] {destFile}, writeInstructions); } - private static File getShadowFile(File destFile) { + private static File getShadowFile(final File destFile) { return new File(destFile.getParent(), ".NEW_" + destFile.getName()); } @VisibleForTesting - static File getBackupFile(File destFile) { + static File getBackupFile(final File destFile) { return new File(destFile.getParent(), ".OLD_" + destFile.getName()); } @@ -318,15 +339,16 @@ private static String minusParquetSuffix(@NotNull final String s) { * Generates the index file path relative to the table destination file path. * * @param tableDest Destination path for the main table containing these indexing columns - * @param columnNames Array of names of the indexing columns + * @param columnNames Array of indexing column names * * @return The relative index file path. For example, for table with destination {@code "table.parquet"} and * indexing column {@code "IndexingColName"}, the method will return - * {@code ".dh_metadata/indexes/IndexingColName/index_IndexingColName_table.parquet"} + * {@code ".dh_metadata/indexes/IndexingColName/index_IndexingColName_table.parquet"} on unix systems. */ public static String getRelativeIndexFilePath(@NotNull final File tableDest, @NotNull final String[] columnNames) { final String columns = String.join(",", columnNames); - return String.format(".dh_metadata/indexes/%s/index_%s_%s", columns, columns, tableDest.getName()); + return String.format(".dh_metadata%sindexes%s%s%sindex_%s_%s", File.separator, File.separator, columns, + File.separator, columns, tableDest.getName()); } /** @@ -483,15 +505,400 @@ private static List indexInfoBuilderHelper( return indexInfoList; } + /** + * 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()}. 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()}. + * + * @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, + @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 List> partitioningColumns = definition.getPartitioningColumns(); + if (partitioningColumns.isEmpty()) { + throw new IllegalArgumentException("Table must have partitioning columns to write partitioned data"); + } + final String[] partitioningColNames = partitioningColumns.stream() + .map(ColumnDefinition::getName) + .toArray(String[]::new); + final PartitionedTable partitionedTable = sourceTable.partitionBy(partitioningColNames); + final TableDefinition keyTableDefinition = TableDefinition.of(partitioningColumns); + 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()); + } + + /** + * 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[][])}. + * + * @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, + @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); + writeKeyValuePartitionedTableImpl(partitionedTable, keyTableDefinition, leafDefinition, destinationDir, + writeInstructions, indexColumnArr, Optional.empty()); + } + + /** + * Write a partitioned table to disk in a key=value partitioning format with the already computed definition for the + * key table and leaf table. + * + * @param partitionedTable The partitioned table to write + * @param keyTableDefinition The definition for key columns + * @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 + * 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)} + */ + 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 Optional sourceTable) { + if (leafDefinition.numColumns() == 0) { + throw new IllegalArgumentException("Cannot write a partitioned parquet table without any non-partitioning " + + "columns"); + } + final String baseName = writeInstructions.baseNameForPartitionedParquetData(); + final boolean hasPartitionInName = baseName.contains(PARTITIONS_TOKEN); + final boolean hasIndexInName = baseName.contains(FILE_INDEX_TOKEN); + final boolean hasUUIDInName = baseName.contains(UUID_TOKEN); + if (!partitionedTable.uniqueKeys() && !hasIndexInName && !hasUUIDInName) { + throw new IllegalArgumentException( + "Cannot write a partitioned parquet table with non-unique keys without {i} or {uuid} in the base " + + "name because there can be multiple partitions with the same key values"); + } + // Note that there can be multiple constituents with the same key values, so cannot directly use the + // partitionedTable.constituentFor(keyValues) method, and we need to group them together + final String[] partitioningColumnNames = partitionedTable.keyColumnNames().toArray(String[]::new); + final Table withGroupConstituents = partitionedTable.table().groupBy(partitioningColumnNames); + // For each row, accumulate the partition values in a key=value format + final List> partitionStringsList = new ArrayList<>(); + final long numRows = withGroupConstituents.size(); + for (long i = 0; i < numRows; i++) { + partitionStringsList.add(new ArrayList<>(partitioningColumnNames.length)); + } + Arrays.stream(partitioningColumnNames).forEach(columnName -> { + final PartitionFormatter partitionFormatter = PartitionFormatter.getFormatterForType( + withGroupConstituents.getColumnSource(columnName).getType()); + try (final CloseableIterator valueIterator = withGroupConstituents.columnIterator(columnName)) { + int row = 0; + while (valueIterator.hasNext()) { + final String partitioningValue = partitionFormatter.format(valueIterator.next()); + partitionStringsList.get(row).add(columnName + "=" + partitioningValue); + row++; + } + } + }); + // For the constituent column for each row, accumulate the constituent tables and build the final file paths + final Collection
partitionedData = new ArrayList<>(); + final Collection destinations = new ArrayList<>(); + try (final CloseableIterator> constituentIterator = + withGroupConstituents.objectColumnIterator(partitionedTable.constituentColumnName())) { + int row = 0; + while (constituentIterator.hasNext()) { + final ObjectVector constituentVector = constituentIterator.next(); + final List partitionStrings = partitionStringsList.get(row); + final File relativePath = new File(destinationRoot, String.join(File.separator, partitionStrings)); + int count = 0; + for (final Table constituent : constituentVector) { + String filename = baseName; + if (hasPartitionInName) { + filename = baseName.replace(PARTITIONS_TOKEN, String.join("_", partitionStrings)); + } + if (hasIndexInName) { + filename = filename.replace(FILE_INDEX_TOKEN, Integer.toString(count)); + } + if (hasUUIDInName) { + filename = filename.replace(UUID_TOKEN, UUID.randomUUID().toString()); + } + filename += PARQUET_FILE_EXTENSION; + destinations.add(new File(relativePath, filename)); + partitionedData.add(constituent); + count++; + } + row++; + } + } + final MessageType partitioningColumnsSchema; + if (writeInstructions.generateMetadataFiles()) { + // Generate schema for partitioning columns for _common_metadata file. The schema for remaining columns will + // be inferred at the time of writing the parquet files and merged with the common schema. + partitioningColumnsSchema = + getSchemaForTable(partitionedTable.table(), keyTableDefinition, writeInstructions); + } else { + partitioningColumnsSchema = null; + } + final Table[] partitionedDataArray = partitionedData.toArray(Table[]::new); + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + final Map> computedCache = + 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, + new File(destinationRoot), computedCache); + if (dataIndexes != null) { + dataIndexes.clear(); + } + } + } + + /** + * 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) { + return null; + } + final List dataIndexes = new ArrayList<>(indexColumnArr.length * tables.length); + for (final Table table : tables) { + for (final String[] indexCols : indexColumnArr) { + dataIndexes.add(DataIndexer.getOrCreateDataIndex(table, indexCols)); + } + } + return dataIndexes; + } + + /** + * 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, + @NotNull final TableDefinition definition) { + final Collection> keyColumnDefinitions = new ArrayList<>(keyColumnNames.size()); + for (final String keyColumnName : keyColumnNames) { + final ColumnDefinition keyColumnDef = definition.getColumn(keyColumnName); + if (keyColumnDef != null) { + keyColumnDefinitions.add(keyColumnDef); + } + } + return TableDefinition.of(keyColumnDefinitions); + } + + /** + * Using the provided definition and key column names, create a sub table definition for the non-key columns. + */ + private static TableDefinition getNonKeyTableDefinition(@NotNull final Collection keyColumnNames, + @NotNull final TableDefinition definition) { + final Collection> nonKeyColumnDefinition = definition.getColumns().stream() + .filter(columnDefinition -> !keyColumnNames.contains(columnDefinition.getName())) + .collect(Collectors.toList()); + return TableDefinition.of(nonKeyColumnDefinition); + } + + /** + * If the definition has any big decimal columns, precompute the precision and scale values for big decimal columns + * for the merged table so that all the constituent parquet files are written with the same schema, precision and + * scale values. We only need to perform the merge operation if there is a big decimal column in the definition. + * That is why this method accepts a supplier instead of the table itself. + */ + private static Map> buildComputedCache( + @NotNull final Supplier
mergedTableSupplier, + @NotNull final TableDefinition definition) { + final Map> computedCache = new HashMap<>(); + Table mergedTable = null; + final List> leafColumnDefinitions = definition.getColumns(); + for (final ColumnDefinition columnDefinition : leafColumnDefinitions) { + if (columnDefinition.getDataType() == BigDecimal.class) { + if (mergedTable == null) { + mergedTable = mergedTableSupplier.get(); + } + final String columnName = columnDefinition.getName(); + final ColumnSource bigDecimalColumnSource = mergedTable.getColumnSource(columnName); + TypeInfos.getPrecisionAndScale(computedCache, columnName, mergedTable.getRowSet(), + () -> bigDecimalColumnSource); + } + } + return computedCache; + } + /** * Writes tables to disk in parquet format to a supplied set of destinations. * * @param sources The tables to write - * @param definition The common schema for all 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 destinations 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 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 @@ -503,19 +910,66 @@ public static void writeParquetTables( @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, + @Nullable final MessageType partitioningColumnsSchema, + @Nullable final File metadataRootDir, + @NotNull final Map> computedCache) { Require.eq(sources.length, "sources.length", destinations.length, "destinations.length"); if (definition.numColumns() == 0) { throw new TableDataException("Cannot write a parquet table with zero columns"); } Arrays.stream(destinations).forEach(ParquetTools::deleteBackupFile); - // Write tables and index files at temporary shadow file paths in the same directory to prevent overwriting - // any existing files + // Write all files at temporary shadow file paths in the same directory to prevent overwriting any existing + // data in case of failure final File[] shadowDestFiles = Arrays.stream(destinations).map(ParquetTools::getShadowFile).toArray(File[]::new); final File[] firstCreatedDirs = Arrays.stream(shadowDestFiles).map(ParquetTools::prepareDestinationFileLocation).toArray(File[]::new); + final ParquetMetadataFileWriter metadataFileWriter; + if (writeInstructions.generateMetadataFiles()) { + if (metadataRootDir == null) { + throw new IllegalArgumentException("Metadata root directory must be set when writing metadata files"); + } + metadataFileWriter = + new ParquetMetadataFileWriterImpl(metadataRootDir, destinations, partitioningColumnsSchema); + } else { + metadataFileWriter = NullParquetMetadataFileWriter.INSTANCE; + } + // List of shadow files, to clean up in case of exceptions final List shadowFiles = new ArrayList<>(); // List of all destination files (including index files), to roll back in case of exceptions @@ -529,7 +983,9 @@ public static void writeParquetTables( shadowFiles.add(shadowDestFiles[tableIdx]); final Table source = sources[tableIdx]; ParquetTableWriter.write(source, definition, writeInstructions, shadowDestFiles[tableIdx].getPath(), - Collections.emptyMap(), (List) null); + destinations[tableIdx].getPath(), Collections.emptyMap(), + (List) null, metadataFileWriter, + computedCache); } } else { // Create index info for each table and write the table and index files to shadow path @@ -553,24 +1009,46 @@ public static void writeParquetTables( final Table sourceTable = sources[tableIdx]; ParquetTableWriter.write(sourceTable, definition, writeInstructions, - shadowDestFiles[tableIdx].getPath(), Collections.emptyMap(), indexInfoList); + shadowDestFiles[tableIdx].getPath(), tableDestination.getPath(), Collections.emptyMap(), + indexInfoList, metadataFileWriter, computedCache); } } - // Write to shadow files was successful + // Write the combined metadata files to shadow destinations + final File metadataDestFile, shadowMetadataFile, commonMetadataDestFile, shadowCommonMetadataFile; + if (writeInstructions.generateMetadataFiles()) { + metadataDestFile = new File(metadataRootDir, METADATA_FILE_NAME); + shadowMetadataFile = ParquetTools.getShadowFile(metadataDestFile); + shadowFiles.add(shadowMetadataFile); + commonMetadataDestFile = new File(metadataRootDir, COMMON_METADATA_FILE_NAME); + shadowCommonMetadataFile = ParquetTools.getShadowFile(commonMetadataDestFile); + shadowFiles.add(shadowCommonMetadataFile); + metadataFileWriter.writeMetadataFiles(shadowMetadataFile.getAbsolutePath(), + shadowCommonMetadataFile.getAbsolutePath()); + } else { + metadataDestFile = shadowMetadataFile = commonMetadataDestFile = shadowCommonMetadataFile = null; + } + + // Write to shadow files was successful, now replace the original files with the shadow files for (int tableIdx = 0; tableIdx < sources.length; tableIdx++) { destFiles.add(destinations[tableIdx]); installShadowFile(destinations[tableIdx], shadowDestFiles[tableIdx]); if (indexInfoLists != null) { final List indexInfoList = indexInfoLists.get(tableIdx); for (final ParquetTableWriter.IndexWritingInfo info : indexInfoList) { - final File indexDestFile = info.metadataFilePath; + final File indexDestFile = info.destFileForMetadata; final File shadowIndexFile = info.destFile; destFiles.add(indexDestFile); installShadowFile(indexDestFile, shadowIndexFile); } } } + if (writeInstructions.generateMetadataFiles()) { + destFiles.add(metadataDestFile); + installShadowFile(metadataDestFile, shadowMetadataFile); + destFiles.add(commonMetadataDestFile); + installShadowFile(commonMetadataDestFile, shadowCommonMetadataFile); + } } catch (Exception e) { for (final File file : destFiles) { rollbackFile(file); @@ -600,22 +1078,32 @@ public static void writeParquetTables( * @implNote This only examines the first source table. The writing code will compute missing indexes for the other * source tables. */ - private static String[][] indexedColumnNames(@NotNull Table @NotNull [] sources) { + @NotNull + private static String[][] indexedColumnNames(@NotNull final Table @NotNull [] sources) { if (sources.length == 0) { - return new String[0][]; + return EMPTY_INDEXES; } - // Use the first table as the source of indexed columns - final Table firstTable = sources[0]; - final DataIndexer dataIndexer = DataIndexer.existingOf(firstTable.getRowSet()); + return indexedColumnNames(sources[0]); + } + + /** + * Examine the source table to retrieve the list of indexes as String[] arrays. + * + * @param source The table from which to retrieve the indexes + * @return An array containing the indexes as String[] arrays. + */ + @NotNull + private static String[][] indexedColumnNames(@NotNull final Table source) { + final DataIndexer dataIndexer = DataIndexer.existingOf(source.getRowSet()); if (dataIndexer == null) { - return new String[0][]; + return EMPTY_INDEXES; } final List dataIndexes = dataIndexer.dataIndexes(true); if (dataIndexes.isEmpty()) { - return new String[0][]; + return EMPTY_INDEXES; } - final Map> nameToColumn = firstTable.getColumnSourceMap(); + final Map> nameToColumn = source.getColumnSourceMap(); // We disregard collisions, here; any mapped name is an adequate choice. final Map, String> columnToName = nameToColumn.entrySet().stream() .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); @@ -710,17 +1198,17 @@ private static Table readTableInternal( if (sourceFileName.endsWith(PARQUET_FILE_EXTENSION)) { return readSingleFileTable(source, instructions); } - if (sourceFileName.equals(ParquetMetadataFileLayout.METADATA_FILE_NAME)) { + if (sourceFileName.equals(METADATA_FILE_NAME)) { return readPartitionedTableWithMetadata(sourceFile.getParentFile(), instructions); } - if (sourceFileName.equals(ParquetMetadataFileLayout.COMMON_METADATA_FILE_NAME)) { + if (sourceFileName.equals(COMMON_METADATA_FILE_NAME)) { return readPartitionedTableWithMetadata(sourceFile.getParentFile(), instructions); } throw new TableDataException( "Source file " + sourceFile + " does not appear to be a parquet file or metadata file"); } if (sourceAttr.isDirectory()) { - final Path metadataPath = sourcePath.resolve(ParquetMetadataFileLayout.METADATA_FILE_NAME); + final Path metadataPath = sourcePath.resolve(METADATA_FILE_NAME); if (Files.exists(metadataPath)) { return readPartitionedTableWithMetadata(sourceFile, instructions); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java index f2e379552c0..12a53b15be0 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java @@ -116,6 +116,10 @@ static Pair getCodecAndArgs( return new ImmutablePair<>(SerializableCodec.class.getName(), null); } + /** + * Get the precision and scale for a given big decimal column. If already cached, fetch it directly, else compute it + * by scanning the entire column and store the values in the cache. + */ public static PrecisionAndScale getPrecisionAndScale( @NotNull final Map> computedCache, @NotNull final String columnName, diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFileHelper.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFileHelper.java deleted file mode 100644 index 1c9c0edd907..00000000000 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFileHelper.java +++ /dev/null @@ -1,18 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.parquet.table.layout; - -import io.deephaven.parquet.table.ParquetTableWriter; - -import java.nio.file.Path; - -final class ParquetFileHelper { - /** - * Used as a filter to select relevant parquet files while reading all files in a directory. - */ - static boolean fileNameMatches(final Path path) { - final String fileName = path.getFileName().toString(); - return fileName.endsWith(ParquetTableWriter.PARQUET_FILE_EXTENSION) && fileName.charAt(0) != '.'; - } -} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFlatPartitionedLayout.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFlatPartitionedLayout.java index 8d758b82118..2ac7970e9ea 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFlatPartitionedLayout.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetFlatPartitionedLayout.java @@ -7,6 +7,7 @@ import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import io.deephaven.parquet.base.ParquetUtils; import org.jetbrains.annotations.NotNull; import java.io.File; @@ -49,7 +50,7 @@ public String toString() { @Override public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { try (final DirectoryStream parquetFileStream = - Files.newDirectoryStream(tableRootDirectory.toPath(), ParquetFileHelper::fileNameMatches)) { + Files.newDirectoryStream(tableRootDirectory.toPath(), ParquetUtils::fileNameMatches)) { for (final Path parquetFilePath : parquetFileStream) { ParquetTableLocationKey locationKey = cache.get(parquetFilePath); if (locationKey == null) { diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetKeyValuePartitionedLayout.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetKeyValuePartitionedLayout.java index 6d5226d1313..edebbbcc195 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetKeyValuePartitionedLayout.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetKeyValuePartitionedLayout.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.impl.locations.local.LocationTableBuilderDefinition; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.location.ParquetTableLocationKey; +import io.deephaven.parquet.base.ParquetUtils; import org.jetbrains.annotations.NotNull; import java.io.File; @@ -27,7 +28,7 @@ public ParquetKeyValuePartitionedLayout( @NotNull final TableDefinition tableDefinition, @NotNull final ParquetInstructions readInstructions) { super(tableRootDirectory, - ParquetFileHelper::fileNameMatches, + ParquetUtils::fileNameMatches, () -> new LocationTableBuilderDefinition(tableDefinition), (path, partitions) -> new ParquetTableLocationKey(path.toFile(), 0, partitions, readInstructions), Math.toIntExact(tableDefinition.getColumnStream().filter(ColumnDefinition::isPartitioning).count())); @@ -38,7 +39,7 @@ public ParquetKeyValuePartitionedLayout( final int maxPartitioningLevels, @NotNull final ParquetInstructions readInstructions) { super(tableRootDirectory, - ParquetFileHelper::fileNameMatches, + ParquetUtils::fileNameMatches, () -> new LocationTableBuilderCsv(tableRootDirectory), (path, partitions) -> new ParquetTableLocationKey(path.toFile(), 0, partitions, readInstructions), maxPartitioningLevels); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetMetadataFileLayout.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/layout/ParquetMetadataFileLayout.java index b27e28d3311..6f279f4651c 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 @@ -12,13 +12,16 @@ import io.deephaven.parquet.table.ParquetTools; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.parquet.base.ParquetUtils; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.base.ParquetFileReader; +import org.apache.commons.io.FilenameUtils; import org.apache.parquet.format.converter.ParquetMetadataConverter; import io.deephaven.util.type.TypeUtils; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.parquet.format.RowGroup; +import org.apache.parquet.hadoop.metadata.FileMetaData; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -36,6 +39,10 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static io.deephaven.parquet.base.ParquetUtils.COMMON_METADATA_FILE_NAME; +import static io.deephaven.parquet.base.ParquetUtils.METADATA_FILE_NAME; +import static io.deephaven.parquet.base.ParquetUtils.METADATA_KEY; +import static io.deephaven.parquet.base.ParquetUtils.getPerFileMetadataKey; import static java.util.stream.Collectors.toMap; /** @@ -45,19 +52,17 @@ *

* Note that we expect to find the following files: *

    - *
  • {@code _metadata} - A file containing Parquet metadata for all {@link RowGroup row groups} in all - * {@code .parquet} files for the entire data set, including schema information non-partitioning columns and key-value - * metadata
  • - *
  • {@code _common_metadata} (optional) - A file containing Parquet metadata with schema information that - * applies to the entire data set, including partitioning columns that are inferred from file paths rather than - * explicitly written in {@link org.apache.parquet.format.ColumnChunk column chunks} within {@code .parquet} files
  • + *
  • {@value ParquetUtils#METADATA_FILE_NAME} - A file containing Parquet metadata for all {@link RowGroup row groups} + * in all {@code .parquet} files for the entire data set, including schema information non-partitioning columns and + * key-value metadata
  • + *
  • {@value ParquetUtils#COMMON_METADATA_FILE_NAME} (optional) - A file containing Parquet metadata with + * schema information that applies to the entire data set, including partitioning columns that are inferred from file + * paths rather than explicitly written in {@link org.apache.parquet.format.ColumnChunk column chunks} within + * {@code .parquet} files
  • *
*/ public class ParquetMetadataFileLayout implements TableLocationKeyFinder { - public static final String METADATA_FILE_NAME = "_metadata"; - public static final String COMMON_METADATA_FILE_NAME = "_common_metadata"; - private final File metadataFile; private final File commonMetadataFile; @@ -139,21 +144,20 @@ public ParquetMetadataFileLayout( final Map partitionKeyToParser = partitioningColumns.stream().collect(toMap( ColumnDefinition::getName, cd -> PartitionParser.lookupSupported(cd.getDataType(), cd.getComponentType()))); - final Map fileNameToRowGroupIndices = new LinkedHashMap<>(); + final Map filePathToRowGroupIndices = new LinkedHashMap<>(); final List rowGroups = metadataFileReader.fileMetaData.getRow_groups(); final int numRowGroups = rowGroups.size(); for (int rgi = 0; rgi < numRowGroups; ++rgi) { - fileNameToRowGroupIndices - .computeIfAbsent(rowGroups.get(rgi).getColumns().get(0).getFile_path(), fn -> new TIntArrayList()) - .add(rgi); + final String relativePath = + FilenameUtils.separatorsToSystem(rowGroups.get(rgi).getColumns().get(0).getFile_path()); + filePathToRowGroupIndices.computeIfAbsent(relativePath, fn -> new TIntArrayList()).add(rgi); } final File directory = metadataFile.getParentFile(); final MutableInt partitionOrder = new MutableInt(0); - keys = fileNameToRowGroupIndices.entrySet().stream().map(entry -> { - final String filePathString = entry.getKey(); + keys = filePathToRowGroupIndices.entrySet().stream().map(entry -> { + final String relativePathString = entry.getKey(); final int[] rowGroupIndices = entry.getValue().toArray(); - - if (filePathString == null || filePathString.isEmpty()) { + if (relativePathString == null || relativePathString.isEmpty()) { throw new TableDataException(String.format( "Missing parquet file name for row groups %s in %s", Arrays.toString(rowGroupIndices), metadataFile)); @@ -161,12 +165,12 @@ public ParquetMetadataFileLayout( final LinkedHashMap> partitions = partitioningColumns.isEmpty() ? null : new LinkedHashMap<>(); if (partitions != null) { - final Path filePath = Paths.get(filePathString); + final Path filePath = Paths.get(relativePathString); final int numPartitions = filePath.getNameCount() - 1; if (numPartitions != partitioningColumns.size()) { throw new TableDataException(String.format( "Unexpected number of path elements in %s for partitions %s", - filePathString, partitions.keySet())); + relativePathString, partitions.keySet())); } final boolean useHiveStyle = filePath.getName(0).toString().contains("="); for (int pi = 0; pi < numPartitions; ++pi) { @@ -178,7 +182,7 @@ public ParquetMetadataFileLayout( if (pathComponents.length != 2) { throw new TableDataException(String.format( "Unexpected path format found for hive-style partitioning from %s for %s", - filePathString, metadataFile)); + relativePathString, metadataFile)); } partitionKey = instructions.getColumnNameFromParquetColumnNameOrDefault(pathComponents[0]); partitionValueRaw = pathComponents[1]; @@ -191,20 +195,48 @@ public ParquetMetadataFileLayout( if (partitions.containsKey(partitionKey)) { throw new TableDataException(String.format( "Unexpected duplicate partition key %s when parsing %s for %s", - partitionKey, filePathString, metadataFile)); + partitionKey, relativePathString, metadataFile)); } partitions.put(partitionKey, partitionValue); } } - final ParquetTableLocationKey tlk = new ParquetTableLocationKey(new File(directory, filePathString), + final File partitionFile = new File(directory, relativePathString); + final ParquetTableLocationKey tlk = new ParquetTableLocationKey(partitionFile, partitionOrder.getAndIncrement(), partitions, inputInstructions); tlk.setFileReader(metadataFileReader); - tlk.setMetadata(metadataFileMetadata); + tlk.setMetadata(getParquetMetadataForFile(relativePathString, metadataFileMetadata)); tlk.setRowGroupIndices(rowGroupIndices); return tlk; }).collect(Collectors.toList()); } + /** + * This method takes the {@link ParquetMetadata} from the metadata file, extracts the key-value metadata specific to + * the provided file, and creates a new {@link ParquetMetadata} for this file. + * + * @param parquetFileRelativePath The parquet file path relative to the root directory containing the metadata file + * @param metadataFileMetadata The overall metadata in the metadata file + */ + private static ParquetMetadata getParquetMetadataForFile(@NotNull final String parquetFileRelativePath, + @NotNull final ParquetMetadata metadataFileMetadata) { + final String fileMetadataString = metadataFileMetadata.getFileMetaData().getKeyValueMetaData() + .get(getPerFileMetadataKey(parquetFileRelativePath)); + final ParquetMetadata fileMetadata; + if (fileMetadataString != null) { + // Create a new file metadata object using the key-value metadata for this file + final Map keyValueMetadata = Map.of(METADATA_KEY, fileMetadataString); + fileMetadata = new ParquetMetadata( + new FileMetaData(metadataFileMetadata.getFileMetaData().getSchema(), + keyValueMetadata, + metadataFileMetadata.getFileMetaData().getCreatedBy()), + metadataFileMetadata.getBlocks()); + } else { + // File specific metadata not found, use the metadata file's metadata + fileMetadata = metadataFileMetadata; + } + return fileMetadata; + } + public String toString() { return ParquetMetadataFileLayout.class.getSimpleName() + '[' + metadataFile + ',' + commonMetadataFile + ']'; } 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 69d30847418..99c321eeb34 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 @@ -183,11 +183,8 @@ protected ColumnLocation makeColumnLocation(@NotNull final String columnName) { final String[] columnPath = parquetColumnNameToPath.get(parquetColumnName); final List nameList = columnPath == null ? Collections.singletonList(parquetColumnName) : Arrays.asList(columnPath); - final ColumnChunkReader[] columnChunkReaders; - try (final SeekableChannelContext channelContext = getChannelProvider().makeSingleUseContext()) { - columnChunkReaders = Arrays.stream(getRowGroupReaders()) - .map(rgr -> rgr.getColumnChunk(nameList, channelContext)).toArray(ColumnChunkReader[]::new); - } + final ColumnChunkReader[] columnChunkReaders = Arrays.stream(getRowGroupReaders()) + .map(rgr -> rgr.getColumnChunk(nameList)).toArray(ColumnChunkReader[]::new); final boolean exists = Arrays.stream(columnChunkReaders).anyMatch(ccr -> ccr != null && ccr.numRows() > 0); return new ParquetColumnLocation<>(this, columnName, parquetColumnName, exists ? columnChunkReaders : null); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java index b8fdcb1c5be..a044c309cea 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 @@ -8,8 +8,8 @@ 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.table.ParquetTableWriter; import io.deephaven.parquet.base.ParquetFileReader; +import org.apache.commons.io.FilenameUtils; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.format.RowGroup; import org.apache.parquet.hadoop.metadata.ParquetMetadata; @@ -23,6 +23,7 @@ import java.util.Map; import java.util.stream.IntStream; +import static io.deephaven.parquet.base.ParquetUtils.PARQUET_FILE_EXTENSION; import static io.deephaven.base.FileUtils.convertToURI; /** @@ -76,8 +77,8 @@ private static URI validateParquetFile(@NotNull final File file) { } private static URI validateParquetFile(@NotNull final URI parquetFileUri) { - if (!parquetFileUri.getRawPath().endsWith(ParquetTableWriter.PARQUET_FILE_EXTENSION)) { - throw new IllegalArgumentException("Parquet file must end in " + ParquetTableWriter.PARQUET_FILE_EXTENSION); + if (!parquetFileUri.getRawPath().endsWith(PARQUET_FILE_EXTENSION)) { + throw new IllegalArgumentException("Parquet file must end in " + PARQUET_FILE_EXTENSION); } return parquetFileUri; } @@ -190,7 +191,8 @@ public synchronized int[] getRowGroupIndices() { // While it seems that row group *could* have column chunks splayed out into multiple files, // we're not expecting that in this code path. To support it, discovery tools should figure out // the row groups for a partition themselves and call setRowGroupReaders. - final String filePath = rowGroups.get(rgi).getColumns().get(0).getFile_path(); + final String filePath = + FilenameUtils.separatorsToSystem(rowGroups.get(rgi).getColumns().get(0).getFile_path()); return filePath == null || convertToURI(filePath, false).equals(uri); }).toArray(); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java index 699a9cf3712..ba3ca21a328 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java @@ -37,7 +37,7 @@ public abstract class ColumnChunkPageStore implements PageStore>, Page, SafeCloseable, Releasable { final PageCache pageCache; - private final ColumnChunkReader columnChunkReader; + final ColumnChunkReader columnChunkReader; private final long mask; private final ToPage toPage; @@ -62,7 +62,7 @@ private CreatorResult( private static boolean canUseOffsetIndexBasedPageStore( @NotNull final ColumnChunkReader columnChunkReader, @NotNull final ColumnDefinition columnDefinition) { - if (columnChunkReader.getOffsetIndex() == null) { + if (!columnChunkReader.hasOffsetIndex()) { return false; } final String version = columnChunkReader.getVersion(); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java index 9c5b6bb778c..650554d3d79 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java @@ -11,6 +11,7 @@ import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelContext.ContextHolder; import org.apache.parquet.internal.column.columnindex.OffsetIndex; import org.jetbrains.annotations.NotNull; @@ -27,6 +28,7 @@ */ final class OffsetIndexBasedColumnChunkPageStore extends ColumnChunkPageStore { private static final long PAGE_SIZE_NOT_FIXED = -1; + private static final int NUM_PAGES_NOT_INITIALIZED = -1; private static final class PageState { private volatile WeakReference> pageRef; @@ -36,8 +38,9 @@ private static final class PageState { } } - private final OffsetIndex offsetIndex; - private final int numPages; + private volatile boolean isInitialized; // This class is initialized when reading the first page + private OffsetIndex offsetIndex; + private int numPages; /** * Fixed number of rows per page. Set as positive value if first ({@link #numPages}-1) pages have equal number of * rows, else equal to {@value #PAGE_SIZE_NOT_FIXED}. We cannot find the number of rows in the last page size from @@ -46,9 +49,9 @@ private static final class PageState { * the same size and calculate the page number as {@code row_index / fixed_page_size -> page_number}. If it is * greater than {@link #numPages}, we will infer that the row is coming from last page. */ - private final long fixedPageSize; - private final AtomicReferenceArray> pageStates; - private final ColumnChunkReader.ColumnPageDirectAccessor columnPageDirectAccessor; + private long fixedPageSize; + private AtomicReferenceArray> pageStates; + private ColumnChunkReader.ColumnPageDirectAccessor columnPageDirectAccessor; OffsetIndexBasedColumnChunkPageStore( @NotNull final PageCache pageCache, @@ -56,26 +59,43 @@ private static final class PageState { final long mask, @NotNull final ToPage toPage) throws IOException { super(pageCache, columnChunkReader, mask, toPage); - offsetIndex = columnChunkReader.getOffsetIndex(); - Assert.neqNull(offsetIndex, "offsetIndex"); - numPages = offsetIndex.getPageCount(); - Assert.gtZero(numPages, "numPages"); - pageStates = new AtomicReferenceArray<>(numPages); - columnPageDirectAccessor = columnChunkReader.getPageAccessor(); - - if (numPages == 1) { - fixedPageSize = numRows(); + numPages = NUM_PAGES_NOT_INITIALIZED; + fixedPageSize = PAGE_SIZE_NOT_FIXED; + } + + private void ensureInitialized(@Nullable final FillContext fillContext) { + if (isInitialized) { return; } - boolean isPageSizeFixed = true; - final long firstPageSize = offsetIndex.getFirstRowIndex(1) - offsetIndex.getFirstRowIndex(0); - for (int i = 2; i < numPages; ++i) { - if (offsetIndex.getFirstRowIndex(i) - offsetIndex.getFirstRowIndex(i - 1) != firstPageSize) { - isPageSizeFixed = false; - break; + synchronized (this) { + if (isInitialized) { + return; + } + try (final ContextHolder holder = SeekableChannelContext.ensureContext( + columnChunkReader.getChannelsProvider(), innerFillContext(fillContext))) { + offsetIndex = columnChunkReader.getOffsetIndex(holder.get()); + } + Assert.neqNull(offsetIndex, "offsetIndex"); + numPages = offsetIndex.getPageCount(); + Assert.gtZero(numPages, "numPages"); + pageStates = new AtomicReferenceArray<>(numPages); + columnPageDirectAccessor = columnChunkReader.getPageAccessor(offsetIndex); + + if (numPages == 1) { + fixedPageSize = numRows(); + return; } + boolean isPageSizeFixed = true; + final long firstPageSize = offsetIndex.getFirstRowIndex(1) - offsetIndex.getFirstRowIndex(0); + for (int i = 2; i < numPages; ++i) { + if (offsetIndex.getFirstRowIndex(i) - offsetIndex.getFirstRowIndex(i - 1) != firstPageSize) { + isPageSizeFixed = false; + break; + } + } + fixedPageSize = isPageSizeFixed ? firstPageSize : PAGE_SIZE_NOT_FIXED; + isInitialized = true; } - fixedPageSize = isPageSizeFixed ? firstPageSize : PAGE_SIZE_NOT_FIXED; } /** @@ -138,7 +158,18 @@ private ChunkPage getPageImpl(@Nullable FillContext fillContext, int pageN @Override @NotNull - public ChunkPage getPageContaining(@Nullable final FillContext fillContext, long rowKey) { + public ChunkPage getPageContaining(@Nullable final FillContext fillContext, final long rowKey) { + // We don't really use chunk capacity in our FillContext. In practice, however, this method is only invoked with + // a null FillContext for single-element "get" methods. + try (final FillContext allocatedFillContext = fillContext != null ? null : makeFillContext(1, null)) { + final FillContext fillContextToUse = fillContext != null ? fillContext : allocatedFillContext; + ensureInitialized(fillContextToUse); + return getPageContainingImpl(fillContextToUse, rowKey); + } + } + + @NotNull + private ChunkPage getPageContainingImpl(@Nullable final FillContext fillContext, long rowKey) { rowKey &= mask(); Require.inRange(rowKey, "rowKey", numRows(), "numRows"); 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 d9b35726664..531348f2d17 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 @@ -14,13 +14,16 @@ import io.deephaven.engine.primitive.function.FloatConsumer; import io.deephaven.engine.primitive.function.ShortConsumer; import io.deephaven.engine.primitive.iterator.CloseableIterator; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.PartitionedTable; +import io.deephaven.engine.table.PartitionedTableFactory; +import io.deephaven.engine.table.impl.SourceTable; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.SourceTable; import io.deephaven.engine.table.impl.dataindex.DataIndexUtils; import io.deephaven.engine.table.impl.indexer.DataIndexer; -import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.select.FormulaEvaluationException; import io.deephaven.engine.table.impl.select.FunctionalColumn; import io.deephaven.engine.table.impl.select.SelectColumn; @@ -39,6 +42,7 @@ import io.deephaven.parquet.table.location.ParquetTableLocationKey; import io.deephaven.parquet.table.pagestore.ColumnChunkPageStore; import io.deephaven.parquet.table.transfer.StringDictionary; +import io.deephaven.qst.type.Type; import io.deephaven.stringset.ArrayStringSet; import io.deephaven.stringset.StringSet; import io.deephaven.test.types.OutOfBandTest; @@ -70,16 +74,44 @@ import java.net.URI; import java.time.Duration; import java.time.Instant; -import java.util.*; +import java.time.LocalDate; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.Objects; import java.util.function.DoubleConsumer; import java.util.function.Function; import java.util.function.IntConsumer; import java.util.function.LongConsumer; +import java.util.stream.Collectors; import static io.deephaven.base.FileUtils.convertToURI; import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; -import static io.deephaven.engine.util.TableTools.*; -import static io.deephaven.parquet.table.ParquetTools.*; +import static io.deephaven.engine.util.TableTools.booleanCol; +import static io.deephaven.engine.util.TableTools.byteCol; +import static io.deephaven.engine.util.TableTools.charCol; +import static io.deephaven.engine.util.TableTools.doubleCol; +import static io.deephaven.engine.util.TableTools.emptyTable; +import static io.deephaven.engine.util.TableTools.floatCol; +import static io.deephaven.engine.util.TableTools.instantCol; +import static io.deephaven.engine.util.TableTools.intCol; +import static io.deephaven.engine.util.TableTools.longCol; +import static io.deephaven.engine.util.TableTools.merge; +import static io.deephaven.engine.util.TableTools.newTable; +import static io.deephaven.engine.util.TableTools.shortCol; +import static io.deephaven.engine.util.TableTools.stringCol; +import static io.deephaven.parquet.table.ParquetTools.readFlatPartitionedTable; +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.util.QueryConstants.*; import static org.junit.Assert.*; @@ -311,23 +343,9 @@ public void indexByLongKey() { final Table fromDisk = checkSingleTable(testTable, dest); // Validate the indexes and lookup functions. - ColumnSource[] columns = Arrays.stream(new String[] {"someLong"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - DataIndex fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); - Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); - - columns = Arrays.stream(new String[] {"someInt", "someLong"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); - Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); - - columns = Arrays.stream(new String[] {"someLong", "someInt"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); - Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); + verifyIndexingInfoExists(fromDisk, "someLong"); + verifyIndexingInfoExists(fromDisk, "someInt", "someLong"); + verifyIndexingInfoExists(fromDisk, "someLong", "someInt"); } @Test @@ -348,23 +366,9 @@ public void indexByStringKey() { final Table fromDisk = checkSingleTable(testTable, dest); // Validate the indexes and lookup functions. - ColumnSource[] columns = Arrays.stream(new String[] {"someString"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - DataIndex fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); - Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); - - columns = Arrays.stream(new String[] {"someInt", "someString"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); - Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); - - columns = Arrays.stream(new String[] {"someString", "someInt"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); - Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); + verifyIndexingInfoExists(fromDisk, "someString"); + verifyIndexingInfoExists(fromDisk, "someInt", "someString"); + verifyIndexingInfoExists(fromDisk, "someString", "someInt"); } @Test @@ -380,39 +384,33 @@ public void indexByBigInt() { DataIndexer.getOrCreateDataIndex(testTable, "someBigInt"); DataIndexer.getOrCreateDataIndex(testTable, "someInt", "someBigInt"); - final File dest = new File(rootFile, "ParquetTest_groupByBigInt_test.parquet"); writeTable(testTable, dest); final Table fromDisk = checkSingleTable(testTable, dest); // Validate the indexes and lookup functions. - ColumnSource[] columns = Arrays.stream(new String[] {"someBigInt"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - DataIndex fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); - Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); - - columns = Arrays.stream(new String[] {"someInt", "someBigInt"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); - Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); + verifyIndexingInfoExists(fromDisk, "someBigInt"); + verifyIndexingInfoExists(fromDisk, "someInt", "someBigInt"); + verifyIndexingInfoExists(fromDisk, "someBigInt", "someInt"); + } - columns = Arrays.stream(new String[] {"someBigInt", "someInt"}).map(fromDisk::getColumnSource) - .toArray(ColumnSource[]::new); - fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + private static void verifyIndexingInfoExists(final Table table, final String... columnNames) { + assertTrue(DataIndexer.hasDataIndex(table, columnNames)); + final DataIndex fullIndex = DataIndexer.getDataIndex(table, columnNames); Assert.neqNull(fullIndex, "fullIndex"); - assertLookupFromTable(fromDisk, fullIndex, columns); + assertLookupFromTable(table, fullIndex, columnNames); } - private void assertLookupFromTable( + private static void assertLookupFromTable( final Table sourceTable, final DataIndex fullIndex, - final ColumnSource[] columns) { + final String... columnNames) { + final ColumnSource[] columns = Arrays.stream(columnNames).map(sourceTable::getColumnSource) + .toArray(ColumnSource[]::new); final DataIndex.RowKeyLookup fullIndexRowKeyLookup = fullIndex.rowKeyLookup(columns); final ColumnSource fullIndexRowSetColumn = fullIndex.rowSetColumn(); - ChunkSource.WithPrev tableKeys = DataIndexUtils.makeBoxedKeySource(columns); + final ChunkSource.WithPrev tableKeys = DataIndexUtils.makeBoxedKeySource(columns); // Iterate through the entire source table and verify the lookup row set is valid and contains this row. try (final RowSet.Iterator rsIt = sourceTable.getRowSet().iterator(); @@ -538,6 +536,651 @@ private static void writeReadTableTest(final Table table, final File dest, checkSingleTable(table, dest); } + @Test + public void basicParquetWithMetadataTest() { + final Table table = TableTools.emptyTable(5).update("A=(int)i", "B=(long)i", "C=(double)i"); + final String filename = "basicParquetWithMetadataTest.parquet"; + final File destFile = new File(rootFile, filename); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + writeTable(table, destFile, writeInstructions); + + final File metadataFile = new File(rootFile, "_metadata"); + assertTrue(metadataFile.exists()); + assertTrue(new File(rootFile, "_common_metadata").exists()); + + final Table fromDisk = readTable(destFile); + assertTableEquals(table, fromDisk); + + final Table fromDiskWithMetadata = readTable(metadataFile); + assertTableEquals(table, fromDiskWithMetadata); + } + + @Test + public void parquetWithIndexingDataAndMetadataTest() { + final File parentDir = new File(rootFile, "tempDir"); + final int[] data = new int[500 * 4]; + for (int i = 0; i < data.length; i++) { + data[i] = i / 4; + } + final Table indexedTable = newTable(TableTools.intCol("vvv", data)); + DataIndexer.getOrCreateDataIndex(indexedTable, "vvv"); + + final File destFile = new File(parentDir, "parquetWithIndexingDataAndMetadataTest.parquet"); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + writeTable(indexedTable, destFile, writeInstructions); + + final Table fromDisk = readTable(destFile); + assertTableEquals(indexedTable, fromDisk); + verifyIndexingInfoExists(fromDisk, "vvv"); + + final File metadataFile = new File(parentDir, "_metadata"); + final Table fromDiskWithMetadata = readTable(metadataFile); + assertTableEquals(indexedTable, fromDiskWithMetadata); + verifyIndexingInfoExists(fromDiskWithMetadata, "vvv"); + } + + @Test + public void flatPartitionedParquetWithMetadataTest() throws IOException { + // Create an empty parent directory + final File parentDir = new File(rootFile, "tempDir"); + parentDir.mkdir(); + assertTrue(parentDir.exists() && parentDir.isDirectory() && parentDir.list().length == 0); + + final Table someTable = TableTools.emptyTable(5).update("A=(int)i"); + final File firstDataFile = new File(parentDir, "data1.parquet"); + 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); + final Table source = readTable(parentDir).select(); + + // Now write with metadata files + parentDir.delete(); + parentDir.mkdir(); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + writeParquetTables(new Table[] {someTable, someTable}, someTable.getDefinition(), writeInstructions, + new File[] {firstDataFile, secondDataFile}, null); + + final Table fromDisk = readTable(parentDir); + assertTableEquals(source, fromDisk); + + final File metadataFile = new File(parentDir, "_metadata"); + final Table fromDiskWithMetadata = readTable(metadataFile); + assertTableEquals(source, fromDiskWithMetadata); + + // Now replace the underlying data files with empty files and read the size from metadata file verifying that + // we can read the size without touching the data + firstDataFile.delete(); + firstDataFile.createNewFile(); + secondDataFile.delete(); + secondDataFile.createNewFile(); + final Table fromDiskWithMetadataWithoutData = readTable(metadataFile); + assertEquals(source.size(), fromDiskWithMetadataWithoutData.size()); + + // Now write with flat partitioned parquet files to different directories with metadata file + 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); + fail("Expected exception when writing the metadata files for tables with different parent directories"); + } catch (final RuntimeException expected) { + } + } + + @Test + public void flatPartitionedParquetWithBigDecimalMetadataTest() throws IOException { + final File parentDir = new File(rootFile, "tempDir"); + + // Both tables have different precision for big decimal column + final Table firstTable = TableTools.emptyTable(5).update("bdColumn = java.math.BigDecimal.valueOf((double)ii)"); + final Table secondTable = + TableTools.emptyTable(5).update("bdColumn = java.math.BigDecimal.valueOf((double)(ii*0.001))"); + final File firstDataFile = new File(parentDir, "data1.parquet"); + final File secondDataFile = new File(parentDir, "data2.parquet"); + + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + final Table[] sources = new Table[] {firstTable, secondTable}; + writeParquetTables(sources, firstTable.getDefinition(), writeInstructions, + new File[] {firstDataFile, secondDataFile}, null); + + // Merge the tables and compute the precision and scale as per the union of the two tables + final Table expected = + maybeFixBigDecimal(PartitionedTableFactory.ofTables(firstTable.getDefinition(), sources).merge()); + + final Table fromDisk = readTable(parentDir).select(); + assertTableEquals(expected, fromDisk); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_metadata")); + assertTableEquals(expected, fromDiskWithMetadata); + } + + @Test + public void writeKeyValuePartitionedDataWithIntegerPartitionsTest() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofInt("PC1").withPartitioning(), + ColumnDefinition.ofInt("PC2").withPartitioning(), + ColumnDefinition.ofLong("I")); + final Table indexedtable = ((QueryTable) TableTools.emptyTable(1_000_000) + .updateView("PC1 = (int)(ii%3)", + "PC2 = (int)(ii%2)", + "I = ii")) + .withDefinitionUnsafe(definition); + DataIndexer.getOrCreateDataIndex(indexedtable, "I"); + + final File parentDir = new File(rootFile, "writeKeyValuePartitionedDataTest"); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .setBaseNameForPartitionedParquetData("data") + .build(); + writeKeyValuePartitionedTable(indexedtable, parentDir.getAbsolutePath(), writeInstructions); + + // Verify that metadata files are generated + assertTrue(new File(parentDir, "_common_metadata").exists()); + assertTrue(new File(parentDir, "_metadata").exists()); + + // Verify that the partitioning and indexing data exists + for (int PC1 = 0; PC1 <= 2; PC1++) { + for (int PC2 = 0; PC2 <= 1; PC2++) { + final File dir = new File(parentDir, "PC1=" + PC1 + File.separator + "PC2=" + PC2); + assertTrue(dir.exists() && dir.isDirectory()); + final File dataFile = new File(dir, "data.parquet"); + assertTrue(dataFile.exists() && dataFile.isFile()); + final File indexFile = new File(dir, ".dh_metadata/indexes/I/index_I_data.parquet"); + assertTrue(indexFile.exists() && indexFile.isFile()); + } + } + + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); + verifyIndexingInfoExists(fromDisk, "I"); + fromDisk.where("I == 3").select(); + assertTableEquals(indexedtable.sort("PC1", "PC2"), fromDisk.sort("PC1", "PC2")); + + final File commonMetadata = new File(parentDir, "_common_metadata"); + final Table fromDiskWithMetadata = readTable(commonMetadata); + fromDiskWithMetadata.where("I == 3").select(); + assertTableEquals(indexedtable.sort("PC1", "PC2"), fromDiskWithMetadata.sort("PC1", "PC2")); + + // Write the same table without generating metadata files + final File parentDirWithoutMetadata = new File(rootFile, "writeKeyValuePartitionedDataWithoutMetadata"); + writeKeyValuePartitionedTable(indexedtable, parentDirWithoutMetadata.getAbsolutePath(), EMPTY); + + // Verify that no metadata files are generated + assertFalse(new File(parentDirWithoutMetadata, "_common_metadata").exists()); + assertFalse(new File(parentDirWithoutMetadata, "_metadata").exists()); + + // Verify that the partitioning and indexing data exists + for (int PC1 = 0; PC1 <= 2; PC1++) { + for (int PC2 = 0; PC2 <= 1; PC2++) { + final File dir = new File(parentDirWithoutMetadata, "PC1=" + PC1 + File.separator + "PC2=" + PC2); + assertTrue(dir.exists() && dir.isDirectory()); + final File[] fileList = dir.listFiles(); + for (final File dataFile : fileList) { + // hidden indexing data + assertTrue(dataFile.getName().equals(".dh_metadata") + || dataFile.getName().endsWith(".parquet")); + } + } + } + final Table fromDiskWithoutMetadata = readKeyValuePartitionedTable(parentDirWithoutMetadata, EMPTY); + assertTableEquals(fromDisk, fromDiskWithoutMetadata); + verifyIndexingInfoExists(fromDiskWithoutMetadata, "I"); + } + + @Test + public void writeKeyValuePartitionedDataWithNoNonPartitioningColumnsTest() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofInt("PC1").withPartitioning(), + ColumnDefinition.ofInt("PC2").withPartitioning()); + final Table inputData = ((QueryTable) TableTools.emptyTable(20) + .updateView("PC1 = (int)(ii%3)", + "PC2 = (int)(ii%2)")) + .withDefinitionUnsafe(definition); + + final File parentDir = new File(rootFile, "writeKeyValuePartitionedDataTest"); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .setBaseNameForPartitionedParquetData("data") + .build(); + try { + writeKeyValuePartitionedTable(inputData, parentDir.getAbsolutePath(), writeInstructions); + fail("Expected exception when writing the partitioned table with no non-partitioning columns"); + } catch (final RuntimeException expected) { + } + } + + @Test + public void writeKeyValuePartitionedDataWithNonUniqueKeys() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofInt("PC1").withPartitioning(), + ColumnDefinition.ofLong("I")); + final Table inputData = ((QueryTable) TableTools.emptyTable(10) + .updateView("PC1 = (int)(ii%3)", + "I = ii")) + .withDefinitionUnsafe(definition); + final PartitionedTable partitionedTable = inputData.partitionBy("PC1"); + final Table internalTable = partitionedTable.table(); + final Table internalTableDuplicated = merge(internalTable, internalTable); + final PartitionedTable partitionedTableWithDuplicatedKeys = PartitionedTableFactory.of(internalTableDuplicated); + assertFalse(partitionedTableWithDuplicatedKeys.uniqueKeys()); + final File parentDir = new File(rootFile, "writeKeyValuePartitionedDataWithNonUniqueKeys"); + ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .setBaseNameForPartitionedParquetData("{partitions}-data") + .build(); + try { + writeKeyValuePartitionedTable(partitionedTableWithDuplicatedKeys, parentDir.getAbsolutePath(), + writeInstructions); + fail("Expected exception when writing the partitioned table with non-unique keys without {i} or {uuid} in " + + "base name"); + } catch (final RuntimeException expected) { + } + + // Write the partitioned table with non-unique keys with {i} in the base name + writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .setBaseNameForPartitionedParquetData("{partitions}-data-{i}") + .build(); + writeKeyValuePartitionedTable(partitionedTableWithDuplicatedKeys, parentDir.getAbsolutePath(), + writeInstructions); + + // Verify that the partitioned data exists + for (int PC1 = 0; PC1 <= 2; PC1++) { + final File dir = new File(parentDir, "PC1=" + PC1 + File.separator); + assertTrue(dir.exists() && dir.isDirectory()); + final String[] dataFileList = dir.list(); + assertEquals(2, dataFileList.length); + for (final String dataFile : dataFileList) { + assertTrue(dataFile.equals("PC1=" + PC1 + "-data-0.parquet") || + dataFile.equals("PC1=" + PC1 + "-data-1.parquet")); + } + } + + final Table expected = merge(inputData, inputData); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); + assertTableEquals(expected.sort("PC1"), fromDisk.sort("PC1")); + + final File commonMetadata = new File(parentDir, "_common_metadata"); + final Table fromDiskWithMetadata = readTable(commonMetadata); + assertTableEquals(expected.sort("PC1"), fromDiskWithMetadata.sort("PC1")); + + FileUtils.deleteRecursively(parentDir); + + // Write the partitioned table with non-unique keys with {uuid} in the base name + writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .setBaseNameForPartitionedParquetData("data-{uuid}") + .build(); + writeKeyValuePartitionedTable(partitionedTableWithDuplicatedKeys, parentDir.getAbsolutePath(), + writeInstructions); + + // Verify that the partitioned data exists with uuid in names + for (int PC1 = 0; PC1 <= 2; PC1++) { + final File dir = new File(parentDir, "PC1=" + PC1 + File.separator); + assertTrue(dir.exists() && dir.isDirectory()); + final String[] dataFileList = dir.list(); + assertEquals(2, dataFileList.length); + for (final String dataFile : dataFileList) { + assertTrue(dataFile.startsWith("data-") && dataFile.endsWith(".parquet")); + } + } + + FileUtils.deleteRecursively(parentDir); + + // Write the partitioned table with non-unique keys without a base name + writeKeyValuePartitionedTable(partitionedTableWithDuplicatedKeys, parentDir.getAbsolutePath(), + writeInstructions); + + // Verify that the partitioned data exists with uuid in names + for (int PC1 = 0; PC1 <= 2; PC1++) { + final File dir = new File(parentDir, "PC1=" + PC1 + File.separator); + assertTrue(dir.exists() && dir.isDirectory()); + final String[] dataFileList = dir.list(); + assertEquals(2, dataFileList.length); + for (final String dataFile : dataFileList) { + assertTrue(dataFile.endsWith(".parquet")); + } + } + } + + @Test + public void writeKeyValuePartitionedDataWithNullKeys() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofInt("PC1").withPartitioning(), + ColumnDefinition.ofLong("I")); + final Table inputData = ((QueryTable) TableTools.emptyTable(10) + .updateView("PC1 = (ii%2==0)? null : (int)(ii%2)", + "I = ii")) + .withDefinitionUnsafe(definition); + final PartitionedTable partitionedTable = inputData.partitionBy("PC1"); + final File parentDir = new File(rootFile, "writeKeyValuePartitionedDataWithNullKeys"); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + writeKeyValuePartitionedTable(partitionedTable, parentDir.getAbsolutePath(), writeInstructions); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY).select(); + assertTableEquals(inputData.sort("PC1"), fromDisk.sort("PC1")); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")).select(); + assertTableEquals(inputData.sort("PC1"), fromDiskWithMetadata.sort("PC1")); + } + + @Test + public void writeKeyValuePartitionedDataWithMixedPartitionsTest() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofInt("PC1").withPartitioning(), + ColumnDefinition.ofChar("PC2"), + ColumnDefinition.ofString("PC3"), + ColumnDefinition.ofInt("PC4").withPartitioning(), + ColumnDefinition.ofLong("II"), + ColumnDefinition.ofInt("I")); + final Table inputData = ((QueryTable) TableTools.emptyTable(10) + .updateView("PC1 = (int)(ii%3)", + "PC2 = (char)(65 + (ii % 2))", + "PC3 = java.time.LocalDate.ofEpochDay(i%2).toString()", + "PC4 = (int)(ii%4)", + "II = ii", + "I = i")) + .withDefinitionUnsafe(definition); + + // We skip one partitioning and one non partitioing column in the definition, and add some more partitioning + // and non-partitioning columns + final TableDefinition tableDefinitionToWrite = TableDefinition.of( + ColumnDefinition.ofInt("PC1").withPartitioning(), + ColumnDefinition.ofChar("PC2").withPartitioning(), + ColumnDefinition.ofString("PC3").withPartitioning(), + ColumnDefinition.ofInt("I"), + ColumnDefinition.ofInt("J")); + + final File parentDir = new File(rootFile, "writeKeyValuePartitionedDataTest"); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + writeKeyValuePartitionedTable(inputData, tableDefinitionToWrite, parentDir.getPath(), writeInstructions); + + // Verify that the partitioned data exists + for (int PC1 = 0; PC1 <= 2; PC1++) { + for (int idx = 0; idx <= 1; idx++) { + final char PC2 = (char) ('A' + idx); + final String PC3 = java.time.LocalDate.ofEpochDay(idx).toString(); + final File dir = new File(parentDir, "PC1=" + PC1 + File.separator + "PC2=" + PC2 + + File.separator + "PC3=" + PC3); + assertTrue(dir.exists() && dir.isDirectory()); + final String[] dataFileList = dir.list(); + assertEquals(1, dataFileList.length); + assertTrue(dataFileList[0].endsWith(".parquet")); + } + } + + // Give then updated table definition used to write the data, we drop the column "II" and add a new column "J" + final Table expected = inputData.dropColumns("II", "PC4").updateView("J = (int)null"); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); + assertTableEquals(expected.sort("PC1", "PC2"), fromDisk.sort("PC1", "PC2")); + + final File commonMetadata = new File(parentDir, "_common_metadata"); + final Table fromDiskWithMetadata = readTable(commonMetadata); + assertTableEquals(expected.sort("PC1", "PC2"), fromDiskWithMetadata.sort("PC1", "PC2")); + + // Delete some files from the partitioned data and read the required rows to verify that we only read the + // required partitions + FileUtils.deleteRecursivelyOnNFS(new File(parentDir, "PC1=0")); + FileUtils.deleteRecursivelyOnNFS(new File(parentDir, "PC1=1")); + assertTableEquals(expected.where("PC1 == 2").sort("PC1", "PC2", "PC3"), + readTable(commonMetadata).where("PC1 == 2").sort("PC1", "PC2", "PC3")); + } + + @Test + public void someMoreKeyValuePartitionedTestsWithComplexKeys() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofString("symbol").withPartitioning(), + ColumnDefinition.ofString("epic_collection_id"), + ColumnDefinition.ofString("epic_request_id"), + ColumnDefinition.ofLong("I")); + final Table inputData = ((QueryTable) TableTools.emptyTable(10) + .updateView("symbol = (i % 2 == 0) ? `AA` : `BB`", + "epic_collection_id = (i % 2 == 0) ? `fss_tick%1234%4321` : `fss_tick%5678%8765`", + "epic_request_id = (i % 2 == 0) ? `223ea-asd43` : `98dce-oiu23`", + "I = ii")) + .withDefinitionUnsafe(definition); + + final File parentDir = new File(rootFile, "someTest"); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + final String[] partitioningCols = new String[] {"symbol", "epic_collection_id", "epic_request_id"}; + final PartitionedTable partitionedTable = inputData.partitionBy(partitioningCols); + writeKeyValuePartitionedTable(partitionedTable, parentDir.getPath(), writeInstructions); + + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); + for (final String col : partitioningCols) { + assertTrue(fromDisk.getDefinition().getColumn(col).isPartitioning()); + } + assertTableEquals(inputData.sort("symbol", "epic_collection_id"), + fromDisk.sort("symbol", "epic_collection_id")); + + final File commonMetadata = new File(parentDir, "_common_metadata"); + final Table fromDiskWithMetadata = readTable(commonMetadata); + assertTableEquals(inputData.sort("symbol", "epic_collection_id"), + fromDiskWithMetadata.sort("symbol", "epic_collection_id")); + } + + @Test + public void testAllPartitioningColumnTypes() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofString("PC1").withPartitioning(), + ColumnDefinition.ofBoolean("PC2").withPartitioning(), + ColumnDefinition.ofChar("PC3").withPartitioning(), + ColumnDefinition.ofByte("PC4").withPartitioning(), + ColumnDefinition.ofShort("PC5").withPartitioning(), + ColumnDefinition.ofInt("PC6").withPartitioning(), + ColumnDefinition.ofLong("PC7").withPartitioning(), + ColumnDefinition.ofFloat("PC8").withPartitioning(), + ColumnDefinition.ofDouble("PC9").withPartitioning(), + ColumnDefinition.of("PC10", Type.find(BigInteger.class)).withPartitioning(), + ColumnDefinition.of("PC11", Type.find(BigDecimal.class)).withPartitioning(), + ColumnDefinition.of("PC12", Type.find(Instant.class)).withPartitioning(), + ColumnDefinition.of("PC13", Type.find(LocalDate.class)).withPartitioning(), + ColumnDefinition.of("PC14", Type.find(LocalTime.class)).withPartitioning(), + ColumnDefinition.ofInt("data")); + + final Table inputData = ((QueryTable) TableTools.emptyTable(10).updateView( + "PC1 = (ii%2 == 0) ? null: ((ii%3 == 0) ? `AA` : `BB`)", + "PC2 = (ii%2 == 0) ? null : (ii % 3 == 0)", + "PC3 = (ii%2 == 0) ? null : (char)(65 + (ii % 2))", + "PC4 = (ii%2 == 0) ? null : (byte)(ii % 2)", + "PC5 = (ii%2 == 0) ? null : (short)(ii % 2)", + "PC6 = (ii%2 == 0) ? null : (int)(ii%3)", + "PC7 = (ii%2 == 0) ? null : (long)(ii%2)", + "PC8 = (ii%2 == 0) ? null : (float)(ii % 2)", + "PC9 = (ii%2 == 0) ? null : (double)(ii % 2)", + "PC10 = (ii%2 == 0) ? null : java.math.BigInteger.valueOf(ii)", + "PC11 = (ii%2 == 0) ? null : ((ii%3 == 0) ? java.math.BigDecimal.valueOf((double)ii) : java.math.BigDecimal.valueOf((double)(ii*0.001)))", + "PC12 = (ii%2 == 0) ? null : java.time.Instant.ofEpochSecond(ii)", + "PC13 = (ii%2 == 0) ? null : java.time.LocalDate.ofEpochDay(ii)", + "PC14 = (ii%2 == 0) ? null : java.time.LocalTime.of(i%24, i%60, (i+10)%60)", + "data = (int)(ii)")) + .withDefinitionUnsafe(definition); + + final File parentDir = new File(rootFile, "testAllPartitioningColumnTypes"); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + writeKeyValuePartitionedTable(inputData, parentDir.getPath(), writeInstructions); + + // Verify that we can read the partition values, but types like LocalDate or LocalTime will be read as strings + // Therefore, we cannot compare the tables directly + readKeyValuePartitionedTable(parentDir, EMPTY).select(); + + // Reading with metadata file should deduce the correct type, so we can compare the tables + final File commonMetadata = new File(parentDir, "_common_metadata"); + final Table fromDiskWithMetadata = readTable(commonMetadata); + final String[] partitioningColumns = definition.getPartitioningColumns().stream() + .map(ColumnDefinition::getName).toArray(String[]::new); + assertTableEquals(inputData.sort(partitioningColumns), fromDiskWithMetadata.sort(partitioningColumns)); + } + + @Test + public void testAllNonPartitioningColumnTypes() { + final TableDefinition definition = TableDefinition.of( + ColumnDefinition.ofString("PC1").withPartitioning(), + ColumnDefinition.ofInt("PC2").withPartitioning(), + ColumnDefinition.ofString("NPC1"), + ColumnDefinition.ofBoolean("NPC2"), + ColumnDefinition.ofChar("NPC3"), + ColumnDefinition.ofByte("NPC4"), + ColumnDefinition.ofShort("NPC5"), + ColumnDefinition.ofInt("NPC6"), + ColumnDefinition.ofLong("NPC7"), + ColumnDefinition.ofFloat("NPC8"), + ColumnDefinition.ofDouble("NPC9"), + ColumnDefinition.of("NPC10", Type.find(BigInteger.class)), + ColumnDefinition.of("bdColumn", Type.find(BigDecimal.class)), + ColumnDefinition.of("NPC12", Type.find(Instant.class)), + ColumnDefinition.of("NPC13", Type.find(LocalDate.class)), + ColumnDefinition.of("NPC14", Type.find(LocalTime.class))); + + Table inputData = ((QueryTable) TableTools.emptyTable(10).updateView( + "PC1 = (ii%2 == 0) ? `AA` : `BB`", + "PC2 = (int)(ii%3)", + "NPC1 = (ii%2 == 0) ? `AA` : `BB`", + "NPC2 = (ii % 2 == 0)", + "NPC3 = (char)(65 + (ii % 2))", + "NPC4 = (byte)(ii % 2)", + "NPC5 = (short)(ii % 2)", + "NPC6 = (int)(ii%3)", + "NPC7 = (long)(ii%2)", + "NPC8 = (float)(ii % 2)", + "NPC9 = (double)(ii % 2)", + "NPC10 = java.math.BigInteger.valueOf(ii)", + "bdColumn = (ii%2 == 0) ? java.math.BigDecimal.valueOf((double)ii) : java.math.BigDecimal.valueOf((double)(ii*0.001))", + "NPC12 = java.time.Instant.ofEpochSecond(ii)", + "NPC13 = java.time.LocalDate.ofEpochDay(ii)", + "NPC14 = java.time.LocalTime.of(i%24, i%60, (i+10)%60)")) + .withDefinitionUnsafe(definition); + + final File parentDir = new File(rootFile, "testAllNonPartitioningColumnTypes"); + final ParquetInstructions writeInstructions = ParquetInstructions.builder() + .setGenerateMetadataFiles(true) + .build(); + + // The First API we test is 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 + final Table bigDecimalFixedInputData = maybeFixBigDecimal(inputData); + + final String[] partitioningColumns = definition.getPartitioningColumns().stream() + .map(ColumnDefinition::getName).toArray(String[]::new); + { + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY).select(); + assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), fromDisk.sort(partitioningColumns)); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); + assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), + fromDiskWithMetadata.sort(partitioningColumns)); + FileUtils.deleteRecursively(parentDir); + } + + // Next API we test is to pass additional indexing columns + final String indexColumn = "NPC5"; + final String[][] indexColumns = new String[][] {{indexColumn}}; + { + writeKeyValuePartitionedTable(inputData, parentDir.getPath(), writeInstructions, indexColumns); + assertFalse(DataIndexer.hasDataIndex(inputData, indexColumn)); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); + verifyIndexingInfoExists(fromDisk, indexColumn); + assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), fromDisk.sort(partitioningColumns)); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); + assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), + fromDiskWithMetadata.sort(partitioningColumns)); + FileUtils.deleteRecursively(parentDir); + } + + // Next API we test is passing the partitioned table without any table definition + final PartitionedTable partitionedTable = inputData.partitionBy("PC1"); + { + writeKeyValuePartitionedTable(partitionedTable, parentDir.getPath(), writeInstructions); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY).select(); + assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), fromDisk.sort(partitioningColumns)); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); + assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), + fromDiskWithMetadata.sort(partitioningColumns)); + FileUtils.deleteRecursively(parentDir); + } + + // Next API we test is to pass additional indexing columns with partitioned table and no definition + { + writeKeyValuePartitionedTable(partitionedTable, parentDir.getPath(), writeInstructions, indexColumns); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); + verifyIndexingInfoExists(fromDisk, "NPC5"); + assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), fromDisk.sort(partitioningColumns)); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); + assertTableEquals(bigDecimalFixedInputData.sort(partitioningColumns), + fromDiskWithMetadata.sort(partitioningColumns)); + FileUtils.deleteRecursively(parentDir); + } + + // Next API we test is 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() + .filter(cd -> (cd.getName() != "PC2") && (cd.getName() != "NPC6")) + .collect(Collectors.toList()); + newColumns.add(ColumnDefinition.ofInt("NPC15")); + final TableDefinition newDefinition = TableDefinition.of(newColumns); + final Table expected = bigDecimalFixedInputData.dropColumns("PC2", "NPC6").updateView("NPC15 = (int)null"); + { + writeKeyValuePartitionedTable(inputData, newDefinition, parentDir.getAbsolutePath(), writeInstructions); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY).select(); + assertTableEquals(expected.sort("PC1"), fromDisk.sort("PC1")); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); + assertTableEquals(expected.sort("PC1"), fromDiskWithMetadata.sort("PC1")); + FileUtils.deleteRecursively(parentDir); + } + + // Next API to test takes table with updated definition and additional indexing columns + { + writeKeyValuePartitionedTable(inputData, newDefinition, parentDir.getAbsolutePath(), writeInstructions, + indexColumns); + assertFalse(DataIndexer.hasDataIndex(inputData, indexColumn)); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); + verifyIndexingInfoExists(fromDisk, indexColumn); + assertTableEquals(expected.sort("PC1"), fromDisk.sort("PC1")); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); + assertTableEquals(expected.sort("PC1"), fromDiskWithMetadata.sort("PC1")); + FileUtils.deleteRecursively(parentDir); + } + + // Next API we test is passing the partitioned table with an updated table definition + { + writeKeyValuePartitionedTable(partitionedTable, newDefinition, parentDir.getPath(), writeInstructions); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY).select(); + assertTableEquals(expected.sort("PC1"), fromDisk.sort("PC1")); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); + assertTableEquals(expected.sort("PC1"), fromDiskWithMetadata.sort("PC1")); + FileUtils.deleteRecursively(parentDir); + } + + // Next API we test is passing the indexing columns with partitioned table and an updated table definition + { + writeKeyValuePartitionedTable(partitionedTable, newDefinition, parentDir.getPath(), writeInstructions, + indexColumns); + final Table fromDisk = readKeyValuePartitionedTable(parentDir, EMPTY); + verifyIndexingInfoExists(fromDisk, "NPC5"); + assertTableEquals(expected.sort("PC1"), fromDisk.sort("PC1")); + final Table fromDiskWithMetadata = readTable(new File(parentDir, "_common_metadata")); + assertTableEquals(expected.sort("PC1"), fromDiskWithMetadata.sort("PC1")); + FileUtils.deleteRecursively(parentDir); + } + } + + @Test public void testVectorColumns() { final Table table = getTableFlat(20000, true, false); @@ -790,7 +1433,7 @@ private Table maybeFixBigDecimal(Table toFix) { private static Table readParquetFileFromGitLFS(final File dest) { try { - return readSingleFileTable(dest, EMPTY); + return readTable(dest, EMPTY); } catch (final RuntimeException e) { if (e.getCause() instanceof InvalidParquetFileException) { final String InvalidParquetFileErrorMsgString = "Invalid parquet file detected, please ensure the " + @@ -980,6 +1623,18 @@ public void basicWriteTests() { basicWriteTestsImpl(MULTI_WRITER); } + @Test + public void readPartitionedDataGeneratedOnWindows() { + final String path = ParquetTableReadWriteTest.class + .getResource("/referencePartitionedDataFromWindows").getFile(); + final Table partitionedDataFromWindows = readParquetFileFromGitLFS(new File(path)).select(); + final Table expected = TableTools.newTable( + longCol("year", 2019, 2020, 2021, 2021, 2022, 2022), + longCol("n_legs", 5, 2, 4, 100, 2, 4), + stringCol("animal", "Brittle stars", "Flamingo", "Dog", "Centipede", "Parrot", "Horse")); + assertTableEquals(expected, partitionedDataFromWindows.sort("year")); + } + private static void basicWriteTestsImpl(TestParquetTableWriter writer) { // Create an empty parent directory final File parentDir = new File(rootFile, "tempDir"); @@ -1448,7 +2103,7 @@ public void readChangedUnderlyingFileTests() { readChangedUnderlyingFileTestsImpl(MULTI_WRITER); } - public void readChangedUnderlyingFileTestsImpl(TestParquetTableWriter writer) { + private void readChangedUnderlyingFileTestsImpl(TestParquetTableWriter writer) { // Write a table to parquet file and read it back final Table tableToSave = TableTools.emptyTable(5).update("A=(int)i", "B=(long)i", "C=(double)i"); final String filename = "readChangedUnderlyingFileTests.parquet"; @@ -1468,9 +2123,9 @@ public void readChangedUnderlyingFileTestsImpl(TestParquetTableWriter writer) { // Read back fromDisk. Since the underlying file has changed, we expect this to fail. try { - fromDisk.coalesce(); - TestCase.fail("Expected TableDataException"); - } catch (TableDataException ignored) { + fromDisk.select(); + TestCase.fail("Expected exception"); + } catch (RuntimeException ignored) { // expected } } diff --git a/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/_common_metadata b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/_common_metadata new file mode 100644 index 00000000000..439dd5c41b3 Binary files /dev/null and b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/_common_metadata differ diff --git a/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/_metadata b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/_metadata new file mode 100644 index 00000000000..d474961e0cb Binary files /dev/null and b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/_metadata differ diff --git a/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2019/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2019/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet new file mode 100644 index 00000000000..000b0d8799f --- /dev/null +++ b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2019/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:07c780c6a7092f19dd25f0c36070ca5a1e754e8fe8cf54305b56a6546fe7e4c0 +size 950 diff --git a/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2020/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2020/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet new file mode 100644 index 00000000000..2d110f68cf1 --- /dev/null +++ b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2020/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:5e76b0d962781c58d9cb5e9229b66ff2165b326e2a21bbc80f2fd71235468b14 +size 915 diff --git a/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2021/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2021/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet new file mode 100644 index 00000000000..58c883c132f --- /dev/null +++ b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2021/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:100ee2b1f38680e5536e40ee7eda1aaf5c0def26219fa993c3ccfd7b35fa2507 +size 919 diff --git a/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2022/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2022/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet new file mode 100644 index 00000000000..0850d74f0f0 --- /dev/null +++ b/extensions/parquet/table/src/test/resources/referencePartitionedDataFromWindows/year=2022/f6b06fe97c9d4692ba0abcf48f8eb32f-0.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:819cb36fd6d76c59de6c6ef11b424f1f4d42c88749d07d257b7d599a85a1c94e +size 915 diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index 4da7874ff7a..ea6b36504a4 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -6,14 +6,14 @@ Parquet files. """ from dataclasses import dataclass from enum import Enum -from typing import List, Optional, Union, Dict +from typing import List, Optional, Union, Dict, Sequence import jpy from deephaven import DHError from deephaven.column import Column from deephaven.dtypes import DType -from deephaven.table import Table +from deephaven.table import Table, PartitionedTable from deephaven.experimental import s3 _JParquetTools = jpy.get_type("io.deephaven.parquet.table.ParquetTools") @@ -43,6 +43,8 @@ def _build_parquet_instructions( is_refreshing: bool = False, for_read: bool = True, force_build: bool = False, + generate_metadata_files: Optional[bool] = None, + base_name: Optional[str] = None, special_instructions: Optional[s3.S3Instructions] = None, ): if not any( @@ -55,10 +57,12 @@ def _build_parquet_instructions( is_legacy_parquet, target_page_size is not None, is_refreshing, + generate_metadata_files is not None, + base_name is not None, special_instructions is not None ] ): - return None + return _JParquetInstructions.EMPTY builder = _JParquetInstructions.builder() if col_instructions is not None: @@ -92,6 +96,12 @@ def _build_parquet_instructions( if is_refreshing: builder.setIsRefreshing(is_refreshing) + if generate_metadata_files: + builder.setGenerateMetadataFiles(generate_metadata_files) + + if base_name: + builder.setBaseNameForPartitionedParquetData(base_name) + if special_instructions is not None: builder.setSpecialInstructions(special_instructions.j_object) @@ -144,8 +154,8 @@ def read( Args: path (str): the file or directory to examine - col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while reading, None by - default. + col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while reading particular + columns, default is None, which means no specialization for any column is_legacy_parquet (bool): if the parquet data is legacy is_refreshing (bool): if the parquet data represents a refreshing source file_layout (Optional[ParquetFileLayout]): the parquet file layout, by default None. When None, the layout is @@ -210,6 +220,10 @@ 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 delete(path: str) -> None: """ Deletes a Parquet table on disk. @@ -234,20 +248,124 @@ def write( max_dictionary_keys: Optional[int] = None, max_dictionary_size: Optional[int] = None, target_page_size: Optional[int] = None, + generate_metadata_files: Optional[bool] = None, + index_columns: Optional[Sequence[Sequence[str]]] = None ) -> None: """ Write a table to a Parquet file. Args: table (Table): the source table path (str): the destination file path; the file name should end in a ".parquet" extension. If the path - includes non-existing directories they are created. If there is an error, any intermediate directories + 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, default is None - col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while writing, default is None - compression_codec_name (Optional[str]): the default compression codec to use, if not specified, defaults to SNAPPY - max_dictionary_keys (Optional[int]): the maximum dictionary keys allowed, if not specified, defaults to 2^20 (1,048,576) - max_dictionary_size (Optional[int]): the maximum dictionary size (in bytes) allowed, defaults to 2^20 (1,048,576) + 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 + 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", + "SNAPPY", "GZIP", "LZO", "LZ4", "LZ4_RAW", "ZSTD", etc. If not specified, defaults to "SNAPPY". + max_dictionary_keys (Optional[int]): the maximum number of unique keys the writer should add to a dictionary page + before switching to non-dictionary encoding, never evaluated for non-String columns, defaults to 2^20 (1,048,576) + max_dictionary_size (Optional[int]): the maximum number of bytes the writer should add to the dictionary before + switching to non-dictionary encoding, never evaluated for non-String columns, defaults to 2^20 (1,048,576) target_page_size (Optional[int]): the target page size in bytes, if not specified, defaults to 2^20 bytes (1 MiB) + generate_metadata_files (Optional[bool]): whether to generate parquet _metadata and _common_metadata files, + defaults to False. Generating these files can help speed up reading of partitioned parquet data because these + files contain metadata (including schema) about the entire dataset, which can be used to skip reading some + files. + index_columns (Optional[Sequence[Sequence[str]]]): sequence of sequence containing the column names for indexes + to persist. The write operation will store the index info for the provided columns as sidecar tables. For + example, if the input is [["Col1"], ["Col1", "Col2"]], the write operation will store the index info for + ["Col1"] and for ["Col1", "Col2"]. By default, data indexes to write are determined by those present on the + source table. This argument can be 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. + Raises: + DHError + """ + try: + write_instructions = _build_parquet_instructions( + col_instructions=col_instructions, + compression_codec_name=compression_codec_name, + max_dictionary_keys=max_dictionary_keys, + max_dictionary_size=max_dictionary_size, + target_page_size=target_page_size, + for_read=False, + generate_metadata_files=generate_metadata_files, + ) + 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) + except Exception as e: + raise DHError(e, "failed to write to parquet data.") from e + + +def write_partitioned( + table: Union[Table, PartitionedTable], + destination_dir: str, + col_definitions: Optional[List[Column]] = None, + col_instructions: Optional[List[ColumnInstruction]] = None, + compression_codec_name: Optional[str] = None, + max_dictionary_keys: Optional[int] = None, + max_dictionary_size: Optional[int] = None, + target_page_size: Optional[int] = None, + base_name: Optional[str] = None, + generate_metadata_files: Optional[bool] = None, + index_columns: Optional[Sequence[Sequence[str]]] = None +) -> None: + """ Write table to disk in parquet format with the partitioning columns written as "key=value" format in a nested + directory structure. For example, for a partitioned column "date", we will have a directory structure like + "date=2021-01-01/.parquet", "date=2021-01-02/.parquet", etc. where "2021-01-01" and + "2021-01-02" are the partition values and "" is passed as an optional parameter. All the necessary + subdirectories are created if they do not exist. + + Args: + 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. + 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 + 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", + "SNAPPY", "GZIP", "LZO", "LZ4", "LZ4_RAW", "ZSTD", etc. If not specified, defaults to "SNAPPY". + max_dictionary_keys (Optional[int]): the maximum number of unique keys the writer should add to a dictionary page + before switching to non-dictionary encoding; never evaluated for non-String columns , defaults to 2^20 (1,048,576) + max_dictionary_size (Optional[int]): the maximum number of bytes the writer should add to the dictionary before + switching to non-dictionary encoding, never evaluated for non-String columns, defaults to 2^20 (1,048,576) + target_page_size (Optional[int]): the target page size in bytes, if not specified, defaults to 2^20 bytes (1 MiB) + base_name (Optional[str]): The base name for the individual partitioned tables, if not specified, defaults to + `{uuid}`, so files will have names of the format `.parquet` where `uuid` is a randomly generated UUID. + Users can provide the following tokens in the base_name: + - The token `{uuid}` will be replaced with a random UUID. For example, a base name of + "table-{uuid}" will result in files named like "table-8e8ab6b2-62f2-40d1-8191-1c5b70c5f330.parquet.parquet". + - The token `{partitions}` will be replaced with an underscore-delimited, concatenated string of + partition values. For example, for a base name of "{partitions}-table" and partitioning columns "PC1" and + "PC2", the file name is generated by concatenating the partition values "PC1=pc1" and "PC2=pc2" + with an underscore followed by "-table.parquet", like "PC1=pc1_PC2=pc2-table.parquet". + - The token `{i}` will be replaced with an automatically incremented integer for files in a directory. For + example, a base name of "table-{i}" will result in files named like "PC=partition1/table-0.parquet", + "PC=partition1/table-1.parquet", etc. + generate_metadata_files (Optional[bool]): whether to generate parquet _metadata and _common_metadata files, + defaults to False. Generating these files can help speed up reading of partitioned parquet data because these + files contain metadata (including schema) about the entire dataset, which can be used to skip reading some + files. + index_columns (Optional[Sequence[Sequence[str]]]): sequence of sequence containing the column names for indexes + to persist. The write operation will store the index info for the provided columns as sidecar tables. For + example, if the input is [["Col1"], ["Col1", "Col2"]], the write operation will store the index info for + ["Col1"] and for ["Col1", "Col2"]. By default, data indexes to write are determined by those present on the + source table. This argument can be 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. Raises: DHError @@ -260,22 +378,28 @@ def write( max_dictionary_size=max_dictionary_size, target_page_size=target_page_size, for_read=False, + generate_metadata_files=generate_metadata_files, + base_name=base_name, ) table_definition = None if col_definitions is not None: table_definition = _JTableDefinition.of([col.j_column_definition for col in col_definitions]) - if table_definition: - if write_instructions: - _JParquetTools.writeTable(table.j_table, path, table_definition, write_instructions) + 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.writeTable(table.j_table, _JFile(path), table_definition) + _JParquetTools.writeKeyValuePartitionedTable(table.j_object, destination_dir, write_instructions, + index_columns_array) else: - if write_instructions: - _JParquetTools.writeTable(table.j_table, _JFile(path), write_instructions) + if table_definition: + _JParquetTools.writeKeyValuePartitionedTable(table.j_object, table_definition, destination_dir, + write_instructions) else: - _JParquetTools.writeTable(table.j_table, path) + _JParquetTools.writeKeyValuePartitionedTable(table.j_object, destination_dir, write_instructions) except Exception as e: raise DHError(e, "failed to write to parquet data.") from e @@ -289,27 +413,38 @@ def batch_write( max_dictionary_keys: Optional[int] = None, max_dictionary_size: Optional[int] = None, target_page_size: Optional[int] = None, - grouping_cols: Optional[List[str]] = None, + generate_metadata_files: Optional[bool] = None, + index_columns: Optional[Sequence[Sequence[str]]] = None ): """ Writes tables to disk in parquet format to a supplied set of paths. - If you specify grouping columns, there must already be grouping information for those columns in the sources. - This can be accomplished with .groupBy().ungroup() or .sort(). - Note that either all the tables are written out successfully or none is. Args: tables (List[Table]): the source tables - paths (List[str]): the destinations paths. Any non existing directories in the paths provided are + 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 + col_definitions (List[Column]): the column definitions to use for writing. col_instructions (Optional[List[ColumnInstruction]]): instructions for customizations while writing - compression_codec_name (Optional[str]): the compression codec to use, if not specified, defaults to SNAPPY - max_dictionary_keys (Optional[int]): the maximum dictionary keys allowed, if not specified, defaults to 2^20 (1,048,576) - max_dictionary_size (Optional[int]): the maximum dictionary size (in bytes) allowed, defaults to 2^20 (1,048,576) + 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". + max_dictionary_keys (Optional[int]): the maximum number of unique keys the writer should add to a dictionary page + before switching to non-dictionary encoding; never evaluated for non-String columns , defaults to 2^20 (1,048,576) + max_dictionary_size (Optional[int]): the maximum number of bytes the writer should add to the dictionary before + switching to non-dictionary encoding, never evaluated for non-String columns, defaults to 2^20 (1,048,576) target_page_size (Optional[int]): the target page size in bytes, if not specified, defaults to 2^20 bytes (1 MiB) - grouping_cols (Optional[List[str]]): the group column names + generate_metadata_files (Optional[bool]): whether to generate parquet _metadata and _common_metadata files, + defaults to False. Generating these files can help speed up reading of partitioned parquet data because these + files contain metadata (including schema) about the entire dataset, which can be used to skip reading some + files. + index_columns (Optional[Sequence[Sequence[str]]]): sequence of sequence containing the column names for indexes + to persist. The write operation will store the index info for the provided columns as sidecar tables. For + example, if the input is [["Col1"], ["Col1", "Col2"]], the write operation will store the index info for + ["Col1"] and for ["Col1", "Col2"]. By default, data indexes to write are determined by those present on the + source table. This argument can be 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. Raises: DHError @@ -322,13 +457,15 @@ def batch_write( max_dictionary_size=max_dictionary_size, target_page_size=target_page_size, for_read=False, + generate_metadata_files=generate_metadata_files, ) table_definition = _JTableDefinition.of([col.j_column_definition for col in col_definitions]) - if grouping_cols: + 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), grouping_cols) + _j_file_array(paths), index_columns_array) else: _JParquetTools.writeTables([t.j_table for t in tables], table_definition, _j_file_array(paths)) diff --git a/py/server/tests/test_parquet.py b/py/server/tests/test_parquet.py index 782048e45f3..04627e1c392 100644 --- a/py/server/tests/test_parquet.py +++ b/py/server/tests/test_parquet.py @@ -6,15 +6,17 @@ import shutil import tempfile import unittest +import fnmatch import pandas import pyarrow.parquet from deephaven import DHError, empty_table, dtypes, new_table from deephaven import arrow as dharrow -from deephaven.column import InputColumn, Column, ColumnType +from deephaven.column import InputColumn, Column, ColumnType, string_col, int_col from deephaven.pandas import to_pandas, to_table -from deephaven.parquet import write, batch_write, read, delete, ColumnInstruction, ParquetFileLayout +from deephaven.parquet import (write, batch_write, read, delete, ColumnInstruction, ParquetFileLayout, + write_partitioned) from tests.testbase import BaseTestCase from deephaven.experimental import s3 @@ -113,7 +115,8 @@ def test_crd_with_instructions(self): # Reading with self.subTest(msg="read_table(str)"): - table2 = read(path=file_location, col_instructions=[col_inst, col_inst1], file_layout=ParquetFileLayout.SINGLE_FILE) + table2 = read(path=file_location, col_instructions=[col_inst, col_inst1], + file_layout=ParquetFileLayout.SINGLE_FILE) self.assert_table_equals(table, table2) # Delete @@ -322,7 +325,9 @@ def test_dictionary_encoding(self): "someIntColumn = i" ]) # Force "longStringColumn" to use non-dictionary encoding - write(dh_table, "data_from_dh.parquet", max_dictionary_size=100) + write(dh_table, "data_from_dh.parquet", max_dictionary_size=100, generate_metadata_files=True) + self.verify_metadata_files(".") + from_disk = read('data_from_dh.parquet', file_layout=ParquetFileLayout.SINGLE_FILE) self.assert_table_equals(dh_table, from_disk) @@ -334,6 +339,12 @@ def test_dictionary_encoding(self): self.assertTrue((metadata.row_group(0).column(2).path_in_schema == 'someIntColumn') & ('RLE_DICTIONARY' not in str(metadata.row_group(0).column(2).encodings))) + def verify_metadata_files(self, root_dir): + metadata_file_path = os.path.join(root_dir, '_metadata') + self.assertTrue(os.path.exists(metadata_file_path)) + common_metadata_file_path = os.path.join(root_dir, '_common_metadata') + self.assertTrue(os.path.exists(common_metadata_file_path)) + def test_dates_and_time(self): dh_table = empty_table(10000).update(formulas=[ "someDateColumn = i % 10 == 0 ? null : java.time.LocalDate.ofEpochDay(i)", @@ -576,6 +587,115 @@ def test_read_parquet_from_s3(self): read("s3://dh-s3-parquet-test1/multiColFile.parquet", special_instructions=s3_instructions).select() # TODO(deephaven-core#5064): Add support for local S3 testing + def verify_index_files(self, index_dir_path, expected_num_index_files=1): + self.assertTrue(os.path.exists(index_dir_path)) + self.assertTrue(len(os.listdir(index_dir_path)) == expected_num_index_files) + + def test_write_partitioned_data(self): + source = new_table([ + string_col("X", ["Aa", "Bb", "Aa", "Cc", "Bb", "Aa", "Bb", "Bb", "Cc"]), + string_col("Y", ["M", "N", "O", "N", "P", "M", "O", "P", "M"]), + int_col("Number", [55, 76, 20, 130, 230, 50, 73, 137, 214]), + ]) + partitioned_table = source.partition_by(by="X") + base_name = "test-{uuid}" + max_dictionary_keys = 1024 + + root_dir = os.path.join(self.temp_dir.name, "test_partitioned_writing") + if os.path.exists(root_dir): + shutil.rmtree(root_dir) + + def verify_table_from_disk(table): + self.assertTrue(len(table.columns)) + self.assertTrue(table.columns[0].name == "X") + self.assertTrue(table.columns[0].column_type == ColumnType.PARTITIONING) + self.assert_table_equals(table.select().sort("X", "Y"), source.sort("X", "Y")) + + def verify_file_names(): + partition_dir_path = os.path.join(root_dir, 'X=Aa') + self.assertTrue(os.path.exists(partition_dir_path)) + self.assertTrue(len(os.listdir(partition_dir_path)) == 1) + parquet_file = os.listdir(partition_dir_path)[0] + self.assertTrue(fnmatch.fnmatch(parquet_file, 'test-*.parquet')) + + # Test all different APIs + write_partitioned(partitioned_table, destination_dir=root_dir, base_name=base_name, + max_dictionary_keys=max_dictionary_keys) + from_disk = read(root_dir) + definition = from_disk.columns + verify_table_from_disk(from_disk) + verify_file_names() + from_disk = None + + shutil.rmtree(root_dir) + write_partitioned(partitioned_table, destination_dir=root_dir, + max_dictionary_keys=max_dictionary_keys, generate_metadata_files=True) + verify_table_from_disk(read(root_dir)) + self.verify_metadata_files(root_dir) + + shutil.rmtree(root_dir) + write_partitioned(partitioned_table, destination_dir=root_dir, base_name=base_name) + verify_table_from_disk(read(root_dir)) + verify_file_names() + + shutil.rmtree(root_dir) + write_partitioned(partitioned_table, destination_dir=root_dir) + verify_table_from_disk(read(root_dir)) + + shutil.rmtree(root_dir) + write_partitioned(partitioned_table, destination_dir=root_dir, index_columns=[["Y"], ["Y", "Number"]]) + verify_table_from_disk(read(root_dir)) + self.verify_index_files(os.path.join(root_dir, "X=Aa/.dh_metadata/indexes/Y")) + self.verify_index_files(os.path.join(root_dir, "X=Aa/.dh_metadata/indexes/Y,Number")) + + shutil.rmtree(root_dir) + write_partitioned(source, col_definitions=definition, destination_dir=root_dir, + base_name=base_name, max_dictionary_keys=max_dictionary_keys) + verify_table_from_disk(read(root_dir)) + verify_file_names() + + shutil.rmtree(root_dir) + write_partitioned(source, col_definitions=definition, destination_dir=root_dir, + max_dictionary_keys=max_dictionary_keys, generate_metadata_files=True) + verify_table_from_disk(read(root_dir)) + self.verify_metadata_files(root_dir) + + shutil.rmtree(root_dir) + write_partitioned(source, col_definitions=definition, destination_dir=root_dir, + base_name=base_name) + verify_table_from_disk(read(root_dir)) + verify_file_names() + + shutil.rmtree(root_dir) + write_partitioned(source, col_definitions=definition, destination_dir=root_dir) + verify_table_from_disk(read(root_dir)) + + shutil.rmtree(root_dir) + write_partitioned(source, col_definitions=definition, destination_dir=root_dir, + index_columns=[["Y"], ["Y", "Number"]]) + verify_table_from_disk(read(root_dir)) + self.verify_index_files(os.path.join(root_dir, "X=Aa/.dh_metadata/indexes/Y")) + self.verify_index_files(os.path.join(root_dir, "X=Aa/.dh_metadata/indexes/Y,Number")) + + def test_write_with_index_columns(self): + first_table = empty_table(10).update(formulas=["x=i", "y=(double)(i/10.0)", "z=(double)(i*i)"]) + write(first_table, "data_from_dh.parquet", index_columns=[["x"], ["y", "z"]]) + from_disk = read("data_from_dh.parquet") + self.assert_table_equals(first_table, from_disk) + self.verify_index_files(".dh_metadata/indexes/x") + self.verify_index_files(".dh_metadata/indexes/y,z") + shutil.rmtree(".dh_metadata") + + second_table = empty_table(10).update(formulas=["x=i*5", "y=(double)(i/5.0)", "z=(double)(i*i*i)"]) + batch_write([first_table, second_table], ["X.parquet", "Y.parquet"], index_columns=[["x"], ["y", "z"]], + col_definitions=first_table.columns) + from_disk_first_table = read("X.parquet") + self.assert_table_equals(first_table, from_disk_first_table) + from_disk_second_table = read("Y.parquet") + self.assert_table_equals(second_table, from_disk_second_table) + 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) + if __name__ == '__main__': unittest.main()