From bc72a9e0d3fcd0bde03c90961d378c98aa817c0d Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 11:09:36 -0400 Subject: [PATCH 01/11] Add support for Delta snapshot -- by version and latest snapshot. --- .../druid/delta/input/DeltaInputSource.java | 30 ++-- .../druid/delta/snapshot/LatestSnapshot.java | 34 +++++ .../delta/snapshot/SnapshotByVersion.java | 48 ++++++ .../druid/delta/snapshot/SnapshotInfo.java | 36 +++++ .../druid/delta/input/DeltaInputRowTest.java | 2 +- .../delta/input/DeltaInputSourceTest.java | 137 ++++++++++++++---- .../druid/delta/input/SnapshotDeltaTable.java | 129 +++++++++++++++++ .../src/test/resources/create_delta_table.py | 99 +++++++++---- 8 files changed, 447 insertions(+), 68 deletions(-) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotByVersion.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotInfo.java create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 01a18e9bc857..95da71be7780 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -52,6 +52,8 @@ import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.delta.filter.DeltaFilter; +import org.apache.druid.delta.snapshot.LatestSnapshot; +import org.apache.druid.delta.snapshot.SnapshotInfo; import org.apache.druid.error.InvalidInput; import org.apache.druid.utils.Streams; import org.apache.hadoop.conf.Configuration; @@ -97,11 +99,15 @@ public class DeltaInputSource implements SplittableInputSource @Nullable private final DeltaFilter filter; + @JsonProperty + private final SnapshotInfo snapshotInfo; + @JsonCreator public DeltaInputSource( @JsonProperty("tablePath") final String tablePath, @JsonProperty("deltaSplit") @Nullable final DeltaSplit deltaSplit, - @JsonProperty("filter") @Nullable final DeltaFilter filter + @JsonProperty("filter") @Nullable final DeltaFilter filter, + @JsonProperty("snapshot") @Nullable final SnapshotInfo snapshotInfo ) { if (tablePath == null) { @@ -110,6 +116,7 @@ public DeltaInputSource( this.tablePath = tablePath; this.deltaSplit = deltaSplit; this.filter = filter; + this.snapshotInfo = snapshotInfo == null ? new LatestSnapshot() : snapshotInfo; } @Override @@ -152,15 +159,15 @@ public InputSourceReader reader( } } else { final Table table = Table.forPath(engine, tablePath); - final Snapshot latestSnapshot = getLatestSnapshotForTable(table, engine); + final Snapshot snapshot = getSnapshotForTable(table, engine); - final StructType fullSnapshotSchema = latestSnapshot.getSchema(engine); + final StructType fullSnapshotSchema = snapshot.getSchema(engine); final StructType prunedSchema = pruneSchema( fullSnapshotSchema, inputRowSchema.getColumnsFilter() ); - final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(engine); + final ScanBuilder scanBuilder = snapshot.getScanBuilder(engine); if (filter != null) { scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema)); } @@ -206,17 +213,17 @@ public Stream> createSplits(InputFormat inputFormat, @Nul } final Engine engine = createDeltaEngine(); - final Snapshot latestSnapshot; + final Snapshot snapshot; final Table table = Table.forPath(engine, tablePath); try { - latestSnapshot = getLatestSnapshotForTable(table, engine); + snapshot = getSnapshotForTable(table, engine); } catch (TableNotFoundException e) { throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); } - final StructType fullSnapshotSchema = latestSnapshot.getSchema(engine); + final StructType fullSnapshotSchema = snapshot.getSchema(engine); - final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(engine); + final ScanBuilder scanBuilder = snapshot.getScanBuilder(engine); if (filter != null) { scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema)); } @@ -254,7 +261,8 @@ public InputSource withSplit(InputSplit split) return new DeltaInputSource( tablePath, split.get(), - filter + filter, + snapshotInfo ); } @@ -333,7 +341,7 @@ private CloseableIterator getTransformedDataIterator( ); } - private Snapshot getLatestSnapshotForTable(final Table table, final Engine engine) + private Snapshot getSnapshotForTable(final Table table, final Engine engine) { // Setting the LogStore class loader before calling the Delta Kernel snapshot API is required as a workaround with // the 3.2.0 Delta Kernel because the Kernel library cannot instantiate the LogStore class otherwise. Please see @@ -341,7 +349,7 @@ private Snapshot getLatestSnapshotForTable(final Table table, final Engine engin final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(LogStore.class.getClassLoader()); - return table.getLatestSnapshot(engine); + return snapshotInfo.getSnapshot(table, engine); } finally { Thread.currentThread().setContextClassLoader(currCtxCl); diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java new file mode 100644 index 000000000000..0b36cd3eb2d5 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.delta.snapshot; + +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.engine.Engine; + +public class LatestSnapshot implements SnapshotInfo +{ + @Override + public Snapshot getSnapshot(Table table, Engine engine) + { + System.out.println("LATEST SNAPSHOT"); + return table.getLatestSnapshot(engine); + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotByVersion.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotByVersion.java new file mode 100644 index 000000000000..9ceec398b0dd --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotByVersion.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.delta.snapshot; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.engine.Engine; +import org.apache.druid.error.InvalidInput; + +public class SnapshotByVersion implements SnapshotInfo +{ + @JsonProperty + private final Long version; + + @JsonCreator + public SnapshotByVersion(@JsonProperty("version") final Long version) + { + if (version == null) { + throw InvalidInput.exception("version cannot be empty or null for version snapshot."); + } + this.version = version; + } + + @Override + public Snapshot getSnapshot(Table table, Engine engine) + { + return table.getSnapshotAsOfVersion(engine, version); + } +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotInfo.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotInfo.java new file mode 100644 index 000000000000..a34a8b96a8bf --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotInfo.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.delta.snapshot; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.delta.kernel.Snapshot; +import io.delta.kernel.Table; +import io.delta.kernel.engine.Engine; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "latest", value = LatestSnapshot.class), + @JsonSubTypes.Type(name = "version", value = SnapshotByVersion.class), +}) +public interface SnapshotInfo +{ + Snapshot getSnapshot(Table table, Engine engine); +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index 4c1b57c434a2..9ba1f3ddb475 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -124,7 +124,7 @@ public void testDeltaInputRow( @ParameterizedTest(name = "{index}:with context {0}") public void testReadNonExistentTable() { - final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null, null); MatcherAssert.assertThat( Assert.assertThrows( diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index e6bcf9f5fc87..ff29f31f7922 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -31,6 +31,9 @@ import org.apache.druid.delta.filter.DeltaLessThanOrEqualsFilter; import org.apache.druid.delta.filter.DeltaNotFilter; import org.apache.druid.delta.filter.DeltaOrFilter; +import org.apache.druid.delta.snapshot.LatestSnapshot; +import org.apache.druid.delta.snapshot.SnapshotByTime; +import org.apache.druid.delta.snapshot.SnapshotByVersion; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; @@ -65,30 +68,35 @@ public static class TablePathParameterTests public static Object[][] data() { return new Object[][]{ +// { +// NonPartitionedDeltaTable.DELTA_TABLE_PATH, +// NonPartitionedDeltaTable.FULL_SCHEMA, +// NonPartitionedDeltaTable.EXPECTED_ROWS +// }, +// { +// NonPartitionedDeltaTable.DELTA_TABLE_PATH, +// NonPartitionedDeltaTable.SCHEMA_1, +// NonPartitionedDeltaTable.EXPECTED_ROWS +// }, +// { +// NonPartitionedDeltaTable.DELTA_TABLE_PATH, +// NonPartitionedDeltaTable.SCHEMA_2, +// NonPartitionedDeltaTable.EXPECTED_ROWS +// }, +// { +// PartitionedDeltaTable.DELTA_TABLE_PATH, +// PartitionedDeltaTable.FULL_SCHEMA, +// PartitionedDeltaTable.EXPECTED_ROWS +// }, +// { +// ComplexTypesDeltaTable.DELTA_TABLE_PATH, +// ComplexTypesDeltaTable.FULL_SCHEMA, +// ComplexTypesDeltaTable.EXPECTED_ROWS +// }, { - NonPartitionedDeltaTable.DELTA_TABLE_PATH, - NonPartitionedDeltaTable.FULL_SCHEMA, - NonPartitionedDeltaTable.EXPECTED_ROWS - }, - { - NonPartitionedDeltaTable.DELTA_TABLE_PATH, - NonPartitionedDeltaTable.SCHEMA_1, - NonPartitionedDeltaTable.EXPECTED_ROWS - }, - { - NonPartitionedDeltaTable.DELTA_TABLE_PATH, - NonPartitionedDeltaTable.SCHEMA_2, - NonPartitionedDeltaTable.EXPECTED_ROWS - }, - { - PartitionedDeltaTable.DELTA_TABLE_PATH, - PartitionedDeltaTable.FULL_SCHEMA, - PartitionedDeltaTable.EXPECTED_ROWS - }, - { - ComplexTypesDeltaTable.DELTA_TABLE_PATH, - ComplexTypesDeltaTable.FULL_SCHEMA, - ComplexTypesDeltaTable.EXPECTED_ROWS + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + SnapshotDeltaTable.EXPECTED_ROWS } }; } @@ -103,7 +111,7 @@ public static Object[][] data() @Test public void testSampleDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, null); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); @@ -137,7 +145,74 @@ public void testSampleDeltaTable() throws IOException @Test public void testReadDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, null); + final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + final List actualReadRows = readAllRows(inputSourceReader); + validateRows(expectedRows, actualReadRows, schema); + } + + // These should be in its own suite test or something + @Test + public void testReadSnapshotVersion0() throws IOException + { + // Only 5 records as expected + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(0L)); + + + + // Throws exception bc version = 3 doesn't exist! +// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(3L)); + + // Reads all the records, including ones from 0, 1 and 2 snapshot versions!!!!! WEIRD +// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new LatestSnapshot()); + final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + final List actualReadRows = readAllRows(inputSourceReader); + validateRows(expectedRows, actualReadRows, schema); + } + + @Test + public void testReadSnapshotVersion2() throws IOException + { + // Only 5 records as expected +// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(3L)); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new LatestSnapshot()); + + final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + final List actualReadRows = readAllRows(inputSourceReader); + validateRows(expectedRows, actualReadRows, schema); + } + + @Test + public void testReadSnapshotByTime() throws IOException + { + // Only 5 records as expected +// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(3L)); +// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByTime(1725461643515L)); + + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByTime(1725461642515L)); + + final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + final List actualReadRows = readAllRows(inputSourceReader); + validateRows(expectedRows, actualReadRows, schema); + } + + @Test + public void testReadSnapshotVersion3() throws IOException + { + // Only 5 records as expected + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(3L)); + + final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); + final List actualReadRows = readAllRows(inputSourceReader); + validateRows(expectedRows, actualReadRows, schema); + } + + @Test + public void testReadSnapshotVersionLatest() throws IOException + { + // Only 5 records as expected + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, null); + final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); final List actualReadRows = readAllRows(inputSourceReader); validateRows(expectedRows, actualReadRows, schema); @@ -269,7 +344,7 @@ public static Object[][] data() @Test public void testSampleDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, filter); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, filter, null); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); @@ -311,7 +386,7 @@ private static List> filterExpectedRows( @Test public void testReadDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, filter); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, filter, null); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); final List actualReadRows = readAllRows(inputSourceReader); validateRows(expectedRows, actualReadRows, schema); @@ -326,7 +401,7 @@ public void testNullTable() MatcherAssert.assertThat( Assert.assertThrows( DruidException.class, - () -> new DeltaInputSource(null, null, null) + () -> new DeltaInputSource(null, null, null, null) ), DruidExceptionMatcher.invalidInput().expectMessageIs( "tablePath cannot be null." @@ -337,7 +412,7 @@ public void testNullTable() @Test public void testSplitNonExistentTable() { - final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null, null); MatcherAssert.assertThat( Assert.assertThrows( @@ -353,7 +428,7 @@ public void testSplitNonExistentTable() @Test public void testReadNonExistentTable() { - final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null, null); MatcherAssert.assertThat( Assert.assertThrows( @@ -391,11 +466,13 @@ private static void validateRows( final InputRowSchema schema ) { + System.out.println("actual rows size: " + actualReadRows.size()); Assert.assertEquals(expectedRows.size(), actualReadRows.size()); for (int idx = 0; idx < expectedRows.size(); idx++) { final Map expectedRow = expectedRows.get(idx); final InputRow actualInputRow = actualReadRows.get(idx); + System.out.println("Actual row" + actualInputRow); for (String key : expectedRow.keySet()) { if (!schema.getColumnsFilter().apply(key)) { Assert.assertNull(actualInputRow.getRaw(key)); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java new file mode 100644 index 000000000000..9cde4ad886bf --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.delta.input; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.AutoTypeColumnSchema; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Refer to extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md to generate the + * sample complex types Delta Lake table used in the unit tests. + * + */ +public class SnapshotDeltaTable +{ + /** + * The Delta table path used by unit tests. + */ +// public static final String DELTA_TABLE_PATH = "src/test/resources/snapshot-table"; + public static final String DELTA_TABLE_PATH = "src/test/resources/snapshot-table"; + + /** + * The list of dimensions in the Delta table {@link #DELTA_TABLE_PATH}. + */ + public static final List DIMENSIONS = ImmutableList.of( + "id", + "array_info", + "struct_info", + "map_info" + ); + + /** + * The expected set of rows from the first checkpoint file {@code {@link #DELTA_TABLE_PATH}/_delta_log/00000000000000000000.json} + */ + private static final List> SPLIT_0_EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "id", 0L, + "array_info", ImmutableList.of(0, 1, 2, 3), + "struct_info", ImmutableMap.of("id", 0L, "snapshotVersion", "0"), + "map_info", ImmutableMap.of("key1", 0, "snapshotVersion", 0) + ), + ImmutableMap.of( + "id", 1L, + "array_info", ImmutableList.of(1, 2, 3, 4), + "struct_info", ImmutableMap.of("id", 1L, "snapshotVersion", "0"), + "map_info", ImmutableMap.of("key1", 1, "snapshotVersion", 0) + ), + ImmutableMap.of( + "id", 2L, + "array_info", ImmutableList.of(2, 3, 4, 5), + "struct_info", ImmutableMap.of("id", 2L, "snapshotVersion", "0"), + "map_info", ImmutableMap.of("key1", 2, "snapshotVersion", 0) + ), + ImmutableMap.of( + "id", 3L, + "array_info", ImmutableList.of(3, 4, 5, 6), + "struct_info", ImmutableMap.of("id", 3L, "snapshotVersion", "0"), + "map_info", ImmutableMap.of("key1", 3, "snapshotVersion", 0) + ), + ImmutableMap.of( + "id", 4L, + "array_info", ImmutableList.of(4, 5, 6, 7), + "struct_info", ImmutableMap.of("id", 4L, "snapshotVersion", "0"), + "map_info", ImmutableMap.of("key1", 4, "snapshotVersion", 0) + ) + ) + ); + + /** + * Mapping of checkpoint file identifier to the list of expected rows in that checkpoint. + */ + public static final Map>> SPLIT_TO_EXPECTED_ROWS = new HashMap<>( + ImmutableMap.of( + 0, SPLIT_0_EXPECTED_ROWS + ) + ); + + /** + * Complete set of expected rows across all checkpoint files for {@link #DELTA_TABLE_PATH}. + */ + public static final List> EXPECTED_ROWS = SPLIT_TO_EXPECTED_ROWS.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + + /** + * The Druid schema used for ingestion of {@link #DELTA_TABLE_PATH}. + */ + public static final InputRowSchema FULL_SCHEMA = new InputRowSchema( + new TimestampSpec("na", "posix", DateTimes.of("2024-01-01")), + new DimensionsSpec( + ImmutableList.of( + new AutoTypeColumnSchema("id", null), + new AutoTypeColumnSchema("array_info", null), + new AutoTypeColumnSchema("struct_info", null), + new AutoTypeColumnSchema("map_info", null) + ) + ), + ColumnsFilter.all() + ); +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index a116513b01dd..b56855f50423 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -16,11 +16,21 @@ # limitations under the License. import argparse +from enum import Enum + from delta import * import pyspark from pyspark.sql.types import MapType, StructType, StructField, ShortType, StringType, TimestampType, LongType, IntegerType, DoubleType, FloatType, DateType, BooleanType, ArrayType +from pyspark.sql.functions import expr from datetime import datetime, timedelta import random +from delta.tables import DeltaTable + + +class TableType(Enum): + SIMPLE = "simple" + COMPLEX = "complex" + SNAPSHOTS = "snapshots" def config_spark_with_delta_lake(): @@ -38,22 +48,6 @@ def config_spark_with_delta_lake(): def create_dataset_with_complex_types(num_records): - """ - Create a mock dataset with records containing complex types like arrays, structs and maps. - - Parameters: - - num_records (int): Number of records to generate. - - Returns: - - Tuple: A tuple containing a list of records and the corresponding schema. - - List of Records: Each record is a tuple representing a row of data. - - StructType: The schema defining the structure of the records. - - Example: - ```python - data, schema = create_dataset_with_complex_types(10) - ``` - """ schema = StructType([ StructField("id", LongType(), False), StructField("array_info", ArrayType(IntegerType(), True), True), @@ -86,6 +80,51 @@ def create_dataset_with_complex_types(num_records): return data, schema +def create_dataset_for_snapshot(num_records): + schema = StructType([ + StructField("id", LongType(), False), + StructField("array_info", ArrayType(IntegerType(), True), True), + StructField("struct_info", StructType([ + StructField("id", LongType(), False), + StructField("nextId", LongType(), False) + ])), + StructField("map_info", MapType(StringType(), IntegerType())) + ]) + + data = [] + + for idx in range(num_records): + record = ( + idx, + (idx, idx + 1, idx + 2, idx + 3), + (idx, idx + 1), + {"snapshotVersion": 0} + ) + data.append(record) + return data, schema + + +def update_table(spark, schema, delta_table_path): + delta_table = DeltaTable.forPath(spark, delta_table_path) + + # Snapshot 1: remove record with id = 2; result = (id=1, id=3, id=4, id=5) + delta_table.delete(condition="id=2") + + # Snapshot 2: do a partial update of snapshotInfo map for id = 3 ; result = (id=1, id=3, id=4, id=5) + delta_table.update( + condition="id=3", + set={"map_info": expr("map('snapshotVersion', 2)")} + ) + + # Snapshot 3: New records to be appended; result = (id=1, id=3, id=4, id=5, id=2, id=6) + append_data = [ + (2, [2, 2, 3, 3], (10, 11), {"snapshotVersion": 3}), + (6, [6, 6, 7, 7], (11, 12), {"snapshotVersion": 3}) + ] + append_df = spark.createDataFrame(append_data, schema) + append_df.write.format("delta").mode("append").save(delta_table_path) + + def create_dataset(num_records): """ Generate a mock employee dataset with different datatypes for testing purposes. @@ -136,24 +175,24 @@ def create_dataset(num_records): data.append(record) return data, schema - def main(): parser = argparse.ArgumentParser(description="Script to write a Delta Lake table.", formatter_class=argparse.ArgumentDefaultsHelpFormatter) - parser.add_argument("--gen_complex_types", type=bool, default=False, help="Generate a Delta table with records" - " containing complex types like structs," - " maps and arrays.") + parser.add_argument('--delta_table_type', type=lambda t: TableType[t.upper()], choices=TableType, + default=TableType.SIMPLE, help='Choose a Delta table type to generate.') + parser.add_argument("--gen_snapshots", type=bool, default=False, help="Generate multiple snapshots by updating and " + "deleting records from the Delta table.") parser.add_argument('--save_path', default=None, required=True, help="Save path for Delta table") parser.add_argument('--save_mode', choices=('append', 'overwrite'), default="append", help="Specify write mode (append/overwrite)") - parser.add_argument('--partitioned_by', choices=("date", "name"), default=None, + parser.add_argument('--partitioned_by', choices=("date", "name", "id"), default=None, help="Column to partition the Delta table") parser.add_argument('--num_records', type=int, default=5, help="Specify number of Delta records to write") args = parser.parse_args() - is_gen_complex_types = args.gen_complex_types + delta_table_type = args.delta_table_type save_mode = args.save_mode save_path = args.save_path num_records = args.num_records @@ -161,21 +200,29 @@ def main(): spark = config_spark_with_delta_lake() - if is_gen_complex_types: + if delta_table_type == TableType.SIMPLE: + data, schema = create_dataset(num_records=num_records) + elif delta_table_type == TableType.COMPLEX: data, schema = create_dataset_with_complex_types(num_records=num_records) + elif delta_table_type == TableType.SNAPSHOTS: + data, schema = create_dataset_for_snapshot(num_records) else: - data, schema = create_dataset(num_records=num_records) + args.print_help() + raise Exception("Unknown value specified for --delta_table_type") df = spark.createDataFrame(data, schema=schema) if not partitioned_by: df.write.format("delta").mode(save_mode).save(save_path) else: - df.write.format("delta").partitionBy("name").mode(save_mode).save(save_path) + df.write.format("delta").partitionBy(partitioned_by).mode(save_mode).save(save_path) df.show() print(f"Generated Delta table records partitioned by {partitioned_by} in {save_path} in {save_mode} mode" - f" with {num_records} records.") + f" with {num_records} records with {delta_table_type}.") + + if delta_table_type == TableType.SNAPSHOTS: + update_table(spark, schema, save_path) if __name__ == "__main__": From dc39dcb797d7459a8e0619969169e4c0e9089886 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 11:09:59 -0400 Subject: [PATCH 02/11] Add snapshot table. --- .../_delta_log/.00000000000000000000.json.crc | Bin 0 -> 36 bytes .../_delta_log/.00000000000000000001.json.crc | Bin 0 -> 16 bytes .../_delta_log/.00000000000000000002.json.crc | Bin 0 -> 20 bytes .../_delta_log/.00000000000000000003.json.crc | Bin 0 -> 20 bytes .../_delta_log/00000000000000000000.json | 8 ++++++++ .../_delta_log/00000000000000000001.json | 2 ++ .../_delta_log/00000000000000000002.json | 3 +++ .../_delta_log/00000000000000000003.json | 3 +++ ...1d2-9642-bceaedfa7a73.c000.snappy.parquet.crc | Bin 0 -> 24 bytes ...7f-41d2-9642-bceaedfa7a73.c000.snappy.parquet | Bin 0 -> 1852 bytes ...afe-a0bd-6b9beeff31a2.c000.snappy.parquet.crc | Bin 0 -> 24 bytes ...46-4afe-a0bd-6b9beeff31a2.c000.snappy.parquet | Bin 0 -> 1854 bytes ...869-ac2f-42830c33a74f.c000.snappy.parquet.crc | Bin 0 -> 24 bytes ...625-8c55-6d50090aba83.c000.snappy.parquet.crc | Bin 0 -> 24 bytes ...60-4869-ac2f-42830c33a74f.c000.snappy.parquet | Bin 0 -> 1885 bytes ...ca-4625-8c55-6d50090aba83.c000.snappy.parquet | Bin 0 -> 1852 bytes ...64b-92f6-c329a83c5932.c000.snappy.parquet.crc | Bin 0 -> 24 bytes ...40b-849d-a9cb12bed5af.c000.snappy.parquet.crc | Bin 0 -> 24 bytes ...40-464b-92f6-c329a83c5932.c000.snappy.parquet | Bin 0 -> 1853 bytes ...31-440b-849d-a9cb12bed5af.c000.snappy.parquet | Bin 0 -> 1853 bytes ...b45-bfe1-282a2a4e6825.c000.snappy.parquet.crc | Bin 0 -> 24 bytes ...9a-4b45-bfe1-282a2a4e6825.c000.snappy.parquet | Bin 0 -> 1854 bytes ...29f-b85d-f21c310c0425.c000.snappy.parquet.crc | Bin 0 -> 24 bytes ...1b-429f-b85d-f21c310c0425.c000.snappy.parquet | Bin 0 -> 1885 bytes 24 files changed, 16 insertions(+) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000000.json.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000001.json.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000002.json.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000003.json.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000000.json create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000001.json create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000002.json create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000003.json create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00003-a1401dfe-2f46-4afe-a0bd-6b9beeff31a2.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00003-a1401dfe-2f46-4afe-a0bd-6b9beeff31a2.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/.part-00009-fb1b9372-3b9a-4b45-bfe1-282a2a4e6825.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/part-00009-fb1b9372-3b9a-4b45-bfe1-282a2a4e6825.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/.part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000000.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..ece75cfbcbdb54f321ab2e8e09ccc08c6d39d510 GIT binary patch literal 36 scmYc;N@ieSU}A{*E3jQIV`FUN{4Y^kdIS1cXC!SdRju9H&Z-a$0P!RZ`2YX_ literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000001.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..3cce5f4b5944cb6f66e71dbf51d3a342de1c126e GIT binary patch literal 16 XcmYc;N@ieSU}9ME@VL>Uk1ksPDeeYD literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000002.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000002.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..2b570b36eb3f623c6917ca367df35ee6b180592e GIT binary patch literal 20 ccmYc;N@ieSU}E_3GC6P0);sb$xHG;207?@Gc>n+a literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000003.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000003.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..e0b0c65987d1ba4454123e2e2087a6c8c0f6bec4 GIT binary patch literal 20 bcmYc;N@ieSU}DI!tuA(MXH$9dFCZ8IHRc9M literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000000.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000000.json new file mode 100644 index 000000000000..e9de73758864 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000000.json @@ -0,0 +1,8 @@ +{"commitInfo":{"timestamp":1725461637518,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[\"id\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"5","numOutputRows":"5","numOutputBytes":"9265"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"9b50d4f7-9a37-4a04-9d57-dc756ac249e0"}} +{"metaData":{"id":"f4f6337e-3a4a-4963-846b-b9bdcf599831","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"array_info\",\"type\":{\"type\":\"array\",\"elementType\":\"integer\",\"containsNull\":true},\"nullable\":true,\"metadata\":{}},{\"name\":\"struct_info\",\"type\":{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"nextId\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]},\"nullable\":true,\"metadata\":{}},{\"name\":\"map_info\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"integer\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["id"],"configuration":{},"createdTime":1725461634674}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"add":{"path":"id=0/part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet","partitionValues":{"id":"0"},"size":1852,"modificationTime":1725461637399,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"struct_info\":{\"id\":0,\"nextId\":1}},\"maxValues\":{\"struct_info\":{\"id\":0,\"nextId\":1}},\"nullCount\":{\"array_info\":0,\"struct_info\":{\"id\":0,\"nextId\":0},\"map_info\":0}}"}} +{"add":{"path":"id=1/part-00003-a1401dfe-2f46-4afe-a0bd-6b9beeff31a2.c000.snappy.parquet","partitionValues":{"id":"1"},"size":1854,"modificationTime":1725461637399,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"struct_info\":{\"id\":1,\"nextId\":2}},\"maxValues\":{\"struct_info\":{\"id\":1,\"nextId\":2}},\"nullCount\":{\"array_info\":0,\"struct_info\":{\"id\":0,\"nextId\":0},\"map_info\":0}}"}} +{"add":{"path":"id=2/part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet","partitionValues":{"id":"2"},"size":1852,"modificationTime":1725461637399,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"struct_info\":{\"id\":2,\"nextId\":3}},\"maxValues\":{\"struct_info\":{\"id\":2,\"nextId\":3}},\"nullCount\":{\"array_info\":0,\"struct_info\":{\"id\":0,\"nextId\":0},\"map_info\":0}}"}} +{"add":{"path":"id=3/part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet","partitionValues":{"id":"3"},"size":1853,"modificationTime":1725461637399,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"struct_info\":{\"id\":3,\"nextId\":4}},\"maxValues\":{\"struct_info\":{\"id\":3,\"nextId\":4}},\"nullCount\":{\"array_info\":0,\"struct_info\":{\"id\":0,\"nextId\":0},\"map_info\":0}}"}} +{"add":{"path":"id=4/part-00009-fb1b9372-3b9a-4b45-bfe1-282a2a4e6825.c000.snappy.parquet","partitionValues":{"id":"4"},"size":1854,"modificationTime":1725461637399,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"struct_info\":{\"id\":4,\"nextId\":5}},\"maxValues\":{\"struct_info\":{\"id\":4,\"nextId\":5}},\"nullCount\":{\"array_info\":0,\"struct_info\":{\"id\":0,\"nextId\":0},\"map_info\":0}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000001.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000001.json new file mode 100644 index 000000000000..96233b5a7fa6 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1725461641116,"operation":"DELETE","operationParameters":{"predicate":"[\"(id#966L = 2)\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"1852","numCopiedRows":"0","numDeletionVectorsAdded":"0","numDeletionVectorsRemoved":"0","numAddedChangeFiles":"0","executionTimeMs":"376","numDeletionVectorsUpdated":"0","numDeletedRows":"1","scanTimeMs":"375","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"c3c7b793-ed2a-4909-85ec-d83a21ae7af5"}} +{"remove":{"path":"id=2/part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet","deletionTimestamp":1725461640732,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"id":"2"},"size":1852,"stats":"{\"numRecords\":1}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000002.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000002.json new file mode 100644 index 000000000000..4b2759c09fd5 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000002.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1725461642486,"operation":"UPDATE","operationParameters":{"predicate":"[\"(id#966L = 3)\"]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"1853","numCopiedRows":"0","numDeletionVectorsAdded":"0","numDeletionVectorsRemoved":"0","numAddedChangeFiles":"0","executionTimeMs":"502","numDeletionVectorsUpdated":"0","scanTimeMs":"247","numAddedFiles":"1","numUpdatedRows":"1","numAddedBytes":"1853","rewriteTimeMs":"253"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"bfca5ef6-5491-418e-87b1-f4e56a6dceb2"}} +{"add":{"path":"id=3/part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet","partitionValues":{"id":"3"},"size":1853,"modificationTime":1725461642482,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"struct_info\":{\"id\":3,\"nextId\":4}},\"maxValues\":{\"struct_info\":{\"id\":3,\"nextId\":4}},\"nullCount\":{\"array_info\":0,\"struct_info\":{\"id\":0,\"nextId\":0},\"map_info\":0}}"}} +{"remove":{"path":"id=3/part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet","deletionTimestamp":1725461642485,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"id":"3"},"size":1853}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000003.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000003.json new file mode 100644 index 000000000000..ccb171408655 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000003.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1725461643493,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"2","numOutputRows":"2","numOutputBytes":"3770"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"e7ae23b7-161d-4226-a58b-705ff3aa252a"}} +{"add":{"path":"id=2/part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet","partitionValues":{"id":"2"},"size":1885,"modificationTime":1725461643476,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"struct_info\":{\"id\":10,\"nextId\":11}},\"maxValues\":{\"struct_info\":{\"id\":10,\"nextId\":11}},\"nullCount\":{\"array_info\":0,\"struct_info\":{\"id\":0,\"nextId\":0},\"map_info\":0}}"}} +{"add":{"path":"id=6/part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet","partitionValues":{"id":"6"},"size":1885,"modificationTime":1725461643490,"dataChange":true,"stats":"{\"numRecords\":1,\"minValues\":{\"struct_info\":{\"id\":11,\"nextId\":12}},\"maxValues\":{\"struct_info\":{\"id\":11,\"nextId\":12}},\"nullCount\":{\"array_info\":0,\"struct_info\":{\"id\":0,\"nextId\":0},\"map_info\":0}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..210b74accb3ccee24ee5b653a78843c953638984 GIT binary patch literal 24 fcmYc;N@ieSU}6Z`p;&P#E$UDFmJF#)W`!vLUNQ*2 literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f6337403471ee3384a394648a67f2504c0467595 GIT binary patch literal 1852 zcmb7F&x_ke6dsSXikdFn1TupJZZU!t8*qe;*NJ16lC+mJEQ={i4^6@{wkB&4Sym)D z$+8$qDLrmNPC53{TW`6Qu#mkax#f`LFX%PUb4l}NMzX~^ghV#pH}8G#y>H$RZSV4r zO@c@lK1cWe_}8cs6OINk5c=A}2qEn&tzzi^qe`V-fumHxUjThW9P;bKe@;~imM$hj zD;-U^2#|Vm*weSq{)EtrI1y4jfV)J#Bd>q|`)Ikq0fm~PfPg{|qZI(A5sTB|IQxkw zX)un!&mk^(eE8snR`NT2N%53y#1BIDX>41%lq_AS2)1di2IlIqt(jOh7c9YCh$ZvN zF;2F1TvGxU3`D+MU8N=&0u^LeX-u|q>99h2xp;s{oQv-;*%l(WKDn}-#w@wDlMaS_ z#GqqsY@K9D!uEE8=;j!zZES7;wyj%w7^E4&;I@jb6FlT29%TamWtt_EL8csZM0Bv@ zz*mA1@q5`1zQCCy78gNJR-ruOZf!sI^QN(bzq}8w*pnpn&AZCvyQ)rg9{x{HK$j zk^2s)+Av|y&Zb65Vt2h^OEshZ^Sd^vy549wS6bsi@DQv)VMo! z=Vrh|KXtm-_MM20glWDxU{>&%o3w#C-<)j~EQm6`%M;KK#!<$CDBYTbq0@zx;?uel z!GZNdh%FR#r{Lps_ovhPEN^K87ZZPkZ{W}1PB@Ns;q-|a)mnd^_l-}#Q1PmmC76{F zETDwB%?k*(3nX#*npmi+)PDcjrM)pl*42Cv&W(~4kUkDcFhLZPLz`em+OfU%YNhP1 z+!`YYA%xOPg3rD7UXoh~f#?tjfgDo^IprV7rH9_yZ)P7_thCs|)_(K7e!ux1tM1p| z*9jtR(jo7D{qWbBGST3u5*48@bc_&E&M5`;yMIstWO;^X5&psJOJb52A7A~hmI;Vg)fDM|cHK7(CL(fE#Gp8)U zoQgT~;xSH!eq57t7RZ8ucqHmF)yOVTK}MMdq?<_7LM9$zGD@`f6tW>i2z_>SAsn$_ zZ!_%g@BxFKHL?CV3j%hq>H1qk=!&}D0pDg(FM4hm5e#9=SpSTBe8Bxk_@51_?{9tpRd1Fo>bYU6k3TT$mdRa1rR3&Le z6dPsZmL(>@SVpSR%F&e6Qc46%lOTK6^_e!0@|!@pUnG~2H7$G^CaF~hH7x*E zIxI6uV9jguSbwO5(y7Ec!ez1$7c8s)tmdND63(Y1>j-z|rfyCF;KwI(&L>ahrJ!=R zM1DZlJ+WQ?NhNNxB-5oE;0)b^5kT>`=-%v<2Y4?4-j$$x^tnQ@rO`La66`@$*N4Hj z%|;AH+m=&q@5c+Tml}4{t|9%xoJV-avtc3bjCo`S!=LN{kC?+Erhilp&FEmn&9)hj zyID22T<$rc+5Yj+^w~g|W}5@%*gXrAHc%&zr$=nFkb(v&R1(K$LX;P#w~cA<^Ry6gisfLIsXM(&7&&- literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..570f9137742eefc5e9f4ae200a358951b1cbbff8 GIT binary patch literal 24 gcmYc;N@ieSU}ET7`A=~3h5xHL)O2M(+?Qkr0BD#A6#xJL literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..9439d2a2018c3b71903e03de8d5c9bfb7d9cf08c GIT binary patch literal 24 fcmYc;N@ieSU}6w$O?95ma7$n5@yq^AW`!vLQ^5#V literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c4fc6cb1a728d33eadd4a94c450be350557878a1 GIT binary patch literal 1885 zcmb7F&x_ke6dsRct(tYZ3G56Kh+;%6HsA^yuN!AECEZ@yvJ?}R9MXhkY)v*Ivg}B5 z+GSa2A>>ec43u6%D4~ZQ+RKtdXnV+^{RavSp@%~DR46I*l0zVQGm>m|-IT=G=*@fI z%=gXvA@|-n+axMEMwZE!_wN3VyRc9Z(k+Y-Qs74rsge`q``ZtGS+Ix>VU1`A9qUTK z>jI)YLnWmGz&D6Z{<`(mKSwQsVW|-vp{0(5U~~kK;*;UNdvy2b#s9)BrZ9m!L)OV} zH}2i9~jafC$?+u8ENOCul0;dnzI6oC{)YDU6yTW(2 zuCZXkp^ZqTh71KfHMC(#g>Y0^MiIrPWnOUBp+sg))yz&orj(XTBADv*dWn8fa@HN) zN-LF?jiMEqymW=kx@cZ;CC~~FM8DO@GJT>D7dbbf8C0{4bj@5T-VkmE447`v9$>n& z!p|(AnYP2s3$+0rNFVV0q!CKLQOVoLId}n2mbfX`f#AwpXJ!emd2f;HSHX24x%RO| zDrOBq_NK*{6+k3j(uH*F?BniS56(G&{BWPnVTw}z6jHt}lk>>AqsT(OKuTJ3DW*p^ z0o={YD8_D9MzJaqe7XPxPs>n0`cR?R(dpmHG9c=PF^aZbHfGT0u55YtS~~tlx#hOq z1~N|0A>yln3)69R!V@gRlm@q$#Kb!=Cy$18e_v&^C3)TckJn{eN^}Q+F4o2ZNtX`5ao$*)OeevZll)pEo1k*G^1f0;fSpwm9z7#Is z8FN{c-gjR)wHNo0@j=!IZw{ks);ZOt+ibbb>Pmm&2VT|duQofaX5+Ni?w@L{@ar9( zh~8F@{ttT6gP_ubM+FZa)T@7i;zbXlev?k2P+xw0^Hzpm>A^e!}rhBBEeCJiqK3$5+N9g zNO2{8ufBhIuzx#0;Vv%lJ4McrPxoKGnNIzHLMf(zfI)zJVhsHN)-_qE;Sn7H>l7A3t)_HyBFULkmS9fBoOxyoCtcgB z$vKMz3I^hlsEbr3D?kP5Me31OB8{dpaTAk%qWz7@k`N*E^||Sw&-@$9L1&G38C0x} zwLQ%D+4{2Mt@NNO%Hjg})@4n0oggF_!e+6yi@Utby-@h?4MKm=3F8165gF{*v138; z_?_^)E&S9j>!-Wo#d5ss?i+l4`4)2roIvRGEYc*<%%a6OVBL{>zxUevFdqEqx) zr09(t&4^k`iD0VIHM(M|3vrR+icvHdl`LLCo}lb-7KG?Mg&d?GBw``cD-v0+BURDo z@~48+z^<4s(w6Bhj7vS1VE3#6#@al}V}Wv4CMS_On%W*F$yEk5ngXn}S;msUn$+g8 zo+^aWQG~ULi)1=ZSeE~B&PA>xIFI(tO}sGCbrT8z-`bfIK6xr{1eIShxq{5!KqaZ& zHdNv=M`T)b37nyN&;ls_7u6fD@&KO&z{dhqkB%gYO_jct4#FIiHLd5bTCC5YwXLYj zty}THYlWIsw<<_GGT{;2bS)T&n*$zNe($!`n#66L->5O;Jf)3k87mf literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c280624b6e0c125c05e5847dba7e42bffeb1ab43 GIT binary patch literal 24 gcmYc;N@ieSU}C7>n994!a@FRllOLzO41TEq0A%h8Gynhq literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..bcc14ed7f2d955705a18b677249fa7d3139102d6 GIT binary patch literal 24 gcmYc;N@ieSU}E6QoZq*}a@FRQtIHc-2ESAQ0ARogxBvhE literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..fe2e2bc21dd3a92a1757945e9e45f714996944ca GIT binary patch literal 1853 zcmb7FO^eh>7_P2lV$@j&!737HmnJY_gf?{3)7mzJsC!wLVQH3O534d%cgpEtIvta= zIt&fsLGd;RFM9K^C&3?}%Q7rpggq|1;!XSkf`>i2`&Q+n!}K7U{z%pP^*r@{B)yA2 zH3=eJ_`G@i@2}rfi4I$hXb7D&FhWQ>mF8`q|P!b|R*@!0rOMME<$|;*Z7L4sdEI2Lv2?7%hQe9I!AR zj+38x6noiKhAMc$O&$T8G!yGNUDGI*q@>PS zf;wN8)Jyv~nZ|xe$yk+K40+4MRjQLAP(fyuhNPFR_A1447n3lP-c`tkSc2uX@!Rt?qn;)GyuTgJvA?(-235@CNdPNK;m zNgY%~RM2D3O&40g?qf;+wt$195Z06DJzqc1Ap5E=!p4LD{jA?T}kt zD{XhHZe91d@5Wa5%8nJVkxQ>Ij>h4UZ^#Zpvf%Cwh;p@4Dx#f?8E!ce_Vmj7e=Y8$V50pPD z3V20Uo?fcU=LoL{{xFB Bqj>-T literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..69c69629aecc51efc53aff9737e4939f8687672a GIT binary patch literal 1853 zcmb7F&x;dR9DjK;JC5z@BKRINkd`p8!wTt;kd4N$6x&{GSxP92hpH?undfG3G8tzk z>aqmHgQd5<*1M-dFTJVAB3{HEODo>|3j_~6s^9lsW-@H7Xp-H0-}l$&^S(bOy>EVM z5=6T2e*W8k_l{MG2}gq%2z_B;gphVzs~GxQR5?=D;kX2UU_;>aC2`1u*DrrLUL{z@ zm?5X{e9(qej=xMf!{guE&2D({m09>AK=s!2Lv2?7_ER~8nHMX zjc@KH|9vSsQ7b%M6TT@a#o404kGtq~8o zah2$H8<~c^R=yS72C`zhNqcU*F>m!;g6zcr%(Z2dy8`95PQFI&LFN50vsx8U2Nl4| zhh;7atVL}Z>koradLXg(ag{8qf)(|j*Id+E!bLQ8_wmNU)Ga6g{P1Kh_>`$U5mesj zuTYxrIy$B8pt}e;E`SnJy?iq6P|g=_=Y#)8S`1jtXJBelkLXb={joM zow{=+;Gv&7-HUrp#74q2-yAS2_aaQ%Kt0=>EtM>YGQPtT&=1B@#)2sQaT1137gmZ- z>rMm*)(;`KP}H5AkJH_oPU{73X#*F5KfpKe3wI|RM>}x(NJO>PpB8=j<1dsyDhhz2 zj9>vJ%dCl5de-ea#&F`-_ yJFRBptl#dRX|3}?`)sS>cg{2hKI6^PO}^gf_`_6)qgVgH!unSAN1BN)8uIbh`}I8aessDQuGR@6 zZPFqCet7wpQYIQ4RiYwvLdOUp<(N`XFa3v#AeZ0+26PBM;B}IiCBF|qu{u{{0JRolAyd;k#8}WsZv4jo1DDDA2@(S3XiCPnC@iO#GBsp`+ z63nTXGcO+CWatMqIcJFk3n%%0xHNT(|~jnXag$SX4{GJCZWs{^Var%Q#63RXekA;lhEXu? zM{z(Aks{c!>%@ZU^IxN%9pR@9*eK14*Q#;W!?*eF)?Ma}If2lHWuz;hSw?GR$q-VN zq!m$Yl#LBbgAN(XNHtnHnvz;diD0VH9eT^sI&qWYj#0ECTNa<7PSAF^3qtgfN{-Vv z3bBzDlu6NOB26_`^0$K9KvqoGXx9olv;NK`$ewk5rp==~5hxFez>% literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/.part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/.part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..5b96f827b0fb62b7c910ea7f987aebd8faa7d666 GIT binary patch literal 24 fcmYc;N@ieSU}E@|)m+)VRB9oIny&1J`;zPcT<{0! literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..976494b582ae50ed64fce006612dc02a456426d2 GIT binary patch literal 1885 zcmb7FPly{;7=L*)nVoin+rqxb3=9$mGi)Ia3E9|m38ii?DvN}&c+i&RCG)x+I+>(1 zliFojP>>$PV~--GlpZ|jW$_^D!Gk?{6BO|vxThYhc=90V_ul-;uxU#hHktRn@Atjm z?|pw}2d`gh6P27ItK`Su{{BwSf6izUQ<^F}S6vgrCwJi5lm3<}l7#z_`MY;VU8uTG#W+EyR1 zUDZ$nKTZgSum(1kc)-UzOay*8PNLZ;Ndr_wRPbZpOBY(mZzXSf0;iAJG(Qz@HPcg1 zT<3c`H(4;_&_<-zK!yUA2HLWuLO7}_y@+DdGA}!uP$ILTYG$t_Q%cJv5lnUZQH6d| zaW);@N-LF?jiNPKw{(Zhrf6PqCC~~FL_gQaD*Z6*D#z9HNU7%<(U z1Hkm>)t;|_rs#pW6>0-KkUrq~P9u~)P|11Z96g06OI*QqB)Ia{nXd%bqP5KRli)g% zT!+{qHM4;rdqpwk1rUjsbRnHOhq%Acg9`>AKi;Pcn6i{dLdv}=c^5hN6O+I88KL7%&_<=va< z_#4%Z+jU#WIJ1C=Zv-w($Bh|J+-UN?JLU=VSi+3Q%D$cKO}V{cr@e1C?W;Zyyx87& zZ{H5tSh(h61nv}Ggh?NmvoYFE5A(x>@A3%jqe+-BKaAg=1%bT*^TiLEb_g3b4B%{G zXxh0Rdt?9Lpjq&i#&MDO8G3!MfZM?&+=bn9GNv>BV$oNg|3dkDqaY~K2p%v(-)0Gf z+xb$sd}l0VRa!5;aB8m{Amgp95ndff^{jL1ZMWTV+x4~K%n!V}H(YP`I_=guuRA>3 qS?8nfxlYULoo$Ug#@ny8`FgA8c|&%_ybSNQ@*n!RiqHW5vHlOIrK|-2 literal 0 HcmV?d00001 From 3a6138708f3f6e11fd74058cc6d75dac971ec994 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 11:14:21 -0400 Subject: [PATCH 03/11] Rename for better consistency --- .../delta/snapshot/{SnapshotInfo.java => Snapshot.java} | 0 .../{SnapshotByVersion.java => VersionedSnapshot.java} | 5 ++--- 2 files changed, 2 insertions(+), 3 deletions(-) rename extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/{SnapshotInfo.java => Snapshot.java} (100%) rename extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/{SnapshotByVersion.java => VersionedSnapshot.java} (90%) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotInfo.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java similarity index 100% rename from extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotInfo.java rename to extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotByVersion.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java similarity index 90% rename from extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotByVersion.java rename to extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java index 9ceec398b0dd..955e45902107 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/SnapshotByVersion.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java @@ -21,12 +21,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.delta.kernel.Snapshot; import io.delta.kernel.Table; import io.delta.kernel.engine.Engine; import org.apache.druid.error.InvalidInput; -public class SnapshotByVersion implements SnapshotInfo +public class SnapshotByVersion implements Snapshot { @JsonProperty private final Long version; @@ -41,7 +40,7 @@ public SnapshotByVersion(@JsonProperty("version") final Long version) } @Override - public Snapshot getSnapshot(Table table, Engine engine) + public io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine) { return table.getSnapshotAsOfVersion(engine, version); } From 4c51a2841bdeca89242bae15868060e398c3eb3f Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 12:23:35 -0400 Subject: [PATCH 04/11] Cleanup and tests. --- .../druid/delta/input/DeltaInputSource.java | 26 ++- .../druid/delta/snapshot/LatestSnapshot.java | 5 +- .../apache/druid/delta/snapshot/Snapshot.java | 7 +- .../delta/snapshot/VersionedSnapshot.java | 15 +- .../delta/input/DeltaInputSourceTest.java | 188 ++++++++---------- .../druid/delta/input/SnapshotDeltaTable.java | 93 ++++----- .../src/test/resources/create_delta_table.py | 21 +- .../_delta_log/.00000000000000000000.json.crc | Bin 36 -> 24 bytes .../_delta_log/.00000000000000000001.json.crc | Bin 16 -> 16 bytes .../_delta_log/.00000000000000000002.json.crc | Bin 20 -> 20 bytes .../_delta_log/.00000000000000000003.json.crc | Bin 20 -> 16 bytes .../_delta_log/00000000000000000000.json | 12 +- .../_delta_log/00000000000000000001.json | 4 +- .../_delta_log/00000000000000000002.json | 6 +- .../_delta_log/00000000000000000003.json | 6 +- ...-9642-bceaedfa7a73.c000.snappy.parquet.crc | Bin 24 -> 0 bytes ...41d2-9642-bceaedfa7a73.c000.snappy.parquet | Bin 1852 -> 0 bytes ...-a0bd-6b9beeff31a2.c000.snappy.parquet.crc | Bin 24 -> 0 bytes ...4afe-a0bd-6b9beeff31a2.c000.snappy.parquet | Bin 1854 -> 0 bytes ...-ac2f-42830c33a74f.c000.snappy.parquet.crc | Bin 24 -> 0 bytes ...-8c55-6d50090aba83.c000.snappy.parquet.crc | Bin 24 -> 0 bytes ...4869-ac2f-42830c33a74f.c000.snappy.parquet | Bin 1885 -> 0 bytes ...4625-8c55-6d50090aba83.c000.snappy.parquet | Bin 1852 -> 0 bytes ...-92f6-c329a83c5932.c000.snappy.parquet.crc | Bin 24 -> 0 bytes ...-849d-a9cb12bed5af.c000.snappy.parquet.crc | Bin 24 -> 0 bytes ...464b-92f6-c329a83c5932.c000.snappy.parquet | Bin 1853 -> 0 bytes ...440b-849d-a9cb12bed5af.c000.snappy.parquet | Bin 1853 -> 0 bytes ...-bfe1-282a2a4e6825.c000.snappy.parquet.crc | Bin 24 -> 0 bytes ...4b45-bfe1-282a2a4e6825.c000.snappy.parquet | Bin 1854 -> 0 bytes ...-b85d-f21c310c0425.c000.snappy.parquet.crc | Bin 24 -> 0 bytes ...429f-b85d-f21c310c0425.c000.snappy.parquet | Bin 1885 -> 0 bytes 31 files changed, 181 insertions(+), 202 deletions(-) delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00001-f159dd53-417f-41d2-9642-bceaedfa7a73.c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00003-a1401dfe-2f46-4afe-a0bd-6b9beeff31a2.c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00003-a1401dfe-2f46-4afe-a0bd-6b9beeff31a2.c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/.part-00009-fb1b9372-3b9a-4b45-bfe1-282a2a4e6825.c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/part-00009-fb1b9372-3b9a-4b45-bfe1-282a2a4e6825.c000.snappy.parquet delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/.part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet.crc delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 95da71be7780..0675d370f70d 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -26,7 +26,6 @@ import com.google.common.primitives.Ints; import io.delta.kernel.Scan; import io.delta.kernel.ScanBuilder; -import io.delta.kernel.Snapshot; import io.delta.kernel.Table; import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.data.FilteredColumnarBatch; @@ -53,7 +52,7 @@ import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.delta.filter.DeltaFilter; import org.apache.druid.delta.snapshot.LatestSnapshot; -import org.apache.druid.delta.snapshot.SnapshotInfo; +import org.apache.druid.delta.snapshot.Snapshot; import org.apache.druid.error.InvalidInput; import org.apache.druid.utils.Streams; import org.apache.hadoop.conf.Configuration; @@ -69,9 +68,9 @@ import java.util.stream.Stream; /** - * Input source to ingest data from a Delta Lake. This input source reads the latest snapshot from a Delta table - * specified by {@code tablePath} parameter. If {@code filter} is specified, it's used at the Kernel level - * for data pruning. The filtering behavior is as follows: + * Input source to ingest data from a Delta Lake. This input source reads the given {@code snapshot} from a Delta + * table specified by {@code tablePath} parameter. If {@code snapshot} is unspecified, it defaults to the latest snapshot. + * If {@code filter} is specified, it's used at the Kernel level for data pruning. The filtering behavior is as follows: *
    *
  • When a filter is applied on a partitioned table using the partitioning columns, the filtering is guaranteed.
  • *
  • When a filter is applied on non-partitioned columns, the filtering is best-effort as the Delta @@ -81,7 +80,6 @@ *

    * We leverage the Delta Kernel APIs to interact with a Delta table. The Kernel API abstracts away the * complexities of the Delta protocol itself. - * Note: currently, the Kernel table API only supports reading from the latest snapshot. *

    */ public class DeltaInputSource implements SplittableInputSource @@ -100,14 +98,14 @@ public class DeltaInputSource implements SplittableInputSource private final DeltaFilter filter; @JsonProperty - private final SnapshotInfo snapshotInfo; + private final Snapshot snapshot; @JsonCreator public DeltaInputSource( @JsonProperty("tablePath") final String tablePath, @JsonProperty("deltaSplit") @Nullable final DeltaSplit deltaSplit, @JsonProperty("filter") @Nullable final DeltaFilter filter, - @JsonProperty("snapshot") @Nullable final SnapshotInfo snapshotInfo + @JsonProperty("snapshot") @Nullable final Snapshot snapshot ) { if (tablePath == null) { @@ -116,7 +114,7 @@ public DeltaInputSource( this.tablePath = tablePath; this.deltaSplit = deltaSplit; this.filter = filter; - this.snapshotInfo = snapshotInfo == null ? new LatestSnapshot() : snapshotInfo; + this.snapshot = snapshot == null ? new LatestSnapshot() : snapshot; } @Override @@ -159,7 +157,7 @@ public InputSourceReader reader( } } else { final Table table = Table.forPath(engine, tablePath); - final Snapshot snapshot = getSnapshotForTable(table, engine); + final io.delta.kernel.Snapshot snapshot = getSnapshotForTable(table, engine); final StructType fullSnapshotSchema = snapshot.getSchema(engine); final StructType prunedSchema = pruneSchema( @@ -213,7 +211,7 @@ public Stream> createSplits(InputFormat inputFormat, @Nul } final Engine engine = createDeltaEngine(); - final Snapshot snapshot; + final io.delta.kernel.Snapshot snapshot; final Table table = Table.forPath(engine, tablePath); try { snapshot = getSnapshotForTable(table, engine); @@ -262,7 +260,7 @@ public InputSource withSplit(InputSplit split) tablePath, split.get(), filter, - snapshotInfo + snapshot ); } @@ -341,7 +339,7 @@ private CloseableIterator getTransformedDataIterator( ); } - private Snapshot getSnapshotForTable(final Table table, final Engine engine) + private io.delta.kernel.Snapshot getSnapshotForTable(final Table table, final Engine engine) { // Setting the LogStore class loader before calling the Delta Kernel snapshot API is required as a workaround with // the 3.2.0 Delta Kernel because the Kernel library cannot instantiate the LogStore class otherwise. Please see @@ -349,7 +347,7 @@ private Snapshot getSnapshotForTable(final Table table, final Engine engine) final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(LogStore.class.getClassLoader()); - return snapshotInfo.getSnapshot(table, engine); + return snapshot.getSnapshot(table, engine); } finally { Thread.currentThread().setContextClassLoader(currCtxCl); diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java index 0b36cd3eb2d5..a7dc2b905e65 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java @@ -19,14 +19,13 @@ package org.apache.druid.delta.snapshot; -import io.delta.kernel.Snapshot; import io.delta.kernel.Table; import io.delta.kernel.engine.Engine; -public class LatestSnapshot implements SnapshotInfo +public class LatestSnapshot implements Snapshot { @Override - public Snapshot getSnapshot(Table table, Engine engine) + public io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine) { System.out.println("LATEST SNAPSHOT"); return table.getLatestSnapshot(engine); diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java index a34a8b96a8bf..28afbc9e2742 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java @@ -21,16 +21,15 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import io.delta.kernel.Snapshot; import io.delta.kernel.Table; import io.delta.kernel.engine.Engine; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "latest", value = LatestSnapshot.class), - @JsonSubTypes.Type(name = "version", value = SnapshotByVersion.class), + @JsonSubTypes.Type(name = "version", value = VersionedSnapshot.class), }) -public interface SnapshotInfo +public interface Snapshot { - Snapshot getSnapshot(Table table, Engine engine); + io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java index 955e45902107..f73b5fa0b882 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java @@ -23,15 +23,17 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.delta.kernel.Table; import io.delta.kernel.engine.Engine; +import io.delta.kernel.exceptions.KernelException; +import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; -public class SnapshotByVersion implements Snapshot +public class VersionedSnapshot implements Snapshot { @JsonProperty private final Long version; @JsonCreator - public SnapshotByVersion(@JsonProperty("version") final Long version) + public VersionedSnapshot(@JsonProperty("version") final Long version) { if (version == null) { throw InvalidInput.exception("version cannot be empty or null for version snapshot."); @@ -42,6 +44,13 @@ public SnapshotByVersion(@JsonProperty("version") final Long version) @Override public io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine) { - return table.getSnapshotAsOfVersion(engine, version); + try { + return table.getSnapshotAsOfVersion(engine, version); + } catch(KernelException ke) { + throw InvalidInput.exception( + "Error reading snapshot version[%s] from tablePath[%s]: [%s]", + version, table.getPath(engine), ke.getMessage() + ); + } } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index ff29f31f7922..e5b1ebc76f7b 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -32,8 +32,8 @@ import org.apache.druid.delta.filter.DeltaNotFilter; import org.apache.druid.delta.filter.DeltaOrFilter; import org.apache.druid.delta.snapshot.LatestSnapshot; -import org.apache.druid.delta.snapshot.SnapshotByTime; -import org.apache.druid.delta.snapshot.SnapshotByVersion; +import org.apache.druid.delta.snapshot.Snapshot; +import org.apache.druid.delta.snapshot.VersionedSnapshot; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; @@ -68,36 +68,72 @@ public static class TablePathParameterTests public static Object[][] data() { return new Object[][]{ -// { -// NonPartitionedDeltaTable.DELTA_TABLE_PATH, -// NonPartitionedDeltaTable.FULL_SCHEMA, -// NonPartitionedDeltaTable.EXPECTED_ROWS -// }, -// { -// NonPartitionedDeltaTable.DELTA_TABLE_PATH, -// NonPartitionedDeltaTable.SCHEMA_1, -// NonPartitionedDeltaTable.EXPECTED_ROWS -// }, -// { -// NonPartitionedDeltaTable.DELTA_TABLE_PATH, -// NonPartitionedDeltaTable.SCHEMA_2, -// NonPartitionedDeltaTable.EXPECTED_ROWS -// }, -// { -// PartitionedDeltaTable.DELTA_TABLE_PATH, -// PartitionedDeltaTable.FULL_SCHEMA, -// PartitionedDeltaTable.EXPECTED_ROWS -// }, -// { -// ComplexTypesDeltaTable.DELTA_TABLE_PATH, -// ComplexTypesDeltaTable.FULL_SCHEMA, -// ComplexTypesDeltaTable.EXPECTED_ROWS -// }, + { + NonPartitionedDeltaTable.DELTA_TABLE_PATH, + NonPartitionedDeltaTable.FULL_SCHEMA, + null, + NonPartitionedDeltaTable.EXPECTED_ROWS + }, + { + NonPartitionedDeltaTable.DELTA_TABLE_PATH, + NonPartitionedDeltaTable.SCHEMA_1, + null, + NonPartitionedDeltaTable.EXPECTED_ROWS + }, + { + NonPartitionedDeltaTable.DELTA_TABLE_PATH, + NonPartitionedDeltaTable.SCHEMA_2, + null, + NonPartitionedDeltaTable.EXPECTED_ROWS + }, + { + PartitionedDeltaTable.DELTA_TABLE_PATH, + PartitionedDeltaTable.FULL_SCHEMA, + null, + PartitionedDeltaTable.EXPECTED_ROWS + }, + { + ComplexTypesDeltaTable.DELTA_TABLE_PATH, + ComplexTypesDeltaTable.FULL_SCHEMA, + null, + ComplexTypesDeltaTable.EXPECTED_ROWS + }, { SnapshotDeltaTable.DELTA_TABLE_PATH, SnapshotDeltaTable.FULL_SCHEMA, - SnapshotDeltaTable.EXPECTED_ROWS - } + new VersionedSnapshot(0L), + SnapshotDeltaTable.V0_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + new VersionedSnapshot(1L), + SnapshotDeltaTable.V1_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + new VersionedSnapshot(2L), + SnapshotDeltaTable.V2_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + new VersionedSnapshot(3L), + SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + new LatestSnapshot(), + SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + null, + SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS + }, }; } @@ -106,12 +142,14 @@ public static Object[][] data() @Parameterized.Parameter(1) public InputRowSchema schema; @Parameterized.Parameter(2) + public Snapshot snapshot; + @Parameterized.Parameter(3) public List> expectedRows; @Test public void testSampleDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshot); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); @@ -145,74 +183,7 @@ public void testSampleDeltaTable() throws IOException @Test public void testReadDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, null); - final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); - final List actualReadRows = readAllRows(inputSourceReader); - validateRows(expectedRows, actualReadRows, schema); - } - - // These should be in its own suite test or something - @Test - public void testReadSnapshotVersion0() throws IOException - { - // Only 5 records as expected - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(0L)); - - - - // Throws exception bc version = 3 doesn't exist! -// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(3L)); - - // Reads all the records, including ones from 0, 1 and 2 snapshot versions!!!!! WEIRD -// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new LatestSnapshot()); - final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); - final List actualReadRows = readAllRows(inputSourceReader); - validateRows(expectedRows, actualReadRows, schema); - } - - @Test - public void testReadSnapshotVersion2() throws IOException - { - // Only 5 records as expected -// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(3L)); - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new LatestSnapshot()); - - final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); - final List actualReadRows = readAllRows(inputSourceReader); - validateRows(expectedRows, actualReadRows, schema); - } - - @Test - public void testReadSnapshotByTime() throws IOException - { - // Only 5 records as expected -// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(3L)); -// final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByTime(1725461643515L)); - - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByTime(1725461642515L)); - - final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); - final List actualReadRows = readAllRows(inputSourceReader); - validateRows(expectedRows, actualReadRows, schema); - } - - @Test - public void testReadSnapshotVersion3() throws IOException - { - // Only 5 records as expected - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, new SnapshotByVersion(3L)); - - final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); - final List actualReadRows = readAllRows(inputSourceReader); - validateRows(expectedRows, actualReadRows, schema); - } - - @Test - public void testReadSnapshotVersionLatest() throws IOException - { - // Only 5 records as expected - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, null); - + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshot); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); final List actualReadRows = readAllRows(inputSourceReader); validateRows(expectedRows, actualReadRows, schema); @@ -440,6 +411,25 @@ public void testReadNonExistentTable() ) ); } + + @Test + public void testReadNonExistentSnapshot() + { + final DeltaInputSource deltaInputSource = new DeltaInputSource( + SnapshotDeltaTable.DELTA_TABLE_PATH, + null, + null, + new VersionedSnapshot(100L) + ); + + MatcherAssert.assertThat( + Assert.assertThrows( + DruidException.class, + () -> deltaInputSource.reader(null, null, null) + ), + DruidExceptionMatcher.invalidInput().expectMessageContains("Error reading snapshot version[100]") + ); + } } private static List sampleAllRows(InputSourceReader reader) throws IOException @@ -466,13 +456,11 @@ private static void validateRows( final InputRowSchema schema ) { - System.out.println("actual rows size: " + actualReadRows.size()); Assert.assertEquals(expectedRows.size(), actualReadRows.size()); for (int idx = 0; idx < expectedRows.size(); idx++) { final Map expectedRow = expectedRows.get(idx); final InputRow actualInputRow = actualReadRows.get(idx); - System.out.println("Actual row" + actualInputRow); for (String key : expectedRow.keySet()) { if (!schema.getColumnsFilter().apply(key)) { Assert.assertNull(actualInputRow.getRaw(key)); diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java index 9cde4ad886bf..d42661593bcd 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java @@ -29,10 +29,8 @@ import org.apache.druid.segment.AutoTypeColumnSchema; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** * Refer to extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md to generate the @@ -44,72 +42,71 @@ public class SnapshotDeltaTable /** * The Delta table path used by unit tests. */ -// public static final String DELTA_TABLE_PATH = "src/test/resources/snapshot-table"; public static final String DELTA_TABLE_PATH = "src/test/resources/snapshot-table"; - /** - * The list of dimensions in the Delta table {@link #DELTA_TABLE_PATH}. - */ - public static final List DIMENSIONS = ImmutableList.of( - "id", - "array_info", - "struct_info", - "map_info" - ); - - /** - * The expected set of rows from the first checkpoint file {@code {@link #DELTA_TABLE_PATH}/_delta_log/00000000000000000000.json} - */ - private static final List> SPLIT_0_EXPECTED_ROWS = new ArrayList<>( + public static final List> V0_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( ImmutableList.of( ImmutableMap.of( "id", 0L, - "array_info", ImmutableList.of(0, 1, 2, 3), - "struct_info", ImmutableMap.of("id", 0L, "snapshotVersion", "0"), - "map_info", ImmutableMap.of("key1", 0, "snapshotVersion", 0) + "map_info", ImmutableMap.of("snapshotVersion", 0) ), ImmutableMap.of( "id", 1L, - "array_info", ImmutableList.of(1, 2, 3, 4), - "struct_info", ImmutableMap.of("id", 1L, "snapshotVersion", "0"), - "map_info", ImmutableMap.of("key1", 1, "snapshotVersion", 0) + "map_info", ImmutableMap.of("snapshotVersion", 0) ), ImmutableMap.of( "id", 2L, - "array_info", ImmutableList.of(2, 3, 4, 5), - "struct_info", ImmutableMap.of("id", 2L, "snapshotVersion", "0"), - "map_info", ImmutableMap.of("key1", 2, "snapshotVersion", 0) - ), + "map_info", ImmutableMap.of("snapshotVersion", 0) + ) + ) + ); + + public static final List> V1_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( ImmutableMap.of( - "id", 3L, - "array_info", ImmutableList.of(3, 4, 5, 6), - "struct_info", ImmutableMap.of("id", 3L, "snapshotVersion", "0"), - "map_info", ImmutableMap.of("key1", 3, "snapshotVersion", 0) + "id", 0L, + "map_info", ImmutableMap.of("snapshotVersion", 0) ), ImmutableMap.of( - "id", 4L, - "array_info", ImmutableList.of(4, 5, 6, 7), - "struct_info", ImmutableMap.of("id", 4L, "snapshotVersion", "0"), - "map_info", ImmutableMap.of("key1", 4, "snapshotVersion", 0) + "id", 2L, + "map_info", ImmutableMap.of("snapshotVersion", 0) ) ) ); - /** - * Mapping of checkpoint file identifier to the list of expected rows in that checkpoint. - */ - public static final Map>> SPLIT_TO_EXPECTED_ROWS = new HashMap<>( - ImmutableMap.of( - 0, SPLIT_0_EXPECTED_ROWS + public static final List> V2_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "id", 2L, + "map_info", ImmutableMap.of("snapshotVersion", 2) + ), + ImmutableMap.of( + "id", 0L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ) ) ); - /** - * Complete set of expected rows across all checkpoint files for {@link #DELTA_TABLE_PATH}. - */ - public static final List> EXPECTED_ROWS = SPLIT_TO_EXPECTED_ROWS.values().stream() - .flatMap(List::stream) - .collect(Collectors.toList()); + public static final List> LATEST_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "id", 1L, + "map_info", ImmutableMap.of("snapshotVersion", 3) + ), + ImmutableMap.of( + "id", 4L, + "map_info", ImmutableMap.of("snapshotVersion", 3) + ), + ImmutableMap.of( + "id", 2L, + "map_info", ImmutableMap.of("snapshotVersion", 2) + ), + ImmutableMap.of( + "id", 0L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ) + ) + ); /** * The Druid schema used for ingestion of {@link #DELTA_TABLE_PATH}. @@ -119,8 +116,6 @@ public class SnapshotDeltaTable new DimensionsSpec( ImmutableList.of( new AutoTypeColumnSchema("id", null), - new AutoTypeColumnSchema("array_info", null), - new AutoTypeColumnSchema("struct_info", null), new AutoTypeColumnSchema("map_info", null) ) ), diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index b56855f50423..feb675fa30be 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -83,11 +83,6 @@ def create_dataset_with_complex_types(num_records): def create_dataset_for_snapshot(num_records): schema = StructType([ StructField("id", LongType(), False), - StructField("array_info", ArrayType(IntegerType(), True), True), - StructField("struct_info", StructType([ - StructField("id", LongType(), False), - StructField("nextId", LongType(), False) - ])), StructField("map_info", MapType(StringType(), IntegerType())) ]) @@ -96,8 +91,6 @@ def create_dataset_for_snapshot(num_records): for idx in range(num_records): record = ( idx, - (idx, idx + 1, idx + 2, idx + 3), - (idx, idx + 1), {"snapshotVersion": 0} ) data.append(record) @@ -107,19 +100,19 @@ def create_dataset_for_snapshot(num_records): def update_table(spark, schema, delta_table_path): delta_table = DeltaTable.forPath(spark, delta_table_path) - # Snapshot 1: remove record with id = 2; result = (id=1, id=3, id=4, id=5) - delta_table.delete(condition="id=2") + # Snapshot 1: remove record with id = 2; result : (id=0, id=2) + delta_table.delete(condition="id=1") - # Snapshot 2: do a partial update of snapshotInfo map for id = 3 ; result = (id=1, id=3, id=4, id=5) + # Snapshot 2: do a partial update of snapshotInfo map for id = 2 ; result : (id=2, id=0) delta_table.update( - condition="id=3", + condition="id=2", set={"map_info": expr("map('snapshotVersion', 2)")} ) - # Snapshot 3: New records to be appended; result = (id=1, id=3, id=4, id=5, id=2, id=6) + # Snapshot 3: New records to be appended; result : (id=1, id=4, id=2, id=0) append_data = [ - (2, [2, 2, 3, 3], (10, 11), {"snapshotVersion": 3}), - (6, [6, 6, 7, 7], (11, 12), {"snapshotVersion": 3}) + (1, {"snapshotVersion": 3}), + (4, {"snapshotVersion": 3}) ] append_df = spark.createDataFrame(append_data, schema) append_df.write.format("delta").mode("append").save(delta_table_path) diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000000.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000000.json.crc index ece75cfbcbdb54f321ab2e8e09ccc08c6d39d510..8d01e0c37e778f8f28c0beed4dc9790018d863f0 100644 GIT binary patch literal 24 fcmYc;N@ieSU}ErADZiE7s1w?(UCCj>_)io7PY(v) literal 36 scmYc;N@ieSU}A{*E3jQIV`FUN{4Y^kdIS1cXC!SdRju9H&Z-a$0P!RZ`2YX_ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000001.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000001.json.crc index 3cce5f4b5944cb6f66e71dbf51d3a342de1c126e..1215445f9f283cd8d82b6ac32aa43148e2092047 100644 GIT binary patch literal 16 XcmYc;N@ieSU}E6^wD7VOhh!}PBNhaR literal 16 XcmYc;N@ieSU}9ME@VL>Uk1ksPDeeYD diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000002.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000002.json.crc index 2b570b36eb3f623c6917ca367df35ee6b180592e..012c2ad2f3c306a9ab34c5b8abf35983a0ee383c 100644 GIT binary patch literal 20 bcmYc;N@ieSU}9)0>9Km^yD@r;&bmVYK8Oc| literal 20 ccmYc;N@ieSU}E_3GC6P0);sb$xHG;207?@Gc>n+a diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000003.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000003.json.crc index e0b0c65987d1ba4454123e2e2087a6c8c0f6bec4..03c7cb26637b43d821559c70d1c89cef7e71d06f 100644 GIT binary patch literal 16 XcmYc;N@ieSU}CUaRTxops^Ag;BRBH$RZSV4r zO@c@lK1cWe_}8cs6OINk5c=A}2qEn&tzzi^qe`V-fumHxUjThW9P;bKe@;~imM$hj zD;-U^2#|Vm*weSq{)EtrI1y4jfV)J#Bd>q|`)Ikq0fm~PfPg{|qZI(A5sTB|IQxkw zX)un!&mk^(eE8snR`NT2N%53y#1BIDX>41%lq_AS2)1di2IlIqt(jOh7c9YCh$ZvN zF;2F1TvGxU3`D+MU8N=&0u^LeX-u|q>99h2xp;s{oQv-;*%l(WKDn}-#w@wDlMaS_ z#GqqsY@K9D!uEE8=;j!zZES7;wyj%w7^E4&;I@jb6FlT29%TamWtt_EL8csZM0Bv@ zz*mA1@q5`1zQCCy78gNJR-ruOZf!sI^QN(bzq}8w*pnpn&AZCvyQ)rg9{x{HK$j zk^2s)+Av|y&Zb65Vt2h^OEshZ^Sd^vy549wS6bsi@DQv)VMo! z=Vrh|KXtm-_MM20glWDxU{>&%o3w#C-<)j~EQm6`%M;KK#!<$CDBYTbq0@zx;?uel z!GZNdh%FR#r{Lps_ovhPEN^K87ZZPkZ{W}1PB@Ns;q-|a)mnd^_l-}#Q1PmmC76{F zETDwB%?k*(3nX#*npmi+)PDcjrM)pl*42Cv&W(~4kUkDcFhLZPLz`em+OfU%YNhP1 z+!`YYA%xOPg3rD7UXoh~f#?tjfgDo^IprV7rH9_yZ)P7_thCs|)_(K7e!ux1tM1p| z*9jtR(jo7D{qWbBGST3u5*48@bc_&E&M5`;yMIstWO;^X5&psJOJb52A7A~hmI;Vg)fDM|cHK7(CL(fE#Gp8)U zoQgT~;xSH!eq57t7RZ8ucqHmF)yOVTK}MMdq?<_7LM9$zGD@`f6tW>i2z_>SAsn$_ zZ!_%g@BxFKHL?CV3j%hq>H1qk=!&}D0pDg(FM4hm5e#9=SpSTBe8Bxk_@51_?{9tpRd1Fo>bYU6k3TT$mdRa1rR3&Le z6dPsZmL(>@SVpSR%F&e6Qc46%lOTK6^_e!0@|!@pUnG~2H7$G^CaF~hH7x*E zIxI6uV9jguSbwO5(y7Ec!ez1$7c8s)tmdND63(Y1>j-z|rfyCF;KwI(&L>ahrJ!=R zM1DZlJ+WQ?NhNNxB-5oE;0)b^5kT>`=-%v<2Y4?4-j$$x^tnQ@rO`La66`@$*N4Hj z%|;AH+m=&q@5c+Tml}4{t|9%xoJV-avtc3bjCo`S!=LN{kC?+Erhilp&FEmn&9)hj zyID22T<$rc+5Yj+^w~g|W}5@%*gXrAHc%&zr$=nFkb(v&R1(K$LX;P#w~cA<^Ry6gisfLIsXM(&7&&- diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet.crc deleted file mode 100644 index 570f9137742eefc5e9f4ae200a358951b1cbbff8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 24 gcmYc;N@ieSU}ET7`A=~3h5xHL)O2M(+?Qkr0BD#A6#xJL diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00005-0b5dd6e9-24ca-4625-8c55-6d50090aba83.c000.snappy.parquet.crc deleted file mode 100644 index 9439d2a2018c3b71903e03de8d5c9bfb7d9cf08c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 24 fcmYc;N@ieSU}6w$O?95ma7$n5@yq^AW`!vLQ^5#V diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00004-71fe37b0-6360-4869-ac2f-42830c33a74f.c000.snappy.parquet deleted file mode 100644 index c4fc6cb1a728d33eadd4a94c450be350557878a1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1885 zcmb7F&x_ke6dsRct(tYZ3G56Kh+;%6HsA^yuN!AECEZ@yvJ?}R9MXhkY)v*Ivg}B5 z+GSa2A>>ec43u6%D4~ZQ+RKtdXnV+^{RavSp@%~DR46I*l0zVQGm>m|-IT=G=*@fI z%=gXvA@|-n+axMEMwZE!_wN3VyRc9Z(k+Y-Qs74rsge`q``ZtGS+Ix>VU1`A9qUTK z>jI)YLnWmGz&D6Z{<`(mKSwQsVW|-vp{0(5U~~kK;*;UNdvy2b#s9)BrZ9m!L)OV} zH}2i9~jafC$?+u8ENOCul0;dnzI6oC{)YDU6yTW(2 zuCZXkp^ZqTh71KfHMC(#g>Y0^MiIrPWnOUBp+sg))yz&orj(XTBADv*dWn8fa@HN) zN-LF?jiMEqymW=kx@cZ;CC~~FM8DO@GJT>D7dbbf8C0{4bj@5T-VkmE447`v9$>n& z!p|(AnYP2s3$+0rNFVV0q!CKLQOVoLId}n2mbfX`f#AwpXJ!emd2f;HSHX24x%RO| zDrOBq_NK*{6+k3j(uH*F?BniS56(G&{BWPnVTw}z6jHt}lk>>AqsT(OKuTJ3DW*p^ z0o={YD8_D9MzJaqe7XPxPs>n0`cR?R(dpmHG9c=PF^aZbHfGT0u55YtS~~tlx#hOq z1~N|0A>yln3)69R!V@gRlm@q$#Kb!=Cy$18e_v&^C3)TckJn{eN^}Q+F4o2ZNtX`5ao$*)OeevZll)pEo1k*G^1f0;fSpwm9z7#Is z8FN{c-gjR)wHNo0@j=!IZw{ks);ZOt+ibbb>Pmm&2VT|duQofaX5+Ni?w@L{@ar9( zh~8F@{ttT6gP_ubM+FZa)T@7i;zbXlev?k2P+xw0^Hzpm>A^e!}rhBBEeCJiqK3$5+N9g zNO2{8ufBhIuzx#0;Vv%lJ4McrPxoKGnNIzHLMf(zfI)zJVhsHN)-_qE;Sn7H>l7A3t)_HyBFULkmS9fBoOxyoCtcgB z$vKMz3I^hlsEbr3D?kP5Me31OB8{dpaTAk%qWz7@k`N*E^||Sw&-@$9L1&G38C0x} zwLQ%D+4{2Mt@NNO%Hjg})@4n0oggF_!e+6yi@Utby-@h?4MKm=3F8165gF{*v138; z_?_^)E&S9j>!-Wo#d5ss?i+l4`4)2roIvRGEYc*<%%a6OVBL{>zxUevFdqEqx) zr09(t&4^k`iD0VIHM(M|3vrR+icvHdl`LLCo}lb-7KG?Mg&d?GBw``cD-v0+BURDo z@~48+z^<4s(w6Bhj7vS1VE3#6#@al}V}Wv4CMS_On%W*F$yEk5ngXn}S;msUn$+g8 zo+^aWQG~ULi)1=ZSeE~B&PA>xIFI(tO}sGCbrT8z-`bfIK6xr{1eIShxq{5!KqaZ& zHdNv=M`T)b37nyN&;ls_7u6fD@&KO&z{dhqkB%gYO_jct4#FIiHLd5bTCC5YwXLYj zty}THYlWIsw<<_GGT{;2bS)T&n*$zNe($!`n#66L->5O;Jf)3k87mf diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet.crc deleted file mode 100644 index c280624b6e0c125c05e5847dba7e42bffeb1ab43..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 24 gcmYc;N@ieSU}C7>n994!a@FRllOLzO41TEq0A%h8Gynhq diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/.part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet.crc deleted file mode 100644 index bcc14ed7f2d955705a18b677249fa7d3139102d6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 24 gcmYc;N@ieSU}E6QoZq*}a@FRQtIHc-2ESAQ0ARogxBvhE diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00000-ba044e48-6240-464b-92f6-c329a83c5932.c000.snappy.parquet deleted file mode 100644 index fe2e2bc21dd3a92a1757945e9e45f714996944ca..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1853 zcmb7FO^eh>7_P2lV$@j&!737HmnJY_gf?{3)7mzJsC!wLVQH3O534d%cgpEtIvta= zIt&fsLGd;RFM9K^C&3?}%Q7rpggq|1;!XSkf`>i2`&Q+n!}K7U{z%pP^*r@{B)yA2 zH3=eJ_`G@i@2}rfi4I$hXb7D&FhWQ>mF8`q|P!b|R*@!0rOMME<$|;*Z7L4sdEI2Lv2?7%hQe9I!AR zj+38x6noiKhAMc$O&$T8G!yGNUDGI*q@>PS zf;wN8)Jyv~nZ|xe$yk+K40+4MRjQLAP(fyuhNPFR_A1447n3lP-c`tkSc2uX@!Rt?qn;)GyuTgJvA?(-235@CNdPNK;m zNgY%~RM2D3O&40g?qf;+wt$195Z06DJzqc1Ap5E=!p4LD{jA?T}kt zD{XhHZe91d@5Wa5%8nJVkxQ>Ij>h4UZ^#Zpvf%Cwh;p@4Dx#f?8E!ce_Vmj7e=Y8$V50pPD z3V20Uo?fcU=LoL{{xFB Bqj>-T diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=3/part-00007-2c883e4b-2e31-440b-849d-a9cb12bed5af.c000.snappy.parquet deleted file mode 100644 index 69c69629aecc51efc53aff9737e4939f8687672a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1853 zcmb7F&x;dR9DjK;JC5z@BKRINkd`p8!wTt;kd4N$6x&{GSxP92hpH?undfG3G8tzk z>aqmHgQd5<*1M-dFTJVAB3{HEODo>|3j_~6s^9lsW-@H7Xp-H0-}l$&^S(bOy>EVM z5=6T2e*W8k_l{MG2}gq%2z_B;gphVzs~GxQR5?=D;kX2UU_;>aC2`1u*DrrLUL{z@ zm?5X{e9(qej=xMf!{guE&2D({m09>AK=s!2Lv2?7_ER~8nHMX zjc@KH|9vSsQ7b%M6TT@a#o404kGtq~8o zah2$H8<~c^R=yS72C`zhNqcU*F>m!;g6zcr%(Z2dy8`95PQFI&LFN50vsx8U2Nl4| zhh;7atVL}Z>koradLXg(ag{8qf)(|j*Id+E!bLQ8_wmNU)Ga6g{P1Kh_>`$U5mesj zuTYxrIy$B8pt}e;E`SnJy?iq6P|g=_=Y#)8S`1jtXJBelkLXb={joM zow{=+;Gv&7-HUrp#74q2-yAS2_aaQ%Kt0=>EtM>YGQPtT&=1B@#)2sQaT1137gmZ- z>rMm*)(;`KP}H5AkJH_oPU{73X#*F5KfpKe3wI|RM>}x(NJO>PpB8=j<1dsyDhhz2 zj9>vJ%dCl5de-ea#&F`-_ yJFRBptl#dRX|3}?`)sS>cg{2hKI6^PO}^gf_`_6)qgVgH!unSAN1BN)8uIbh`}I8aessDQuGR@6 zZPFqCet7wpQYIQ4RiYwvLdOUp<(N`XFa3v#AeZ0+26PBM;B}IiCBF|qu{u{{0JRolAyd;k#8}WsZv4jo1DDDA2@(S3XiCPnC@iO#GBsp`+ z63nTXGcO+CWatMqIcJFk3n%%0xHNT(|~jnXag$SX4{GJCZWs{^Var%Q#63RXekA;lhEXu? zM{z(Aks{c!>%@ZU^IxN%9pR@9*eK14*Q#;W!?*eF)?Ma}If2lHWuz;hSw?GR$q-VN zq!m$Yl#LBbgAN(XNHtnHnvz;diD0VH9eT^sI&qWYj#0ECTNa<7PSAF^3qtgfN{-Vv z3bBzDlu6NOB26_`^0$K9KvqoGXx9olv;NK`$ewk5rp==~5hxFez>% diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/.part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/.part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet.crc deleted file mode 100644 index 5b96f827b0fb62b7c910ea7f987aebd8faa7d666..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 24 fcmYc;N@ieSU}E@|)m+)VRB9oIny&1J`;zPcT<{0! diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=6/part-00009-3c56f0f3-261b-429f-b85d-f21c310c0425.c000.snappy.parquet deleted file mode 100644 index 976494b582ae50ed64fce006612dc02a456426d2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1885 zcmb7FPly{;7=L*)nVoin+rqxb3=9$mGi)Ia3E9|m38ii?DvN}&c+i&RCG)x+I+>(1 zliFojP>>$PV~--GlpZ|jW$_^D!Gk?{6BO|vxThYhc=90V_ul-;uxU#hHktRn@Atjm z?|pw}2d`gh6P27ItK`Su{{BwSf6izUQ<^F}S6vgrCwJi5lm3<}l7#z_`MY;VU8uTG#W+EyR1 zUDZ$nKTZgSum(1kc)-UzOay*8PNLZ;Ndr_wRPbZpOBY(mZzXSf0;iAJG(Qz@HPcg1 zT<3c`H(4;_&_<-zK!yUA2HLWuLO7}_y@+DdGA}!uP$ILTYG$t_Q%cJv5lnUZQH6d| zaW);@N-LF?jiNPKw{(Zhrf6PqCC~~FL_gQaD*Z6*D#z9HNU7%<(U z1Hkm>)t;|_rs#pW6>0-KkUrq~P9u~)P|11Z96g06OI*QqB)Ia{nXd%bqP5KRli)g% zT!+{qHM4;rdqpwk1rUjsbRnHOhq%Acg9`>AKi;Pcn6i{dLdv}=c^5hN6O+I88KL7%&_<=va< z_#4%Z+jU#WIJ1C=Zv-w($Bh|J+-UN?JLU=VSi+3Q%D$cKO}V{cr@e1C?W;Zyyx87& zZ{H5tSh(h61nv}Ggh?NmvoYFE5A(x>@A3%jqe+-BKaAg=1%bT*^TiLEb_g3b4B%{G zXxh0Rdt?9Lpjq&i#&MDO8G3!MfZM?&+=bn9GNv>BV$oNg|3dkDqaY~K2p%v(-)0Gf z+xb$sd}l0VRa!5;aB8m{Amgp95ndff^{jL1ZMWTV+x4~K%n!V}H(YP`I_=guuRA>3 qS?8nfxlYULoo$Ug#@ny8`FgA8c|&%_ybSNQ@*n!RiqHW5vHlOIrK|-2 From 3ee2bdf907e5cd16d330d40f30179761e7004f07 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 12:47:04 -0400 Subject: [PATCH 05/11] Doc updates. --- .../extensions-contrib/delta-lake.md | 7 +--- docs/ingestion/input-sources.md | 40 +++++++++++++++---- .../src/test/resources/README.md | 21 ++++++++-- .../src/test/resources/create_delta_table.py | 2 - 4 files changed, 51 insertions(+), 19 deletions(-) diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md index 503fbfdc55df..88f3a2c77f37 100644 --- a/docs/development/extensions-contrib/delta-lake.md +++ b/docs/development/extensions-contrib/delta-lake.md @@ -51,9 +51,4 @@ java \ -c "org.apache.druid.extensions.contrib:druid-deltalake-extensions:" ``` -See [Loading community extensions](../../configuration/extensions.md#loading-community-extensions) for more information. - -## Known limitations - -This extension relies on the Delta Kernel API and can only read from the latest Delta table snapshot. Ability to read from -arbitrary snapshots is tracked [here](https://github.com/delta-io/delta/issues/2581). \ No newline at end of file +See [Loading community extensions](../../configuration/extensions.md#loading-community-extensions) for more information. \ No newline at end of file diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index fb8e1f98c91f..1b2e02c8d770 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -1147,11 +1147,12 @@ To use the Delta Lake input source, load the extension [`druid-deltalake-extensi You can use the Delta input source to read data stored in a Delta Lake table. For a given table, the input source scans the latest snapshot from the configured table. Druid ingests the underlying delta files from the table. -| Property|Description|Required| -|---------|-----------|--------| -| type|Set this value to `delta`.|yes| -| tablePath|The location of the Delta table.|yes| -| filter|The JSON Object that filters data files within a snapshot.|no| +| Property|Description| Default|Required | +|---------|-----------|-----------------| +|type|Set this value to `delta`.| None|yes| +|tablePath|The location of the Delta table.|None|yes| +|filter|The JSON Object that filters data files within a snapshot.|None|no| +|snapshot|The JSON Object that specifies a Delta table snapshot to read from.|latest|no| ### Delta filter object @@ -1224,7 +1225,27 @@ filters on partitioned columns. | column | The table column to apply the filter on. | yes | | value | The value to use in the filter. | yes | -The following is a sample spec to read all records from the Delta table `/delta-table/foo`: +### Delta snapshot object + +You can use the `snapshot` to read the Delta table at the specific snapshot. This input source supports the following +snapshot types: `latest` and `version`. If `snapshot` is not specified, it defaults to the latest snapshot. + +`latest` snapshot: + +| Property | Description | Required | +|----------|------------------------------------|-------------| +| type | Set this value to `latest`. | no (default)| + + +`version` snapshot: + +| Property | Description | Required | +|----------|------------------------------------------|----------| +| type | Set this value to `version`. | yes | +| version | The Delta snapshot version to read from. | yes | + + +The following is a sample spec to read all records from the latest record from Delta table `/delta-table/foo`: ```json ... @@ -1237,7 +1258,8 @@ The following is a sample spec to read all records from the Delta table `/delta- } ``` -The following is a sample spec to read records from the Delta table `/delta-table/foo` to select records where `name = 'Employee4' and age >= 30`: +The following is a sample spec to read records from the Delta table `/delta-table/foo` snapshot version `3` to select records where +`name = 'Employee4' and age >= 30`: ```json ... @@ -1260,6 +1282,10 @@ The following is a sample spec to read records from the Delta table `/delta-tabl "value": "30" } ] + }, + "snapshot": { + "type": "version", + "version": 3 } }, } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md index f45b33ab62cf..03cf1505a815 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md @@ -44,18 +44,20 @@ Delta table to `resources/employee-delta-table`. You can override the defaults b ```shell python3 create_delta_table.py -h -usage: create_delta_table.py [-h] --save_path SAVE_PATH [--save_mode {append,overwrite}] [--partitioned_by {date,name}] [--num_records NUM_RECORDS] +usage: create_delta_table.py [-h] [--delta_table_type {TableType.SIMPLE,TableType.COMPLEX,TableType.SNAPSHOTS}] --save_path SAVE_PATH [--save_mode {append,overwrite}] [--partitioned_by {date,name,id}] [--num_records NUM_RECORDS] Script to write a Delta Lake table. options: -h, --help show this help message and exit + --delta_table_type {TableType.SIMPLE,TableType.COMPLEX,TableType.SNAPSHOTS} + Choose a Delta table type to generate. (default: TableType.SIMPLE) --save_path SAVE_PATH Save path for Delta table (default: None) --save_mode {append,overwrite} Specify write mode (append/overwrite) (default: append) - --partitioned_by {date,name} - Partitioned by columns (default: None) + --partitioned_by {date,name,id} + Column to partition the Delta table (default: None) --num_records NUM_RECORDS Specify number of Delta records to write (default: 5) ``` @@ -90,8 +92,19 @@ The resulting Delta table is checked in to the repo. The expectated rows to be u The test data in `resources/complex-types-table` contains 5 Delta records generated with 1 snapshot. The table was generated by running the following commands: ```shell -python3 create_delta_table.py --save_path=complex-types-table --num_records=5 --gen_complex_types=True +python3 create_delta_table.py --save_path=complex-types-table --delta_table_type=complex ``` The resulting Delta table is checked in to the repo. The expectated rows to be used in tests are updated in `ComplexTypesDeltaTable.java` accordingly. + +### Snapshots table `snapshot-table`: + +The test data in `resources/snapshot-table` contains 4 Delta snapshots with delete, update and removal of recoreds across +snapshots. The table was generated by running the following commands: +```shell +python3 create_delta_table.py --save_path=snapshot-table --partitioned_by=id --delta_table_type=snapshots --num_records=3 +``` + +The resulting Delta table is checked in to the repo. The expectated rows to be used in tests are updated in +`SnapshotDeltaTable.java` accordingly. diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index feb675fa30be..df04104dbaee 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -174,8 +174,6 @@ def main(): parser.add_argument('--delta_table_type', type=lambda t: TableType[t.upper()], choices=TableType, default=TableType.SIMPLE, help='Choose a Delta table type to generate.') - parser.add_argument("--gen_snapshots", type=bool, default=False, help="Generate multiple snapshots by updating and " - "deleting records from the Delta table.") parser.add_argument('--save_path', default=None, required=True, help="Save path for Delta table") parser.add_argument('--save_mode', choices=('append', 'overwrite'), default="append", help="Specify write mode (append/overwrite)") From 2f2aaadd461bcd888760d7987213132b93130c38 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 12:50:26 -0400 Subject: [PATCH 06/11] Checkstyle fixes --- .../org/apache/druid/delta/snapshot/VersionedSnapshot.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java index f73b5fa0b882..9529e52c06ce 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java @@ -24,7 +24,6 @@ import io.delta.kernel.Table; import io.delta.kernel.engine.Engine; import io.delta.kernel.exceptions.KernelException; -import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; public class VersionedSnapshot implements Snapshot @@ -46,7 +45,8 @@ public io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine) { try { return table.getSnapshotAsOfVersion(engine, version); - } catch(KernelException ke) { + } + catch (KernelException ke) { throw InvalidInput.exception( "Error reading snapshot version[%s] from tablePath[%s]: [%s]", version, table.getPath(engine), ke.getMessage() From 5117511d0dbed7470d7110420e9e15a1892b088b Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 12:57:08 -0400 Subject: [PATCH 07/11] Python script docs. --- .../src/test/resources/create_delta_table.py | 33 +++++++++++++++++-- 1 file changed, 31 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index df04104dbaee..7f7bf71deeb5 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -48,6 +48,19 @@ def config_spark_with_delta_lake(): def create_dataset_with_complex_types(num_records): + """ + Create a mock dataset with records containing complex types like arrays, structs and maps. + Parameters: + - num_records (int): Number of records to generate. + Returns: + - Tuple: A tuple containing a list of records and the corresponding schema. + - List of Records: Each record is a tuple representing a row of data. + - StructType: The schema defining the structure of the records. + Example: + ```python + data, schema = create_dataset_with_complex_types(10) + ``` +""" schema = StructType([ StructField("id", LongType(), False), StructField("array_info", ArrayType(IntegerType(), True), True), @@ -80,7 +93,18 @@ def create_dataset_with_complex_types(num_records): return data, schema -def create_dataset_for_snapshot(num_records): +def create_snapshots_table(num_records): + """ + Create a mock dataset for snapshots. + Parameters: + - num_records (int): Number of records to generate. + Returns: + - Tuple: A tuple containing a list of records and the corresponding schema pertaining to a single snapshot. + Example: + ```python + data, schema = create_snapshots_table(5) + ``` + """ schema = StructType([ StructField("id", LongType(), False), StructField("map_info", MapType(StringType(), IntegerType())) @@ -98,6 +122,10 @@ def create_dataset_for_snapshot(num_records): def update_table(spark, schema, delta_table_path): + """ + Update table at the specified delta path with updates: deletion, partial upsert, and insertion. + Each update generates a distinct snapshot for the Delta table. + """ delta_table = DeltaTable.forPath(spark, delta_table_path) # Snapshot 1: remove record with id = 2; result : (id=0, id=2) @@ -168,6 +196,7 @@ def create_dataset(num_records): data.append(record) return data, schema + def main(): parser = argparse.ArgumentParser(description="Script to write a Delta Lake table.", formatter_class=argparse.ArgumentDefaultsHelpFormatter) @@ -196,7 +225,7 @@ def main(): elif delta_table_type == TableType.COMPLEX: data, schema = create_dataset_with_complex_types(num_records=num_records) elif delta_table_type == TableType.SNAPSHOTS: - data, schema = create_dataset_for_snapshot(num_records) + data, schema = create_snapshots_table(num_records) else: args.print_help() raise Exception("Unknown value specified for --delta_table_type") From daaba889aba97e077ff78df924046a9ba46fbffc Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 13:18:28 -0400 Subject: [PATCH 08/11] Moar Testz! --- .../druid/delta/input/DeltaInputSource.java | 6 ++ .../druid/delta/snapshot/LatestSnapshot.java | 1 - .../delta/snapshot/VersionedSnapshot.java | 2 +- .../druid/delta/input/DeltaInputRowTest.java | 3 +- .../input/DeltaInputSourceSerdeTest.java | 78 +++++++++++++++++++ .../druid/delta/input/RowSerdeTest.java | 4 +- .../druid/delta/input/SnapshotDeltaTable.java | 5 ++ 7 files changed, 95 insertions(+), 4 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 0675d370f70d..3a8a3fee7f96 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -365,4 +365,10 @@ DeltaFilter getFilter() { return filter; } + + @VisibleForTesting + Snapshot getSnapshot() + { + return snapshot; + } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java index a7dc2b905e65..86f466e8129c 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java @@ -27,7 +27,6 @@ public class LatestSnapshot implements Snapshot @Override public io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine) { - System.out.println("LATEST SNAPSHOT"); return table.getLatestSnapshot(engine); } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java index 9529e52c06ce..e3ab4d631930 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java @@ -35,7 +35,7 @@ public class VersionedSnapshot implements Snapshot public VersionedSnapshot(@JsonProperty("version") final Long version) { if (version == null) { - throw InvalidInput.exception("version cannot be empty or null for version snapshot."); + throw InvalidInput.exception("version cannot be empty or null for versioned snapshot reads."); } this.version = version; } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index 9ba1f3ddb475..4de745b1927b 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -55,7 +55,8 @@ public static Collection data() Object[][] data = new Object[][]{ {NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.FULL_SCHEMA, NonPartitionedDeltaTable.DIMENSIONS, NonPartitionedDeltaTable.EXPECTED_ROWS}, {PartitionedDeltaTable.DELTA_TABLE_PATH, PartitionedDeltaTable.FULL_SCHEMA, PartitionedDeltaTable.DIMENSIONS, PartitionedDeltaTable.EXPECTED_ROWS}, - {ComplexTypesDeltaTable.DELTA_TABLE_PATH, ComplexTypesDeltaTable.FULL_SCHEMA, ComplexTypesDeltaTable.DIMENSIONS, ComplexTypesDeltaTable.EXPECTED_ROWS} + {ComplexTypesDeltaTable.DELTA_TABLE_PATH, ComplexTypesDeltaTable.FULL_SCHEMA, ComplexTypesDeltaTable.DIMENSIONS, ComplexTypesDeltaTable.EXPECTED_ROWS}, + {SnapshotDeltaTable.DELTA_TABLE_PATH, SnapshotDeltaTable.FULL_SCHEMA, SnapshotDeltaTable.DIMENSIONS, SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS} }; return Arrays.asList(data); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java index b6b223d9e0a6..7822ec9ddf09 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java @@ -21,10 +21,13 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.exc.InvalidTypeIdException; import com.fasterxml.jackson.databind.exc.ValueInstantiationException; import org.apache.druid.delta.common.DeltaLakeDruidModule; import org.apache.druid.delta.filter.DeltaAndFilter; import org.apache.druid.delta.filter.DeltaLessThanFilter; +import org.apache.druid.delta.snapshot.LatestSnapshot; +import org.apache.druid.delta.snapshot.VersionedSnapshot; import org.apache.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; @@ -45,6 +48,7 @@ public void testDeltaInputSourceDeserializationWithNoFilter() throws JsonProcess final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); Assert.assertNull(deltaInputSource.getFilter()); + Assert.assertTrue(deltaInputSource.getSnapshot() instanceof LatestSnapshot); } @Test @@ -63,6 +67,7 @@ public void testDeltaInputSourceDeserializationWithLessThanFilter() throws JsonP final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); Assert.assertTrue(deltaInputSource.getFilter() instanceof DeltaLessThanFilter); + Assert.assertTrue(deltaInputSource.getSnapshot() instanceof LatestSnapshot); } @Test @@ -91,6 +96,7 @@ public void testDeltaInputSourceDeserializationWithAndFilter() throws JsonProces final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); Assert.assertEquals("s3://foo/bar/baz", deltaInputSource.getTablePath()); Assert.assertTrue(deltaInputSource.getFilter() instanceof DeltaAndFilter); + Assert.assertTrue(deltaInputSource.getSnapshot() instanceof LatestSnapshot); } @Test @@ -139,4 +145,76 @@ public void testDeltaInputSourceDeserializationWithNoFilterColumn() exception.getCause().getMessage() ); } + + @Test + public void testDeltaInputSourceLatestSnapshot() throws JsonProcessingException + { + final String payload = "{\n" + + " \"type\": \"delta\",\n" + + " \"tablePath\": \"foo/bar\",\n" + + " \"snapshot\": {\n" + + " \"type\": \"latest\"\n" + + " }\n" + + " }"; + + final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); + Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); + Assert.assertTrue(deltaInputSource.getSnapshot() instanceof LatestSnapshot); + } + + @Test + public void testDeltaInputSourceVersionedSnapshot() throws JsonProcessingException + { + final String payload = "{\n" + + " \"type\": \"delta\",\n" + + " \"tablePath\": \"foo/bar\",\n" + + " \"snapshot\": {\n" + + " \"type\": \"version\",\n" + + " \"version\": 56\n" + + " }\n" + + " }"; + + final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); + Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); + Assert.assertTrue(deltaInputSource.getSnapshot() instanceof VersionedSnapshot); + } + + @Test + public void testDeltaInputSourceVersionSnapshotMissingVersion() + { + final String payload = "{\n" + + " \"type\": \"delta\",\n" + + " \"tablePath\": \"foo/bar\",\n" + + " \"snapshot\": {\n" + + " \"type\": \"version\"\n" + + " }\n" + + " }"; + + final ValueInstantiationException exception = Assert.assertThrows( + ValueInstantiationException.class, + () -> OBJECT_MAPPER.readValue(payload, DeltaInputSource.class) + ); + + Assert.assertEquals( + "version cannot be empty or null for versioned snapshot reads.", + exception.getCause().getMessage() + ); + } + + @Test + public void testDeltaInputSourceInvalidSnapshot() + { + final String payload = "{\n" + + " \"type\": \"delta\",\n" + + " \"tablePath\": \"foo/bar\",\n" + + " \"snapshot\": {\n" + + " \"type\": \"jdldld\"\n" + + " }\n" + + " }"; + + Assert.assertThrows( + InvalidTypeIdException.class, + () -> OBJECT_MAPPER.readValue(payload, DeltaInputSource.class) + ); + } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java index 7ac3eec09ef6..5a0eff0fd160 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java @@ -37,7 +37,9 @@ public static Collection data() { Object[][] data = new Object[][]{ {NonPartitionedDeltaTable.DELTA_TABLE_PATH}, - {PartitionedDeltaTable.DELTA_TABLE_PATH} + {PartitionedDeltaTable.DELTA_TABLE_PATH}, + {ComplexTypesDeltaTable.DELTA_TABLE_PATH}, + {SnapshotDeltaTable.DELTA_TABLE_PATH} }; return Arrays.asList(data); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java index d42661593bcd..781fa087a159 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java @@ -44,6 +44,11 @@ public class SnapshotDeltaTable */ public static final String DELTA_TABLE_PATH = "src/test/resources/snapshot-table"; + /** + * The list of dimensions in the Delta table {@link #DELTA_TABLE_PATH}. + */ + public static final List DIMENSIONS = ImmutableList.of("id", "map_info"); + public static final List> V0_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( ImmutableList.of( ImmutableMap.of( From d11ddea37b7a85b8d9ca4af435b6a1ac3a10111d Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 14:26:44 -0400 Subject: [PATCH 09/11] Add the missing parquet Delta files. --- ...4856-a5a8-516e5b35ef44.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet | Bin 0 -> 869 bytes ...4603-92a8-92ec51e4bdb9.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...4a2e-81aa-7bc8140b0f09.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet | Bin 0 -> 869 bytes ...c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet | Bin 0 -> 869 bytes ...4c46-b2ca-db8d7288d345.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...446c-b4f1-ab0c2e762044.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...388-4c46-b2ca-db8d7288d345.c000.snappy.parquet | Bin 0 -> 869 bytes ...1b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet | Bin 0 -> 869 bytes ...4b3d-8d6c-dd90dd3db251.c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet | Bin 0 -> 869 bytes 12 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/.part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet.crc create mode 100644 extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c1b3cedd81104fe55e3da14bea10c21cd0f5290c GIT binary patch literal 16 YcmYc;N@ieSU}Bhguwl`{f74F`04f{?xBvhE literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..8a0655de859eddccd03a1a34e332edcafaae36dd GIT binary patch literal 869 zcmZ`&&1%~~5FW1+)f)8~7L~s6oHnefm+p$J;$EcqzM#2Y|LqG`{nI zzKKwxu_AaYqw(xhz&fka5dEO(Y)%L14L$Fk*Wyq{m&15`p{9}&=n!6=%5doV=K)Sb z&PSlN?y_uq0Kx+-)Wkecqre~QNgl}hFU_{e3)2QmSrZp;{b=T?srAVoi zFv5Hm8r74H;WhvU`ws=!h{4HFQ=AHb{UUUa{VdTpV9Zh6|2C5MoVC(t17m)YYnGo_ zG*d9Q?x+8groScB0(Mto#Hh0Gr9G6^5@p0(AXEO;of(p9Ov*T|a&~^E}bRs%mKp9}PM3uv!Y3V%4GOEtbpo zOW-ex{vS`HdNFX_w(OQ=*Y`&=*K_L5sBL#!cJtWjjE-7uHSQd@nojqqId-J74{gS{BRQjks<0AX`gZ%{vS={0P literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..b19268a21f0b9f3b9a254deacd0716c1decb5547 GIT binary patch literal 16 YcmYc;N@ieSU}88b^6SvTf74F`04hBO(EtDd literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c1b3cedd81104fe55e3da14bea10c21cd0f5290c GIT binary patch literal 16 YcmYc;N@ieSU}Bhguwl`{f74F`04f{?xBvhE literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3ae3011478e0474e0f027c571760c820b5e65689 GIT binary patch literal 869 zcmZ`&%WB&|6djEnCF@=BT{Q1Rsy7f&WtQm1GBBnizwC&-7>2Nsp*Xh(!1+)|5Bt> zf+?mx4bAHD#&8<|MSBkg*ocA2KvS3sA^HQ8UHV-kuaP+iasS&$-gDMUpAO9VajseZ zg`$~)xphDN?=<}#CIZnf8pURne$#eAT8ptDng|(6kW*yJitBvcY^xcEAK0qh W>^ja!?&}u3sPs{Nghlq@2m1^8bKN!o literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..8a0655de859eddccd03a1a34e332edcafaae36dd GIT binary patch literal 869 zcmZ`&&1%~~5FW1+)f)8~7L~s6oHnefm+p$J;$EcqzM#2Y|LqG`{nI zzKKwxu_AaYqw(xhz&fka5dEO(Y)%L14L$Fk*Wyq{m&15`p{9}&=n!6=%5doV=K)Sb z&PSlN?y_uq0Kx+-)Wkecqre~QNgl}hFU_{e3)2QmSrZp;{b=T?srAVoi zFv5Hm8r74H;WhvU`ws=!h{4HFQ=AHb{UUUa{VdTpV9Zh6|2C5MoVC(t17m)YYnGo_ zG*d9Q?x+8groScB0(Mto#Hh0Gr9G6^5@p0(AXEO;of(p9Ov*T|a&~^E}bRs%mKp9}PM3uv!Y3V%4GOEtbpo zOW-ex{vS`HdNFX_w(OQ=*Y`&=*K_L5sBL#!cJtWjjE-7uHSQd@nojqqId-J74{gS{BRQjks<0AX`gZ%{vS={0P literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..dce55f335fc1ce0d89bd0f2b0c755b2b60e6c0e9 GIT binary patch literal 16 XcmYc;N@ieSU}Es`VBN6r-}KV}A;|@W literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c1b3cedd81104fe55e3da14bea10c21cd0f5290c GIT binary patch literal 16 YcmYc;N@ieSU}Bhguwl`{f74F`04f{?xBvhE literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..03888f094fe8ede43d2f681da6081408a8816398 GIT binary patch literal 869 zcmZ`&OKaOe5FW1` z*ZzXuOE38mJ@im|?6FYjwU_>kjvjVHOfS;TH}lQoo8A7S7d8cYN*~jo?>~R7>y)5b zrB#3iCjd}}a;e%Uu!HB=mtlMnBrjDIyMd2^f*SPK?@u4L@9=g{3tq}DLuzQdMB_X6 z=$i;78Y_Y;8I5PZ0@hiLhUf=PXLCA8Pw08`v>t~tx*W#i3pJILK!@KM!yk za%N&=r^2@{)y?o)dNV~UK&I1#2eca`VFqW5|sd^t^Bt8Jk(4_=ed zgCJ@>SSNA)2}0SX3efcvHBr$DeG>SIbp81G%=1JKtE#0bd^F_9!)hsDidBcAw^%OU zE`dKQ`hPHu8pXg3+p^o1-Pj+^T+eAZqeHvfwp+(eXLQs)RO8NZyXAC`T4P5l`=PB4 WTV2N)$pg*A8)f)8~7L~s6oHnefm+p$J;$EcqzM#2Y|LqG`{nI zzKKwxu_AaYqw(xhz&fka5dEO(Y)%L14L$Fk*Wyq{m&15`p{9}&=n!6=%5doV=K)Sb z&PSlN?y_uq0Kx+-)Wkecqre~QNgl}hFU_{e3)2QmSrZp;{b=T?srAVoi zFv5Hm8r74H;WhvU`ws=!h{4HFQ=AHb{UUUa{VdTpV9Zh6|2C5MoVC(t17m)YYnGo_ zG*d9Q?x+8groScB0(Mto#Hh0Gr9G6^5@p0(AXEO;of(p9Ov*T|a&~^E}bRs%mKp9}PM3uv!Y3V%4GOEtbpo zOW-ex{vS`HdNFX_w(OQ=*Y`&=*K_L5sBL#!cJtWjjE-7uHSQd@nojqqId-J74{gS{BRQjks<0AX`gZ%{vS={0P literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/.part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/.part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..b19268a21f0b9f3b9a254deacd0716c1decb5547 GIT binary patch literal 16 YcmYc;N@ieSU}88b^6SvTf74F`04hBO(EtDd literal 0 HcmV?d00001 diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3ae3011478e0474e0f027c571760c820b5e65689 GIT binary patch literal 869 zcmZ`&%WB&|6djEnCF@=BT{Q1Rsy7f&WtQm1GBBnizwC&-7>2Nsp*Xh(!1+)|5Bt> zf+?mx4bAHD#&8<|MSBkg*ocA2KvS3sA^HQ8UHV-kuaP+iasS&$-gDMUpAO9VajseZ zg`$~)xphDN?=<}#CIZnf8pURne$#eAT8ptDng|(6kW*yJitBvcY^xcEAK0qh W>^ja!?&}u3sPs{Nghlq@2m1^8bKN!o literal 0 HcmV?d00001 From f5bbefa188ca82c524c2c972e1a326788089bed5 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Wed, 4 Sep 2024 22:24:21 -0400 Subject: [PATCH 10/11] Javadocs and cleanup. --- .../apache/druid/delta/snapshot/LatestSnapshot.java | 3 +++ .../org/apache/druid/delta/snapshot/Snapshot.java | 11 +++++++++++ .../druid/delta/snapshot/VersionedSnapshot.java | 3 +++ .../druid/delta/input/DeltaInputSourceSerdeTest.java | 2 +- .../src/test/resources/README.md | 6 +++--- .../src/test/resources/create_delta_table.py | 3 +-- 6 files changed, 22 insertions(+), 6 deletions(-) diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java index 86f466e8129c..2a3fde82e3da 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java @@ -22,6 +22,9 @@ import io.delta.kernel.Table; import io.delta.kernel.engine.Engine; +/** + * Represents a snapshot that always points to the latest available snapshot in a Delta table. + */ public class LatestSnapshot implements Snapshot { @Override diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java index 28afbc9e2742..0974fc0085b3 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java @@ -24,6 +24,9 @@ import io.delta.kernel.Table; import io.delta.kernel.engine.Engine; +/** + * Represents a snapshot of a Delta table that can be retrieved. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "latest", value = LatestSnapshot.class), @@ -31,5 +34,13 @@ }) public interface Snapshot { + + /** + * Retrieves a snapshot of the given Delta table using the provided engine. + * + * @param table the Delta table from which to retrieve the snapshot + * @param engine the engine used to retrieve the snapshot + * @return the snapshot of the Delta table, as determined by the implementation + */ io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java index e3ab4d631930..9c7b0f7a363b 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java @@ -26,6 +26,9 @@ import io.delta.kernel.exceptions.KernelException; import org.apache.druid.error.InvalidInput; +/** + * Represents a Delta table snapshot identified by a specific version. + */ public class VersionedSnapshot implements Snapshot { @JsonProperty diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java index 7822ec9ddf09..4561cbebe46e 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java @@ -180,7 +180,7 @@ public void testDeltaInputSourceVersionedSnapshot() throws JsonProcessingExcepti } @Test - public void testDeltaInputSourceVersionSnapshotMissingVersion() + public void testDeltaInputSourceVersionedSnapshotMissingVersion() { final String payload = "{\n" + " \"type\": \"delta\",\n" diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md index 03cf1505a815..41aec3e3016e 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md @@ -90,7 +90,7 @@ The resulting Delta table is checked in to the repo. The expectated rows to be u ### Complex types table `complex-types-table`: The test data in `resources/complex-types-table` contains 5 Delta records generated with 1 snapshot. -The table was generated by running the following commands: +The table was generated by running the following command: ```shell python3 create_delta_table.py --save_path=complex-types-table --delta_table_type=complex ``` @@ -100,8 +100,8 @@ The resulting Delta table is checked in to the repo. The expectated rows to be u ### Snapshots table `snapshot-table`: -The test data in `resources/snapshot-table` contains 4 Delta snapshots with delete, update and removal of recoreds across -snapshots. The table was generated by running the following commands: +The test data in `resources/snapshot-table` contains 4 Delta snapshots with delete, update and removal of records across +snapshots. The table was generated by running the following command: ```shell python3 create_delta_table.py --save_path=snapshot-table --partitioned_by=id --delta_table_type=snapshots --num_records=3 ``` diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index 7f7bf71deeb5..ffca055e73c1 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -26,7 +26,6 @@ import random from delta.tables import DeltaTable - class TableType(Enum): SIMPLE = "simple" COMPLEX = "complex" @@ -60,7 +59,7 @@ def create_dataset_with_complex_types(num_records): ```python data, schema = create_dataset_with_complex_types(10) ``` -""" + """ schema = StructType([ StructField("id", LongType(), False), StructField("array_info", ArrayType(IntegerType(), True), True), From 75f43ebe1236e3307dbb0d2ade14df1cbf20c735 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 6 Sep 2024 11:06:33 -0400 Subject: [PATCH 11/11] Remove snapshot json object in favor of a less verbose flattened structure. --- docs/ingestion/input-sources.md | 29 +------- .../druid/delta/input/DeltaInputSource.java | 31 ++++---- .../druid/delta/snapshot/LatestSnapshot.java | 35 ---------- .../apache/druid/delta/snapshot/Snapshot.java | 46 ------------ .../delta/snapshot/VersionedSnapshot.java | 59 ---------------- .../input/DeltaInputSourceSerdeTest.java | 70 +------------------ .../delta/input/DeltaInputSourceTest.java | 37 ++++------ website/.spelling | 1 + 8 files changed, 37 insertions(+), 271 deletions(-) delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java delete mode 100644 extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index 1b2e02c8d770..5ab283002148 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -1152,7 +1152,7 @@ the latest snapshot from the configured table. Druid ingests the underlying delt |type|Set this value to `delta`.| None|yes| |tablePath|The location of the Delta table.|None|yes| |filter|The JSON Object that filters data files within a snapshot.|None|no| -|snapshot|The JSON Object that specifies a Delta table snapshot to read from.|latest|no| +|snapshotVersion|The snapshot version to read from the Delta table. An integer value must be specified.|Latest|no| ### Delta filter object @@ -1225,27 +1225,7 @@ filters on partitioned columns. | column | The table column to apply the filter on. | yes | | value | The value to use in the filter. | yes | -### Delta snapshot object - -You can use the `snapshot` to read the Delta table at the specific snapshot. This input source supports the following -snapshot types: `latest` and `version`. If `snapshot` is not specified, it defaults to the latest snapshot. - -`latest` snapshot: - -| Property | Description | Required | -|----------|------------------------------------|-------------| -| type | Set this value to `latest`. | no (default)| - - -`version` snapshot: - -| Property | Description | Required | -|----------|------------------------------------------|----------| -| type | Set this value to `version`. | yes | -| version | The Delta snapshot version to read from. | yes | - - -The following is a sample spec to read all records from the latest record from Delta table `/delta-table/foo`: +The following is a sample spec to read all records from the latest snapshot from Delta table `/delta-table/foo`: ```json ... @@ -1283,10 +1263,7 @@ The following is a sample spec to read records from the Delta table `/delta-tabl } ] }, - "snapshot": { - "type": "version", - "version": 3 - } + "snapshotVersion": 3 }, } ``` diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 3a8a3fee7f96..c4c2f2668b04 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -26,6 +26,7 @@ import com.google.common.primitives.Ints; import io.delta.kernel.Scan; import io.delta.kernel.ScanBuilder; +import io.delta.kernel.Snapshot; import io.delta.kernel.Table; import io.delta.kernel.data.ColumnarBatch; import io.delta.kernel.data.FilteredColumnarBatch; @@ -51,8 +52,6 @@ import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.delta.filter.DeltaFilter; -import org.apache.druid.delta.snapshot.LatestSnapshot; -import org.apache.druid.delta.snapshot.Snapshot; import org.apache.druid.error.InvalidInput; import org.apache.druid.utils.Streams; import org.apache.hadoop.conf.Configuration; @@ -68,8 +67,8 @@ import java.util.stream.Stream; /** - * Input source to ingest data from a Delta Lake. This input source reads the given {@code snapshot} from a Delta - * table specified by {@code tablePath} parameter. If {@code snapshot} is unspecified, it defaults to the latest snapshot. + * Input source to ingest data from a Delta Lake. This input source reads the given {@code snapshotVersion} from a Delta + * table specified by {@code tablePath} parameter, or the latest snapshot if it's not specified. * If {@code filter} is specified, it's used at the Kernel level for data pruning. The filtering behavior is as follows: *
      *
    • When a filter is applied on a partitioned table using the partitioning columns, the filtering is guaranteed.
    • @@ -98,14 +97,14 @@ public class DeltaInputSource implements SplittableInputSource private final DeltaFilter filter; @JsonProperty - private final Snapshot snapshot; + private final Long snapshotVersion; @JsonCreator public DeltaInputSource( @JsonProperty("tablePath") final String tablePath, @JsonProperty("deltaSplit") @Nullable final DeltaSplit deltaSplit, @JsonProperty("filter") @Nullable final DeltaFilter filter, - @JsonProperty("snapshot") @Nullable final Snapshot snapshot + @JsonProperty("snapshotVersion") @Nullable final Long snapshotVersion ) { if (tablePath == null) { @@ -114,7 +113,7 @@ public DeltaInputSource( this.tablePath = tablePath; this.deltaSplit = deltaSplit; this.filter = filter; - this.snapshot = snapshot == null ? new LatestSnapshot() : snapshot; + this.snapshotVersion = snapshotVersion; } @Override @@ -157,7 +156,7 @@ public InputSourceReader reader( } } else { final Table table = Table.forPath(engine, tablePath); - final io.delta.kernel.Snapshot snapshot = getSnapshotForTable(table, engine); + final Snapshot snapshot = getSnapshotForTable(table, engine); final StructType fullSnapshotSchema = snapshot.getSchema(engine); final StructType prunedSchema = pruneSchema( @@ -211,7 +210,7 @@ public Stream> createSplits(InputFormat inputFormat, @Nul } final Engine engine = createDeltaEngine(); - final io.delta.kernel.Snapshot snapshot; + final Snapshot snapshot; final Table table = Table.forPath(engine, tablePath); try { snapshot = getSnapshotForTable(table, engine); @@ -260,7 +259,7 @@ public InputSource withSplit(InputSplit split) tablePath, split.get(), filter, - snapshot + snapshotVersion ); } @@ -339,7 +338,7 @@ private CloseableIterator getTransformedDataIterator( ); } - private io.delta.kernel.Snapshot getSnapshotForTable(final Table table, final Engine engine) + private Snapshot getSnapshotForTable(final Table table, final Engine engine) { // Setting the LogStore class loader before calling the Delta Kernel snapshot API is required as a workaround with // the 3.2.0 Delta Kernel because the Kernel library cannot instantiate the LogStore class otherwise. Please see @@ -347,7 +346,11 @@ private io.delta.kernel.Snapshot getSnapshotForTable(final Table table, final En final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(LogStore.class.getClassLoader()); - return snapshot.getSnapshot(table, engine); + if (snapshotVersion != null) { + return table.getSnapshotAsOfVersion(engine, snapshotVersion); + } else { + return table.getLatestSnapshot(engine); + } } finally { Thread.currentThread().setContextClassLoader(currCtxCl); @@ -367,8 +370,8 @@ DeltaFilter getFilter() } @VisibleForTesting - Snapshot getSnapshot() + Long getSnapshotVersion() { - return snapshot; + return snapshotVersion; } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java deleted file mode 100644 index 2a3fde82e3da..000000000000 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/LatestSnapshot.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.delta.snapshot; - -import io.delta.kernel.Table; -import io.delta.kernel.engine.Engine; - -/** - * Represents a snapshot that always points to the latest available snapshot in a Delta table. - */ -public class LatestSnapshot implements Snapshot -{ - @Override - public io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine) - { - return table.getLatestSnapshot(engine); - } -} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java deleted file mode 100644 index 0974fc0085b3..000000000000 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/Snapshot.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.delta.snapshot; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import io.delta.kernel.Table; -import io.delta.kernel.engine.Engine; - -/** - * Represents a snapshot of a Delta table that can be retrieved. - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "latest", value = LatestSnapshot.class), - @JsonSubTypes.Type(name = "version", value = VersionedSnapshot.class), -}) -public interface Snapshot -{ - - /** - * Retrieves a snapshot of the given Delta table using the provided engine. - * - * @param table the Delta table from which to retrieve the snapshot - * @param engine the engine used to retrieve the snapshot - * @return the snapshot of the Delta table, as determined by the implementation - */ - io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine); -} diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java deleted file mode 100644 index 9c7b0f7a363b..000000000000 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/snapshot/VersionedSnapshot.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.delta.snapshot; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import io.delta.kernel.Table; -import io.delta.kernel.engine.Engine; -import io.delta.kernel.exceptions.KernelException; -import org.apache.druid.error.InvalidInput; - -/** - * Represents a Delta table snapshot identified by a specific version. - */ -public class VersionedSnapshot implements Snapshot -{ - @JsonProperty - private final Long version; - - @JsonCreator - public VersionedSnapshot(@JsonProperty("version") final Long version) - { - if (version == null) { - throw InvalidInput.exception("version cannot be empty or null for versioned snapshot reads."); - } - this.version = version; - } - - @Override - public io.delta.kernel.Snapshot getSnapshot(Table table, Engine engine) - { - try { - return table.getSnapshotAsOfVersion(engine, version); - } - catch (KernelException ke) { - throw InvalidInput.exception( - "Error reading snapshot version[%s] from tablePath[%s]: [%s]", - version, table.getPath(engine), ke.getMessage() - ); - } - } -} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java index 4561cbebe46e..d8eb083cf57b 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java @@ -21,13 +21,10 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.exc.InvalidTypeIdException; import com.fasterxml.jackson.databind.exc.ValueInstantiationException; import org.apache.druid.delta.common.DeltaLakeDruidModule; import org.apache.druid.delta.filter.DeltaAndFilter; import org.apache.druid.delta.filter.DeltaLessThanFilter; -import org.apache.druid.delta.snapshot.LatestSnapshot; -import org.apache.druid.delta.snapshot.VersionedSnapshot; import org.apache.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; @@ -48,7 +45,6 @@ public void testDeltaInputSourceDeserializationWithNoFilter() throws JsonProcess final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); Assert.assertNull(deltaInputSource.getFilter()); - Assert.assertTrue(deltaInputSource.getSnapshot() instanceof LatestSnapshot); } @Test @@ -67,7 +63,6 @@ public void testDeltaInputSourceDeserializationWithLessThanFilter() throws JsonP final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); Assert.assertTrue(deltaInputSource.getFilter() instanceof DeltaLessThanFilter); - Assert.assertTrue(deltaInputSource.getSnapshot() instanceof LatestSnapshot); } @Test @@ -96,7 +91,6 @@ public void testDeltaInputSourceDeserializationWithAndFilter() throws JsonProces final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); Assert.assertEquals("s3://foo/bar/baz", deltaInputSource.getTablePath()); Assert.assertTrue(deltaInputSource.getFilter() instanceof DeltaAndFilter); - Assert.assertTrue(deltaInputSource.getSnapshot() instanceof LatestSnapshot); } @Test @@ -147,74 +141,16 @@ public void testDeltaInputSourceDeserializationWithNoFilterColumn() } @Test - public void testDeltaInputSourceLatestSnapshot() throws JsonProcessingException + public void testDeltaInputSourceWithSnapshotVersion() throws JsonProcessingException { final String payload = "{\n" + " \"type\": \"delta\",\n" + " \"tablePath\": \"foo/bar\",\n" - + " \"snapshot\": {\n" - + " \"type\": \"latest\"\n" - + " }\n" - + " }"; - - final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); - Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); - Assert.assertTrue(deltaInputSource.getSnapshot() instanceof LatestSnapshot); - } - - @Test - public void testDeltaInputSourceVersionedSnapshot() throws JsonProcessingException - { - final String payload = "{\n" - + " \"type\": \"delta\",\n" - + " \"tablePath\": \"foo/bar\",\n" - + " \"snapshot\": {\n" - + " \"type\": \"version\",\n" - + " \"version\": 56\n" - + " }\n" + + " \"snapshotVersion\": 56\n" + " }"; final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); - Assert.assertTrue(deltaInputSource.getSnapshot() instanceof VersionedSnapshot); - } - - @Test - public void testDeltaInputSourceVersionedSnapshotMissingVersion() - { - final String payload = "{\n" - + " \"type\": \"delta\",\n" - + " \"tablePath\": \"foo/bar\",\n" - + " \"snapshot\": {\n" - + " \"type\": \"version\"\n" - + " }\n" - + " }"; - - final ValueInstantiationException exception = Assert.assertThrows( - ValueInstantiationException.class, - () -> OBJECT_MAPPER.readValue(payload, DeltaInputSource.class) - ); - - Assert.assertEquals( - "version cannot be empty or null for versioned snapshot reads.", - exception.getCause().getMessage() - ); - } - - @Test - public void testDeltaInputSourceInvalidSnapshot() - { - final String payload = "{\n" - + " \"type\": \"delta\",\n" - + " \"tablePath\": \"foo/bar\",\n" - + " \"snapshot\": {\n" - + " \"type\": \"jdldld\"\n" - + " }\n" - + " }"; - - Assert.assertThrows( - InvalidTypeIdException.class, - () -> OBJECT_MAPPER.readValue(payload, DeltaInputSource.class) - ); + Assert.assertEquals((Long) 56L, deltaInputSource.getSnapshotVersion()); } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index e5b1ebc76f7b..cbbcaefb3ceb 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -19,6 +19,7 @@ package org.apache.druid.delta.input; +import io.delta.kernel.exceptions.KernelException; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; @@ -31,9 +32,6 @@ import org.apache.druid.delta.filter.DeltaLessThanOrEqualsFilter; import org.apache.druid.delta.filter.DeltaNotFilter; import org.apache.druid.delta.filter.DeltaOrFilter; -import org.apache.druid.delta.snapshot.LatestSnapshot; -import org.apache.druid.delta.snapshot.Snapshot; -import org.apache.druid.delta.snapshot.VersionedSnapshot; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.DateTimes; @@ -101,31 +99,25 @@ public static Object[][] data() { SnapshotDeltaTable.DELTA_TABLE_PATH, SnapshotDeltaTable.FULL_SCHEMA, - new VersionedSnapshot(0L), + 0L, SnapshotDeltaTable.V0_SNAPSHOT_EXPECTED_ROWS }, { SnapshotDeltaTable.DELTA_TABLE_PATH, SnapshotDeltaTable.FULL_SCHEMA, - new VersionedSnapshot(1L), + 1L, SnapshotDeltaTable.V1_SNAPSHOT_EXPECTED_ROWS }, { SnapshotDeltaTable.DELTA_TABLE_PATH, SnapshotDeltaTable.FULL_SCHEMA, - new VersionedSnapshot(2L), + 2L, SnapshotDeltaTable.V2_SNAPSHOT_EXPECTED_ROWS }, { SnapshotDeltaTable.DELTA_TABLE_PATH, SnapshotDeltaTable.FULL_SCHEMA, - new VersionedSnapshot(3L), - SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS - }, - { - SnapshotDeltaTable.DELTA_TABLE_PATH, - SnapshotDeltaTable.FULL_SCHEMA, - new LatestSnapshot(), + 3L, SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS }, { @@ -133,7 +125,7 @@ public static Object[][] data() SnapshotDeltaTable.FULL_SCHEMA, null, SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS - }, + } }; } @@ -142,14 +134,14 @@ public static Object[][] data() @Parameterized.Parameter(1) public InputRowSchema schema; @Parameterized.Parameter(2) - public Snapshot snapshot; + public Long snapshotVersion; @Parameterized.Parameter(3) public List> expectedRows; @Test public void testSampleDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshot); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshotVersion); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); @@ -183,7 +175,7 @@ public void testSampleDeltaTable() throws IOException @Test public void testReadDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshot); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshotVersion); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); final List actualReadRows = readAllRows(inputSourceReader); validateRows(expectedRows, actualReadRows, schema); @@ -419,15 +411,12 @@ public void testReadNonExistentSnapshot() SnapshotDeltaTable.DELTA_TABLE_PATH, null, null, - new VersionedSnapshot(100L) + 100L ); - MatcherAssert.assertThat( - Assert.assertThrows( - DruidException.class, - () -> deltaInputSource.reader(null, null, null) - ), - DruidExceptionMatcher.invalidInput().expectMessageContains("Error reading snapshot version[100]") + Assert.assertThrows( + KernelException.class, + () -> deltaInputSource.reader(null, null, null) ); } } diff --git a/website/.spelling b/website/.spelling index f8e319d07824..8175755f8046 100644 --- a/website/.spelling +++ b/website/.spelling @@ -2417,3 +2417,4 @@ percentilesSpectatorHistogram quantilesFromDDSketch quantileFromDDSketch collapsingLowestDense +snapshotVersion