Skip to content

Commit

Permalink
[Backport] Extension to read and ingest Delta Lake tables (#15755) (#…
Browse files Browse the repository at this point in the history
…15812)

First commit: clean backport of Extension to read and ingest Delta Lake tables #15755
Second commit: change version from 30.0.0 to 29.0.0
  • Loading branch information
abhishekrb19 authored Jan 31, 2024
1 parent 4b60c7e commit 8438d0f
Show file tree
Hide file tree
Showing 64 changed files with 2,303 additions and 9 deletions.
4 changes: 4 additions & 0 deletions distribution/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -257,6 +257,8 @@
<argument>org.apache.druid.extensions:druid-kubernetes-extensions</argument>
<argument>-c</argument>
<argument>org.apache.druid.extensions:druid-catalog</argument>
<argument>-c</argument>
<argument>org.apache.druid.extensions:druid-deltalake-extensions</argument>
<argument>${druid.distribution.pulldeps.opts}</argument>
</arguments>
</configuration>
Expand Down Expand Up @@ -453,6 +455,8 @@
<argument>-c</argument>
<argument>org.apache.druid.extensions:druid-iceberg-extensions</argument>
<argument>-c</argument>
<argument>org.apache.druid.extensions:druid-deltalake-extensions</argument>
<argument>-c</argument>
<argument>org.apache.druid.extensions.contrib:druid-spectator-histogram</argument>
</arguments>
</configuration>
Expand Down
44 changes: 44 additions & 0 deletions docs/development/extensions-contrib/delta-lake.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
---
id: delta-lake
title: "Delta Lake extension"
---

<!--
~ 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.
-->

## Delta Lake extension


Delta Lake is an open source storage framework that enables building a
Lakehouse architecture with various compute engines. [DeltaLakeInputSource](../../ingestion/input-sources.md#delta-lake-input-source) lets
you ingest data stored in a Delta Lake table into Apache Druid. To use the Delta Lake extension, add the `druid-deltalake-extensions` to the list of loaded extensions.
See [Loading extensions](../../configuration/extensions.md#loading-extensions) for more information.

The Delta input source reads the configured Delta Lake table and extracts all the underlying delta files in the table's latest snapshot.
These Delta Lake files are versioned Parquet files.

## Version support

The Delta Lake extension uses the Delta Kernel introduced in Delta Lake 3.0.0, which is compatible with Apache Spark 3.5.x.
Older versions are unsupported, so consider upgrading to Delta Lake 3.0.x or higher to use this extension.

## Known limitations

- This extension relies on the Delta Kernel API and can only read from the latest Delta table snapshot.
- Column filtering isn't supported. The extension reads all columns in the configured table.
39 changes: 33 additions & 6 deletions docs/ingestion/input-sources.md
Original file line number Diff line number Diff line change
Expand Up @@ -715,6 +715,13 @@ rolled-up datasource `wikipedia_rollup` by grouping on hour, "countryName", and
to `true` to enable a compatibility mode where the timestampSpec is ignored.
:::

The [secondary partitioning method](native-batch.md#partitionsspec) determines the requisite number of concurrent worker tasks that run in parallel to complete ingestion with the Combining input source.
Set this value in `maxNumConcurrentSubTasks` in `tuningConfig` based on the secondary partitioning method:
- `range` or `single_dim` partitioning: greater than or equal to 1
- `hashed` or `dynamic` partitioning: greater than or equal to 2

For more information on the `maxNumConcurrentSubTasks` field, see [Implementation considerations](native-batch.md#implementation-considerations).

## SQL input source

The SQL input source is used to read data directly from RDBMS.
Expand Down Expand Up @@ -817,7 +824,7 @@ The following is an example of a Combining input source spec:
## Iceberg input source

:::info
To use the Iceberg input source, add the `druid-iceberg-extensions` extension.
To use the Iceberg input source, load the extension [`druid-iceberg-extensions`](../development/extensions-contrib/iceberg.md).
:::

You use the Iceberg input source to read data stored in the Iceberg table format. For a given table, the input source scans up to the latest Iceberg snapshot from the configured Hive catalog. Druid ingests the underlying live data files using the existing input source formats.
Expand Down Expand Up @@ -1006,11 +1013,31 @@ This input source provides the following filters: `and`, `equals`, `interval`, a
|type|Set this value to `not`.|yes|
|filter|The iceberg filter on which logical NOT is applied|yes|

## Delta Lake input source

:::info
To use the Delta Lake input source, load the extension [`druid-deltalake-extensions`](../development/extensions-contrib/delta-lake.md).
:::

The [secondary partitioning method](native-batch.md#partitionsspec) determines the requisite number of concurrent worker tasks that run in parallel to complete ingestion with the Combining input source.
Set this value in `maxNumConcurrentSubTasks` in `tuningConfig` based on the secondary partitioning method:
- `range` or `single_dim` partitioning: greater than or equal to 1
- `hashed` or `dynamic` partitioning: greater than or equal to 2
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 following is a sample spec:

```json
...
"ioConfig": {
"type": "index_parallel",
"inputSource": {
"type": "delta",
"tablePath": "/delta-table/directory"
},
}
}
```

| Property|Description|Required|
|---------|-----------|--------|
| type|Set this value to `delta`.|yes|
| tablePath|The location of the Delta table.|yes|

For more information on the `maxNumConcurrentSubTasks` field, see [Implementation considerations](native-batch.md#implementation-considerations).
156 changes: 156 additions & 0 deletions extensions-contrib/druid-deltalake-extensions/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,156 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">

<groupId>org.apache.druid.extensions</groupId>
<artifactId>druid-deltalake-extensions</artifactId>
<name>druid-deltalake-extensions</name>
<description>Delta Lake connector for Druid</description>

<parent>
<artifactId>druid</artifactId>
<groupId>org.apache.druid</groupId>
<version>29.0.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>

<properties>
<delta-kernel.version>3.0.0</delta-kernel.version>
</properties>

<dependencies>
<dependency>
<groupId>io.delta</groupId>
<artifactId>delta-kernel-api</artifactId>
<version>${delta-kernel.version}</version>
</dependency>

<dependency>
<groupId>io.delta</groupId>
<artifactId>delta-kernel-defaults</artifactId>
<version>${delta-kernel.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-api</artifactId>
<version>${hadoop.compile.version}</version>
<scope>compile</scope>
</dependency>

<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.google.inject</groupId>
<artifactId>guice</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<version>2.12.7.1</version>
</dependency>
<dependency>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil-core</artifactId>
<version>8.5.4</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-aws</artifactId>
<version>${hadoop.compile.version}</version>
<scope>runtime</scope>
<exclusions>
<exclusion>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-bundle</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-all</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-core</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.owasp</groupId>
<artifactId>dependency-check-maven</artifactId>
<configuration>
<skip>true</skip>
</configuration>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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.common;

import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.inject.Binder;
import org.apache.druid.delta.input.DeltaInputSource;
import org.apache.druid.error.DruidException;
import org.apache.druid.initialization.DruidModule;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;

import java.util.Collections;
import java.util.List;

public class DeltaLakeDruidModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return Collections.singletonList(
new SimpleModule("DeltaLakeDruidModule")
.registerSubtypes(
new NamedType(DeltaInputSource.class, DeltaInputSource.TYPE_KEY)
)
);
}

@Override
public void configure(Binder binder)
{
final Configuration conf = new Configuration();
conf.setClassLoader(getClass().getClassLoader());

ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
FileSystem.get(conf);
}
catch (Exception ex) {
throw DruidException.forPersona(DruidException.Persona.DEVELOPER)
.ofCategory(DruidException.Category.UNCATEGORIZED)
.build(ex, "Problem during fileSystem class level initialization");
}
finally {
Thread.currentThread().setContextClassLoader(currCtxCl);
}
}

}

Loading

0 comments on commit 8438d0f

Please sign in to comment.