Skip to content

Commit 5b32488

Browse files
committed
[Feature][Paimon] paimon sink support savemode
1 parent a70d696 commit 5b32488

File tree

12 files changed

+698
-57
lines changed

12 files changed

+698
-57
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,215 @@
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.catalog;
19+
20+
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
21+
import org.apache.seatunnel.api.table.catalog.Catalog;
22+
import org.apache.seatunnel.api.table.catalog.CatalogTable;
23+
import org.apache.seatunnel.api.table.catalog.PhysicalColumn;
24+
import org.apache.seatunnel.api.table.catalog.TablePath;
25+
import org.apache.seatunnel.api.table.catalog.TableSchema;
26+
import org.apache.seatunnel.api.table.catalog.exception.CatalogException;
27+
import org.apache.seatunnel.api.table.catalog.exception.DatabaseAlreadyExistException;
28+
import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException;
29+
import org.apache.seatunnel.api.table.catalog.exception.TableAlreadyExistException;
30+
import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException;
31+
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
32+
import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig;
33+
import org.apache.seatunnel.connectors.seatunnel.paimon.utils.SchemaUtil;
34+
35+
import org.apache.paimon.catalog.Identifier;
36+
import org.apache.paimon.table.FileStoreTable;
37+
import org.apache.paimon.table.Table;
38+
import org.apache.paimon.types.DataField;
39+
40+
import lombok.extern.slf4j.Slf4j;
41+
42+
import java.io.Closeable;
43+
import java.io.IOException;
44+
import java.util.List;
45+
46+
@Slf4j
47+
public class PaimonCatalog implements Catalog, PaimonTable {
48+
private static final String DEFAULT_DATABASE = "default";
49+
50+
private String catalogName;
51+
private ReadonlyConfig readonlyConfig;
52+
private PaimonCatalogLoader paimonCatalogLoader;
53+
private org.apache.paimon.catalog.Catalog catalog;
54+
55+
public PaimonCatalog(String catalogName, ReadonlyConfig readonlyConfig) {
56+
this.readonlyConfig = readonlyConfig;
57+
this.catalogName = catalogName;
58+
this.paimonCatalogLoader = new PaimonCatalogLoader(new PaimonSinkConfig(readonlyConfig));
59+
}
60+
61+
@Override
62+
public void open() throws CatalogException {
63+
this.catalog = paimonCatalogLoader.loadCatalog();
64+
}
65+
66+
@Override
67+
public void close() throws CatalogException {
68+
if (catalog != null && catalog instanceof Closeable) {
69+
try {
70+
((Closeable) catalog).close();
71+
} catch (IOException e) {
72+
log.error("Error while closing IcebergCatalog.", e);
73+
throw new CatalogException(e);
74+
}
75+
}
76+
}
77+
78+
@Override
79+
public String name() {
80+
return this.catalogName;
81+
}
82+
83+
@Override
84+
public String getDefaultDatabase() throws CatalogException {
85+
return DEFAULT_DATABASE;
86+
}
87+
88+
@Override
89+
public boolean databaseExists(String databaseName) throws CatalogException {
90+
return catalog.databaseExists(databaseName);
91+
}
92+
93+
@Override
94+
public List<String> listDatabases() throws CatalogException {
95+
return catalog.listDatabases();
96+
}
97+
98+
@Override
99+
public List<String> listTables(String databaseName)
100+
throws CatalogException, DatabaseNotExistException {
101+
try {
102+
return catalog.listTables(databaseName);
103+
} catch (org.apache.paimon.catalog.Catalog.DatabaseNotExistException e) {
104+
throw new DatabaseNotExistException(this.catalogName, databaseName);
105+
}
106+
}
107+
108+
@Override
109+
public boolean tableExists(TablePath tablePath) throws CatalogException {
110+
return catalog.tableExists(toIdentifier(tablePath));
111+
}
112+
113+
@Override
114+
public CatalogTable getTable(TablePath tablePath)
115+
throws CatalogException, TableNotExistException {
116+
try {
117+
FileStoreTable paimonFileStoreTableTable = (FileStoreTable) getPaimonTable(tablePath);
118+
return toCatalogTable(paimonFileStoreTableTable, tablePath);
119+
} catch (Exception e) {
120+
throw new TableNotExistException(this.catalogName, tablePath);
121+
}
122+
}
123+
124+
@Override
125+
public Table getPaimonTable(TablePath tablePath)
126+
throws CatalogException, TableNotExistException {
127+
try {
128+
return catalog.getTable(toIdentifier(tablePath));
129+
} catch (org.apache.paimon.catalog.Catalog.TableNotExistException e) {
130+
throw new TableNotExistException(this.catalogName, tablePath);
131+
}
132+
}
133+
134+
@Override
135+
public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreIfExists)
136+
throws TableAlreadyExistException, DatabaseNotExistException, CatalogException {
137+
try {
138+
catalog.createTable(
139+
toIdentifier(tablePath),
140+
SchemaUtil.toPaimonSchema(table.getTableSchema()),
141+
ignoreIfExists);
142+
} catch (org.apache.paimon.catalog.Catalog.TableAlreadyExistException e) {
143+
throw new TableAlreadyExistException(this.catalogName, tablePath);
144+
} catch (org.apache.paimon.catalog.Catalog.DatabaseNotExistException e) {
145+
throw new DatabaseNotExistException(this.catalogName, tablePath.getDatabaseName());
146+
}
147+
}
148+
149+
@Override
150+
public void dropTable(TablePath tablePath, boolean ignoreIfNotExists)
151+
throws TableNotExistException, CatalogException {
152+
try {
153+
catalog.dropTable(toIdentifier(tablePath), ignoreIfNotExists);
154+
} catch (org.apache.paimon.catalog.Catalog.TableNotExistException e) {
155+
throw new TableNotExistException(this.catalogName, tablePath);
156+
}
157+
}
158+
159+
@Override
160+
public void createDatabase(TablePath tablePath, boolean ignoreIfExists)
161+
throws DatabaseAlreadyExistException, CatalogException {
162+
try {
163+
catalog.createDatabase(tablePath.getDatabaseName(), ignoreIfExists);
164+
} catch (org.apache.paimon.catalog.Catalog.DatabaseAlreadyExistException e) {
165+
throw new DatabaseAlreadyExistException(this.catalogName, tablePath.getDatabaseName());
166+
}
167+
}
168+
169+
@Override
170+
public void dropDatabase(TablePath tablePath, boolean ignoreIfNotExists)
171+
throws DatabaseNotExistException, CatalogException {
172+
try {
173+
catalog.dropDatabase(tablePath.getDatabaseName(), ignoreIfNotExists, true);
174+
} catch (Exception e) {
175+
throw new DatabaseNotExistException(this.catalogName, tablePath.getDatabaseName());
176+
}
177+
}
178+
179+
private CatalogTable toCatalogTable(
180+
FileStoreTable paimonFileStoreTableTable, TablePath tablePath) {
181+
org.apache.paimon.schema.TableSchema schema = paimonFileStoreTableTable.schema();
182+
List<DataField> dataFields = schema.fields();
183+
TableSchema.Builder builder = TableSchema.builder();
184+
dataFields.forEach(
185+
dataField -> {
186+
String name = dataField.name();
187+
SeaTunnelDataType<?> seaTunnelType =
188+
SchemaUtil.toSeaTunnelType(dataField.type());
189+
PhysicalColumn physicalColumn =
190+
PhysicalColumn.of(
191+
name,
192+
seaTunnelType,
193+
(Long) null,
194+
true,
195+
null,
196+
dataField.description());
197+
builder.column(physicalColumn);
198+
});
199+
200+
List<String> partitionKeys = schema.partitionKeys();
201+
202+
return CatalogTable.of(
203+
org.apache.seatunnel.api.table.catalog.TableIdentifier.of(
204+
catalogName, tablePath.getDatabaseName(), tablePath.getTableName()),
205+
builder.build(),
206+
paimonFileStoreTableTable.options(),
207+
partitionKeys,
208+
null,
209+
catalogName);
210+
}
211+
212+
private Identifier toIdentifier(TablePath tablePath) {
213+
return Identifier.create(tablePath.getDatabaseName(), tablePath.getTableName());
214+
}
215+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,54 @@
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.catalog;
19+
20+
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
21+
import org.apache.seatunnel.api.configuration.util.OptionRule;
22+
import org.apache.seatunnel.api.table.catalog.Catalog;
23+
import org.apache.seatunnel.api.table.factory.CatalogFactory;
24+
import org.apache.seatunnel.api.table.factory.Factory;
25+
import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig;
26+
27+
import com.google.auto.service.AutoService;
28+
29+
@AutoService(Factory.class)
30+
public class PaimonCatalogFactory implements CatalogFactory {
31+
@Override
32+
public Catalog createCatalog(String catalogName, ReadonlyConfig readonlyConfig) {
33+
return new PaimonCatalog(catalogName, readonlyConfig);
34+
}
35+
36+
@Override
37+
public String factoryIdentifier() {
38+
return "Paimon";
39+
}
40+
41+
@Override
42+
public OptionRule optionRule() {
43+
return OptionRule.builder()
44+
.required(
45+
PaimonSinkConfig.WAREHOUSE,
46+
PaimonSinkConfig.DATABASE,
47+
PaimonSinkConfig.TABLE)
48+
.optional(
49+
PaimonSinkConfig.HDFS_SITE_PATH,
50+
PaimonSinkConfig.SCHEMA_SAVE_MODE,
51+
PaimonSinkConfig.DATA_SAVE_MODE)
52+
.build();
53+
}
54+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,61 @@
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.catalog;
19+
20+
import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig;
21+
22+
import org.apache.commons.lang3.StringUtils;
23+
import org.apache.hadoop.conf.Configuration;
24+
import org.apache.hadoop.fs.Path;
25+
import org.apache.paimon.catalog.Catalog;
26+
import org.apache.paimon.catalog.CatalogContext;
27+
import org.apache.paimon.catalog.CatalogFactory;
28+
import org.apache.paimon.options.Options;
29+
30+
import lombok.extern.slf4j.Slf4j;
31+
32+
import java.io.Serializable;
33+
import java.util.HashMap;
34+
import java.util.Map;
35+
36+
import static org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonConfig.WAREHOUSE;
37+
38+
@Slf4j
39+
public class PaimonCatalogLoader implements Serializable {
40+
private PaimonSinkConfig config;
41+
42+
public PaimonCatalogLoader(PaimonSinkConfig config) {
43+
this.config = config;
44+
}
45+
46+
public Catalog loadCatalog() {
47+
// When using the seatunel engine, set the current class loader to prevent loading failures
48+
Thread.currentThread().setContextClassLoader(PaimonCatalogLoader.class.getClassLoader());
49+
final String warehouse = config.getWarehouse();
50+
final Map<String, String> optionsMap = new HashMap<>();
51+
optionsMap.put(WAREHOUSE.key(), warehouse);
52+
final Options options = Options.fromMap(optionsMap);
53+
final Configuration hadoopConf = new Configuration();
54+
String hdfsSitePathOptional = config.getHdfsSitePath();
55+
if (StringUtils.isNotBlank(hdfsSitePathOptional)) {
56+
hadoopConf.addResource(new Path(hdfsSitePathOptional));
57+
}
58+
final CatalogContext catalogContext = CatalogContext.create(options, hadoopConf);
59+
return CatalogFactory.createCatalog(catalogContext);
60+
}
61+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,28 @@
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.catalog;
19+
20+
import org.apache.seatunnel.api.table.catalog.TablePath;
21+
import org.apache.seatunnel.api.table.catalog.exception.CatalogException;
22+
import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException;
23+
24+
import org.apache.paimon.table.Table;
25+
26+
public interface PaimonTable {
27+
Table getPaimonTable(TablePath tablePath) throws CatalogException, TableNotExistException;
28+
}

seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java

+8-1
Original file line numberDiff line numberDiff line change
@@ -22,20 +22,27 @@
2222
import org.apache.seatunnel.api.sink.SeaTunnelSink;
2323
import org.apache.seatunnel.api.source.SeaTunnelSource;
2424

25+
import java.io.Serializable;
2526
import java.util.List;
2627

2728
/**
2829
* Utility class to store configuration options, used by {@link SeaTunnelSource} and {@link
2930
* SeaTunnelSink}.
3031
*/
31-
public class PaimonConfig {
32+
public class PaimonConfig implements Serializable {
3233

3334
public static final Option<String> WAREHOUSE =
3435
Options.key("warehouse")
3536
.stringType()
3637
.noDefaultValue()
3738
.withDescription("The warehouse path of paimon");
3839

40+
public static final Option<String> CATALOG_NAME =
41+
Options.key("catalog_name")
42+
.stringType()
43+
.defaultValue("paimon")
44+
.withDescription(" the iceberg catalog name");
45+
3946
public static final Option<String> DATABASE =
4047
Options.key("database")
4148
.stringType()

0 commit comments

Comments
 (0)