Skip to content

Commit 2bfb97e

Browse files
authored
[Feature][Iceberg] Support read multi-table (#8524)
1 parent 021af14 commit 2bfb97e

File tree

20 files changed

+665
-329
lines changed

20 files changed

+665
-329
lines changed

Diff for: docs/en/connector-v2/source/Iceberg.md

+29-21
Original file line numberDiff line numberDiff line change
@@ -71,11 +71,12 @@ libfb303-xxx.jar
7171

7272
## Source Options
7373

74-
| Name | Type | Required | Default | Description |
74+
| Name | Type | Required | Default | Description |
7575
|--------------------------|---------|----------|----------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
7676
| catalog_name | string | yes | - | User-specified catalog name. |
7777
| namespace | string | yes | - | The iceberg database name in the backend catalog. |
78-
| table | string | yes | - | The iceberg table name in the backend catalog. |
78+
| table | string | no | - | The iceberg table name in the backend catalog. |
79+
| table_list | string | no | - | The iceberg table list in the backend catalog. |
7980
| iceberg.catalog.config | map | yes | - | Specify the properties for initializing the Iceberg catalog, which can be referenced in this file:"https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/CatalogProperties.java" |
8081
| hadoop.config | map | no | - | Properties passed through to the Hadoop configuration |
8182
| iceberg.hadoop-conf-path | string | no | - | The specified loading paths for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files. |
@@ -87,6 +88,7 @@ libfb303-xxx.jar
8788
| use_snapshot_id | long | no | - | Instructs this scan to look for use the given snapshot ID. |
8889
| use_snapshot_timestamp | long | no | - | Instructs this scan to look for use the most recent snapshot as of the given time in milliseconds. timestamp – the timestamp in millis since the Unix epoch |
8990
| stream_scan_strategy | enum | no | FROM_LATEST_SNAPSHOT | Starting strategy for stream mode execution, Default to use `FROM_LATEST_SNAPSHOT` if don’t specify any value,The optional values are:<br/>TABLE_SCAN_THEN_INCREMENTAL: Do a regular table scan then switch to the incremental mode.<br/>FROM_LATEST_SNAPSHOT: Start incremental mode from the latest snapshot inclusive.<br/>FROM_EARLIEST_SNAPSHOT: Start incremental mode from the earliest snapshot inclusive.<br/>FROM_SNAPSHOT_ID: Start incremental mode from a snapshot with a specific id inclusive.<br/>FROM_SNAPSHOT_TIMESTAMP: Start incremental mode from a snapshot with a specific timestamp inclusive. |
91+
| increment.scan-interval | long | no | 2000 | The interval of increment scan(mills) |
9092
| common-options | | no | - | Source plugin common parameters, please refer to [Source Common Options](../source-common-options.md) for details. |
9193

9294
## Task Example
@@ -101,25 +103,6 @@ env {
101103
102104
source {
103105
Iceberg {
104-
schema {
105-
fields {
106-
f2 = "boolean"
107-
f1 = "bigint"
108-
f3 = "int"
109-
f4 = "bigint"
110-
f5 = "float"
111-
f6 = "double"
112-
f7 = "date"
113-
f9 = "timestamp"
114-
f10 = "timestamp"
115-
f11 = "string"
116-
f12 = "bytes"
117-
f13 = "bytes"
118-
f14 = "decimal(19,9)"
119-
f15 = "array<int>"
120-
f16 = "map<string, int>"
121-
}
122-
}
123106
catalog_name = "seatunnel"
124107
iceberg.catalog.config={
125108
type = "hadoop"
@@ -141,6 +124,31 @@ sink {
141124
}
142125
```
143126

127+
### Multi-Table Read:
128+
129+
```hocon
130+
source {
131+
Iceberg {
132+
catalog_name = "seatunnel"
133+
iceberg.catalog.config = {
134+
type = "hadoop"
135+
warehouse = "file:///tmp/seatunnel/iceberg/hadoop/"
136+
}
137+
namespace = "database1"
138+
table_list = [
139+
{
140+
table = "table_1
141+
},
142+
{
143+
table = "table_2
144+
}
145+
]
146+
147+
plugin_output = "iceberg"
148+
}
149+
}
150+
```
151+
144152
### Hadoop S3 Catalog:
145153

146154
```hocon

Diff for: seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java

+45-36
Original file line numberDiff line numberDiff line change
@@ -19,18 +19,19 @@
1919

2020
package org.apache.seatunnel.connectors.seatunnel.iceberg.config;
2121

22-
import org.apache.seatunnel.shade.com.typesafe.config.Config;
23-
2422
import org.apache.seatunnel.api.configuration.Option;
2523
import org.apache.seatunnel.api.configuration.Options;
2624
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
25+
import org.apache.seatunnel.api.table.catalog.TablePath;
2726
import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergStreamScanStrategy;
2827

29-
import org.apache.iceberg.expressions.Expression;
30-
3128
import lombok.Getter;
3229
import lombok.ToString;
3330

31+
import java.util.Collections;
32+
import java.util.List;
33+
import java.util.stream.Collectors;
34+
3435
import static org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergStreamScanStrategy.FROM_LATEST_SNAPSHOT;
3536

3637
@Getter
@@ -74,45 +75,53 @@ public class SourceConfig extends CommonConfig {
7475
.defaultValue(FROM_LATEST_SNAPSHOT)
7576
.withDescription(" the iceberg strategy of stream scanning");
7677

77-
private Long startSnapshotTimestamp;
78-
private Long startSnapshotId;
79-
private Long endSnapshotId;
78+
public static final Option<List<SourceTableConfig>> KEY_TABLE_LIST =
79+
Options.key("table_list")
80+
.listType(SourceTableConfig.class)
81+
.noDefaultValue()
82+
.withDescription(" the iceberg tables");
8083

81-
private Long useSnapshotId;
82-
private Long useSnapshotTimestamp;
84+
public static final Option<Long> KEY_INCREMENT_SCAN_INTERVAL =
85+
Options.key("increment.scan-interval")
86+
.longType()
87+
.defaultValue(2000L)
88+
.withDescription(" the interval of increment scan(mills)");
8389

84-
private IcebergStreamScanStrategy streamScanStrategy = KEY_STREAM_SCAN_STRATEGY.defaultValue();
85-
private Expression filter;
86-
private Long splitSize;
87-
private Integer splitLookback;
88-
private Long splitOpenFileCost;
90+
private long incrementScanInterval;
91+
private List<SourceTableConfig> tableList;
8992

9093
public SourceConfig(ReadonlyConfig readonlyConfig) {
9194
super(readonlyConfig);
92-
Config pluginConfig = readonlyConfig.toConfig();
93-
if (pluginConfig.hasPath(KEY_START_SNAPSHOT_TIMESTAMP.key())) {
94-
this.startSnapshotTimestamp = pluginConfig.getLong(KEY_START_SNAPSHOT_TIMESTAMP.key());
95-
}
96-
if (pluginConfig.hasPath(KEY_START_SNAPSHOT_ID.key())) {
97-
this.startSnapshotId = pluginConfig.getLong(KEY_START_SNAPSHOT_ID.key());
98-
}
99-
if (pluginConfig.hasPath(KEY_END_SNAPSHOT_ID.key())) {
100-
this.endSnapshotId = pluginConfig.getLong(KEY_END_SNAPSHOT_ID.key());
101-
}
102-
if (pluginConfig.hasPath(KEY_USE_SNAPSHOT_ID.key())) {
103-
this.useSnapshotId = pluginConfig.getLong(KEY_USE_SNAPSHOT_ID.key());
104-
}
105-
if (pluginConfig.hasPath(KEY_USE_SNAPSHOT_TIMESTAMP.key())) {
106-
this.useSnapshotTimestamp = pluginConfig.getLong(KEY_USE_SNAPSHOT_TIMESTAMP.key());
107-
}
108-
if (pluginConfig.hasPath(KEY_STREAM_SCAN_STRATEGY.key())) {
109-
this.streamScanStrategy =
110-
pluginConfig.getEnum(
111-
IcebergStreamScanStrategy.class, KEY_STREAM_SCAN_STRATEGY.key());
95+
this.incrementScanInterval = readonlyConfig.get(KEY_INCREMENT_SCAN_INTERVAL);
96+
if (this.getTable() != null) {
97+
SourceTableConfig tableConfig =
98+
SourceTableConfig.builder()
99+
.namespace(this.getNamespace())
100+
.table(this.getTable())
101+
.startSnapshotTimestamp(
102+
readonlyConfig.get(KEY_START_SNAPSHOT_TIMESTAMP))
103+
.startSnapshotId(readonlyConfig.get(KEY_START_SNAPSHOT_ID))
104+
.endSnapshotId(readonlyConfig.get(KEY_END_SNAPSHOT_ID))
105+
.useSnapshotId(readonlyConfig.get(KEY_USE_SNAPSHOT_ID))
106+
.useSnapshotTimestamp(readonlyConfig.get(KEY_USE_SNAPSHOT_TIMESTAMP))
107+
.streamScanStrategy(readonlyConfig.get(KEY_STREAM_SCAN_STRATEGY))
108+
.build();
109+
this.tableList = Collections.singletonList(tableConfig);
110+
} else {
111+
this.tableList =
112+
readonlyConfig.get(KEY_TABLE_LIST).stream()
113+
.map(
114+
tableConfig ->
115+
tableConfig.setNamespace(
116+
SourceConfig.this.getNamespace()))
117+
.collect(Collectors.toList());
112118
}
113119
}
114120

115-
public static SourceConfig loadConfig(ReadonlyConfig pluginConfig) {
116-
return new SourceConfig(pluginConfig);
121+
public SourceTableConfig getTableConfig(TablePath tablePath) {
122+
return tableList.stream()
123+
.filter(tableConfig -> tableConfig.getTablePath().equals(tablePath))
124+
.findFirst()
125+
.get();
117126
}
118127
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,81 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
20+
package org.apache.seatunnel.connectors.seatunnel.iceberg.config;
21+
22+
import org.apache.seatunnel.api.table.catalog.TablePath;
23+
import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergStreamScanStrategy;
24+
import org.apache.seatunnel.connectors.seatunnel.iceberg.utils.SchemaUtils;
25+
26+
import org.apache.iceberg.catalog.TableIdentifier;
27+
import org.apache.iceberg.expressions.Expression;
28+
29+
import lombok.AllArgsConstructor;
30+
import lombok.Builder;
31+
import lombok.Data;
32+
import lombok.experimental.Tolerate;
33+
34+
import java.io.Serializable;
35+
36+
import static org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig.KEY_STREAM_SCAN_STRATEGY;
37+
38+
@AllArgsConstructor
39+
@Data
40+
@Builder
41+
public class SourceTableConfig implements Serializable {
42+
private String namespace;
43+
private String table;
44+
45+
private Long startSnapshotTimestamp;
46+
private Long startSnapshotId;
47+
private Long endSnapshotId;
48+
49+
private Long useSnapshotId;
50+
private Long useSnapshotTimestamp;
51+
52+
private IcebergStreamScanStrategy streamScanStrategy = KEY_STREAM_SCAN_STRATEGY.defaultValue();
53+
private Expression filter;
54+
private Long splitSize;
55+
private Integer splitLookback;
56+
private Long splitOpenFileCost;
57+
58+
@Tolerate
59+
public SourceTableConfig() {}
60+
61+
public TablePath getTablePath() {
62+
String[] paths = table.split("\\.");
63+
if (paths.length == 1) {
64+
return TablePath.of(namespace, table);
65+
}
66+
if (paths.length == 2) {
67+
return TablePath.of(paths[0], paths[1]);
68+
}
69+
String namespace = table.substring(0, table.lastIndexOf("\\."));
70+
return TablePath.of(namespace, table);
71+
}
72+
73+
public TableIdentifier getTableIdentifier() {
74+
return SchemaUtils.toIcebergTableIdentifier(getTablePath());
75+
}
76+
77+
public SourceTableConfig setNamespace(String namespace) {
78+
this.namespace = namespace;
79+
return this;
80+
}
81+
}

Diff for: seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/sink/IcebergSinkWriter.java

+1
Original file line numberDiff line numberDiff line change
@@ -158,6 +158,7 @@ public void close() throws IOException {
158158
if (writer != null) {
159159
writer.close();
160160
}
161+
icebergTableLoader.close();
161162
} finally {
162163
results.clear();
163164
}

0 commit comments

Comments
 (0)