Support for reading Delta Lake table snapshots (#17004)

Problem
Currently, the delta input source only supports reading from the latest snapshot of the given Delta Lake table. This is a known documented limitation.

Description
Add support for reading Delta snapshot. By default, the Druid-Delta connector reads the latest snapshot of the Delta table in order to preserve compatibility. Users can specify a snapshotVersion to ingest change data events from Delta tables into Druid.

In the future, we can also add support for time-based snapshot reads. The Delta API to read time-based snapshots is not clear currently.
This commit is contained in:
Abhishek Radhakrishnan 2024-09-09 04:42:48 -04:00 committed by GitHub
parent 51fe3c08ab
commit aa833a711c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
31 changed files with 364 additions and 57 deletions

View File

@ -52,8 +52,3 @@ java \
``` ```
See [Loading community extensions](../../configuration/extensions.md#loading-community-extensions) for more information. 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).

View File

@ -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 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. the latest snapshot from the configured table. Druid ingests the underlying delta files from the table.
| Property|Description|Required| | Property|Description| Default|Required |
|---------|-----------|--------| |---------|-----------|-----------------|
| type|Set this value to `delta`.|yes| |type|Set this value to `delta`.| None|yes|
| tablePath|The location of the Delta table.|yes| |tablePath|The location of the Delta table.|None|yes|
| filter|The JSON Object that filters data files within a snapshot.|no| |filter|The JSON Object that filters data files within a snapshot.|None|no|
|snapshotVersion|The snapshot version to read from the Delta table. An integer value must be specified.|Latest|no|
### Delta filter object ### Delta filter object
@ -1224,7 +1225,7 @@ filters on partitioned columns.
| column | The table column to apply the filter on. | yes | | column | The table column to apply the filter on. | yes |
| value | The value to use in the filter. | 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`: The following is a sample spec to read all records from the latest snapshot from Delta table `/delta-table/foo`:
```json ```json
... ...
@ -1237,7 +1238,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 ```json
... ...
@ -1260,7 +1262,8 @@ The following is a sample spec to read records from the Delta table `/delta-tabl
"value": "30" "value": "30"
} }
] ]
} },
"snapshotVersion": 3
}, },
} }
``` ```

View File

@ -67,9 +67,9 @@ import java.util.stream.Collectors;
import java.util.stream.Stream; 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 * Input source to ingest data from a Delta Lake. This input source reads the given {@code snapshotVersion} from a Delta
* specified by {@code tablePath} parameter. If {@code filter} is specified, it's used at the Kernel level * table specified by {@code tablePath} parameter, or the latest snapshot if it's not specified.
* for data pruning. The filtering behavior is as follows: * If {@code filter} is specified, it's used at the Kernel level for data pruning. The filtering behavior is as follows:
* <ul> * <ul>
* <li> When a filter is applied on a partitioned table using the partitioning columns, the filtering is guaranteed. </li> * <li> When a filter is applied on a partitioned table using the partitioning columns, the filtering is guaranteed. </li>
* <li> When a filter is applied on non-partitioned columns, the filtering is best-effort as the Delta * <li> When a filter is applied on non-partitioned columns, the filtering is best-effort as the Delta
@ -79,7 +79,6 @@ import java.util.stream.Stream;
* <p> * <p>
* We leverage the Delta Kernel APIs to interact with a Delta table. The Kernel API abstracts away the * We leverage the Delta Kernel APIs to interact with a Delta table. The Kernel API abstracts away the
* complexities of the Delta protocol itself. * complexities of the Delta protocol itself.
* Note: currently, the Kernel table API only supports reading from the latest snapshot.
* </p> * </p>
*/ */
public class DeltaInputSource implements SplittableInputSource<DeltaSplit> public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
@ -97,11 +96,15 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
@Nullable @Nullable
private final DeltaFilter filter; private final DeltaFilter filter;
@JsonProperty
private final Long snapshotVersion;
@JsonCreator @JsonCreator
public DeltaInputSource( public DeltaInputSource(
@JsonProperty("tablePath") final String tablePath, @JsonProperty("tablePath") final String tablePath,
@JsonProperty("deltaSplit") @Nullable final DeltaSplit deltaSplit, @JsonProperty("deltaSplit") @Nullable final DeltaSplit deltaSplit,
@JsonProperty("filter") @Nullable final DeltaFilter filter @JsonProperty("filter") @Nullable final DeltaFilter filter,
@JsonProperty("snapshotVersion") @Nullable final Long snapshotVersion
) )
{ {
if (tablePath == null) { if (tablePath == null) {
@ -110,6 +113,7 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
this.tablePath = tablePath; this.tablePath = tablePath;
this.deltaSplit = deltaSplit; this.deltaSplit = deltaSplit;
this.filter = filter; this.filter = filter;
this.snapshotVersion = snapshotVersion;
} }
@Override @Override
@ -152,15 +156,15 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
} }
} else { } else {
final Table table = Table.forPath(engine, tablePath); 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( final StructType prunedSchema = pruneSchema(
fullSnapshotSchema, fullSnapshotSchema,
inputRowSchema.getColumnsFilter() inputRowSchema.getColumnsFilter()
); );
final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(engine); final ScanBuilder scanBuilder = snapshot.getScanBuilder(engine);
if (filter != null) { if (filter != null) {
scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema)); scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema));
} }
@ -206,17 +210,17 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
} }
final Engine engine = createDeltaEngine(); final Engine engine = createDeltaEngine();
final Snapshot latestSnapshot; final Snapshot snapshot;
final Table table = Table.forPath(engine, tablePath); final Table table = Table.forPath(engine, tablePath);
try { try {
latestSnapshot = getLatestSnapshotForTable(table, engine); snapshot = getSnapshotForTable(table, engine);
} }
catch (TableNotFoundException e) { catch (TableNotFoundException e) {
throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); 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) { if (filter != null) {
scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema)); scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema));
} }
@ -254,7 +258,8 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
return new DeltaInputSource( return new DeltaInputSource(
tablePath, tablePath,
split.get(), split.get(),
filter filter,
snapshotVersion
); );
} }
@ -333,7 +338,7 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
); );
} }
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 // 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 // the 3.2.0 Delta Kernel because the Kernel library cannot instantiate the LogStore class otherwise. Please see
@ -341,8 +346,12 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
try { try {
Thread.currentThread().setContextClassLoader(LogStore.class.getClassLoader()); Thread.currentThread().setContextClassLoader(LogStore.class.getClassLoader());
if (snapshotVersion != null) {
return table.getSnapshotAsOfVersion(engine, snapshotVersion);
} else {
return table.getLatestSnapshot(engine); return table.getLatestSnapshot(engine);
} }
}
finally { finally {
Thread.currentThread().setContextClassLoader(currCtxCl); Thread.currentThread().setContextClassLoader(currCtxCl);
} }
@ -359,4 +368,10 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
{ {
return filter; return filter;
} }
@VisibleForTesting
Long getSnapshotVersion()
{
return snapshotVersion;
}
} }

View File

@ -55,7 +55,8 @@ public class DeltaInputRowTest
Object[][] data = new Object[][]{ Object[][] data = new Object[][]{
{NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.FULL_SCHEMA, NonPartitionedDeltaTable.DIMENSIONS, NonPartitionedDeltaTable.EXPECTED_ROWS}, {NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.FULL_SCHEMA, NonPartitionedDeltaTable.DIMENSIONS, NonPartitionedDeltaTable.EXPECTED_ROWS},
{PartitionedDeltaTable.DELTA_TABLE_PATH, PartitionedDeltaTable.FULL_SCHEMA, PartitionedDeltaTable.DIMENSIONS, PartitionedDeltaTable.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); return Arrays.asList(data);
} }
@ -124,7 +125,7 @@ public class DeltaInputRowTest
@ParameterizedTest(name = "{index}:with context {0}") @ParameterizedTest(name = "{index}:with context {0}")
public void testReadNonExistentTable() 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( MatcherAssert.assertThat(
Assert.assertThrows( Assert.assertThrows(

View File

@ -139,4 +139,18 @@ public class DeltaInputSourceSerdeTest
exception.getCause().getMessage() exception.getCause().getMessage()
); );
} }
@Test
public void testDeltaInputSourceWithSnapshotVersion() throws JsonProcessingException
{
final String payload = "{\n"
+ " \"type\": \"delta\",\n"
+ " \"tablePath\": \"foo/bar\",\n"
+ " \"snapshotVersion\": 56\n"
+ " }";
final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class);
Assert.assertEquals("foo/bar", deltaInputSource.getTablePath());
Assert.assertEquals((Long) 56L, deltaInputSource.getSnapshotVersion());
}
} }

View File

@ -19,6 +19,7 @@
package org.apache.druid.delta.input; 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.InputRow;
import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowListPlusRawValues;
import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputRowSchema;
@ -68,27 +69,62 @@ public class DeltaInputSourceTest
{ {
NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.DELTA_TABLE_PATH,
NonPartitionedDeltaTable.FULL_SCHEMA, NonPartitionedDeltaTable.FULL_SCHEMA,
null,
NonPartitionedDeltaTable.EXPECTED_ROWS NonPartitionedDeltaTable.EXPECTED_ROWS
}, },
{ {
NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.DELTA_TABLE_PATH,
NonPartitionedDeltaTable.SCHEMA_1, NonPartitionedDeltaTable.SCHEMA_1,
null,
NonPartitionedDeltaTable.EXPECTED_ROWS NonPartitionedDeltaTable.EXPECTED_ROWS
}, },
{ {
NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.DELTA_TABLE_PATH,
NonPartitionedDeltaTable.SCHEMA_2, NonPartitionedDeltaTable.SCHEMA_2,
null,
NonPartitionedDeltaTable.EXPECTED_ROWS NonPartitionedDeltaTable.EXPECTED_ROWS
}, },
{ {
PartitionedDeltaTable.DELTA_TABLE_PATH, PartitionedDeltaTable.DELTA_TABLE_PATH,
PartitionedDeltaTable.FULL_SCHEMA, PartitionedDeltaTable.FULL_SCHEMA,
null,
PartitionedDeltaTable.EXPECTED_ROWS PartitionedDeltaTable.EXPECTED_ROWS
}, },
{ {
ComplexTypesDeltaTable.DELTA_TABLE_PATH, ComplexTypesDeltaTable.DELTA_TABLE_PATH,
ComplexTypesDeltaTable.FULL_SCHEMA, ComplexTypesDeltaTable.FULL_SCHEMA,
null,
ComplexTypesDeltaTable.EXPECTED_ROWS ComplexTypesDeltaTable.EXPECTED_ROWS
},
{
SnapshotDeltaTable.DELTA_TABLE_PATH,
SnapshotDeltaTable.FULL_SCHEMA,
0L,
SnapshotDeltaTable.V0_SNAPSHOT_EXPECTED_ROWS
},
{
SnapshotDeltaTable.DELTA_TABLE_PATH,
SnapshotDeltaTable.FULL_SCHEMA,
1L,
SnapshotDeltaTable.V1_SNAPSHOT_EXPECTED_ROWS
},
{
SnapshotDeltaTable.DELTA_TABLE_PATH,
SnapshotDeltaTable.FULL_SCHEMA,
2L,
SnapshotDeltaTable.V2_SNAPSHOT_EXPECTED_ROWS
},
{
SnapshotDeltaTable.DELTA_TABLE_PATH,
SnapshotDeltaTable.FULL_SCHEMA,
3L,
SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS
},
{
SnapshotDeltaTable.DELTA_TABLE_PATH,
SnapshotDeltaTable.FULL_SCHEMA,
null,
SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS
} }
}; };
} }
@ -98,12 +134,14 @@ public class DeltaInputSourceTest
@Parameterized.Parameter(1) @Parameterized.Parameter(1)
public InputRowSchema schema; public InputRowSchema schema;
@Parameterized.Parameter(2) @Parameterized.Parameter(2)
public Long snapshotVersion;
@Parameterized.Parameter(3)
public List<Map<String, Object>> expectedRows; public List<Map<String, Object>> expectedRows;
@Test @Test
public void testSampleDeltaTable() throws IOException public void testSampleDeltaTable() throws IOException
{ {
final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null); final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshotVersion);
final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null);
List<InputRowListPlusRawValues> actualSampledRows = sampleAllRows(inputSourceReader); List<InputRowListPlusRawValues> actualSampledRows = sampleAllRows(inputSourceReader);
@ -137,7 +175,7 @@ public class DeltaInputSourceTest
@Test @Test
public void testReadDeltaTable() throws IOException public void testReadDeltaTable() throws IOException
{ {
final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null); final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshotVersion);
final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null);
final List<InputRow> actualReadRows = readAllRows(inputSourceReader); final List<InputRow> actualReadRows = readAllRows(inputSourceReader);
validateRows(expectedRows, actualReadRows, schema); validateRows(expectedRows, actualReadRows, schema);
@ -269,7 +307,7 @@ public class DeltaInputSourceTest
@Test @Test
public void testSampleDeltaTable() throws IOException 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); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null);
List<InputRowListPlusRawValues> actualSampledRows = sampleAllRows(inputSourceReader); List<InputRowListPlusRawValues> actualSampledRows = sampleAllRows(inputSourceReader);
@ -311,7 +349,7 @@ public class DeltaInputSourceTest
@Test @Test
public void testReadDeltaTable() throws IOException 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 InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null);
final List<InputRow> actualReadRows = readAllRows(inputSourceReader); final List<InputRow> actualReadRows = readAllRows(inputSourceReader);
validateRows(expectedRows, actualReadRows, schema); validateRows(expectedRows, actualReadRows, schema);
@ -326,7 +364,7 @@ public class DeltaInputSourceTest
MatcherAssert.assertThat( MatcherAssert.assertThat(
Assert.assertThrows( Assert.assertThrows(
DruidException.class, DruidException.class,
() -> new DeltaInputSource(null, null, null) () -> new DeltaInputSource(null, null, null, null)
), ),
DruidExceptionMatcher.invalidInput().expectMessageIs( DruidExceptionMatcher.invalidInput().expectMessageIs(
"tablePath cannot be null." "tablePath cannot be null."
@ -337,7 +375,7 @@ public class DeltaInputSourceTest
@Test @Test
public void testSplitNonExistentTable() 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( MatcherAssert.assertThat(
Assert.assertThrows( Assert.assertThrows(
@ -353,7 +391,7 @@ public class DeltaInputSourceTest
@Test @Test
public void testReadNonExistentTable() 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( MatcherAssert.assertThat(
Assert.assertThrows( Assert.assertThrows(
@ -365,6 +403,22 @@ public class DeltaInputSourceTest
) )
); );
} }
@Test
public void testReadNonExistentSnapshot()
{
final DeltaInputSource deltaInputSource = new DeltaInputSource(
SnapshotDeltaTable.DELTA_TABLE_PATH,
null,
null,
100L
);
Assert.assertThrows(
KernelException.class,
() -> deltaInputSource.reader(null, null, null)
);
}
} }
private static List<InputRowListPlusRawValues> sampleAllRows(InputSourceReader reader) throws IOException private static List<InputRowListPlusRawValues> sampleAllRows(InputSourceReader reader) throws IOException

View File

@ -37,7 +37,9 @@ public class RowSerdeTest
{ {
Object[][] data = new Object[][]{ Object[][] data = new Object[][]{
{NonPartitionedDeltaTable.DELTA_TABLE_PATH}, {NonPartitionedDeltaTable.DELTA_TABLE_PATH},
{PartitionedDeltaTable.DELTA_TABLE_PATH} {PartitionedDeltaTable.DELTA_TABLE_PATH},
{ComplexTypesDeltaTable.DELTA_TABLE_PATH},
{SnapshotDeltaTable.DELTA_TABLE_PATH}
}; };
return Arrays.asList(data); return Arrays.asList(data);
} }

View File

@ -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.List;
import java.util.Map;
/**
* 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";
/**
* The list of dimensions in the Delta table {@link #DELTA_TABLE_PATH}.
*/
public static final List<String> DIMENSIONS = ImmutableList.of("id", "map_info");
public static final List<Map<String, Object>> V0_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>(
ImmutableList.of(
ImmutableMap.of(
"id", 0L,
"map_info", ImmutableMap.of("snapshotVersion", 0)
),
ImmutableMap.of(
"id", 1L,
"map_info", ImmutableMap.of("snapshotVersion", 0)
),
ImmutableMap.of(
"id", 2L,
"map_info", ImmutableMap.of("snapshotVersion", 0)
)
)
);
public static final List<Map<String, Object>> V1_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>(
ImmutableList.of(
ImmutableMap.of(
"id", 0L,
"map_info", ImmutableMap.of("snapshotVersion", 0)
),
ImmutableMap.of(
"id", 2L,
"map_info", ImmutableMap.of("snapshotVersion", 0)
)
)
);
public static final List<Map<String, Object>> 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)
)
)
);
public static final List<Map<String, Object>> 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}.
*/
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("map_info", null)
)
),
ColumnsFilter.all()
);
}

View File

@ -44,18 +44,20 @@ Delta table to `resources/employee-delta-table`. You can override the defaults b
```shell ```shell
python3 create_delta_table.py -h 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. Script to write a Delta Lake table.
options: options:
-h, --help show this help message and exit -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 SAVE_PATH
Save path for Delta table (default: None) Save path for Delta table (default: None)
--save_mode {append,overwrite} --save_mode {append,overwrite}
Specify write mode (append/overwrite) (default: append) Specify write mode (append/overwrite) (default: append)
--partitioned_by {date,name} --partitioned_by {date,name,id}
Partitioned by columns (default: None) Column to partition the Delta table (default: None)
--num_records NUM_RECORDS --num_records NUM_RECORDS
Specify number of Delta records to write (default: 5) Specify number of Delta records to write (default: 5)
``` ```
@ -88,10 +90,21 @@ The resulting Delta table is checked in to the repo. The expectated rows to be u
### Complex types table `complex-types-table`: ### Complex types table `complex-types-table`:
The test data in `resources/complex-types-table` contains 5 Delta records generated with 1 snapshot. 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 ```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 The resulting Delta table is checked in to the repo. The expectated rows to be used in tests are updated in
`ComplexTypesDeltaTable.java` accordingly. `ComplexTypesDeltaTable.java` accordingly.
### Snapshots table `snapshot-table`:
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
```
The resulting Delta table is checked in to the repo. The expectated rows to be used in tests are updated in
`SnapshotDeltaTable.java` accordingly.

View File

@ -16,11 +16,20 @@
# limitations under the License. # limitations under the License.
import argparse import argparse
from enum import Enum
from delta import * from delta import *
import pyspark import pyspark
from pyspark.sql.types import MapType, StructType, StructField, ShortType, StringType, TimestampType, LongType, IntegerType, DoubleType, FloatType, DateType, BooleanType, ArrayType 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 from datetime import datetime, timedelta
import random import random
from delta.tables import DeltaTable
class TableType(Enum):
SIMPLE = "simple"
COMPLEX = "complex"
SNAPSHOTS = "snapshots"
def config_spark_with_delta_lake(): def config_spark_with_delta_lake():
@ -40,15 +49,12 @@ def config_spark_with_delta_lake():
def create_dataset_with_complex_types(num_records): def create_dataset_with_complex_types(num_records):
""" """
Create a mock dataset with records containing complex types like arrays, structs and maps. Create a mock dataset with records containing complex types like arrays, structs and maps.
Parameters: Parameters:
- num_records (int): Number of records to generate. - num_records (int): Number of records to generate.
Returns: Returns:
- Tuple: A tuple containing a list of records and the corresponding schema. - 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. - List of Records: Each record is a tuple representing a row of data.
- StructType: The schema defining the structure of the records. - StructType: The schema defining the structure of the records.
Example: Example:
```python ```python
data, schema = create_dataset_with_complex_types(10) data, schema = create_dataset_with_complex_types(10)
@ -86,6 +92,59 @@ def create_dataset_with_complex_types(num_records):
return data, schema return data, schema
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()))
])
data = []
for idx in range(num_records):
record = (
idx,
{"snapshotVersion": 0}
)
data.append(record)
return data, schema
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)
delta_table.delete(condition="id=1")
# Snapshot 2: do a partial update of snapshotInfo map for id = 2 ; result : (id=2, id=0)
delta_table.update(
condition="id=2",
set={"map_info": expr("map('snapshotVersion', 2)")}
)
# Snapshot 3: New records to be appended; result : (id=1, id=4, id=2, id=0)
append_data = [
(1, {"snapshotVersion": 3}),
(4, {"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): def create_dataset(num_records):
""" """
Generate a mock employee dataset with different datatypes for testing purposes. Generate a mock employee dataset with different datatypes for testing purposes.
@ -141,19 +200,18 @@ def main():
parser = argparse.ArgumentParser(description="Script to write a Delta Lake table.", parser = argparse.ArgumentParser(description="Script to write a Delta Lake table.",
formatter_class=argparse.ArgumentDefaultsHelpFormatter) formatter_class=argparse.ArgumentDefaultsHelpFormatter)
parser.add_argument("--gen_complex_types", type=bool, default=False, help="Generate a Delta table with records" parser.add_argument('--delta_table_type', type=lambda t: TableType[t.upper()], choices=TableType,
" containing complex types like structs," default=TableType.SIMPLE, help='Choose a Delta table type to generate.')
" maps and arrays.")
parser.add_argument('--save_path', default=None, required=True, help="Save path for 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", parser.add_argument('--save_mode', choices=('append', 'overwrite'), default="append",
help="Specify write mode (append/overwrite)") 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") help="Column to partition the Delta table")
parser.add_argument('--num_records', type=int, default=5, help="Specify number of Delta records to write") parser.add_argument('--num_records', type=int, default=5, help="Specify number of Delta records to write")
args = parser.parse_args() 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_mode = args.save_mode
save_path = args.save_path save_path = args.save_path
num_records = args.num_records num_records = args.num_records
@ -161,21 +219,29 @@ def main():
spark = config_spark_with_delta_lake() spark = config_spark_with_delta_lake()
if is_gen_complex_types: if delta_table_type == TableType.SIMPLE:
data, schema = create_dataset_with_complex_types(num_records=num_records)
else:
data, schema = create_dataset(num_records=num_records) 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_snapshots_table(num_records)
else:
args.print_help()
raise Exception("Unknown value specified for --delta_table_type")
df = spark.createDataFrame(data, schema=schema) df = spark.createDataFrame(data, schema=schema)
if not partitioned_by: if not partitioned_by:
df.write.format("delta").mode(save_mode).save(save_path) df.write.format("delta").mode(save_mode).save(save_path)
else: 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() df.show()
print(f"Generated Delta table records partitioned by {partitioned_by} in {save_path} in {save_mode} mode" 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__": if __name__ == "__main__":

View File

@ -0,0 +1,6 @@
{"commitInfo":{"timestamp":1725465348581,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[\"id\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"3","numOutputRows":"3","numOutputBytes":"2607"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"d52bcd81-2310-417a-acb2-e206a4882383"}}
{"metaData":{"id":"5a4682fa-c3d8-4f49-8825-b8540e20ce93","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"map_info\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"integer\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["id"],"configuration":{},"createdTime":1725465346226}}
{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}
{"add":{"path":"id=0/part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet","partitionValues":{"id":"0"},"size":869,"modificationTime":1725465348507,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}}
{"add":{"path":"id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet","partitionValues":{"id":"1"},"size":869,"modificationTime":1725465348507,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}}
{"add":{"path":"id=2/part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet","partitionValues":{"id":"2"},"size":869,"modificationTime":1725465348506,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}}

View File

@ -0,0 +1,2 @@
{"commitInfo":{"timestamp":1725465352088,"operation":"DELETE","operationParameters":{"predicate":"[\"(id#852L = 1)\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"869","numCopiedRows":"0","numDeletionVectorsAdded":"0","numDeletionVectorsRemoved":"0","numAddedChangeFiles":"0","executionTimeMs":"426","numDeletionVectorsUpdated":"0","numDeletedRows":"1","scanTimeMs":"421","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"5af91bc8-feb5-40e2-b7d0-76acd1038ba7"}}
{"remove":{"path":"id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet","deletionTimestamp":1725465351650,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"id":"1"},"size":869,"stats":"{\"numRecords\":1}"}}

View File

@ -0,0 +1,3 @@
{"commitInfo":{"timestamp":1725465353405,"operation":"UPDATE","operationParameters":{"predicate":"[\"(id#852L = 2)\"]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"869","numCopiedRows":"0","numDeletionVectorsAdded":"0","numDeletionVectorsRemoved":"0","numAddedChangeFiles":"0","executionTimeMs":"448","numDeletionVectorsUpdated":"0","scanTimeMs":"215","numAddedFiles":"1","numUpdatedRows":"1","numAddedBytes":"869","rewriteTimeMs":"232"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"24e06ceb-96fe-44b4-bcf8-9c06604ec5f9"}}
{"add":{"path":"id=2/part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet","partitionValues":{"id":"2"},"size":869,"modificationTime":1725465353400,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}}
{"remove":{"path":"id=2/part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet","deletionTimestamp":1725465353403,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"id":"2"},"size":869}}

View File

@ -0,0 +1,3 @@
{"commitInfo":{"timestamp":1725465354608,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"2","numOutputRows":"2","numOutputBytes":"1738"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"ab44598c-07be-41df-bb30-e5c2f050cfe4"}}
{"add":{"path":"id=1/part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet","partitionValues":{"id":"1"},"size":869,"modificationTime":1725465354584,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}}
{"add":{"path":"id=4/part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet","partitionValues":{"id":"4"},"size":869,"modificationTime":1725465354593,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}}

View File

@ -2417,3 +2417,4 @@ percentilesSpectatorHistogram
quantilesFromDDSketch quantilesFromDDSketch
quantileFromDDSketch quantileFromDDSketch
collapsingLowestDense collapsingLowestDense
snapshotVersion