mirror of https://github.com/apache/druid.git
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:
parent
51fe3c08ab
commit
aa833a711c
|
@ -51,9 +51,4 @@ java \
|
|||
-c "org.apache.druid.extensions.contrib:druid-deltalake-extensions:<VERSION>"
|
||||
```
|
||||
|
||||
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).
|
||||
See [Loading community extensions](../../configuration/extensions.md#loading-community-extensions) for more information.
|
|
@ -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|
|
||||
|snapshotVersion|The snapshot version to read from the Delta table. An integer value must be specified.|Latest|no|
|
||||
|
||||
### Delta filter object
|
||||
|
||||
|
@ -1224,7 +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 |
|
||||
|
||||
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
|
||||
...
|
||||
|
@ -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
|
||||
...
|
||||
|
@ -1260,7 +1262,8 @@ The following is a sample spec to read records from the Delta table `/delta-tabl
|
|||
"value": "30"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"snapshotVersion": 3
|
||||
},
|
||||
}
|
||||
```
|
||||
|
|
|
@ -67,9 +67,9 @@ import java.util.stream.Collectors;
|
|||
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 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:
|
||||
* <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 non-partitioned columns, the filtering is best-effort as the Delta
|
||||
|
@ -79,7 +79,6 @@ import java.util.stream.Stream;
|
|||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*/
|
||||
public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
|
||||
|
@ -97,11 +96,15 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
|
|||
@Nullable
|
||||
private final DeltaFilter filter;
|
||||
|
||||
@JsonProperty
|
||||
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("filter") @Nullable final DeltaFilter filter,
|
||||
@JsonProperty("snapshotVersion") @Nullable final Long snapshotVersion
|
||||
)
|
||||
{
|
||||
if (tablePath == null) {
|
||||
|
@ -110,6 +113,7 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
|
|||
this.tablePath = tablePath;
|
||||
this.deltaSplit = deltaSplit;
|
||||
this.filter = filter;
|
||||
this.snapshotVersion = snapshotVersion;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -152,15 +156,15 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
|
|||
}
|
||||
} 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 +210,17 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
|
|||
}
|
||||
|
||||
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 +258,8 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
|
|||
return new DeltaInputSource(
|
||||
tablePath,
|
||||
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
|
||||
// the 3.2.0 Delta Kernel because the Kernel library cannot instantiate the LogStore class otherwise. Please see
|
||||
|
@ -341,7 +346,11 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
|
|||
final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
|
||||
try {
|
||||
Thread.currentThread().setContextClassLoader(LogStore.class.getClassLoader());
|
||||
return table.getLatestSnapshot(engine);
|
||||
if (snapshotVersion != null) {
|
||||
return table.getSnapshotAsOfVersion(engine, snapshotVersion);
|
||||
} else {
|
||||
return table.getLatestSnapshot(engine);
|
||||
}
|
||||
}
|
||||
finally {
|
||||
Thread.currentThread().setContextClassLoader(currCtxCl);
|
||||
|
@ -359,4 +368,10 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
|
|||
{
|
||||
return filter;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Long getSnapshotVersion()
|
||||
{
|
||||
return snapshotVersion;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,7 +55,8 @@ public class DeltaInputRowTest
|
|||
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);
|
||||
}
|
||||
|
@ -124,7 +125,7 @@ public class DeltaInputRowTest
|
|||
@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(
|
||||
|
|
|
@ -139,4 +139,18 @@ public class DeltaInputSourceSerdeTest
|
|||
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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
@ -68,27 +69,62 @@ public class DeltaInputSourceTest
|
|||
{
|
||||
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,
|
||||
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)
|
||||
public InputRowSchema schema;
|
||||
@Parameterized.Parameter(2)
|
||||
public Long snapshotVersion;
|
||||
@Parameterized.Parameter(3)
|
||||
public List<Map<String, Object>> expectedRows;
|
||||
|
||||
@Test
|
||||
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);
|
||||
|
||||
List<InputRowListPlusRawValues> actualSampledRows = sampleAllRows(inputSourceReader);
|
||||
|
@ -137,7 +175,7 @@ public class DeltaInputSourceTest
|
|||
@Test
|
||||
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 List<InputRow> actualReadRows = readAllRows(inputSourceReader);
|
||||
validateRows(expectedRows, actualReadRows, schema);
|
||||
|
@ -269,7 +307,7 @@ public class DeltaInputSourceTest
|
|||
@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<InputRowListPlusRawValues> actualSampledRows = sampleAllRows(inputSourceReader);
|
||||
|
@ -311,7 +349,7 @@ public class DeltaInputSourceTest
|
|||
@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<InputRow> actualReadRows = readAllRows(inputSourceReader);
|
||||
validateRows(expectedRows, actualReadRows, schema);
|
||||
|
@ -326,7 +364,7 @@ public class DeltaInputSourceTest
|
|||
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 +375,7 @@ public class DeltaInputSourceTest
|
|||
@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 +391,7 @@ public class DeltaInputSourceTest
|
|||
@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(
|
||||
|
@ -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
|
||||
|
|
|
@ -37,7 +37,9 @@ public class RowSerdeTest
|
|||
{
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
|
@ -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)
|
||||
```
|
||||
|
@ -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`:
|
||||
|
||||
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 --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 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.
|
||||
|
|
|
@ -16,11 +16,20 @@
|
|||
# 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():
|
||||
|
@ -40,15 +49,12 @@ 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)
|
||||
|
@ -86,6 +92,59 @@ def create_dataset_with_complex_types(num_records):
|
|||
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):
|
||||
"""
|
||||
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.",
|
||||
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('--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 +219,29 @@ def main():
|
|||
|
||||
spark = config_spark_with_delta_lake()
|
||||
|
||||
if is_gen_complex_types:
|
||||
data, schema = create_dataset_with_complex_types(num_records=num_records)
|
||||
else:
|
||||
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_snapshots_table(num_records)
|
||||
else:
|
||||
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__":
|
||||
|
|
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
|
@ -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}}"}}
|
|
@ -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}"}}
|
|
@ -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}}
|
|
@ -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}}"}}
|
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
|
@ -2417,3 +2417,4 @@ percentilesSpectatorHistogram
|
|||
quantilesFromDDSketch
|
||||
quantileFromDDSketch
|
||||
collapsingLowestDense
|
||||
snapshotVersion
|
||||
|
|
Loading…
Reference in New Issue