Skip to content

Commit 23dbea9

Browse files
committed
[Feature][Paimon] Support the streaming mode for sink #6342
[Feature][Paimon] Support cdc write of paimon sink #6427 [Feature][Paimon] Support auto create db and table for paimon sink
1 parent 7b48a16 commit 23dbea9

File tree

15 files changed

+886
-38
lines changed

15 files changed

+886
-38
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,87 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.seatunnel.connectors.seatunnel.paimon.data;
19+
20+
import org.apache.seatunnel.api.table.type.ArrayType;
21+
import org.apache.seatunnel.api.table.type.DecimalType;
22+
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
23+
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
24+
25+
import org.apache.paimon.types.DataField;
26+
import org.apache.paimon.types.DataType;
27+
import org.apache.paimon.types.DataTypes;
28+
29+
import java.util.concurrent.atomic.AtomicInteger;
30+
31+
public class PaimonTypeMapper {
32+
private static final AtomicInteger fieldId = new AtomicInteger(-1);
33+
34+
public static DataType toPaimonType(SeaTunnelDataType dataType) {
35+
switch (dataType.getSqlType()) {
36+
case BOOLEAN:
37+
return DataTypes.BOOLEAN();
38+
case BYTES:
39+
return DataTypes.BYTES();
40+
case SMALLINT:
41+
return DataTypes.SMALLINT();
42+
case TINYINT:
43+
return DataTypes.TINYINT();
44+
case INT:
45+
return DataTypes.INT();
46+
case BIGINT:
47+
return DataTypes.BIGINT();
48+
case FLOAT:
49+
return DataTypes.FLOAT();
50+
case DOUBLE:
51+
return DataTypes.DOUBLE();
52+
case DECIMAL:
53+
DecimalType decimalType = (DecimalType) dataType;
54+
return DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale());
55+
case ARRAY:
56+
ArrayType arrayType = (ArrayType) dataType;
57+
// converter elementType
58+
DataType elementType = toPaimonType(arrayType.getElementType());
59+
return DataTypes.ARRAY(elementType);
60+
case MAP:
61+
org.apache.seatunnel.api.table.type.MapType mapType =
62+
(org.apache.seatunnel.api.table.type.MapType) dataType;
63+
DataType keyType = toPaimonType(mapType.getKeyType());
64+
DataType valueType = toPaimonType(mapType.getValueType());
65+
return DataTypes.MAP(keyType, valueType);
66+
case ROW:
67+
SeaTunnelRowType seaTunnelRowType = (SeaTunnelRowType) dataType;
68+
DataField[] dataFields = new DataField[seaTunnelRowType.getTotalFields()];
69+
for (int i = 0; i < seaTunnelRowType.getFieldNames().length; i++) {
70+
String field = seaTunnelRowType.getFieldName(i);
71+
SeaTunnelDataType fieldType = seaTunnelRowType.getFieldType(i);
72+
int id = fieldId.incrementAndGet();
73+
dataFields[i] = new DataField(id, field, toPaimonType(fieldType));
74+
}
75+
return DataTypes.ROW(dataFields);
76+
case DATE:
77+
return DataTypes.DATE();
78+
case TIME:
79+
return DataTypes.TIME();
80+
case TIMESTAMP:
81+
return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE();
82+
case STRING:
83+
default:
84+
return DataTypes.STRING();
85+
}
86+
}
87+
}

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

+25-22
Original file line numberDiff line numberDiff line change
@@ -19,14 +19,15 @@
1919

2020
import org.apache.seatunnel.shade.com.typesafe.config.Config;
2121

22-
import org.apache.seatunnel.api.common.PrepareFailException;
22+
import org.apache.seatunnel.api.common.JobContext;
2323
import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode;
2424
import org.apache.seatunnel.api.serialization.DefaultSerializer;
2525
import org.apache.seatunnel.api.serialization.Serializer;
2626
import org.apache.seatunnel.api.sink.SeaTunnelSink;
2727
import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
2828
import org.apache.seatunnel.api.sink.SinkWriter;
29-
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
29+
import org.apache.seatunnel.api.table.catalog.CatalogTable;
30+
import org.apache.seatunnel.api.table.catalog.TableSchema;
3031
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
3132
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
3233
import org.apache.seatunnel.common.config.CheckConfigUtil;
@@ -38,6 +39,7 @@
3839
import org.apache.seatunnel.connectors.seatunnel.paimon.sink.commit.PaimonAggregatedCommitter;
3940
import org.apache.seatunnel.connectors.seatunnel.paimon.sink.commit.PaimonCommitInfo;
4041
import org.apache.seatunnel.connectors.seatunnel.paimon.sink.state.PaimonSinkState;
42+
import org.apache.seatunnel.connectors.seatunnel.paimon.utils.SchemaUtil;
4143

4244
import org.apache.hadoop.conf.Configuration;
4345
import org.apache.hadoop.fs.Path;
@@ -46,6 +48,7 @@
4648
import org.apache.paimon.catalog.CatalogFactory;
4749
import org.apache.paimon.catalog.Identifier;
4850
import org.apache.paimon.options.Options;
51+
import org.apache.paimon.schema.Schema;
4952
import org.apache.paimon.table.Table;
5053

5154
import com.google.auto.service.AutoService;
@@ -72,18 +75,11 @@ public class PaimonSink
7275

7376
private SeaTunnelRowType seaTunnelRowType;
7477

75-
private Config pluginConfig;
76-
7778
private Table table;
7879

79-
@Override
80-
public String getPluginName() {
81-
return PLUGIN_NAME;
82-
}
80+
private JobContext jobContext;
8381

84-
@Override
85-
public void prepare(Config pluginConfig) throws PrepareFailException {
86-
this.pluginConfig = pluginConfig;
82+
public PaimonSink(Config pluginConfig, CatalogTable catalogTable) {
8783
CheckResult result =
8884
CheckConfigUtil.checkAllExists(
8985
pluginConfig, WAREHOUSE.key(), DATABASE.key(), TABLE.key());
@@ -108,43 +104,45 @@ public void prepare(Config pluginConfig) throws PrepareFailException {
108104
final CatalogContext catalogContext = CatalogContext.create(options, hadoopConf);
109105
try (Catalog catalog = CatalogFactory.createCatalog(catalogContext)) {
110106
Identifier identifier = Identifier.create(database, table);
107+
// Auto create if not exists the database and table for paimon
108+
catalog.createDatabase(database, true);
109+
TableSchema tableSchema = catalogTable.getTableSchema();
110+
this.seaTunnelRowType = tableSchema.toPhysicalRowDataType();
111+
Schema paimonTableSchema = SchemaUtil.toPaimonSchema(tableSchema);
112+
catalog.createTable(identifier, paimonTableSchema, true);
111113
this.table = catalog.getTable(identifier);
114+
// todo if source is cdc,need to check primary key of tableSchema
112115
} catch (Exception e) {
113116
String errorMsg =
114117
String.format(
115-
"Failed to get table [%s] from database [%s] on warehouse [%s]",
118+
"Failed to create table [%s] from database [%s] on warehouse [%s]",
116119
database, table, warehouse);
117120
throw new PaimonConnectorException(
118121
PaimonConnectorErrorCode.GET_TABLE_FAILED, errorMsg, e);
119122
}
120123
}
121124

122125
@Override
123-
public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) {
124-
this.seaTunnelRowType = seaTunnelRowType;
125-
}
126-
127-
@Override
128-
public SeaTunnelDataType<SeaTunnelRow> getConsumedType() {
129-
return this.seaTunnelRowType;
126+
public String getPluginName() {
127+
return PLUGIN_NAME;
130128
}
131129

132130
@Override
133131
public SinkWriter<SeaTunnelRow, PaimonCommitInfo, PaimonSinkState> createWriter(
134132
SinkWriter.Context context) throws IOException {
135-
return new PaimonSinkWriter(context, table, seaTunnelRowType);
133+
return new PaimonSinkWriter(context, table, seaTunnelRowType, jobContext);
136134
}
137135

138136
@Override
139137
public Optional<SinkAggregatedCommitter<PaimonCommitInfo, PaimonAggregatedCommitInfo>>
140138
createAggregatedCommitter() throws IOException {
141-
return Optional.of(new PaimonAggregatedCommitter(table));
139+
return Optional.of(new PaimonAggregatedCommitter(table, jobContext));
142140
}
143141

144142
@Override
145143
public SinkWriter<SeaTunnelRow, PaimonCommitInfo, PaimonSinkState> restoreWriter(
146144
SinkWriter.Context context, List<PaimonSinkState> states) throws IOException {
147-
return new PaimonSinkWriter(context, table, seaTunnelRowType, states);
145+
return new PaimonSinkWriter(context, table, seaTunnelRowType, states, jobContext);
148146
}
149147

150148
@Override
@@ -156,4 +154,9 @@ public Optional<Serializer<PaimonAggregatedCommitInfo>> getAggregatedCommitInfoS
156154
public Optional<Serializer<PaimonCommitInfo>> getCommitInfoSerializer() {
157155
return Optional.of(new DefaultSerializer<>());
158156
}
157+
158+
@Override
159+
public void setJobContext(JobContext jobContext) {
160+
this.jobContext = jobContext;
161+
}
159162
}

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

+12
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,14 @@
1717

1818
package org.apache.seatunnel.connectors.seatunnel.paimon.sink;
1919

20+
import org.apache.seatunnel.shade.com.typesafe.config.Config;
21+
2022
import org.apache.seatunnel.api.configuration.util.OptionRule;
23+
import org.apache.seatunnel.api.table.catalog.CatalogTable;
24+
import org.apache.seatunnel.api.table.connector.TableSink;
2125
import org.apache.seatunnel.api.table.factory.Factory;
2226
import org.apache.seatunnel.api.table.factory.TableSinkFactory;
27+
import org.apache.seatunnel.api.table.factory.TableSinkFactoryContext;
2328
import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonConfig;
2429

2530
import com.google.auto.service.AutoService;
@@ -41,4 +46,11 @@ public OptionRule optionRule() {
4146
.optional(PaimonConfig.HDFS_SITE_PATH)
4247
.build();
4348
}
49+
50+
@Override
51+
public TableSink createSink(TableSinkFactoryContext context) {
52+
Config pluginConfig = context.getOptions().toConfig();
53+
CatalogTable catalogTable = context.getCatalogTable();
54+
return () -> new PaimonSink(pluginConfig, catalogTable);
55+
}
4456
}

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

+43-10
Original file line numberDiff line numberDiff line change
@@ -17,21 +17,27 @@
1717

1818
package org.apache.seatunnel.connectors.seatunnel.paimon.sink;
1919

20+
import org.apache.seatunnel.api.common.JobContext;
2021
import org.apache.seatunnel.api.sink.SinkWriter;
2122
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
2223
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
2324
import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorErrorCode;
2425
import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorException;
2526
import org.apache.seatunnel.connectors.seatunnel.paimon.sink.commit.PaimonCommitInfo;
2627
import org.apache.seatunnel.connectors.seatunnel.paimon.sink.state.PaimonSinkState;
28+
import org.apache.seatunnel.connectors.seatunnel.paimon.utils.JobContextUtil;
2729
import org.apache.seatunnel.connectors.seatunnel.paimon.utils.RowConverter;
2830

2931
import org.apache.paimon.data.InternalRow;
3032
import org.apache.paimon.table.Table;
3133
import org.apache.paimon.table.sink.BatchTableCommit;
3234
import org.apache.paimon.table.sink.BatchTableWrite;
33-
import org.apache.paimon.table.sink.BatchWriteBuilder;
3435
import org.apache.paimon.table.sink.CommitMessage;
36+
import org.apache.paimon.table.sink.StreamTableCommit;
37+
import org.apache.paimon.table.sink.StreamTableWrite;
38+
import org.apache.paimon.table.sink.TableCommit;
39+
import org.apache.paimon.table.sink.TableWrite;
40+
import org.apache.paimon.table.sink.WriteBuilder;
3541

3642
import lombok.extern.slf4j.Slf4j;
3743

@@ -50,9 +56,9 @@ public class PaimonSinkWriter
5056

5157
private String commitUser = UUID.randomUUID().toString();
5258

53-
private final BatchWriteBuilder tableWriteBuilder;
59+
private final WriteBuilder tableWriteBuilder;
5460

55-
private final BatchTableWrite tableWrite;
61+
private final TableWrite tableWrite;
5662

5763
private long checkpointId = 0;
5864

@@ -64,37 +70,58 @@ public class PaimonSinkWriter
6470

6571
private final SinkWriter.Context context;
6672

67-
public PaimonSinkWriter(Context context, Table table, SeaTunnelRowType seaTunnelRowType) {
73+
private final JobContext jobContext;
74+
75+
public PaimonSinkWriter(
76+
Context context,
77+
Table table,
78+
SeaTunnelRowType seaTunnelRowType,
79+
JobContext jobContext) {
6880
this.table = table;
69-
this.tableWriteBuilder = this.table.newBatchWriteBuilder().withOverwrite();
81+
this.tableWriteBuilder =
82+
JobContextUtil.isBatchJob(jobContext)
83+
? this.table.newBatchWriteBuilder().withOverwrite()
84+
: this.table.newStreamWriteBuilder();
7085
this.tableWrite = tableWriteBuilder.newWrite();
7186
this.seaTunnelRowType = seaTunnelRowType;
7287
this.context = context;
88+
this.jobContext = jobContext;
7389
}
7490

7591
public PaimonSinkWriter(
7692
Context context,
7793
Table table,
7894
SeaTunnelRowType seaTunnelRowType,
79-
List<PaimonSinkState> states) {
95+
List<PaimonSinkState> states,
96+
JobContext jobContext) {
8097
this.table = table;
81-
this.tableWriteBuilder = this.table.newBatchWriteBuilder().withOverwrite();
98+
this.tableWriteBuilder =
99+
JobContextUtil.isBatchJob(jobContext)
100+
? this.table.newBatchWriteBuilder().withOverwrite()
101+
: this.table.newStreamWriteBuilder();
82102
this.tableWrite = tableWriteBuilder.newWrite();
83103
this.seaTunnelRowType = seaTunnelRowType;
84104
this.context = context;
105+
this.jobContext = jobContext;
85106
if (Objects.isNull(states) || states.isEmpty()) {
86107
return;
87108
}
88109
this.commitUser = states.get(0).getCommitUser();
89110
this.checkpointId = states.get(0).getCheckpointId();
90-
try (BatchTableCommit tableCommit = tableWriteBuilder.newCommit()) {
111+
try (TableCommit tableCommit = tableWriteBuilder.newCommit()) {
91112
List<CommitMessage> commitables =
92113
states.stream()
93114
.map(PaimonSinkState::getCommittables)
94115
.flatMap(List::stream)
95116
.collect(Collectors.toList());
96117
log.info("Trying to recommit states {}", commitables);
97-
tableCommit.commit(commitables);
118+
if (JobContextUtil.isBatchJob(jobContext)) {
119+
log.debug("Trying to recommit states batch mode");
120+
((BatchTableCommit) tableCommit).commit(commitables);
121+
} else {
122+
log.debug("Trying to recommit states streaming mode");
123+
((StreamTableCommit) tableCommit).commit(Objects.hash(commitables), commitables);
124+
}
98125
} catch (Exception e) {
99126
throw new PaimonConnectorException(
100127
PaimonConnectorErrorCode.TABLE_WRITE_COMMIT_FAILED, e);
@@ -117,7 +144,13 @@ public void write(SeaTunnelRow element) throws IOException {
117144
@Override
118145
public Optional<PaimonCommitInfo> prepareCommit() throws IOException {
119146
try {
120-
List<CommitMessage> fileCommittables = tableWrite.prepareCommit();
147+
List<CommitMessage> fileCommittables;
148+
if (JobContextUtil.isBatchJob(jobContext)) {
149+
fileCommittables = ((BatchTableWrite) tableWrite).prepareCommit();
150+
} else {
151+
fileCommittables =
152+
((StreamTableWrite) tableWrite).prepareCommit(false, committables.size());
153+
}
121154
committables.addAll(fileCommittables);
122155
return Optional.of(new PaimonCommitInfo(fileCommittables));
123156
} catch (Exception e) {

0 commit comments

Comments
 (0)