From d2708904a46b1d37c7ee949f97b52a83befc496d Mon Sep 17 00:00:00 2001 From: iture123 Date: Sun, 28 Aug 2022 23:30:34 +0800 Subject: [PATCH 01/44] [Feature][Connector-V2] new connecotor of Elasticsearch source(#2553) --- docs/en/connector-v2/source/Elasticsearch.md | 58 +++++++ plugin-mapping.properties | 3 +- .../elasticsearch/client/EsRestClient.java | 154 ++++++++++++++++-- .../config/EsClusterConnectionConfig.java | 28 ++++ .../elasticsearch/config/SinkConfig.java | 6 - .../config/source/SourceConfig.java | 30 ++++ .../source/SourceConfigDeaultConstant.java | 26 +++ .../dto/source/IndexDocsCount.java | 43 +++++ .../dto/source/ScrollResult.java | 43 +++++ .../dto/source/SourceIndexInfo.java | 68 ++++++++ .../exception/GetIndexDocsCountException.java | 31 ++++ .../exception/ScrollRequestException.java | 30 ++++ .../sink/ElasticsearchSinkWriter.java | 20 +-- .../source/ElasticsearchSource.java | 83 ++++++++++ .../source/ElasticsearchSourceReader.java | 115 +++++++++++++ .../source/ElasticsearchSourceSplit.java | 54 ++++++ .../ElasticsearchSourceSplitEnumerator.java | 115 +++++++++++++ .../source/ElasticsearchSourceState.java | 23 +++ 18 files changed, 892 insertions(+), 38 deletions(-) create mode 100644 docs/en/connector-v2/source/Elasticsearch.md create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java create mode 100644 seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java diff --git a/docs/en/connector-v2/source/Elasticsearch.md b/docs/en/connector-v2/source/Elasticsearch.md new file mode 100644 index 00000000000..77d8176e0a8 --- /dev/null +++ b/docs/en/connector-v2/source/Elasticsearch.md @@ -0,0 +1,58 @@ +# Elasticsearch + +## Description + +Read data from `Elasticsearch`. + +:::tip + +Engine Supported + +* supported `ElasticSearch version is >= 2.x and < 8.x` + +::: + +## Options + +| name | type | required | default value | +|-------------|--------| -------- |---------------| +| hosts | array | yes | - | +| username | string | no | | +| password | string | no | | +| index | string | yes | - | +| source | array | yes | - | +| scroll_time | string | no | 1m | +| scroll_size | int | no | 100 | + + + +### hosts [array] +`Elasticsearch` cluster http address, the format is `host:port` , allowing multiple hosts to be specified. Such as `["host1:9200", "host2:9200"]`. + +### username [string] +x-pack username + +### password [string] +x-pack password + +### index [string] +`Elasticsearch` index name, support * fuzzy matching + +### source [array] +The fields of index. +You can get the document id by specifying the field `_id`.If sink _id to other index,you need specify an alias for _id due to the `Elasticsearch` limit. + +### scroll_time [String] +Amount of time `Elasticsearch` will keep the search context alive for scroll requests. + +### scroll_size [int] +Maximum number of hits to be returned with each `Elasticsearch` scroll request. + +## Examples +```bash +Elasticsearch { + hosts = ["localhost:9200"] + index = "seatunnel-*" + source = ["_id","name","age"] +} +``` diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 0efd44f1f65..09b2a2e020b 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -115,7 +115,8 @@ seatunnel.source.OssFile = connector-file-oss seatunnel.source.Pulsar = connector-pulsar seatunnel.source.Hudi = connector-hudi seatunnel.sink.DingTalk = connector-dingtalk -seatunnel.sink.elasticsearch = connector-elasticsearch +seatunnel.source.Elasticsearch = connector-elasticsearch +seatunnel.sink.Elasticsearch = connector-elasticsearch seatunnel.source.IoTDB = connector-iotdb seatunnel.sink.IoTDB = connector-iotdb seatunnel.sink.Neo4j = connector-neo4j diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java index 5a2a3df0995..42e732cee15 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.commons.lang3.StringUtils; import org.apache.http.HttpHost; import org.apache.http.HttpStatus; @@ -26,27 +27,53 @@ import org.apache.http.auth.UsernamePasswordCredentials; import org.apache.http.client.CredentialsProvider; import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.util.Asserts; import org.apache.http.util.EntityUtils; +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.EsClusterConnectionConfig; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.BulkResponse; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.BulkElasticsearchException; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.GetElasticsearchVersionException; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.GetIndexDocsCountException; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ScrollRequestException; +import org.apache.seatunnel.shade.com.typesafe.config.Config; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import java.io.IOException; -import java.util.List; +import java.util.*; public class EsRestClient { - private static EsRestClient esRestClient; - private static RestClient restClient; + private final RestClient restClient; - private EsRestClient() { + private EsRestClient(RestClient restClient) { + this.restClient = restClient; + } + public static EsRestClient createInstance(Config pluginConfig){ + List hosts = pluginConfig.getStringList(EsClusterConnectionConfig.HOSTS); + String username = null; + String password = null; + if (pluginConfig.hasPath(EsClusterConnectionConfig.USERNAME)) { + username = pluginConfig.getString(EsClusterConnectionConfig.USERNAME); + if (pluginConfig.hasPath(EsClusterConnectionConfig.PASSWORD)) { + password = pluginConfig.getString(EsClusterConnectionConfig.PASSWORD); + } + } + return createInstance(hosts, username, password); } + public static EsRestClient createInstance(List hosts, String username, String password) { + RestClientBuilder restClientBuilder = getRestClientBuilder(hosts, username, password); + return new EsRestClient(restClientBuilder.build()); + } + + private static RestClientBuilder getRestClientBuilder(List hosts, String username, String password) { HttpHost[] httpHosts = new HttpHost[hosts.size()]; for (int i = 0; i < hosts.size(); i++) { @@ -67,14 +94,6 @@ private static RestClientBuilder getRestClientBuilder(List hosts, String return builder; } - public static EsRestClient getInstance(List hosts, String username, String password) { - if (restClient == null) { - RestClientBuilder restClientBuilder = getRestClientBuilder(hosts, username, password); - restClient = restClientBuilder.build(); - esRestClient = new EsRestClient(); - } - return esRestClient; - } public BulkResponse bulk(String requestBody) { Request request = new Request("POST", "_bulk"); @@ -102,7 +121,7 @@ public BulkResponse bulk(String requestBody) { /** * @return version.number, example:2.0.0 */ - public static String getClusterVersion() { + public String getClusterVersion() { Request request = new Request("GET", "/"); try { Response response = restClient.performRequest(request); @@ -120,4 +139,113 @@ public void close() throws IOException { restClient.close(); } + /** + * first time to request search documents by scroll + * call /${index}/_search?scroll=${scroll} + * + * @param index index name + * @param source select fields + * @param scrollTime such as:1m + * @param scrollSize fetch documents count in one request + */ + public ScrollResult searchByScroll(String index, List source, String scrollTime, int scrollSize) { + Map param = new HashMap<>(); + Map query = new HashMap<>(); + query.put("match_all", new HashMap()); + param.put("query", query); + param.put("_source", source); + param.put("sort", new String[]{"_doc"}); + param.put("size", scrollSize); + String endpoint = index + "/_search?scroll=" + scrollTime; + ScrollResult scrollResult = getDocsFromScrollRequest(endpoint, JsonUtils.toJsonString(param)); + return scrollResult; + } + + + /** + * scroll to get result + * call _search/scroll + * + * @param scrollId the scroll id of the last request + * @param scrollTime such as:1m + */ + public ScrollResult searchWithScrollId(String scrollId, String scrollTime) { + Map param = new HashMap<>(); + param.put("scroll_id", scrollId); + param.put("scroll", scrollTime); + ScrollResult scrollResult = getDocsFromScrollRequest("_search/scroll", JsonUtils.toJsonString(param)); + return scrollResult; + } + + + private ScrollResult getDocsFromScrollRequest(String endpoint, String requestBody) { + Request request = new Request("POST", endpoint); + request.setJsonEntity(requestBody); + try { + Response response = restClient.performRequest(request); + if (response == null) { + throw new ScrollRequestException("POST " + endpoint + " response null"); + } + if (response.getStatusLine().getStatusCode() == HttpStatus.SC_OK) { + String entity = EntityUtils.toString(response.getEntity()); + ObjectNode responseJson = JsonUtils.parseObject(entity); + + JsonNode shards = responseJson.get("_shards"); + int totalShards = shards.get("total").intValue(); + int successful = shards.get("successful").intValue(); + Asserts.check(totalShards == successful, String.format("POST %s,total shards(%d)!= successful shards(%d)", endpoint, totalShards, successful)); + + ScrollResult scrollResult = getDocsFromScrollResponse(responseJson); + return scrollResult; + } else { + throw new ScrollRequestException(String.format("POST %s response status code=%d,request boy=%s", endpoint, response.getStatusLine().getStatusCode(), requestBody)); + } + } catch (IOException e) { + throw new ScrollRequestException(String.format("POST %s error,request boy=%s", endpoint, requestBody), e); + + } + } + + private ScrollResult getDocsFromScrollResponse(ObjectNode responseJson) { + ScrollResult scrollResult = new ScrollResult(); + String scrollId = responseJson.get("_scroll_id").asText(); + scrollResult.setScrollId(scrollId); + + JsonNode hitsNode = responseJson.get("hits").get("hits"); + List> docs = new ArrayList<>(hitsNode.size()); + scrollResult.setDocs(docs); + + Iterator iter = hitsNode.iterator(); + while (iter.hasNext()) { + Map doc = new HashMap<>(); + JsonNode hitNode = iter.next(); + doc.put("_index", hitNode.get("_index").textValue()); + doc.put("_id", hitNode.get("_id").textValue()); + Map source = JsonUtils.parseObject(hitNode.get("_source").toPrettyString(), Map.class); + doc.putAll(source); + docs.add(doc); + } + return scrollResult; + } + + public List getIndexDocsCount(String index){ + String endpoint = String.format("_cat/indices/%s?h=index,docsCount&format=json",index); + Request request = new Request("GET", endpoint); + try { + Response response = restClient.performRequest(request); + if (response == null) { + + } + if (response.getStatusLine().getStatusCode() == HttpStatus.SC_OK) { + String entity = EntityUtils.toString(response.getEntity()); + List indexDocsCounts = JsonUtils.toList(entity, IndexDocsCount.class); + return indexDocsCounts; + }else{ + throw new GetIndexDocsCountException(String.format("POST %s response status code=%d", endpoint, response.getStatusLine().getStatusCode())); + } + }catch (IOException ex){ + throw new GetIndexDocsCountException(ex); + } + } + } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java new file mode 100644 index 00000000000..fd482db5efd --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java @@ -0,0 +1,28 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.config; + +public class EsClusterConnectionConfig { + + public static final String HOSTS = "hosts"; + + public static final String USERNAME = "username"; + + public static final String PASSWORD = "password"; + +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java index f747fad8583..94ae604f29a 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java @@ -25,12 +25,6 @@ public class SinkConfig { public static final String INDEX_TYPE = "index_type"; - public static final String USERNAME = "username"; - - public static final String PASSWORD = "password"; - - public static final String HOSTS = "hosts"; - public static final String MAX_BATCH_SIZE = "max_batch_size"; public static final String MAX_RETRY_SIZE = "max_retry_size"; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java new file mode 100644 index 00000000000..8040803079b --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java @@ -0,0 +1,30 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.config.source; + +public class SourceConfig { + + public static final String INDEX = "index"; + + public static final String SOURCE = "source"; + + public static final String SCROLL_TIME = "scroll_time"; + + public static final String SCROLL_SIZE = "scroll_size"; + +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java new file mode 100644 index 00000000000..035b556b6ff --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java @@ -0,0 +1,26 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.config.source; + +public class SourceConfigDeaultConstant { + + public static final String SCROLLL_TIME = "1m"; + + public static final int SCROLLL_SIZE = 100; + +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java new file mode 100644 index 00000000000..314c1ed4867 --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java @@ -0,0 +1,43 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.dto.source; + +public class IndexDocsCount { + + private String index; + /** + * index docs count + */ + private Long docsCount; + + public String getIndex() { + return index; + } + + public void setIndex(String index) { + this.index = index; + } + + public Long getDocsCount() { + return docsCount; + } + + public void setDocsCount(Long docsCount) { + this.docsCount = docsCount; + } +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java new file mode 100644 index 00000000000..a9b086e611c --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java @@ -0,0 +1,43 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.dto.source; + +import java.util.List; +import java.util.Map; + +public class ScrollResult { + + private String scrollId; + private List> docs; + + public String getScrollId() { + return scrollId; + } + + public void setScrollId(String scrollId) { + this.scrollId = scrollId; + } + + public List> getDocs() { + return docs; + } + + public void setDocs(List> docs) { + this.docs = docs; + } +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java new file mode 100644 index 00000000000..a84b677e8ab --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java @@ -0,0 +1,68 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.dto.source; + +import java.io.Serializable; +import java.util.List; + +public class SourceIndexInfo implements Serializable { + private String index; + private List source; + private String scrollTime; + private int scrollSize; + + public SourceIndexInfo(String index, List source, String scrollTime, int scrollSize) { + this.index = index; + this.source = source; + this.scrollTime = scrollTime; + this.scrollSize = scrollSize; + } + + public String getIndex() { + return index; + } + + public void setIndex(String index) { + this.index = index; + } + + + public List getSource() { + return source; + } + + public void setSource(List source) { + this.source = source; + } + + public String getScrollTime() { + return scrollTime; + } + + public void setScrollTime(String scrollTime) { + this.scrollTime = scrollTime; + } + + public int getScrollSize() { + return scrollSize; + } + + public void setScrollSize(int scrollSize) { + this.scrollSize = scrollSize; + } +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java new file mode 100644 index 00000000000..a7b0cb455bd --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java @@ -0,0 +1,31 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.exception; + +public class GetIndexDocsCountException extends RuntimeException { + + + public GetIndexDocsCountException(String message) { + super(message); + } + + public GetIndexDocsCountException(Throwable cause) { + super( cause); + } + +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java new file mode 100644 index 00000000000..f0948341790 --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java @@ -0,0 +1,30 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.exception; + +public class ScrollRequestException extends RuntimeException { + + public ScrollRequestException(String message, Throwable cause) { + super(message, cause); + } + + public ScrollRequestException(String message) { + super(message); + } + +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java index 048f48f441d..66467974088 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java @@ -21,7 +21,6 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.SinkConfig; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.BulkConfig; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.BulkResponse; @@ -50,10 +49,8 @@ public class ElasticsearchSinkWriter implements SinkWrit private final List requestEsList; private EsRestClient esRestClient; - private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchSinkWriter.class); - public ElasticsearchSinkWriter( Context context, SeaTunnelRowType seaTunnelRowType, @@ -62,26 +59,13 @@ public ElasticsearchSinkWriter( this.context = context; IndexInfo indexInfo = new IndexInfo(pluginConfig); - initRestClient(pluginConfig); - ElasticsearchVersion elasticsearchVersion = ElasticsearchVersion.get(EsRestClient.getClusterVersion()); + esRestClient = EsRestClient.createInstance(pluginConfig); + ElasticsearchVersion elasticsearchVersion = ElasticsearchVersion.get(esRestClient.getClusterVersion()); this.seaTunnelRowSerializer = new ElasticsearchRowSerializer(elasticsearchVersion, indexInfo, seaTunnelRowType); this.requestEsList = new ArrayList<>(BulkConfig.MAX_BATCH_SIZE); } - private void initRestClient(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig) { - List hosts = pluginConfig.getStringList(SinkConfig.HOSTS); - String username = null; - String password = null; - if (pluginConfig.hasPath(SinkConfig.USERNAME)) { - username = pluginConfig.getString(SinkConfig.USERNAME); - if (pluginConfig.hasPath(SinkConfig.PASSWORD)) { - password = pluginConfig.getString(SinkConfig.PASSWORD); - } - } - esRestClient = EsRestClient.getInstance(hosts, username, password); - } - @Override public void write(SeaTunnelRow element) { String indexRequestRow = seaTunnelRowSerializer.serializeRow(element); diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java new file mode 100644 index 00000000000..40dec852601 --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java @@ -0,0 +1,83 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.source; + +import com.google.auto.service.AutoService; +import org.apache.seatunnel.api.common.PrepareFailException; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.type.*; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfig; +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import java.util.List; + +@AutoService(SeaTunnelSource.class) +public class ElasticsearchSource implements SeaTunnelSource { + + + private Config pluginConfig; + + private SeaTunnelRowType rowTypeInfo; + + @Override + public String getPluginName() { + return "Elasticsearch"; + } + + @Override + public void prepare(Config pluginConfig) throws PrepareFailException { + this.pluginConfig = pluginConfig; + List source = pluginConfig.getStringList(SourceConfig.SOURCE); + SeaTunnelDataType[] fieldTypes = new SeaTunnelDataType[source.size()]; + for (int i = 0; i < source.size(); i++) { + fieldTypes[i] = BasicType.STRING_TYPE; + } + rowTypeInfo = new SeaTunnelRowType(source.toArray(new String[source.size()]), fieldTypes); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SeaTunnelDataType getProducedType() { + return this.rowTypeInfo; + } + + @Override + public SourceReader createReader(SourceReader.Context readerContext) throws Exception { + // new ElasticsearchSourceReader(); + return new ElasticsearchSourceReader(readerContext, pluginConfig); + } + + @Override + public SourceSplitEnumerator createEnumerator(SourceSplitEnumerator.Context enumeratorContext) throws Exception { + return new ElasticsearchSourceSplitEnumerator(enumeratorContext, pluginConfig); + } + + @Override + public SourceSplitEnumerator restoreEnumerator(SourceSplitEnumerator.Context enumeratorContext, ElasticsearchSourceState checkpointState) throws Exception { + return new ElasticsearchSourceSplitEnumerator(enumeratorContext, pluginConfig); + } + + +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java new file mode 100644 index 00000000000..f862a7fbffb --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java @@ -0,0 +1,115 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.source; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.*; + +public class ElasticsearchSourceReader implements SourceReader { + + protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSourceReader.class); + + Context context; + + private Config pluginConfig; + + private EsRestClient esRestClient; + + Deque splits = new LinkedList<>(); + boolean noMoreSplit; + + public ElasticsearchSourceReader(Context context, Config pluginConfig) { + this.context = context; + this.pluginConfig = pluginConfig; + } + + @Override + public void open() throws Exception { + esRestClient = EsRestClient.createInstance(this.pluginConfig); + } + + @Override + public void close() throws IOException { + esRestClient.close(); + } + + + @Override + @SuppressWarnings("magicnumber") + public void pollNext(Collector output) throws Exception { + ElasticsearchSourceSplit split = splits.poll(); + if (null != split) { + SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo(); + + ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize()); + outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output); + while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) { + scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime()); + outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output); + } + } else if (noMoreSplit) { + // signal to the source that we have reached the end of the data. + LOG.info("Closed the bounded ELasticsearch source"); + context.signalNoMoreElement(); + } else { + Thread.sleep(1000L); + } + } + + private void outputFromScrollResult(ScrollResult scrollResult, List source, Collector output) { + int sourceSize = source.size(); + for (Map doc : scrollResult.getDocs()) { + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(sourceSize); + for (int i = 0; i < sourceSize; i++) { + Object value = doc.get(source.get(i)); + seaTunnelRow.setField(i, String.valueOf(value)); + } + output.collect(seaTunnelRow); + } + } + + @Override + public List snapshotState(long checkpointId) throws Exception { + return new ArrayList<>(splits); + } + + @Override + public void addSplits(List splits) { + this.splits.addAll(splits); + } + + @Override + public void handleNoMoreSplits() { + noMoreSplit = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + + } +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java new file mode 100644 index 00000000000..f9c0303447d --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java @@ -0,0 +1,54 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.source; + +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; + + +public class ElasticsearchSourceSplit implements SourceSplit { + + private SourceIndexInfo sourceIndexInfo; + private int splitId; + + public ElasticsearchSourceSplit(SourceIndexInfo sourceIndexInfo, int splitId) { + this.sourceIndexInfo = sourceIndexInfo; + this.splitId = splitId; + } + + @Override + public String splitId() { + return String.valueOf(splitId); + } + + public SourceIndexInfo getSourceIndexInfo() { + return sourceIndexInfo; + } + + public void setSourceIndexInfo(SourceIndexInfo sourceIndexInfo) { + this.sourceIndexInfo = sourceIndexInfo; + } + + public int getSplitId() { + return splitId; + } + + public void setSplitId(int splitId) { + this.splitId = splitId; + } +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java new file mode 100644 index 00000000000..17ebc8cd267 --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java @@ -0,0 +1,115 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.source; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfigDeaultConstant; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +public class ElasticsearchSourceSplitEnumerator implements SourceSplitEnumerator { + + private SourceSplitEnumerator.Context enumeratorContext; + + private Config pluginConfig; + + private EsRestClient esRestClient; + + public ElasticsearchSourceSplitEnumerator(Context enumeratorContext, Config pluginConfig) { + this.enumeratorContext = enumeratorContext; + this.pluginConfig = pluginConfig; + } + + @Override + public void open() { + esRestClient = EsRestClient.createInstance(pluginConfig); + } + + @Override + public void run() throws Exception { + + } + + @Override + public void close() throws IOException { + esRestClient.close(); + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + + } + + @Override + public int currentUnassignedSplitSize() { + return 0; + } + + @Override + public void handleSplitRequest(int subtaskId) { + + } + + @Override + public void registerReader(int subtaskId) { + String scrolllTime = SourceConfigDeaultConstant.SCROLLL_TIME; + if (pluginConfig.hasPath(SourceConfig.SCROLL_TIME)) { + scrolllTime = pluginConfig.getString(SourceConfig.SCROLL_TIME); + } + int scrollSize = SourceConfigDeaultConstant.SCROLLL_SIZE; + if (pluginConfig.hasPath(SourceConfig.SCROLL_SIZE)) { + scrollSize = pluginConfig.getInt(SourceConfig.SCROLL_SIZE); + } + + List indexDocsCounts = esRestClient.getIndexDocsCount(pluginConfig.getString(SourceConfig.INDEX)); + indexDocsCounts = indexDocsCounts.stream().filter(x -> x.getDocsCount() != null && x.getDocsCount() > 0) + .sorted(Comparator.comparingLong(IndexDocsCount::getDocsCount)).collect(Collectors.toList()); + List splits = new ArrayList<>(); + int parallelism = enumeratorContext.currentParallelism(); + List sources = pluginConfig.getStringList(SourceConfig.SOURCE); + + for (int i = 0; i < indexDocsCounts.size(); i++) { + IndexDocsCount indexDocsCount = indexDocsCounts.get(i); + if (i % parallelism == subtaskId) { + splits.add(new ElasticsearchSourceSplit(new SourceIndexInfo(indexDocsCount.getIndex(), sources, scrolllTime, scrollSize), subtaskId)); + } + } + + enumeratorContext.assignSplit(subtaskId, splits); + enumeratorContext.signalNoMoreSplits(subtaskId); + } + + @Override + public ElasticsearchSourceState snapshotState(long checkpointId) throws Exception { + return null; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + + } +} diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java new file mode 100644 index 00000000000..0bb3b12d00f --- /dev/null +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java @@ -0,0 +1,23 @@ +/* + * 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.seatunnel.connectors.seatunnel.elasticsearch.source; + +import java.io.Serializable; + +public class ElasticsearchSourceState implements Serializable { +} From ddca509059587b18934d236e710f551c82d5d6e0 Mon Sep 17 00:00:00 2001 From: iture123 Date: Fri, 2 Sep 2022 23:30:45 +0800 Subject: [PATCH 02/44] [Feature][Connector-V2] add Elasticsearch e2e test(#2553) --- .../seatunnel-flink-connector-v2-e2e/pom.xml | 7 ++ .../ElasticsearchSourceToConsoleIT.java | 68 +++++++++++++++++++ .../FakeSourceToElasticsearchIT.java | 46 +++++++++++++ .../elasticsearch_to_console.conf | 57 ++++++++++++++++ .../fakesource_to_elasticsearch.conf | 62 +++++++++++++++++ 5 files changed, 240 insertions(+) create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml index 36df6d0d052..63ac585a0bc 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml @@ -53,6 +53,13 @@ org.awaitility awaitility + + org.testcontainers + elasticsearch + 1.17.3 + test + + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java new file mode 100644 index 00000000000..178775e6ec7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java @@ -0,0 +1,68 @@ +/* + * 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.seatunnel.e2e.flink.v2.elasticsearch; + +import org.apache.seatunnel.e2e.flink.FlinkContainer; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; + +public class ElasticsearchSourceToConsoleIT extends FlinkContainer { + + private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchSourceToConsoleIT.class); + + private ElasticsearchContainer container; + + @SuppressWarnings({"checkstyle:MagicNumber", "checkstyle:Indentation"}) + @BeforeEach + public void startElasticsearchContainer() throws InterruptedException{ + container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch")) + .withNetwork(NETWORK) + .withNetworkAliases("elasticsearch") + .withLogConsumer(new Slf4jLogConsumer(LOGGER)); + container.start(); + LOGGER.info("Elasticsearch container started"); + Thread.sleep(5000L); + + } + + @Test + public void testFakeSourceToConsoleSink() throws IOException, InterruptedException { + Container.ExecResult sinkEsResult = executeSeaTunnelFlinkJob("/elasticsearch/fakesource_to_elasticsearch.conf"); + Assertions.assertEquals(0, sinkEsResult.getExitCode()); + Container.ExecResult sourceEsResult = executeSeaTunnelFlinkJob("/elasticsearch/elasticsearch_to_console.conf"); + Assertions.assertEquals(0, sourceEsResult.getExitCode()); + } + + @AfterEach + public void closeContainer() { + if (container != null) { + container.stop(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java new file mode 100644 index 00000000000..e4d08299b94 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java @@ -0,0 +1,46 @@ +package org.apache.seatunnel.e2e.flink.v2.elasticsearch; + +import org.apache.seatunnel.e2e.flink.FlinkContainer; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; + +public class FakeSourceToElasticsearchIT extends FlinkContainer { + private static final Logger LOGGER = LoggerFactory.getLogger(FakeSourceToElasticsearchIT.class); + + private ElasticsearchContainer container; + + @SuppressWarnings({"checkstyle:MagicNumber", "checkstyle:Indentation"}) + @BeforeEach + public void startElasticsearchContainer() throws InterruptedException { + container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch")).withNetwork(NETWORK).withNetworkAliases("elasticsearch").withLogConsumer(new Slf4jLogConsumer(LOGGER)); + container.start(); + LOGGER.info("Elasticsearch container started"); + Thread.sleep(5000L); + + } + + @Test + public void testFakeSourceToConsoleSink() throws IOException, InterruptedException { + Container.ExecResult execResult = executeSeaTunnelFlinkJob("/elasticsearch/fakesource_to_elasticsearch.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + container.stop(); + } + + @AfterEach + public void closeContainer() { + if (container != null) { + container.stop(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf new file mode 100644 index 00000000000..1cf455791b9 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf @@ -0,0 +1,57 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" + #execution.checkpoint.interval = 10000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Elasticsearch { + hosts = ["elasticsearch:9200"] + index = "st_index*" + source = ["_id","name","age"] + result_table_name = "fake" + scroll_size = 100 + scroll_time = "1m" + } + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +transform { + sql { + sql = "select _id as doc_id,name,age from fake" + } + + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/transform/sql +} + +sink { + Console {} + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/category/sink-v2 +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf new file mode 100644 index 00000000000..e8c2b8ac3e8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf @@ -0,0 +1,62 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" + #execution.checkpoint.interval = 10000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake" + schema = { + fields { + name = "string" + age = "int" + } + } + } + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +transform { + sql { + sql = "select name,age from fake" + } + + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/transform/sql +} + +sink { + Elasticsearch{ + hosts = ["elasticsearch:9200"] + index = "st_index" + index_type = "st" + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/category/sink-v2 +} \ No newline at end of file From cef10bedb33a38587452912ad735d58a0b97cf30 Mon Sep 17 00:00:00 2001 From: iture123 Date: Fri, 2 Sep 2022 23:47:48 +0800 Subject: [PATCH 03/44] [Feature][Connector-V2] add license(#2553) --- .../FakeSourceToElasticsearchIT.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java index e4d08299b94..0a9481c8c86 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java @@ -1,3 +1,20 @@ +/* + * 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.seatunnel.e2e.flink.v2.elasticsearch; import org.apache.seatunnel.e2e.flink.FlinkContainer; From 9678fdd2d524623f7a4267c219611b03721c4cdd Mon Sep 17 00:00:00 2001 From: iture123 Date: Sat, 3 Sep 2022 22:41:29 +0800 Subject: [PATCH 04/44] [Feature][Connector-V2]move java files from src/java to src/main/java(#2553) --- .../connectors/seatunnel/elasticsearch/client/EsRestClient.java | 0 .../seatunnel/elasticsearch/config/EsClusterConnectionConfig.java | 0 .../connectors/seatunnel/elasticsearch/config/SinkConfig.java | 0 .../seatunnel/elasticsearch/config/source/SourceConfig.java | 0 .../elasticsearch/config/source/SourceConfigDeaultConstant.java | 0 .../connectors/seatunnel/elasticsearch/constant/BulkConfig.java | 0 .../seatunnel/elasticsearch/constant/ElasticsearchVersion.java | 0 .../connectors/seatunnel/elasticsearch/dto/BulkResponse.java | 0 .../connectors/seatunnel/elasticsearch/dto/IndexInfo.java | 0 .../seatunnel/elasticsearch/dto/source/IndexDocsCount.java | 0 .../seatunnel/elasticsearch/dto/source/ScrollResult.java | 0 .../seatunnel/elasticsearch/dto/source/SourceIndexInfo.java | 0 .../elasticsearch/exception/BulkElasticsearchException.java | 0 .../elasticsearch/exception/GetElasticsearchVersionException.java | 0 .../elasticsearch/exception/GetIndexDocsCountException.java | 0 .../seatunnel/elasticsearch/exception/ScrollRequestException.java | 0 .../elasticsearch/serialize/ElasticsearchRowSerializer.java | 0 .../seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java | 0 .../seatunnel/elasticsearch/serialize/index/IndexSerializer.java | 0 .../elasticsearch/serialize/index/IndexSerializerFactory.java | 0 .../serialize/index/impl/FixedValueIndexSerializer.java | 0 .../serialize/index/impl/VariableIndexSerializer.java | 0 .../elasticsearch/serialize/type/IndexTypeSerializer.java | 0 .../elasticsearch/serialize/type/IndexTypeSerializerFactory.java | 0 .../elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java | 0 .../serialize/type/impl/RequiredIndexTypeSerializer.java | 0 .../seatunnel/elasticsearch/sink/ElasticsearchSink.java | 0 .../seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java | 0 .../seatunnel/elasticsearch/source/ElasticsearchSource.java | 0 .../seatunnel/elasticsearch/source/ElasticsearchSourceReader.java | 0 .../seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java | 0 .../elasticsearch/source/ElasticsearchSourceSplitEnumerator.java | 0 .../seatunnel/elasticsearch/source/ElasticsearchSourceState.java | 0 .../elasticsearch/state/ElasticsearchAggregatedCommitInfo.java | 0 .../seatunnel/elasticsearch/state/ElasticsearchCommitInfo.java | 0 .../seatunnel/elasticsearch/state/ElasticsearchSinkState.java | 0 .../connectors/seatunnel/elasticsearch/util/RegexUtils.java | 0 37 files changed, 0 insertions(+), 0 deletions(-) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/ElasticsearchVersion.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/BulkResponse.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/IndexInfo.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/BulkElasticsearchException.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetElasticsearchVersionException.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializer.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializerFactory.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/FixedValueIndexSerializer.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/RequiredIndexTypeSerializer.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchAggregatedCommitInfo.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchCommitInfo.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchSinkState.java (100%) rename seatunnel-connectors-v2/connector-elasticsearch/src/{ => main}/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java (100%) diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/EsClusterConnectionConfig.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfig.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/source/SourceConfigDeaultConstant.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/ElasticsearchVersion.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/ElasticsearchVersion.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/ElasticsearchVersion.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/ElasticsearchVersion.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/BulkResponse.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/BulkResponse.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/BulkResponse.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/BulkResponse.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/IndexInfo.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/IndexInfo.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/IndexInfo.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/IndexInfo.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/IndexDocsCount.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/BulkElasticsearchException.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/BulkElasticsearchException.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/BulkElasticsearchException.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/BulkElasticsearchException.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetElasticsearchVersionException.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetElasticsearchVersionException.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetElasticsearchVersionException.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetElasticsearchVersionException.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/ScrollRequestException.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializer.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializer.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializer.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializerFactory.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializerFactory.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializerFactory.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/IndexSerializerFactory.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/FixedValueIndexSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/FixedValueIndexSerializer.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/FixedValueIndexSerializer.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/FixedValueIndexSerializer.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/RequiredIndexTypeSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/RequiredIndexTypeSerializer.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/RequiredIndexTypeSerializer.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/RequiredIndexTypeSerializer.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceState.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchAggregatedCommitInfo.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchAggregatedCommitInfo.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchAggregatedCommitInfo.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchCommitInfo.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchCommitInfo.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchCommitInfo.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchCommitInfo.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchSinkState.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchSinkState.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchSinkState.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/state/ElasticsearchSinkState.java diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java similarity index 100% rename from seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java rename to seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java From 62f54f65c1fa651d3dca806fed2dc3bfe5943a2d Mon Sep 17 00:00:00 2001 From: iture123 Date: Sun, 4 Sep 2022 00:25:11 +0800 Subject: [PATCH 05/44] [Feature][Connector-V2]fix call ElasticsearchContainer#close() duplicatedly(#2553) --- .../e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java index 0a9481c8c86..893c8d22c02 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java @@ -51,7 +51,6 @@ public void startElasticsearchContainer() throws InterruptedException { public void testFakeSourceToConsoleSink() throws IOException, InterruptedException { Container.ExecResult execResult = executeSeaTunnelFlinkJob("/elasticsearch/fakesource_to_elasticsearch.conf"); Assertions.assertEquals(0, execResult.getExitCode()); - container.stop(); } @AfterEach From 0ea81ee136a4233386393ad8336ee45d2ba35e65 Mon Sep 17 00:00:00 2001 From: iture123 Date: Sun, 4 Sep 2022 12:09:01 +0800 Subject: [PATCH 06/44] [Feature][Connector-V2]modify method name(#2553) --- .../flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java | 2 +- .../e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java index 178775e6ec7..9fbe9ee4723 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java @@ -52,7 +52,7 @@ public void startElasticsearchContainer() throws InterruptedException{ } @Test - public void testFakeSourceToConsoleSink() throws IOException, InterruptedException { + public void testElasticsearchSourceToConsoleSink() throws IOException, InterruptedException { Container.ExecResult sinkEsResult = executeSeaTunnelFlinkJob("/elasticsearch/fakesource_to_elasticsearch.conf"); Assertions.assertEquals(0, sinkEsResult.getExitCode()); Container.ExecResult sourceEsResult = executeSeaTunnelFlinkJob("/elasticsearch/elasticsearch_to_console.conf"); diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java index 893c8d22c02..a5d864a02cf 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java @@ -48,7 +48,7 @@ public void startElasticsearchContainer() throws InterruptedException { } @Test - public void testFakeSourceToConsoleSink() throws IOException, InterruptedException { + public void testFakeSourceToELasticsearchSink() throws IOException, InterruptedException { Container.ExecResult execResult = executeSeaTunnelFlinkJob("/elasticsearch/fakesource_to_elasticsearch.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } From 19a118d02706ce591170ac4aafdb3fb0bbcd8665 Mon Sep 17 00:00:00 2001 From: iture123 Date: Sun, 4 Sep 2022 12:15:50 +0800 Subject: [PATCH 07/44] [Feature][Connector-V2]format code for Checkstyle(#2553) --- .../elasticsearch/client/EsRestClient.java | 58 ++++++++++--------- .../elasticsearch/config/SinkConfig.java | 6 +- .../elasticsearch/constant/BulkConfig.java | 2 + .../dto/source/ScrollResult.java | 2 +- .../dto/source/SourceIndexInfo.java | 1 - .../exception/GetIndexDocsCountException.java | 3 +- .../serialize/ElasticsearchRowSerializer.java | 25 ++++---- .../serialize/SeaTunnelRowSerializer.java | 1 - .../index/impl/VariableIndexSerializer.java | 4 +- .../serialize/type/IndexTypeSerializer.java | 1 - .../type/IndexTypeSerializerFactory.java | 6 +- .../type/impl/NotIndexTypeSerializer.java | 1 - .../elasticsearch/sink/ElasticsearchSink.java | 5 +- .../sink/ElasticsearchSinkWriter.java | 9 ++- .../source/ElasticsearchSource.java | 16 +++-- .../source/ElasticsearchSourceReader.java | 18 ++++-- .../source/ElasticsearchSourceSplit.java | 1 - .../ElasticsearchSourceSplitEnumerator.java | 3 +- .../elasticsearch/util/RegexUtils.java | 1 - 19 files changed, 90 insertions(+), 73 deletions(-) diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java index 42e732cee15..e943a4d8af4 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java @@ -17,6 +17,18 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.client; +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.EsClusterConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.BulkResponse; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.BulkElasticsearchException; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.GetElasticsearchVersionException; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.GetIndexDocsCountException; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ScrollRequestException; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -29,33 +41,31 @@ import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.util.Asserts; import org.apache.http.util.EntityUtils; -import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.EsClusterConnectionConfig; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.BulkResponse; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.BulkElasticsearchException; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.GetElasticsearchVersionException; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.GetIndexDocsCountException; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.ScrollRequestException; -import org.apache.seatunnel.shade.com.typesafe.config.Config; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; public class EsRestClient { + private static final int CONNECTION_REQUEST_TIMEOUT = 10 * 1000; + + private static final int SOCKET_TIMEOUT = 5 * 60 * 1000; + private final RestClient restClient; private EsRestClient(RestClient restClient) { this.restClient = restClient; } - public static EsRestClient createInstance(Config pluginConfig){ + public static EsRestClient createInstance(Config pluginConfig) { List hosts = pluginConfig.getStringList(EsClusterConnectionConfig.HOSTS); String username = null; String password = null; @@ -73,7 +83,6 @@ public static EsRestClient createInstance(List hosts, String username, S return new EsRestClient(restClientBuilder.build()); } - private static RestClientBuilder getRestClientBuilder(List hosts, String username, String password) { HttpHost[] httpHosts = new HttpHost[hosts.size()]; for (int i = 0; i < hosts.size(); i++) { @@ -83,8 +92,8 @@ private static RestClientBuilder getRestClientBuilder(List hosts, String RestClientBuilder builder = RestClient.builder(httpHosts) .setRequestConfigCallback(requestConfigBuilder -> requestConfigBuilder - .setConnectionRequestTimeout(10 * 1000) - .setSocketTimeout(5 * 60 * 1000)); + .setConnectionRequestTimeout(CONNECTION_REQUEST_TIMEOUT) + .setSocketTimeout(SOCKET_TIMEOUT)); if (StringUtils.isNotEmpty(username)) { CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); @@ -94,7 +103,6 @@ private static RestClientBuilder getRestClientBuilder(List hosts, String return builder; } - public BulkResponse bulk(String requestBody) { Request request = new Request("POST", "_bulk"); request.setJsonEntity(requestBody); @@ -143,10 +151,10 @@ public void close() throws IOException { * first time to request search documents by scroll * call /${index}/_search?scroll=${scroll} * - * @param index index name - * @param source select fields + * @param index index name + * @param source select fields * @param scrollTime such as:1m - * @param scrollSize fetch documents count in one request + * @param scrollSize fetch documents count in one request */ public ScrollResult searchByScroll(String index, List source, String scrollTime, int scrollSize) { Map param = new HashMap<>(); @@ -161,7 +169,6 @@ public ScrollResult searchByScroll(String index, List source, String scr return scrollResult; } - /** * scroll to get result * call _search/scroll @@ -177,7 +184,6 @@ public ScrollResult searchWithScrollId(String scrollId, String scrollTime) { return scrollResult; } - private ScrollResult getDocsFromScrollRequest(String endpoint, String requestBody) { Request request = new Request("POST", endpoint); request.setJsonEntity(requestBody); @@ -228,22 +234,22 @@ private ScrollResult getDocsFromScrollResponse(ObjectNode responseJson) { return scrollResult; } - public List getIndexDocsCount(String index){ - String endpoint = String.format("_cat/indices/%s?h=index,docsCount&format=json",index); + public List getIndexDocsCount(String index) { + String endpoint = String.format("_cat/indices/%s?h=index,docsCount&format=json", index); Request request = new Request("GET", endpoint); try { Response response = restClient.performRequest(request); if (response == null) { - + throw new GetIndexDocsCountException("POST " + endpoint + " response null"); } if (response.getStatusLine().getStatusCode() == HttpStatus.SC_OK) { String entity = EntityUtils.toString(response.getEntity()); List indexDocsCounts = JsonUtils.toList(entity, IndexDocsCount.class); return indexDocsCounts; - }else{ + } else { throw new GetIndexDocsCountException(String.format("POST %s response status code=%d", endpoint, response.getStatusLine().getStatusCode())); } - }catch (IOException ex){ + } catch (IOException ex) { throw new GetIndexDocsCountException(ex); } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java index 94ae604f29a..89375f1d044 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java @@ -29,11 +29,11 @@ public class SinkConfig { public static final String MAX_RETRY_SIZE = "max_retry_size"; - public static void setValue(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig){ - if(pluginConfig.hasPath(MAX_BATCH_SIZE)){ + public static void setValue(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig) { + if (pluginConfig.hasPath(MAX_BATCH_SIZE)) { BulkConfig.MAX_BATCH_SIZE = pluginConfig.getInt(MAX_BATCH_SIZE); } - if(pluginConfig.hasPath(MAX_RETRY_SIZE)){ + if (pluginConfig.hasPath(MAX_RETRY_SIZE)) { BulkConfig.MAX_RETRY_SIZE = pluginConfig.getInt(MAX_RETRY_SIZE); } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java index b6108dc4790..dba8b8dd1ef 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java @@ -27,11 +27,13 @@ public class BulkConfig { * once bulk es include max document size * {@link SinkConfig#MAX_BATCH_SIZE} */ + @SuppressWarnings("checkstyle:MagicNumber") public static int MAX_BATCH_SIZE = 10; /** * the max retry size of bulk es * {@link SinkConfig#MAX_RETRY_SIZE} */ + @SuppressWarnings("checkstyle:MagicNumber") public static int MAX_RETRY_SIZE = 3; } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java index a9b086e611c..7608f3a24bc 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/ScrollResult.java @@ -23,7 +23,7 @@ public class ScrollResult { private String scrollId; - private List> docs; + private List> docs; public String getScrollId() { return scrollId; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java index a84b677e8ab..0f1d018b5b2 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java @@ -41,7 +41,6 @@ public void setIndex(String index) { this.index = index; } - public List getSource() { return source; } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java index a7b0cb455bd..19925a966f4 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/exception/GetIndexDocsCountException.java @@ -19,13 +19,12 @@ public class GetIndexDocsCountException extends RuntimeException { - public GetIndexDocsCountException(String message) { super(message); } public GetIndexDocsCountException(Throwable cause) { - super( cause); + super(cause); } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java index 06c5581bbec..55547ed0dd6 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java @@ -17,24 +17,25 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.IndexInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.IndexInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.index.IndexSerializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.index.IndexSerializerFactory; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type.IndexTypeSerializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type.IndexTypeSerializerFactory; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + import java.util.HashMap; import java.util.Map; /** - * use in elasticsearch version >= 7.* + * use in elasticsearch version >= 7.* */ -public class ElasticsearchRowSerializer implements SeaTunnelRowSerializer{ +public class ElasticsearchRowSerializer implements SeaTunnelRowSerializer { private final SeaTunnelRowType seaTunnelRowType; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -43,13 +44,13 @@ public class ElasticsearchRowSerializer implements SeaTunnelRowSerializer{ private final IndexTypeSerializer indexTypeSerializer; public ElasticsearchRowSerializer(ElasticsearchVersion elasticsearchVersion, IndexInfo indexInfo, SeaTunnelRowType seaTunnelRowType) { - this.indexTypeSerializer = IndexTypeSerializerFactory.getIndexTypeSerializer(elasticsearchVersion,indexInfo.getType()); - this.indexSerializer = IndexSerializerFactory.getIndexSerializer(indexInfo.getIndex(),seaTunnelRowType); + this.indexTypeSerializer = IndexTypeSerializerFactory.getIndexTypeSerializer(elasticsearchVersion, indexInfo.getType()); + this.indexSerializer = IndexSerializerFactory.getIndexSerializer(indexInfo.getIndex(), seaTunnelRowType); this.seaTunnelRowType = seaTunnelRowType; } @Override - public String serializeRow(SeaTunnelRow row){ + public String serializeRow(SeaTunnelRow row) { String[] fieldNames = seaTunnelRowType.getFieldNames(); Map doc = new HashMap<>(fieldNames.length); Object[] fields = row.getFields(); @@ -59,13 +60,13 @@ public String serializeRow(SeaTunnelRow row){ StringBuilder sb = new StringBuilder(); - Map indexInner = new HashMap<>(); + Map indexInner = new HashMap<>(); String index = indexSerializer.serialize(row); - indexInner.put("_index",index); + indexInner.put("_index", index); indexTypeSerializer.fillType(indexInner); - Map> indexParam = new HashMap<>(); - indexParam.put("index",indexInner); + Map> indexParam = new HashMap<>(); + indexParam.put("index", indexInner); try { sb.append(objectMapper.writeValueAsString(indexParam)); sb.append("\n"); diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java index d1fbae8a4f6..53300f984f1 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize; - import org.apache.seatunnel.api.table.type.SeaTunnelRow; public interface SeaTunnelRowSerializer { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java index 7997632359a..2ddfc35c5ce 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java @@ -33,7 +33,7 @@ public class VariableIndexSerializer implements IndexSerializer { private final String index; private final Map fieldIndexMap; - private final String NULL_DEFAULT = "null"; + private final String nullDefault = "null"; public VariableIndexSerializer(SeaTunnelRowType seaTunnelRowType, String index, List fieldNames) { this.index = index; @@ -61,7 +61,7 @@ public String serialize(SeaTunnelRow row) { private String getValue(int fieldIndex, SeaTunnelRow row) { Object valueObj = row.getField(fieldIndex); if (valueObj == null) { - return NULL_DEFAULT; + return nullDefault; } else { return valueObj.toString(); } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java index 3e528058ed1..7d0a395e015 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type; - import java.util.Map; public interface IndexTypeSerializer { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java index 878257cb631..f70a54dc1c4 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java @@ -17,12 +17,14 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type; +import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion.ES2; +import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion.ES5; +import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion.ES6; + import org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type.impl.NotIndexTypeSerializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type.impl.RequiredIndexTypeSerializer; -import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion.*; - public class IndexTypeSerializerFactory { private static final String DEFAULT_TYPE = "st"; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java index fa5afb5b85c..57bfe8116b7 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java @@ -26,7 +26,6 @@ */ public class NotIndexTypeSerializer implements IndexTypeSerializer { - @Override public void fillType(Map indexInner) { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java index a5eac83acd7..70912b9bb1f 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.sink; -import com.google.auto.service.AutoService; import org.apache.seatunnel.api.common.PrepareFailException; import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.sink.SeaTunnelSink; @@ -30,8 +29,9 @@ import org.apache.seatunnel.connectors.seatunnel.elasticsearch.state.ElasticsearchCommitInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.state.ElasticsearchSinkState; -import java.util.Collections; +import com.google.auto.service.AutoService; +import java.util.Collections; @AutoService(SeaTunnelSink.class) public class ElasticsearchSink implements SeaTunnelSink { @@ -41,7 +41,6 @@ public class ElasticsearchSink implements SeaTunnelSink implements SinkWriter { +public class ElasticsearchSinkWriter implements SinkWriter { - private final Context context; + private final SinkWriter.Context context; private final SeaTunnelRowSerializer seaTunnelRowSerializer; private final List requestEsList; @@ -52,7 +55,7 @@ public class ElasticsearchSinkWriter implements SinkWrit private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchSinkWriter.class); public ElasticsearchSinkWriter( - Context context, + SinkWriter.Context context, SeaTunnelRowType seaTunnelRowType, Config pluginConfig, List elasticsearchStates) { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java index 40dec852601..1f639f83687 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSource.java @@ -17,16 +17,21 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.source; -import com.google.auto.service.AutoService; import org.apache.seatunnel.api.common.PrepareFailException; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceReader; import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.table.type.*; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfig; + import org.apache.seatunnel.shade.com.typesafe.config.Config; +import com.google.auto.service.AutoService; + import java.util.List; @AutoService(SeaTunnelSource.class) @@ -64,13 +69,12 @@ public SeaTunnelDataType getProducedType() { } @Override - public SourceReader createReader(SourceReader.Context readerContext) throws Exception { - // new ElasticsearchSourceReader(); + public SourceReader createReader(SourceReader.Context readerContext) { return new ElasticsearchSourceReader(readerContext, pluginConfig); } @Override - public SourceSplitEnumerator createEnumerator(SourceSplitEnumerator.Context enumeratorContext) throws Exception { + public SourceSplitEnumerator createEnumerator(SourceSplitEnumerator.Context enumeratorContext) { return new ElasticsearchSourceSplitEnumerator(enumeratorContext, pluginConfig); } @@ -79,5 +83,5 @@ public SourceSplitEnumerator return new ElasticsearchSourceSplitEnumerator(enumeratorContext, pluginConfig); } - } + diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java index f862a7fbffb..d720265c97b 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java @@ -23,18 +23,24 @@ import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; + import org.apache.seatunnel.shade.com.typesafe.config.Config; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Deque; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; public class ElasticsearchSourceReader implements SourceReader { protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSourceReader.class); - Context context; + SourceReader.Context context; private Config pluginConfig; @@ -43,7 +49,9 @@ public class ElasticsearchSourceReader implements SourceReader splits = new LinkedList<>(); boolean noMoreSplit; - public ElasticsearchSourceReader(Context context, Config pluginConfig) { + private final long pollNextWaitTime = 1000L; + + public ElasticsearchSourceReader(SourceReader.Context context, Config pluginConfig) { this.context = context; this.pluginConfig = pluginConfig; } @@ -58,9 +66,7 @@ public void close() throws IOException { esRestClient.close(); } - @Override - @SuppressWarnings("magicnumber") public void pollNext(Collector output) throws Exception { ElasticsearchSourceSplit split = splits.poll(); if (null != split) { @@ -77,7 +83,7 @@ public void pollNext(Collector output) throws Exception { LOG.info("Closed the bounded ELasticsearch source"); context.signalNoMoreElement(); } else { - Thread.sleep(1000L); + Thread.sleep(pollNextWaitTime); } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java index f9c0303447d..33626f1ed82 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java @@ -20,7 +20,6 @@ import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; - public class ElasticsearchSourceSplit implements SourceSplit { private SourceIndexInfo sourceIndexInfo; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java index 17ebc8cd267..becef4ff7fd 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfigDeaultConstant; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo; + import org.apache.seatunnel.shade.com.typesafe.config.Config; import java.io.IOException; @@ -39,7 +40,7 @@ public class ElasticsearchSourceSplitEnumerator implements SourceSplitEnumerator private EsRestClient esRestClient; - public ElasticsearchSourceSplitEnumerator(Context enumeratorContext, Config pluginConfig) { + public ElasticsearchSourceSplitEnumerator(SourceSplitEnumerator.Context enumeratorContext, Config pluginConfig) { this.enumeratorContext = enumeratorContext; this.pluginConfig = pluginConfig; } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java index 9ccc413ff03..097eca897ae 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.List; import java.util.regex.Matcher; - import java.util.regex.Pattern; public class RegexUtils { From 28e821f164391f7707827ed2910fee90ee2c4a29 Mon Sep 17 00:00:00 2001 From: iture123 Date: Sun, 4 Sep 2022 14:06:56 +0800 Subject: [PATCH 08/44] [Feature][Connector-V2]remove duplicate testcase,add Assert sink(#2553) --- .../ElasticsearchSourceToConsoleIT.java | 29 +++++++++++++++---- .../elasticsearch_to_console.conf | 14 ++++++++- 2 files changed, 37 insertions(+), 6 deletions(-) diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java index 9fbe9ee4723..10e0e8c0a2b 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java @@ -17,8 +17,10 @@ package org.apache.seatunnel.e2e.flink.v2.elasticsearch; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; import org.apache.seatunnel.e2e.flink.FlinkContainer; +import com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -40,7 +42,7 @@ public class ElasticsearchSourceToConsoleIT extends FlinkContainer { @SuppressWarnings({"checkstyle:MagicNumber", "checkstyle:Indentation"}) @BeforeEach - public void startElasticsearchContainer() throws InterruptedException{ + public void startElasticsearchContainer() throws InterruptedException { container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch")) .withNetwork(NETWORK) .withNetworkAliases("elasticsearch") @@ -48,15 +50,32 @@ public void startElasticsearchContainer() throws InterruptedException{ container.start(); LOGGER.info("Elasticsearch container started"); Thread.sleep(5000L); + createIndexDocs(); + } + /** + * create a index,and bulk some documents + */ + private void createIndexDocs() { + EsRestClient esRestClient = EsRestClient.createInstance(Lists.newArrayList(container.getHttpHostAddress()), "", ""); + String requestBody = "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" + + "{\"name\":\"EbvYoFkXtS\",\"age\":18}\n" + + "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" + + "{\"name\":\"LjFMprGLJZ\",\"age\":19}\n" + + "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" + + "{\"name\":\"uJTtAVuSyI\",\"age\":20}\n"; + esRestClient.bulk(requestBody); + try { + esRestClient.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Test public void testElasticsearchSourceToConsoleSink() throws IOException, InterruptedException { - Container.ExecResult sinkEsResult = executeSeaTunnelFlinkJob("/elasticsearch/fakesource_to_elasticsearch.conf"); - Assertions.assertEquals(0, sinkEsResult.getExitCode()); - Container.ExecResult sourceEsResult = executeSeaTunnelFlinkJob("/elasticsearch/elasticsearch_to_console.conf"); - Assertions.assertEquals(0, sourceEsResult.getExitCode()); + Container.ExecResult execResult = executeSeaTunnelFlinkJob("/elasticsearch/elasticsearch_to_console.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); } @AfterEach diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf index 1cf455791b9..9a13c8ec135 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf @@ -51,7 +51,19 @@ transform { sink { Console {} - + Assert { + rules = [ + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, # please go to https://seatunnel.apache.org/docs/category/sink-v2 } \ No newline at end of file From cafea1fa031d26448813c6da241eaf1bb596d08c Mon Sep 17 00:00:00 2001 From: iture123 Date: Sun, 4 Sep 2022 14:18:30 +0800 Subject: [PATCH 09/44] [Feature][Connector-V2] add Elasticsearch e2e spark test(#2553) --- .../seatunnel-spark-connector-v2-e2e/pom.xml | 7 ++ .../ElasticsearchSourceToConsoleIT.java | 88 +++++++++++++++++++ .../FakeSourceToElasticsearchIT.java | 66 ++++++++++++++ .../elasticsearch_to_console.conf | 69 +++++++++++++++ .../fakesource_to_elasticsearch.conf | 62 +++++++++++++ 5 files changed, 292 insertions(+) create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml index 4da27bcf05a..0f88787cd2a 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml @@ -42,6 +42,13 @@ org.awaitility awaitility + + + org.testcontainers + elasticsearch + 1.17.3 + test + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java new file mode 100644 index 00000000000..ae541039e4c --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java @@ -0,0 +1,88 @@ +/* + * 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.seatunnel.e2e.spark.v2.elasticsearch; + +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; +import org.apache.seatunnel.e2e.spark.SparkContainer; + +import com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; + +/** + * This test case is used to verify that the elasticsearch source is able to send data to the console. + * Make sure the SeaTunnel job can submit successfully on spark engine. + */ +public class ElasticsearchSourceToConsoleIT extends SparkContainer { + + private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchSourceToConsoleIT.class); + + private ElasticsearchContainer container; + + @SuppressWarnings({"checkstyle:MagicNumber", "checkstyle:Indentation"}) + @BeforeEach + public void startElasticsearchContainer() throws InterruptedException { + container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch")).withNetwork(NETWORK).withNetworkAliases("elasticsearch").withLogConsumer(new Slf4jLogConsumer(LOGGER)); + container.start(); + LOGGER.info("Elasticsearch container started"); + Thread.sleep(5000L); + createIndexDocs(); + } + + /** + * create a index,and bulk some documents + */ + private void createIndexDocs() { + EsRestClient esRestClient = EsRestClient.createInstance(Lists.newArrayList(container.getHttpHostAddress()), "", ""); + String requestBody = "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" + + "{\"name\":\"EbvYoFkXtS\",\"age\":18}\n" + + "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" + + "{\"name\":\"LjFMprGLJZ\",\"age\":19}\n" + + "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" + + "{\"name\":\"uJTtAVuSyI\",\"age\":20}\n"; + esRestClient.bulk(requestBody); + try { + esRestClient.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Test + public void testElasticsearchSourceToConsoleSink() throws IOException, InterruptedException { + Container.ExecResult execResult = executeSeaTunnelSparkJob("/elasticsearch/elasticsearch_to_console.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + @AfterEach + public void closeContainer() { + if (container != null) { + container.stop(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java new file mode 100644 index 00000000000..33aebc39441 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java @@ -0,0 +1,66 @@ +/* + * 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.seatunnel.e2e.spark.v2.elasticsearch; + +import org.apache.seatunnel.e2e.spark.SparkContainer; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; + +/** + * This test case is used to verify that the fake source is able to send data to the elasticsearch. + * Make sure the SeaTunnel job can submit successfully on spark engine. + */ +public class FakeSourceToElasticsearchIT extends SparkContainer { + + private static final Logger LOGGER = LoggerFactory.getLogger(FakeSourceToElasticsearchIT.class); + + private ElasticsearchContainer container; + + @SuppressWarnings({"checkstyle:MagicNumber", "checkstyle:Indentation"}) + @BeforeEach + public void startElasticsearchContainer() throws InterruptedException { + container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch")).withNetwork(NETWORK).withNetworkAliases("elasticsearch").withLogConsumer(new Slf4jLogConsumer(LOGGER)); + container.start(); + LOGGER.info("Elasticsearch container started"); + Thread.sleep(5000L); + } + + @Test + public void testFakeSourceToElasticsearchSink() throws IOException, InterruptedException { + Container.ExecResult execResult = executeSeaTunnelSparkJob("/elasticsearch/fakesource_to_elasticsearch.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + @AfterEach + public void closeContainer() { + if (container != null) { + container.stop(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf new file mode 100644 index 00000000000..9a13c8ec135 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf @@ -0,0 +1,69 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" + #execution.checkpoint.interval = 10000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Elasticsearch { + hosts = ["elasticsearch:9200"] + index = "st_index*" + source = ["_id","name","age"] + result_table_name = "fake" + scroll_size = 100 + scroll_time = "1m" + } + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +transform { + sql { + sql = "select _id as doc_id,name,age from fake" + } + + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/transform/sql +} + +sink { + Console {} + Assert { + rules = [ + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/category/sink-v2 +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf new file mode 100644 index 00000000000..e8c2b8ac3e8 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf @@ -0,0 +1,62 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" + #execution.checkpoint.interval = 10000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake" + schema = { + fields { + name = "string" + age = "int" + } + } + } + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource +} + +transform { + sql { + sql = "select name,age from fake" + } + + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/transform/sql +} + +sink { + Elasticsearch{ + hosts = ["elasticsearch:9200"] + index = "st_index" + index_type = "st" + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/category/sink-v2 +} \ No newline at end of file From 36d6ba2fc70d8f445718b3ca1ed033fc7c9da7d2 Mon Sep 17 00:00:00 2001 From: iture123 Date: Sun, 4 Sep 2022 18:43:55 +0800 Subject: [PATCH 10/44] [Feature][Connector-V2]fix Elasticsearch e2e spark conf 'env'(#2553) --- .../elasticsearch/elasticsearch_to_console.conf | 11 +++++++---- .../elasticsearch/fakesource_to_elasticsearch.conf | 11 +++++++---- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf index 9a13c8ec135..174f60110a7 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf @@ -19,11 +19,14 @@ ###### env { - # You can set flink configuration here - execution.parallelism = 1 + # You can set spark configuration here + # see available properties defined by spark: https://spark.apache.org/docs/latest/configuration.html#available-properties job.mode = "BATCH" - #execution.checkpoint.interval = 10000 - #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local } source { diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf index e8c2b8ac3e8..ec706d89a87 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf @@ -19,11 +19,14 @@ ###### env { - # You can set flink configuration here - execution.parallelism = 1 + # You can set spark configuration here + # see available properties defined by spark: https://spark.apache.org/docs/latest/configuration.html#available-properties job.mode = "BATCH" - #execution.checkpoint.interval = 10000 - #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local } source { From a7207fb892e1570dfc3e55a124fbf1937bdb81a4 Mon Sep 17 00:00:00 2001 From: iture123 Date: Wed, 7 Sep 2022 00:09:28 +0800 Subject: [PATCH 11/44] [Feature][Connector-V2]fix JsonNode convert to Map(#2553) --- .../seatunnel/elasticsearch/client/EsRestClient.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java index e943a4d8af4..3cf0807484d 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java @@ -29,6 +29,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -61,6 +62,8 @@ public class EsRestClient { private final RestClient restClient; + private final ObjectMapper mapper = new ObjectMapper(); + private EsRestClient(RestClient restClient) { this.restClient = restClient; } @@ -227,7 +230,7 @@ private ScrollResult getDocsFromScrollResponse(ObjectNode responseJson) { JsonNode hitNode = iter.next(); doc.put("_index", hitNode.get("_index").textValue()); doc.put("_id", hitNode.get("_id").textValue()); - Map source = JsonUtils.parseObject(hitNode.get("_source").toPrettyString(), Map.class); + Map source = mapper.convertValue(hitNode.get("_source"), new TypeReference>(){}); doc.putAll(source); docs.add(doc); } From 93bc8bd1168069e191351037f12e7d5a0dc45a98 Mon Sep 17 00:00:00 2001 From: Zongwen Li Date: Wed, 7 Sep 2022 22:16:50 +0800 Subject: [PATCH 12/44] [Improve][build] Improved scope of maven-shade-plugin (#2665) * [Improve][build] Improved scope of maven-shade-plugin * Excludes log dependencies in maven-shade-plugin * common/base module need skip shading * common/base module need skip shading --- pom.xml | 24 +++++++---------- .../connector-common/pom.xml | 26 +++++++++++++++++++ .../connector-dingtalk/pom.xml | 6 +++++ .../connector-file-base/pom.xml | 26 +++++++++++++++++++ .../connector-http-base/pom.xml | 25 ++++++++++++++++++ seatunnel-connectors-v2/pom.xml | 11 ++++++++ .../seatunnel-connectors-flink-sql/pom.xml | 13 ++++++++++ .../seatunnel-connectors-flink/pom.xml | 14 ++++++++++ .../seatunnel-connectors-spark/pom.xml | 14 ++++++++++ seatunnel-core/pom.xml | 15 +++++++++++ seatunnel-core/seatunnel-core-base/pom.xml | 25 ++++++++++++++++++ seatunnel-core/seatunnel-core-starter/pom.xml | 25 ++++++++++++++++++ 12 files changed, 210 insertions(+), 14 deletions(-) diff --git a/pom.xml b/pom.xml index 4a13fe35392..1b184d5a10e 100644 --- a/pom.xml +++ b/pom.xml @@ -581,6 +581,13 @@ true true + + + org.slf4j:* + ch.qos.logback:* + log4j:* + + *:* @@ -600,9 +607,9 @@ shade - - + + + @@ -807,17 +814,6 @@ scala-maven-plugin - - org.apache.maven.plugins - maven-shade-plugin - - - - - org.codehaus.mojo - flatten-maven-plugin - - org.apache.maven.plugins maven-surefire-plugin diff --git a/seatunnel-connectors-v2/connector-common/pom.xml b/seatunnel-connectors-v2/connector-common/pom.xml index 4ed052e493f..702cdc02591 100644 --- a/seatunnel-connectors-v2/connector-common/pom.xml +++ b/seatunnel-connectors-v2/connector-common/pom.xml @@ -28,4 +28,30 @@ 4.0.0 connector-common + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + + true + + + + + + + org.codehaus.mojo + flatten-maven-plugin + + + \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-dingtalk/pom.xml b/seatunnel-connectors-v2/connector-dingtalk/pom.xml index 31faf57abbe..f6efee3f26f 100644 --- a/seatunnel-connectors-v2/connector-dingtalk/pom.xml +++ b/seatunnel-connectors-v2/connector-dingtalk/pom.xml @@ -41,6 +41,12 @@ com.aliyun alibaba-dingtalk-service-sdk ${dingtalk.service.version} + + + log4j + log4j + + diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/pom.xml b/seatunnel-connectors-v2/connector-file/connector-file-base/pom.xml index e2c4f2f6cdc..dce7647dadb 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/pom.xml +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/pom.xml @@ -118,4 +118,30 @@ + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + + true + + + + + + + org.codehaus.mojo + flatten-maven-plugin + + + \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/pom.xml b/seatunnel-connectors-v2/connector-http/connector-http-base/pom.xml index 510e32de739..ba621534019 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/pom.xml +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/pom.xml @@ -60,4 +60,29 @@ + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + + true + + + + + + + org.codehaus.mojo + flatten-maven-plugin + + + \ No newline at end of file diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index facea7d474d..9f424ff960b 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -85,6 +85,17 @@ true + + + org.apache.maven.plugins + maven-shade-plugin + + + + + org.codehaus.mojo + flatten-maven-plugin + diff --git a/seatunnel-connectors/seatunnel-connectors-flink-sql/pom.xml b/seatunnel-connectors/seatunnel-connectors-flink-sql/pom.xml index 1b4e9bea6f6..e225444a3e4 100644 --- a/seatunnel-connectors/seatunnel-connectors-flink-sql/pom.xml +++ b/seatunnel-connectors/seatunnel-connectors-flink-sql/pom.xml @@ -33,5 +33,18 @@ flink-sql-connector-elasticsearch-7 + + + + org.apache.maven.plugins + maven-shade-plugin + + + + org.codehaus.mojo + flatten-maven-plugin + + + \ No newline at end of file diff --git a/seatunnel-connectors/seatunnel-connectors-flink/pom.xml b/seatunnel-connectors/seatunnel-connectors-flink/pom.xml index fa26056bc8b..02447f7f411 100644 --- a/seatunnel-connectors/seatunnel-connectors-flink/pom.xml +++ b/seatunnel-connectors/seatunnel-connectors-flink/pom.xml @@ -103,4 +103,18 @@ + + + + org.apache.maven.plugins + maven-shade-plugin + + + + + org.codehaus.mojo + flatten-maven-plugin + + + diff --git a/seatunnel-connectors/seatunnel-connectors-spark/pom.xml b/seatunnel-connectors/seatunnel-connectors-spark/pom.xml index ccdafa83e94..9c9eed1f8b3 100644 --- a/seatunnel-connectors/seatunnel-connectors-spark/pom.xml +++ b/seatunnel-connectors/seatunnel-connectors-spark/pom.xml @@ -96,4 +96,18 @@ + + + + org.apache.maven.plugins + maven-shade-plugin + + + + + org.codehaus.mojo + flatten-maven-plugin + + + diff --git a/seatunnel-core/pom.xml b/seatunnel-core/pom.xml index 937fe767b8f..b9587e5f20b 100644 --- a/seatunnel-core/pom.xml +++ b/seatunnel-core/pom.xml @@ -39,4 +39,19 @@ seatunnel-flink-starter seatunnel-spark-starter + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + + org.codehaus.mojo + flatten-maven-plugin + + + diff --git a/seatunnel-core/seatunnel-core-base/pom.xml b/seatunnel-core/seatunnel-core-base/pom.xml index a83c2fb10f6..905fce17ac1 100644 --- a/seatunnel-core/seatunnel-core-base/pom.xml +++ b/seatunnel-core/seatunnel-core-base/pom.xml @@ -62,4 +62,29 @@ + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + + true + + + + + + + org.codehaus.mojo + flatten-maven-plugin + + + diff --git a/seatunnel-core/seatunnel-core-starter/pom.xml b/seatunnel-core/seatunnel-core-starter/pom.xml index 38768724377..775bb9a137f 100644 --- a/seatunnel-core/seatunnel-core-starter/pom.xml +++ b/seatunnel-core/seatunnel-core-starter/pom.xml @@ -61,4 +61,29 @@ + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + + true + + + + + + + org.codehaus.mojo + flatten-maven-plugin + + + From 86df925b82f7a7db00d764412bd32427f3de6e46 Mon Sep 17 00:00:00 2001 From: Kerwin <37063904+zhuangchong@users.noreply.github.com> Date: Thu, 8 Sep 2022 09:48:05 +0800 Subject: [PATCH 13/44] [Chore][Doc] Fix postgresql jdbc download link 404 in documentation (#2674) --- docs/en/connector/flink-sql/Jdbc.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/connector/flink-sql/Jdbc.md b/docs/en/connector/flink-sql/Jdbc.md index 113a9e4ac4c..53486d2883b 100644 --- a/docs/en/connector/flink-sql/Jdbc.md +++ b/docs/en/connector/flink-sql/Jdbc.md @@ -15,7 +15,7 @@ A driver dependency is also required to connect to a specified database. Here ar | Driver | Group Id | Artifact Id | JAR | |------------|-------------------|----------------------|---------------| | MySQL | mysql | mysql-connector-java | [Download](https://repo.maven.apache.org/maven2/mysql/mysql-connector-java/) | -| PostgreSQL | org.postgresql | postgresql | [Download](https://jdbc.postgresql.org/download.html) | +| PostgreSQL | org.postgresql | postgresql | [Download](https://jdbc.postgresql.org/download/) | | Derby | org.apache.derby | derby | [Download](http://db.apache.org/derby/derby_downloads.html) | After downloading the driver jars, you need to place the jars into $FLINK_HOME/lib/. From 32c32ef238fd9c365f78bfa94a396423fc291ed0 Mon Sep 17 00:00:00 2001 From: Laglangyue <35491928+laglangyue@users.noreply.github.com> Date: Thu, 8 Sep 2022 14:21:06 +0800 Subject: [PATCH 14/44] [Bug][e2e] the e2e exec error in win10, fix path separator (#2633) * fix the e2e exec error in win10-os,because line separator. --- .../seatunnel/e2e/spark/SparkContainer.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java index 984ecfe88f6..2f74483f006 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java @@ -94,15 +94,14 @@ public Container.ExecResult executeSeaTunnelSparkJob(String confFile) throws IOE master.copyFileToContainer(MountableFile.forHostPath(confPath), targetConfInContainer); // TODO: use start-seatunnel-spark.sh to run the spark job. Need to modified the SparkStarter can find the seatunnel-core-spark.jar. - // Running IT use cases under Windows requires replacing \ with / - String conf = targetConfInContainer.replaceAll("\\\\", "/"); final List command = new ArrayList<>(); - command.add(Paths.get(SEATUNNEL_HOME, "bin", SEATUNNEL_SPARK_BIN).toString()); + String sparkBinPath = Paths.get(SEATUNNEL_HOME, "bin", SEATUNNEL_SPARK_BIN).toString(); + command.add(adaptPathForWin(sparkBinPath)); command.add("--master"); command.add("local"); command.add("--deploy-mode"); command.add("client"); - command.add("--config " + conf); + command.add("--config " + adaptPathForWin(targetConfInContainer)); Container.ExecResult execResult = master.execInContainer("bash", "-c", String.join(" ", command)); LOG.info(execResult.getStdout()); @@ -119,10 +118,10 @@ protected void copySeaTunnelSparkFile() { master.copyFileToContainer(MountableFile.forHostPath(seatunnelCoreSparkJarPath), SPARK_JAR_PATH); // copy bin - String seatunnelFlinkBinPath = Paths.get(PROJECT_ROOT_PATH.toString(), + String seatunnelSparkBinPath = Paths.get(PROJECT_ROOT_PATH.toString(), "seatunnel-core", "seatunnel-spark-starter", "src", "main", "bin", SEATUNNEL_SPARK_BIN).toString(); master.copyFileToContainer( - MountableFile.forHostPath(seatunnelFlinkBinPath), + MountableFile.forHostPath(seatunnelSparkBinPath), Paths.get(SEATUNNEL_BIN, SEATUNNEL_SPARK_BIN).toString()); // copy connectors @@ -155,4 +154,8 @@ private List getConnectorJarFiles() { f -> f.getName().contains("connector-")))) .collect(Collectors.toList()); } + + private String adaptPathForWin(String path) { + return path == null ? "" : path.replaceAll("\\\\", "/"); + } } From 4c6278a36c9c857446c7fbdb286063122b35627f Mon Sep 17 00:00:00 2001 From: Kirs Date: Thu, 8 Sep 2022 16:14:41 +0800 Subject: [PATCH 15/44] set reviewer count is 2 (#2683) --- .asf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index 5f53dec814e..c5d24103072 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -37,7 +37,7 @@ github: strict: true required_pull_request_reviews: dismiss_stale_reviews: true - required_approving_review_count: 1 + required_approving_review_count: 2 notifications: commits: commits@seatunnel.apache.org From ad43f14f87a898efe12f772437792cda75d1fafa Mon Sep 17 00:00:00 2001 From: liugddx <804167098@qq.com> Date: Thu, 8 Sep 2022 19:58:53 +0800 Subject: [PATCH 16/44] [hotfix][e2e] refactor jupiter,add junit-jupiter-api (#2662) Co-authored-by: CalvinKirs Co-authored-by: Zongwen Li --- pom.xml | 28 +++++++++---------- seatunnel-core/pom.xml | 11 ++++++++ seatunnel-server/pom.xml | 15 ++++++++++ seatunnel-server/seatunnel-app/pom.xml | 1 + .../app/controller/UserControllerTest.java | 2 ++ 5 files changed, 43 insertions(+), 14 deletions(-) diff --git a/pom.xml b/pom.xml index 1b184d5a10e..9beb6a1fe2b 100644 --- a/pom.xml +++ b/pom.xml @@ -264,17 +264,11 @@ - org.junit.jupiter - junit-jupiter-engine + org.junit + junit-bom ${junit.version} - test - - - - org.junit.jupiter - junit-jupiter-params - ${junit.version} - test + pom + import @@ -351,13 +345,17 @@ ${commons.logging.version} - + org.springframework.boot - spring-boot-dependencies + spring-boot-starter-web + ${spring-boot.version} + + + + org.springframework.boot + spring-boot-starter-jetty ${spring-boot.version} - pom - import @@ -461,11 +459,13 @@ org.junit.jupiter junit-jupiter-engine + test org.junit.jupiter junit-jupiter-params + test diff --git a/seatunnel-core/pom.xml b/seatunnel-core/pom.xml index b9587e5f20b..c1992914f71 100644 --- a/seatunnel-core/pom.xml +++ b/seatunnel-core/pom.xml @@ -45,6 +45,17 @@ org.apache.maven.plugins maven-shade-plugin + + false + true + + true + + + ch.qos.logback:* + + + diff --git a/seatunnel-server/pom.xml b/seatunnel-server/pom.xml index 36dc537a0c3..1438fcd5b2d 100644 --- a/seatunnel-server/pom.xml +++ b/seatunnel-server/pom.xml @@ -31,6 +31,21 @@ seatunnel-scheduler seatunnel-server-common + + + + + org.springframework.boot + spring-boot-starter-jetty + ${spring-boot.version} + + + org.springframework.boot + spring-boot-starter-aop + ${spring-boot.version} + + + diff --git a/seatunnel-server/seatunnel-app/pom.xml b/seatunnel-server/seatunnel-app/pom.xml index 16af21ebb7e..d485d9f10a9 100644 --- a/seatunnel-server/seatunnel-app/pom.xml +++ b/seatunnel-server/seatunnel-app/pom.xml @@ -161,6 +161,7 @@ org.springframework.boot spring-boot-starter-test + ${spring-boot.version} test diff --git a/seatunnel-server/seatunnel-app/src/test/java/org/apache/seatunnel/app/controller/UserControllerTest.java b/seatunnel-server/seatunnel-app/src/test/java/org/apache/seatunnel/app/controller/UserControllerTest.java index f2c182c4418..4b1b9116d51 100644 --- a/seatunnel-server/seatunnel-app/src/test/java/org/apache/seatunnel/app/controller/UserControllerTest.java +++ b/seatunnel-server/seatunnel-app/src/test/java/org/apache/seatunnel/app/controller/UserControllerTest.java @@ -30,6 +30,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.springframework.boot.test.mock.mockito.MockBean; @@ -37,6 +38,7 @@ import org.springframework.test.web.servlet.MvcResult; import org.springframework.test.web.servlet.result.MockMvcResultHandlers; +@Disabled("todo:this test is not working, waiting fix") public class UserControllerTest extends WebMvcApplicationTest { @MockBean From e9b00d57142b27586681dc1365e968c496553062 Mon Sep 17 00:00:00 2001 From: Laglangyue <35491928+laglangyue@users.noreply.github.com> Date: Fri, 9 Sep 2022 10:31:25 +0800 Subject: [PATCH 17/44] [Bug] fix flink-start didn't load plugins/${pluginName}/lib/* (#2639) * [Bug] fix flink-start didn't load plugins/${pluginName}/lib/* Co-authored-by: laglangyue --- .../seatunnel/common/config/Common.java | 29 +++++++++++++++++++ .../flink/execution/FlinkExecution.java | 20 +++++++++++++ .../core/starter/spark/SparkStarter.java | 23 +-------------- 3 files changed, 50 insertions(+), 22 deletions(-) diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/config/Common.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/config/Common.java index 67034db0490..294cc4965d5 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/config/Common.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/config/Common.java @@ -17,10 +17,18 @@ package org.apache.seatunnel.common.config; +import static java.nio.file.FileVisitOption.FOLLOW_LINKS; + import java.io.File; +import java.io.IOException; import java.net.URISyntaxException; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class Common { @@ -33,6 +41,8 @@ private Common() { */ public static final int COLLECTION_SIZE = 16; + private static final int PLUGIN_LIB_DIR_DEPTH = 3; + private static DeployMode MODE; private static boolean STARTER = false; @@ -132,4 +142,23 @@ public static Path pluginLibDir(String pluginName) { return Paths.get(pluginDir(pluginName).toString(), "lib"); } + /** + * return plugin's dependent jars, which located in 'plugins/${pluginName}/lib/*'. + */ + public static List getPluginsJarDependencies(){ + Path pluginRootDir = Common.pluginRootDir(); + if (!Files.exists(pluginRootDir) || !Files.isDirectory(pluginRootDir)) { + return Collections.emptyList(); + } + try (Stream stream = Files.walk(pluginRootDir, PLUGIN_LIB_DIR_DEPTH, FOLLOW_LINKS)) { + return stream + .filter(it -> pluginRootDir.relativize(it).getNameCount() == PLUGIN_LIB_DIR_DEPTH) + .filter(it -> it.getParent().endsWith("lib")) + .filter(it -> it.getFileName().toString().endsWith(".jar")) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java index c0cc1a67e38..df452dd855a 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.core.starter.flink.execution; import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.core.starter.config.EngineType; import org.apache.seatunnel.core.starter.config.EnvironmentFactory; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; @@ -31,8 +32,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.MalformedURLException; +import java.net.URL; +import java.nio.file.Path; import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; /** * Used to execute a SeaTunnelTask. @@ -54,6 +59,7 @@ public FlinkExecution(Config config) { this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(flinkEnvironment, config.getConfigList("source")); this.transformPluginExecuteProcessor = new TransformExecuteProcessor(flinkEnvironment, config.getConfigList("transform")); this.sinkPluginExecuteProcessor = new SinkExecuteProcessor(flinkEnvironment, config.getConfigList("sink")); + registerPlugin(); } @Override @@ -70,4 +76,18 @@ public void execute() throws TaskExecuteException { throw new TaskExecuteException("Execute Flink job error", e); } } + + private void registerPlugin(){ + List pluginsJarDependencies = Common.getPluginsJarDependencies().stream() + .map(Path::toUri) + .map(uri -> { + try { + return uri.toURL(); + } catch (MalformedURLException e) { + throw new RuntimeException("the uri of jar illegal:" + uri, e); + } + }) + .collect(Collectors.toList()); + flinkEnvironment.registerPlugin(pluginsJarDependencies); + } } diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java index 19479d0e38d..3aaf02c6714 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java @@ -17,8 +17,6 @@ package org.apache.seatunnel.core.starter.spark; -import static java.nio.file.FileVisitOption.FOLLOW_LINKS; - import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.common.config.DeployMode; @@ -63,8 +61,6 @@ */ public class SparkStarter implements Starter { - private static final int PLUGIN_LIB_DIR_DEPTH = 3; - /** * original commandline args */ @@ -130,7 +126,7 @@ public List buildCommands() throws IOException { setSparkConf(); Common.setDeployMode(commandArgs.getDeployMode()); Common.setStarter(true); - this.jars.addAll(getPluginsJarDependencies()); + this.jars.addAll(Common.getPluginsJarDependencies()); this.jars.addAll(getConnectorJarDependencies()); this.appName = this.sparkConf.getOrDefault("spark.app.name", Constants.LOGO); return buildFinal(); @@ -179,23 +175,6 @@ static Map getSparkConf(String configFile) throws FileNotFoundEx .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().unwrapped().toString())); } - /** - * return plugin's dependent jars, which located in 'plugins/${pluginName}/lib/*'. - */ - private List getPluginsJarDependencies() throws IOException { - Path pluginRootDir = Common.pluginRootDir(); - if (!Files.exists(pluginRootDir) || !Files.isDirectory(pluginRootDir)) { - return Collections.emptyList(); - } - try (Stream stream = Files.walk(pluginRootDir, PLUGIN_LIB_DIR_DEPTH, FOLLOW_LINKS)) { - return stream - .filter(it -> pluginRootDir.relativize(it).getNameCount() == PLUGIN_LIB_DIR_DEPTH) - .filter(it -> it.getParent().endsWith("lib")) - .filter(it -> it.getFileName().endsWith("jar")) - .collect(Collectors.toList()); - } - } - /** * return connector's jars, which located in 'connectors/spark/*'. */ From 7278209ca2d345d436351f055010b9a8864b20e0 Mon Sep 17 00:00:00 2001 From: Laglangyue <35491928+laglangyue@users.noreply.github.com> Date: Fri, 9 Sep 2022 15:24:48 +0800 Subject: [PATCH 18/44] [Connector-V2][JDBC-connector] support Jdbc dm (#2377) * [Connector-V2][JDBC-connector] Add DM source and sink connector --- docs/en/connector-v2/sink/Jdbc.md | 41 ++-- docs/en/connector-v2/source/Jdbc.md | 32 ++- pom.xml | 16 +- .../connector-jdbc/pom.xml | 12 +- .../jdbc/internal/dialect/dm/DmdbDialect.java | 40 ++++ .../dialect/dm/DmdbDialectFactory.java | 40 ++++ .../dialect/dm/DmdbJdbcRowConverter.java | 39 ++++ .../internal/dialect/dm/DmdbTypeMapper.java | 198 ++++++++++++++++++ .../connector-jdbc-flink-e2e/pom.xml | 16 ++ .../e2e/flink/v2/jdbc/JdbcDmdbIT.java | 148 +++++++++++++ .../test/resources/jdbc/init_sql/dm_init.conf | 122 +++++++++++ .../jdbc/jdbc_dm_source_and_sink.conf | 57 +++++ .../connector-jdbc-spark-e2e/pom.xml | 17 ++ .../e2e/spark/v2/jdbc/JdbcDmdbIT.java | 151 +++++++++++++ .../test/resources/jdbc/init_sql/dm_init.conf | 122 +++++++++++ .../jdbc/jdbc_dm_source_and_sink.conf | 57 +++++ .../seatunnel-spark-connector-v2-e2e/pom.xml | 1 + .../pom.xml | 1 + seatunnel-server/seatunnel-app/pom.xml | 1 + 19 files changed, 1066 insertions(+), 45 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialect.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialectFactory.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbJdbcRowConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeMapper.java create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/resources/jdbc/init_sql/dm_init.conf create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/resources/jdbc/jdbc_dm_source_and_sink.conf create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/resources/jdbc/init_sql/dm_init.conf create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/resources/jdbc/jdbc_dm_source_and_sink.conf diff --git a/docs/en/connector-v2/sink/Jdbc.md b/docs/en/connector-v2/sink/Jdbc.md index f8c883cec76..ed6285eae12 100644 --- a/docs/en/connector-v2/sink/Jdbc.md +++ b/docs/en/connector-v2/sink/Jdbc.md @@ -15,21 +15,21 @@ Use `Xa transactions` to ensure `exactly-once`. So only support `exactly-once` f ## Options -| name | type | required | default value | -| --- | --- | --- | --- | -| url | String | Yes | - | -| driver | String | Yes | - | -| user | String | No | - | -| password | String | No | - | -| query | String | Yes | - | -| connection_check_timeout_sec | Int | No | 30 | -| max_retries | Int | No | 3 | -| batch_size | Int | No | 300 | -| batch_interval_ms | Int | No | 1000 | -| is_exactly_once | Boolean | No | false | -| xa_data_source_class_name | String | No | - | -| max_commit_attempts | Int | No | 3 | -| transaction_timeout_sec | Int | No | -1 | +| name | type | required | default value | +|------------------------------|---------|----------|---------------| +| url | String | Yes | - | +| driver | String | Yes | - | +| user | String | No | - | +| password | String | No | - | +| query | String | Yes | - | +| connection_check_timeout_sec | Int | No | 30 | +| max_retries | Int | No | 3 | +| batch_size | Int | No | 300 | +| batch_interval_ms | Int | No | 1000 | +| is_exactly_once | Boolean | No | false | +| xa_data_source_class_name | String | No | - | +| max_commit_attempts | Int | No | 3 | +| transaction_timeout_sec | Int | No | -1 | ### driver [string] The jdbc class name used to connect to the remote data source, if you use MySQL the value is com.mysql.cj.jdbc.Driver. @@ -64,7 +64,7 @@ For batch writing, when the number of buffers reaches the number of `batch_size` Whether to enable exactly-once semantics, which will use Xa transactions. If on, you need to set `xa_data_source_class_name`. ### xa_data_source_class_name[string] -The xa data source class name of the database Driver, for example, mysql is `com.mysql.cj.jdbc.MysqlXADataSource` and postgresql is `org.postgresql.xa.PGXADataSource` +The xa data source class name of the database Driver, for example, mysql is `com.mysql.cj.jdbc.MysqlXADataSource`, and please refer to appendix for other data sources ### max_commit_attempts[int] The number of retries for transaction commit failures @@ -76,6 +76,15 @@ The timeout after the transaction is opened, the default is -1 (never timeout). In the case of is_exactly_once = "true", Xa transactions are used. This requires database support, and some databases require some setup. For example, postgres needs to set `max_prepared_transactions > 1` Such as `ALTER SYSTEM set max_prepared_transactions to 10`. +## appendix +there are some reference value for params above. + +| datasource | driver | url | xa_data_source_class_name | maven | +|------------|--------------------------|-------------------------------------------|-------------------------------------|---------------------------------------------------------------| +| mysql | com.mysql.cj.jdbc.Driver | jdbc:mysql://localhost:3306/test | com.mysql.cj.jdbc.MysqlXADataSource | https://mvnrepository.com/artifact/mysql/mysql-connector-java | +| postgresql | org.postgresql.Driver | jdbc:postgresql://localhost:5432/postgres | org.postgresql.xa.PGXADataSource | https://mvnrepository.com/artifact/org.postgresql/postgresql | | +| dm | dm.jdbc.driver.DmDriver | jdbc:dm://localhost:5236 | dm.jdbc.driver.DmdbXADataSource | https://mvnrepository.com/artifact/com.dameng/DmJdbcDriver18 | + ## Example Simple ``` diff --git a/docs/en/connector-v2/source/Jdbc.md b/docs/en/connector-v2/source/Jdbc.md index 5f1e47ac98e..ca229ee8291 100644 --- a/docs/en/connector-v2/source/Jdbc.md +++ b/docs/en/connector-v2/source/Jdbc.md @@ -20,17 +20,17 @@ supports query SQL and can achieve projection effect. ## Options -| name | type | required | default value | -| --- | --- | --- | --- | -| url | String | Yes | - | -| driver | String | Yes | - | -| user | String | No | - | -| password | String | No | - | -| query | String | Yes | - | -| connection_check_timeout_sec | Int | No | 30 | -| partition_column | String | No | - | -| partition_upper_bound | Long | No | - | -| partition_lower_bound | Long | No | - | +| name | type | required | default value | +|------------------------------|--------|----------|---------------| +| url | String | Yes | - | +| driver | String | Yes | - | +| user | String | No | - | +| password | String | No | - | +| query | String | Yes | - | +| connection_check_timeout_sec | Int | No | 30 | +| partition_column | String | No | - | +| partition_upper_bound | Long | No | - | +| partition_lower_bound | Long | No | - | ### driver [string] The jdbc class name used to connect to the remote data source, if you use MySQL the value is com.mysql.cj.jdbc.Driver. @@ -66,6 +66,16 @@ The partition_column min value for scan, if not set SeaTunnel will query databas ## tips If partition_column is not set, it will run in single concurrency, and if partition_column is set, it will be executed in parallel according to the concurrency of tasks. + +## appendix +there are some reference value for params above. + +| datasource | driver | url | maven | +|------------|--------------------------|-------------------------------------------|---------------------------------------------------------------| +| mysql | com.mysql.cj.jdbc.Driver | jdbc:mysql://localhost:3306/test | https://mvnrepository.com/artifact/mysql/mysql-connector-java | +| postgresql | org.postgresql.Driver | jdbc:postgresql://localhost:5432/postgres | https://mvnrepository.com/artifact/org.postgresql/postgresql | | +| dm | dm.jdbc.driver.DmDriver | jdbc:dm://localhost:5236 | https://mvnrepository.com/artifact/com.dameng/DmJdbcDriver18 | + ## Example simple: ```Jdbc { diff --git a/pom.xml b/pom.xml index 9beb6a1fe2b..1a904df22f1 100644 --- a/pom.xml +++ b/pom.xml @@ -135,6 +135,7 @@ 1.18.0 8.0.16 42.3.3 + 8.1.2.141 false false false @@ -206,21 +207,6 @@ seatunnel-config-shade ${seatunnel.config.shade.version} - - - - mysql - mysql-connector-java - ${mysql.version} - test - - - - org.postgresql - postgresql - ${postgresql.version} - - commons-codec commons-codec diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index eb4e1b37d4c..68b6ad2847b 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -31,8 +31,9 @@ 5.2.5-HBase-2.x - 42.3.3 8.0.16 + 42.3.3 + 8.1.2.141 @@ -47,7 +48,12 @@ org.postgresql postgresql - ${pg.version} + ${postgresql.version} + + + com.dameng + DmJdbcDriver18 + ${dm-jdbc.version} @@ -57,4 +63,4 @@ - \ No newline at end of file + diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialect.java new file mode 100644 index 00000000000..f13bb1d7a28 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialect.java @@ -0,0 +1,40 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dm; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; + +public class DmdbDialect implements JdbcDialect { + + @Override + public String dialectName() { + return "DM"; + } + + @Override + public JdbcRowConverter getRowConverter() { + return new DmdbJdbcRowConverter(); + } + + @Override + public JdbcDialectTypeMapper getJdbcDialectTypeMapper() { + return new DmdbTypeMapper(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialectFactory.java new file mode 100644 index 00000000000..0578bba3c58 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialectFactory.java @@ -0,0 +1,40 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dm; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; + +import com.google.auto.service.AutoService; + +/** + * Factory for {@link DmdbDialect}. + */ +@AutoService(JdbcDialectFactory.class) +public class DmdbDialectFactory implements JdbcDialectFactory { + + @Override + public boolean acceptsURL(String url) { + return url.startsWith("jdbc:dm:"); + } + + @Override + public JdbcDialect create() { + return new DmdbDialect(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbJdbcRowConverter.java new file mode 100644 index 00000000000..6aa666314e4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbJdbcRowConverter.java @@ -0,0 +1,39 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dm; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.AbstractJdbcRowConverter; + +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; + +public class DmdbJdbcRowConverter extends AbstractJdbcRowConverter { + + @Override + public String converterName() { + return "DM"; + } + + @Override + public SeaTunnelRow toInternal(ResultSet rs, ResultSetMetaData metaData, SeaTunnelRowType typeInfo) throws SQLException { + return super.toInternal(rs, metaData, typeInfo); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeMapper.java new file mode 100644 index 00000000000..a5aea557163 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbTypeMapper.java @@ -0,0 +1,198 @@ +/* + * 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.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dm; + +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; + +import java.sql.ResultSetMetaData; +import java.sql.SQLException; + +public class DmdbTypeMapper implements JdbcDialectTypeMapper { + + // ============================data types===================== + private static final String DM_BIT = "BIT"; + + // ----------------------------number------------------------- + private static final String DM_NUMERIC = "NUMERIC"; + private static final String DM_NUMBER = "NUMBER"; + private static final String DM_DECIMAL = "DECIMAL"; + /** + * same to DECIMAL + */ + private static final String DM_DEC = "DEC"; + + // ----------------------------int----------------------------- + private static final String DM_INTEGER = "INTEGER"; + private static final String DM_INT = "INT"; + public static final String DM_PLS_INTEGER = "PLS_INTEGER"; + private static final String DM_BIGINT = "BIGINT"; + private static final String DM_TINYINT = "TINYINT"; + private static final String DM_BYTE = "BYTE"; + private static final String DM_SMALLINT = "SMALLINT"; + + // dm float is double for Cpp. + private static final String DM_FLOAT = "FLOAT"; + private static final String DM_DOUBLE = "DOUBLE"; + private static final String DM_DOUBLE_PRECISION = "DOUBLE PRECISION"; + private static final String DM_REAL = "REAL"; + + // DM_CHAR DM_CHARACTER DM_VARCHAR DM_VARCHAR2 max is 32767 + private static final String DM_CHAR = "CHAR"; + private static final String DM_CHARACTER = "CHARACTER"; + private static final String DM_VARCHAR = "VARCHAR"; + private static final String DM_VARCHAR2 = "VARCHAR2"; + private static final String DM_LONGVARCHAR = "LONGVARCHAR"; + private static final String DM_CLOB = "CLOB"; + private static final String DM_TEXT = "TEXT"; + private static final String DM_LONG = "LONG"; + + // ------------------------------time------------------------- + private static final String DM_DATE = "DATE"; + private static final String DM_TIME = "TIME"; + private static final String DM_TIMESTAMP = "TIMESTAMP"; + private static final String DM_DATETIME = "DATETIME"; + + // ---------------------------binary--------------------------- + private static final String DM_BINARY = "BINARY"; + private static final String DM_VARBINARY = "VARBINARY"; + + // -------------------------time interval----------------------- + private static final String DM_INTERVAL_YEAR_TO_MONTH = "INTERVAL YEAR TO MONTH"; + private static final String DM_INTERVAL_YEAR = "INTERVAL YEAR"; + private static final String DM_INTERVAL_MONTH = "INTERVAL MONTH"; + private static final String DM_INTERVAL_DAY = "INTERVAL DAY"; + private static final String DM_INTERVAL_DAY_TO_HOUR = "INTERVAL DAY TO HOUR"; + private static final String DM_INTERVAL_DAY_TO_MINUTE = "INTERVAL DAY TO MINUTE"; + private static final String DM_INTERVAL_DAY_TO_SECOND = "INTERVAL DAY TO SECOND"; + private static final String DM_INTERVAL_HOUR = "INTERVAL HOUR"; + private static final String DM_INTERVAL_HOUR_TO_MINUTE = "INTERVAL HOUR TO MINUTE"; + private static final String DM_INTERVAL_HOUR_TO_SECOND = "INTERVAL HOUR TO SECOND"; + private static final String DM_INTERVAL_MINUTE = "INTERVAL MINUTE"; + private static final String DM_INTERVAL_MINUTE_TO_SECOND = "INTERVAL MINUTE TO SECOND"; + private static final String DM_INTERVAL_SECOND = "INTERVAL SECOND"; + // time zone + private static final String DM_TIME_WITH_TIME_ZONE = "TIME WITH TIME ZONE"; + private static final String DM_TIMESTAMP_WITH_TIME_ZONE = "TIMESTAMP WITH TIME ZONE"; + private static final String TIMESTAMP_WITH_LOCAL_TIME_ZONE = "TIMESTAMP WITH LOCAL TIME ZONE"; + + // ------------------------------blob------------------------- + public static final String DM_BLOB = "BLOB"; + public static final String DM_BFILE = "BFILE"; + public static final String DM_IMAGE = "IMAGE"; + public static final String DM_LONGVARBINARY = "LONGVARBINARY"; + + @Override + @SuppressWarnings("checkstyle:MagicNumber") + public SeaTunnelDataType mapping(ResultSetMetaData metadata, int colIndex) throws SQLException { + String dmdbType = metadata.getColumnTypeName(colIndex).toUpperCase(); + int precision = metadata.getPrecision(colIndex); + switch (dmdbType) { + case DM_BIT: + return BasicType.BOOLEAN_TYPE; + + case DM_INT: + case DM_INTEGER: + case DM_PLS_INTEGER: + return BasicType.INT_TYPE; + + case DM_TINYINT: + case DM_BYTE: + return BasicType.BYTE_TYPE; + + case DM_SMALLINT: + return BasicType.SHORT_TYPE; + + case DM_BIGINT: + return BasicType.LONG_TYPE; + + case DM_NUMERIC: + case DM_NUMBER: + case DM_DECIMAL: + case DM_DEC: + if (precision > 0) { + return new DecimalType(precision, metadata.getScale(colIndex)); + } + return new DecimalType(38, 18); + + case DM_REAL: + return BasicType.FLOAT_TYPE; + + case DM_FLOAT: + case DM_DOUBLE_PRECISION: + case DM_DOUBLE: + return BasicType.DOUBLE_TYPE; + + case DM_CHAR: + case DM_CHARACTER: + case DM_VARCHAR: + case DM_VARCHAR2: + // 100G-1 byte + case DM_TEXT: + case DM_LONG: + case DM_LONGVARCHAR: + case DM_CLOB: + return BasicType.STRING_TYPE; + + case DM_TIMESTAMP: + case DM_DATETIME: + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + + case DM_TIME: + return LocalTimeType.LOCAL_TIME_TYPE; + + case DM_DATE: + return LocalTimeType.LOCAL_DATE_TYPE; + + // 100G-1 byte + case DM_BLOB: + case DM_BINARY: + case DM_VARBINARY: + case DM_LONGVARBINARY: + case DM_IMAGE: + case DM_BFILE: + return PrimitiveByteArrayType.INSTANCE; + + //Doesn't support yet + case DM_INTERVAL_YEAR_TO_MONTH: + case DM_INTERVAL_YEAR: + case DM_INTERVAL_MONTH: + case DM_INTERVAL_DAY: + case DM_INTERVAL_DAY_TO_HOUR: + case DM_INTERVAL_DAY_TO_MINUTE: + case DM_INTERVAL_DAY_TO_SECOND: + case DM_INTERVAL_HOUR: + case DM_INTERVAL_HOUR_TO_MINUTE: + case DM_INTERVAL_HOUR_TO_SECOND: + case DM_INTERVAL_MINUTE: + case DM_INTERVAL_MINUTE_TO_SECOND: + case DM_INTERVAL_SECOND: + case DM_TIME_WITH_TIME_ZONE: + case DM_TIMESTAMP_WITH_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + default: + final String jdbcColumnName = metadata.getColumnName(colIndex); + throw new UnsupportedOperationException( + String.format("Doesn't support Dmdb type '%s' on column '%s' yet.", dmdbType, jdbcColumnName)); + } + } +} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/pom.xml index e4361c3592e..4b4d1d9981e 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/pom.xml @@ -46,6 +46,22 @@ 1.17.3 test + + mysql + mysql-connector-java + ${mysql.version} + provided + + + org.postgresql + postgresql + ${postgresql.version} + + + com.dameng + DmJdbcDriver18 + ${dm-jdbc.version} + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java new file mode 100644 index 00000000000..b88aab09350 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java @@ -0,0 +1,148 @@ +/* + * 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.seatunnel.e2e.flink.v2.jdbc; + +import static org.testcontainers.shaded.org.awaitility.Awaitility.given; + +import org.apache.seatunnel.e2e.flink.FlinkContainer; + +import com.google.common.collect.Lists; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import lombok.extern.slf4j.Slf4j; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; + +import java.io.File; +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +@Slf4j +public class JdbcDmdbIT extends FlinkContainer { + + private static final String DOCKER_IMAGE = "laglangyue/dmdb8"; + private static final String DRIVER_CLASS = "dm.jdbc.driver.DmDriver"; + private static final String HOST = "flink_e2e_dmdb"; + private static final String LOCAL_HOST = "localhost"; + private static final String URL = "jdbc:dm://" + LOCAL_HOST + ":5236"; + private static final String USERNAME = "SYSDBA"; + private static final String PASSWORD = "SYSDBA"; + private static final String DATABASE = "SYSDBA"; + private static final String SOURCE_TABLE = "e2e_table_source"; + private static final String SINK_TABLE = "e2e_table_sink"; + private Connection jdbcConnection; + private GenericContainer dbServer; + + @BeforeEach + public void startDmdbContainer() throws ClassNotFoundException, SQLException { + dbServer = new GenericContainer<>(DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withLogConsumer(new Slf4jLogConsumer(log)); + dbServer.setPortBindings(Lists.newArrayList( + String.format("%s:%s", 5236, 5236))); + Startables.deepStart(Stream.of(dbServer)).join(); + log.info("Dmdb container started"); + // wait for Dmdb fully start + Class.forName(DRIVER_CLASS); + given().ignoreExceptions() + .await() + .atMost(180, TimeUnit.SECONDS) + .untilAsserted(this::initializeJdbcConnection); + initializeJdbcTable(); + } + + private void initializeJdbcConnection() throws SQLException { + jdbcConnection = DriverManager.getConnection(URL, USERNAME, PASSWORD); + } + + /** + * init the table for DM_SERVER, DDL and DML for source and sink + */ + private void initializeJdbcTable() { + java.net.URL resource = FlinkContainer.class.getResource("/jdbc/init_sql/dm_init.conf"); + if (resource == null) { + throw new IllegalArgumentException("can't find find file"); + } + String file = resource.getFile(); + Config config = ConfigFactory.parseFile(new File(file)); + assert config.hasPath("dm_table_source") && config.hasPath("DML") && config.hasPath("dm_table_sink"); + try (Statement statement = jdbcConnection.createStatement()) { + // source + String sourceTableDDL = config.getString("dm_table_source"); + statement.execute(sourceTableDDL); + String insertSQL = config.getString("DML"); + statement.execute(insertSQL); + // sink + String sinkTableDDL = config.getString("dm_table_sink"); + statement.execute(sinkTableDDL); + } catch (SQLException e) { + throw new RuntimeException("Initializing table failed!", e); + } + } + + private void assertHasData(String table) { + try (Connection connection = DriverManager.getConnection(URL, USERNAME, PASSWORD)) { + Statement statement = connection.createStatement(); + String sql = String.format("select * from %s.%s limit 1", DATABASE, table); + ResultSet source = statement.executeQuery(sql); + Assertions.assertTrue(source.next()); + } catch (SQLException e) { + throw new RuntimeException("test dm server image error", e); + } + } + + @AfterEach + public void closeDmdbContainer() throws SQLException { + if (jdbcConnection != null) { + jdbcConnection.close(); + } + if (dbServer != null) { + dbServer.close(); + } + } + + @Test + @DisplayName("JDBC-DM container can be pull") + public void testDMDBImage() { + assertHasData(SOURCE_TABLE); + } + + @Test + @DisplayName("flink JDBC-DM test") + public void testJdbcDmdbSourceAndSink() throws IOException, InterruptedException, SQLException { + assertHasData(SOURCE_TABLE); + Container.ExecResult execResult = executeSeaTunnelFlinkJob("/jdbc/jdbc_dm_source_and_sink.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + assertHasData(SINK_TABLE); + } + +} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/resources/jdbc/init_sql/dm_init.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/resources/jdbc/init_sql/dm_init.conf new file mode 100644 index 00000000000..056c252a1d1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/resources/jdbc/init_sql/dm_init.conf @@ -0,0 +1,122 @@ +# +# 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. +# + +dm_table_source = """ +create table if not exists "SYSDBA".e2e_table_source +( + DM_BIT BIT, + DM_INT INT, + DM_INTEGER INTEGER, + DM_PLS_INTEGER PLS_INTEGER, + DM_TINYINT TINYINT, + + DM_BYTE BYTE, + DM_SMALLINT SMALLINT, + DM_BIGINT BIGINT, + + DM_NUMERIC NUMERIC, + DM_NUMBER NUMBER, + DM_DECIMAL DECIMAL, + DM_DEC DEC, + + DM_REAL REAL, + DM_FLOAT FLOAT, + DM_DOUBLE_PRECISION DOUBLE PRECISION, + DM_DOUBLE DOUBLE, + + DM_CHAR CHAR, + DM_CHARACTER CHARACTER, + DM_VARCHAR VARCHAR, + DM_VARCHAR2 VARCHAR2, + DM_TEXT TEXT, + DM_LONG LONG, + DM_LONGVARCHAR LONGVARCHAR, + DM_CLOB CLOB, + + DM_TIMESTAMP TIMESTAMP, + DM_DATETIME DATETIME, + DM_TIME TIME, + DM_DATE DATE, + + DM_BLOB BLOB, + DM_BINARY BINARY, + DM_VARBINARY VARBINARY, + DM_LONGVARBINARY LONGVARBINARY, + DM_IMAGE IMAGE, + DM_BFILE BFILE +) +""" + +dm_table_sink = """ +create table if not exists "SYSDBA".e2e_table_sink +( + DM_BIT BIT, + DM_INT INT, + DM_INTEGER INTEGER, + DM_PLS_INTEGER PLS_INTEGER, + DM_TINYINT TINYINT, + + DM_BYTE BYTE, + DM_SMALLINT SMALLINT, + DM_BIGINT BIGINT, + + DM_NUMERIC NUMERIC, + DM_NUMBER NUMBER, + DM_DECIMAL DECIMAL, + DM_DEC DEC, + + DM_REAL REAL, + DM_FLOAT FLOAT, + DM_DOUBLE_PRECISION DOUBLE PRECISION, + DM_DOUBLE DOUBLE, + + DM_CHAR CHAR, + DM_CHARACTER CHARACTER, + DM_VARCHAR VARCHAR, + DM_VARCHAR2 VARCHAR2, + DM_TEXT TEXT, + DM_LONG LONG, + DM_LONGVARCHAR LONGVARCHAR, + DM_CLOB CLOB, + + DM_TIMESTAMP TIMESTAMP, + DM_DATETIME DATETIME, + DM_TIME TIME, + DM_DATE DATE, + + DM_BLOB BLOB, + DM_BINARY BINARY, + DM_VARBINARY VARBINARY, + DM_LONGVARBINARY LONGVARBINARY, + DM_IMAGE IMAGE, + DM_BFILE BFILE +) +""" +// only need for source +DML = """ +INSERT INTO "SYSDBA".e2e_table_source ( +DM_BIT, DM_INT, DM_INTEGER, DM_PLS_INTEGER, DM_TINYINT, DM_BYTE, DM_SMALLINT, DM_BIGINT, +DM_NUMERIC, DM_NUMBER, DM_DECIMAL, DM_DEC, DM_REAL, DM_FLOAT, DM_DOUBLE_PRECISION, DM_DOUBLE, +DM_CHAR, DM_CHARACTER, DM_VARCHAR, DM_VARCHAR2, DM_TEXT, DM_LONG, DM_LONGVARCHAR, DM_CLOB, +DM_TIMESTAMP, DM_DATETIME, DM_TIME, DM_DATE, +DM_BLOB, DM_BINARY, DM_VARBINARY, DM_LONGVARBINARY, DM_IMAGE, DM_BFILE) +VALUES +(0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, '1', + 'a', 'a', 'a', 'a', 'a', 'a', 'a', +'2022-08-13 17:35:59.000000', '2022-08-13 17:36:11.000000', '15:45:00', '2022-08-13', +null, null, null, null, null, null) +""" \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/resources/jdbc/jdbc_dm_source_and_sink.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/resources/jdbc/jdbc_dm_source_and_sink.conf new file mode 100644 index 00000000000..ba2468ccfc5 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/resources/jdbc/jdbc_dm_source_and_sink.conf @@ -0,0 +1,57 @@ +# +# 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. +# + +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" + #execution.checkpoint.interval = 10000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + Jdbc { + url = "jdbc:dm://flink_e2e_dmdb:5236" + driver = "dm.jdbc.driver.DmDriver" + connection_check_timeout_sec = 1000 + user = "SYSDBA" + password = "SYSDBA" + query = """select * from "SYSDBA".e2e_table_source""" + } + +} + +transform { + +} + +sink { + Jdbc { + url = "jdbc:dm://flink_e2e_dmdb:5236" + driver = "dm.jdbc.driver.DmDriver" + connection_check_timeout_sec = 1000 + user = "SYSDBA" + password = "SYSDBA" + query = """ +INSERT INTO SYSDBA.e2e_table_sink (DM_BIT, DM_INT, DM_INTEGER, DM_PLS_INTEGER, DM_TINYINT, DM_BYTE, DM_SMALLINT, DM_BIGINT, DM_NUMERIC, DM_NUMBER, + DM_DECIMAL, DM_DEC, DM_REAL, DM_FLOAT, DM_DOUBLE_PRECISION, DM_DOUBLE, DM_CHAR, DM_CHARACTER, DM_VARCHAR, DM_VARCHAR2, DM_TEXT, DM_LONG, + DM_LONGVARCHAR, DM_CLOB, DM_TIMESTAMP, DM_DATETIME, DM_TIME, DM_DATE, DM_BLOB, DM_BINARY, DM_VARBINARY, DM_LONGVARBINARY, DM_IMAGE, DM_BFILE) +VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?) +""" + } +} + diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/pom.xml index 8d96fb66a38..a5242ef72c4 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/pom.xml @@ -40,6 +40,23 @@ ${project.version} test + + mysql + mysql-connector-java + ${mysql.version} + provided + + + + org.postgresql + postgresql + ${postgresql.version} + + + com.dameng + DmJdbcDriver18 + ${dm-jdbc.version} + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java new file mode 100644 index 00000000000..d1eb5043d9e --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java @@ -0,0 +1,151 @@ +/* + * 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.seatunnel.e2e.spark.v2.jdbc; + +import static org.testcontainers.shaded.org.awaitility.Awaitility.given; + +import org.apache.seatunnel.e2e.spark.SparkContainer; + +import com.google.common.collect.Lists; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; +import lombok.extern.slf4j.Slf4j; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +@Slf4j +public class JdbcDmdbIT extends SparkContainer { + + private static final String DM_DOCKER_IMAGE = "laglangyue/dmdb8"; + private static final String DRIVER_CLASS = "dm.jdbc.driver.DmDriver"; + private static final String HOST = "spark_e2e_dmdb"; + private static final String LOCAL_HOST = "localhost"; + private static final String URL = "jdbc:dm://" + LOCAL_HOST + ":5236"; + private static final String USERNAME = "SYSDBA"; + private static final String PASSWORD = "SYSDBA"; + private static final String DATABASE = "SYSDBA"; + private static final String SOURCE_TABLE = "e2e_table_source"; + private static final String SINK_TABLE = "e2e_table_sink"; + private GenericContainer dbServer; + private Connection jdbcConnection; + + @BeforeEach + public void beforeAllForDM() { + try { + dbServer = new GenericContainer<>(DM_DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withLogConsumer(new Slf4jLogConsumer(log)); + dbServer.setPortBindings(Lists.newArrayList("5236:5236")); + Startables.deepStart(Stream.of(dbServer)).join(); + log.info("dmdb container started"); + Class.forName(DRIVER_CLASS); + given().ignoreExceptions() + .await() + .atMost(180, TimeUnit.SECONDS) + .untilAsserted(this::initializeJdbcConnection); + initializeJdbcTable(); + } catch (Exception ex) { + log.error("dm container init failed", ex); + throw new RuntimeException(ex); + } + } + + @AfterEach + public void closeDmdbContainer() throws SQLException { + if (jdbcConnection != null) { + jdbcConnection.close(); + } + if (dbServer != null) { + dbServer.close(); + } + } + + private void initializeJdbcConnection() throws SQLException { + jdbcConnection = DriverManager.getConnection(URL, USERNAME, PASSWORD); + } + + /** + * init the table for DM_SERVER, DDL and DML for source and sink + */ + private void initializeJdbcTable() { + URL resource = JdbcDmdbIT.class.getResource("/jdbc/init_sql/dm_init.conf"); + if (resource == null) { + throw new IllegalArgumentException("can't find find file"); + } + String file = resource.getFile(); + Config config = ConfigFactory.parseFile(new File(file)); + assert config.hasPath("dm_table_source") && config.hasPath("DML") && config.hasPath("dm_table_sink"); + try (Statement statement = jdbcConnection.createStatement()) { + // source + String sourceTableDDL = config.getString("dm_table_source"); + statement.execute(sourceTableDDL); + String insertSQL = config.getString("DML"); + statement.execute(insertSQL); + // sink + String sinkTableDDL = config.getString("dm_table_sink"); + statement.execute(sinkTableDDL); + } catch (SQLException e) { + throw new RuntimeException("Initializing table failed!", e); + } + } + + private void assertHasData(String table) { + try (Connection connection = DriverManager.getConnection(URL, USERNAME, PASSWORD)) { + Statement statement = connection.createStatement(); + String sql = String.format("select * from %s.%s limit 1", DATABASE, table); + ResultSet source = statement.executeQuery(sql); + Assertions.assertTrue(source.next()); + } catch (SQLException e) { + throw new RuntimeException("test dm server image error", e); + } + } + + @Test + @DisplayName("JDBC-DM container can be pull") + public void testDMDBImage() { + assertHasData(SOURCE_TABLE); + } + + @Test + @DisplayName("spark JDBC-DM test for all type mapper") + public void testDMDBSourceToJdbcSink() throws SQLException, IOException, InterruptedException { + Container.ExecResult execResult = executeSeaTunnelSparkJob("/jdbc/jdbc_dm_source_and_sink.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + assertHasData(SINK_TABLE); + } + +} diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/resources/jdbc/init_sql/dm_init.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/resources/jdbc/init_sql/dm_init.conf new file mode 100644 index 00000000000..056c252a1d1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/resources/jdbc/init_sql/dm_init.conf @@ -0,0 +1,122 @@ +# +# 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. +# + +dm_table_source = """ +create table if not exists "SYSDBA".e2e_table_source +( + DM_BIT BIT, + DM_INT INT, + DM_INTEGER INTEGER, + DM_PLS_INTEGER PLS_INTEGER, + DM_TINYINT TINYINT, + + DM_BYTE BYTE, + DM_SMALLINT SMALLINT, + DM_BIGINT BIGINT, + + DM_NUMERIC NUMERIC, + DM_NUMBER NUMBER, + DM_DECIMAL DECIMAL, + DM_DEC DEC, + + DM_REAL REAL, + DM_FLOAT FLOAT, + DM_DOUBLE_PRECISION DOUBLE PRECISION, + DM_DOUBLE DOUBLE, + + DM_CHAR CHAR, + DM_CHARACTER CHARACTER, + DM_VARCHAR VARCHAR, + DM_VARCHAR2 VARCHAR2, + DM_TEXT TEXT, + DM_LONG LONG, + DM_LONGVARCHAR LONGVARCHAR, + DM_CLOB CLOB, + + DM_TIMESTAMP TIMESTAMP, + DM_DATETIME DATETIME, + DM_TIME TIME, + DM_DATE DATE, + + DM_BLOB BLOB, + DM_BINARY BINARY, + DM_VARBINARY VARBINARY, + DM_LONGVARBINARY LONGVARBINARY, + DM_IMAGE IMAGE, + DM_BFILE BFILE +) +""" + +dm_table_sink = """ +create table if not exists "SYSDBA".e2e_table_sink +( + DM_BIT BIT, + DM_INT INT, + DM_INTEGER INTEGER, + DM_PLS_INTEGER PLS_INTEGER, + DM_TINYINT TINYINT, + + DM_BYTE BYTE, + DM_SMALLINT SMALLINT, + DM_BIGINT BIGINT, + + DM_NUMERIC NUMERIC, + DM_NUMBER NUMBER, + DM_DECIMAL DECIMAL, + DM_DEC DEC, + + DM_REAL REAL, + DM_FLOAT FLOAT, + DM_DOUBLE_PRECISION DOUBLE PRECISION, + DM_DOUBLE DOUBLE, + + DM_CHAR CHAR, + DM_CHARACTER CHARACTER, + DM_VARCHAR VARCHAR, + DM_VARCHAR2 VARCHAR2, + DM_TEXT TEXT, + DM_LONG LONG, + DM_LONGVARCHAR LONGVARCHAR, + DM_CLOB CLOB, + + DM_TIMESTAMP TIMESTAMP, + DM_DATETIME DATETIME, + DM_TIME TIME, + DM_DATE DATE, + + DM_BLOB BLOB, + DM_BINARY BINARY, + DM_VARBINARY VARBINARY, + DM_LONGVARBINARY LONGVARBINARY, + DM_IMAGE IMAGE, + DM_BFILE BFILE +) +""" +// only need for source +DML = """ +INSERT INTO "SYSDBA".e2e_table_source ( +DM_BIT, DM_INT, DM_INTEGER, DM_PLS_INTEGER, DM_TINYINT, DM_BYTE, DM_SMALLINT, DM_BIGINT, +DM_NUMERIC, DM_NUMBER, DM_DECIMAL, DM_DEC, DM_REAL, DM_FLOAT, DM_DOUBLE_PRECISION, DM_DOUBLE, +DM_CHAR, DM_CHARACTER, DM_VARCHAR, DM_VARCHAR2, DM_TEXT, DM_LONG, DM_LONGVARCHAR, DM_CLOB, +DM_TIMESTAMP, DM_DATETIME, DM_TIME, DM_DATE, +DM_BLOB, DM_BINARY, DM_VARBINARY, DM_LONGVARBINARY, DM_IMAGE, DM_BFILE) +VALUES +(0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, '1', + 'a', 'a', 'a', 'a', 'a', 'a', 'a', +'2022-08-13 17:35:59.000000', '2022-08-13 17:36:11.000000', '15:45:00', '2022-08-13', +null, null, null, null, null, null) +""" \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/resources/jdbc/jdbc_dm_source_and_sink.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/resources/jdbc/jdbc_dm_source_and_sink.conf new file mode 100644 index 00000000000..466b5b5c008 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/resources/jdbc/jdbc_dm_source_and_sink.conf @@ -0,0 +1,57 @@ +# +# 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. +# + +env { + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:dm://spark_e2e_dmdb:5236" + driver = "dm.jdbc.driver.DmDriver" + connection_check_timeout_sec = 1000 + user = "SYSDBA" + password = "SYSDBA" + query = """select DM_INT,DM_VARCHAR from "SYSDBA".e2e_table_source""" + partition_column = "DM_INT" + } + +} + +transform { + +} + +sink { + Jdbc { + url = "jdbc:dm://spark_e2e_dmdb:5236" + driver = "dm.jdbc.driver.DmDriver" + connection_check_timeout_sec = 1000 + user = "SYSDBA" + password = "SYSDBA" + query = """ + INSERT INTO "SYSDBA".e2e_table_sink(DM_INT,DM_VARCHAR) + values (?,?) +""" + } +} + diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml index 96d8bbb04d9..a7a084bac14 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml @@ -55,6 +55,7 @@ + org.awaitility awaitility diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/pom.xml index aecd54f826f..6e1b2aec5f9 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/pom.xml @@ -49,6 +49,7 @@ org.postgresql postgresql + ${postgresql.version} test diff --git a/seatunnel-server/seatunnel-app/pom.xml b/seatunnel-server/seatunnel-app/pom.xml index d485d9f10a9..6acf7ead60c 100644 --- a/seatunnel-server/seatunnel-app/pom.xml +++ b/seatunnel-server/seatunnel-app/pom.xml @@ -137,6 +137,7 @@ mysql mysql-connector-java + ${mysql.version} provided From 5039752eacc8702477b77e5b9b0946ad87125f9b Mon Sep 17 00:00:00 2001 From: Zongwen Li Date: Fri, 9 Sep 2022 16:00:48 +0800 Subject: [PATCH 19/44] [Improve][e2e] Container only copy required connector jars (#2675) * [Improve][e2e] flink container only copy required connector jars * rename flink-e2e-common * remove useless imported * [Improve][e2e] flink sql container refactoring * remove useless imported * remove useless * [Improve][e2e] spark container only copy required connector jars * change for code review * Use e2e-common module directly * checkstyle * code format --- .../api/configuration/ReadonlyConfig.java | 12 +- seatunnel-e2e/pom.xml | 8 +- seatunnel-e2e/seatunnel-e2e-common/pom.xml | 55 ++++++ .../e2e/common/AbstractContainer.java | 103 +++++++++++ .../e2e/common/AbstractFlinkContainer.java | 121 +++++++++++++ .../e2e/common/AbstractSparkContainer.java | 87 +++++++++ .../seatunnel/e2e/common/ContainerUtil.java | 165 ++++++++++++++++++ .../connector-flink-e2e-base/pom.xml | 11 +- .../seatunnel/e2e/flink/FlinkContainer.java | 150 +++------------- .../seatunnel-flink-connector-v2-e2e/pom.xml | 2 +- .../pom.xml | 12 +- .../seatunnel/e2e/flink/FlinkContainer.java | 149 ++-------------- .../pom.xml | 11 +- .../e2e/flink/sql/FlinkContainer.java | 129 ++------------ .../flink/sql/fake/DatagenToConsoleIT.java | 5 +- .../connector-spark-e2e-base/pom.xml | 11 +- .../seatunnel/e2e/spark/SparkContainer.java | 143 ++------------- .../seatunnel-spark-connector-v2-e2e/pom.xml | 2 +- .../pom.xml | 11 +- .../seatunnel/e2e/spark/SparkContainer.java | 139 ++------------- 20 files changed, 679 insertions(+), 647 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-e2e-common/pom.xml create mode 100644 seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractContainer.java create mode 100644 seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractFlinkContainer.java create mode 100644 seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractSparkContainer.java create mode 100644 seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/ContainerUtil.java diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/ReadonlyConfig.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/ReadonlyConfig.java index beb9d08a9f0..32e8db2734e 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/ReadonlyConfig.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/ReadonlyConfig.java @@ -66,18 +66,24 @@ public T get(Option option) { return getOptional(option).orElseGet(option::defaultValue); } - @SuppressWarnings("MagicNumber") public Map toMap() { if (confData.isEmpty()) { return Collections.emptyMap(); } + Map result = new HashMap<>(); + toMap(result); + return result; + } + + public void toMap(Map result) { + if (confData.isEmpty()) { + return; + } Map flatteningMap = flatteningMap(confData); - Map result = new HashMap<>((flatteningMap.size() << 2) / 3 + 1); for (Map.Entry entry : flatteningMap.entrySet()) { result.put(entry.getKey(), convertToJsonString(entry.getValue())); } - return result; } @SuppressWarnings("unchecked") diff --git a/seatunnel-e2e/pom.xml b/seatunnel-e2e/pom.xml index c9b0b975a77..c8bb6374e78 100644 --- a/seatunnel-e2e/pom.xml +++ b/seatunnel-e2e/pom.xml @@ -27,15 +27,17 @@ pom - seatunnel-flink-e2e - seatunnel-spark-e2e + seatunnel-e2e-common seatunnel-flink-connector-v2-e2e - seatunnel-spark-connector-v2-e2e + seatunnel-flink-e2e seatunnel-flink-sql-e2e + seatunnel-spark-connector-v2-e2e + seatunnel-spark-e2e 4.13.2 + 2.4 diff --git a/seatunnel-e2e/seatunnel-e2e-common/pom.xml b/seatunnel-e2e/seatunnel-e2e-common/pom.xml new file mode 100644 index 00000000000..3ddc46cb5c1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-e2e-common/pom.xml @@ -0,0 +1,55 @@ + + + + + seatunnel-e2e + org.apache.seatunnel + ${revision} + + 4.0.0 + + seatunnel-e2e-common + + + + org.apache.seatunnel + seatunnel-api + ${project.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven-jar-plugin.version} + + false + + + + + test-jar + + + + + + + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractContainer.java new file mode 100644 index 00000000000..f4fb9d1169f --- /dev/null +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractContainer.java @@ -0,0 +1,103 @@ +/* + * 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.seatunnel.e2e.common; + +import static org.apache.seatunnel.e2e.common.ContainerUtil.PROJECT_ROOT_PATH; +import static org.apache.seatunnel.e2e.common.ContainerUtil.adaptPathForWin; +import static org.apache.seatunnel.e2e.common.ContainerUtil.copyConfigFileToContainer; +import static org.apache.seatunnel.e2e.common.ContainerUtil.copyConnectorJarToContainer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; + +public abstract class AbstractContainer { + protected static final Logger LOG = LoggerFactory.getLogger(AbstractContainer.class); + protected static final String START_ROOT_MODULE_NAME = "seatunnel-core"; + + protected final String startModuleName; + + protected final String startModuleFullPath; + + public AbstractContainer() { + String[] modules = getStartModulePath().split(File.separator); + this.startModuleName = modules[modules.length - 1]; + this.startModuleFullPath = PROJECT_ROOT_PATH + File.separator + + START_ROOT_MODULE_NAME + File.separator + getStartModulePath(); + } + + protected abstract String getDockerImage(); + + protected abstract String getStartModulePath(); + + protected abstract String getStartShellName(); + + protected abstract String getConnectorModulePath(); + + protected abstract String getConnectorType(); + + protected abstract String getConnectorNamePrefix(); + + protected abstract String getSeaTunnelHomeInContainer(); + + protected abstract List getExtraStartShellCommands(); + + protected void copySeaTunnelStarter(GenericContainer container) { + String[] modules = getStartModulePath().split(File.separator); + final String startModuleName = modules[modules.length - 1]; + ContainerUtil.copySeaTunnelStarter(container, + startModuleName, + PROJECT_ROOT_PATH + File.separator + START_ROOT_MODULE_NAME + File.separator + getStartModulePath(), + getSeaTunnelHomeInContainer(), + getStartShellName()); + } + + protected Container.ExecResult executeJob(GenericContainer container, String confFile) throws IOException, InterruptedException { + final String confInContainerPath = copyConfigFileToContainer(container, confFile); + // copy connectors + copyConnectorJarToContainer(container, + confFile, + getConnectorModulePath(), + getConnectorNamePrefix(), + getConnectorType(), + getSeaTunnelHomeInContainer()); + return executeCommand(container, confInContainerPath); + } + + protected Container.ExecResult executeCommand(GenericContainer container, String configPath) throws IOException, InterruptedException { + final List command = new ArrayList<>(); + String binPath = Paths.get(getSeaTunnelHomeInContainer(), "bin", getStartShellName()).toString(); + // base command + command.add(adaptPathForWin(binPath)); + command.add("--config"); + command.add(adaptPathForWin(configPath)); + command.addAll(getExtraStartShellCommands()); + + Container.ExecResult execResult = container.execInContainer("bash", "-c", String.join(" ", command)); + LOG.info(execResult.getStdout()); + LOG.error(execResult.getStderr()); + return execResult; + } +} diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractFlinkContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractFlinkContainer.java new file mode 100644 index 00000000000..1f06648eb91 --- /dev/null +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractFlinkContainer.java @@ -0,0 +1,121 @@ +/* + * 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.seatunnel.e2e.common; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Stream; + +/** + * This class is the base class of FlinkEnvironment test. + * The before method will create a Flink cluster, and after method will close the Flink cluster. + * You can use {@link AbstractFlinkContainer#executeSeaTunnelFlinkJob} to submit a seatunnel config and run a seatunnel job. + */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class AbstractFlinkContainer extends AbstractContainer { + + protected static final Logger LOG = LoggerFactory.getLogger(AbstractFlinkContainer.class); + + protected static final String FLINK_SEATUNNEL_HOME = "/tmp/flink/seatunnel"; + + protected static final Network NETWORK = Network.newNetwork(); + + protected static final List DEFAULT_FLINK_PROPERTIES = Arrays.asList( + "jobmanager.rpc.address: jobmanager", + "taskmanager.numberOfTaskSlots: 10", + "parallelism.default: 4", + "env.java.opts: -Doracle.jdbc.timezoneAsRegion=false"); + + protected static final String DEFAULT_DOCKER_IMAGE = "flink:1.13.6-scala_2.11"; + + protected GenericContainer jobManager; + protected GenericContainer taskManager; + + @Override + protected String getDockerImage() { + return DEFAULT_DOCKER_IMAGE; + } + + @Override + protected String getSeaTunnelHomeInContainer() { + return FLINK_SEATUNNEL_HOME; + } + + @BeforeAll + public void before() { + final String dockerImage = getDockerImage(); + final String properties = String.join("\n", getFlinkProperties()); + jobManager = new GenericContainer<>(dockerImage) + .withCommand("jobmanager") + .withNetwork(NETWORK) + .withNetworkAliases("jobmanager") + .withExposedPorts() + .withEnv("FLINK_PROPERTIES", properties) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + taskManager = + new GenericContainer<>(dockerImage) + .withCommand("taskmanager") + .withNetwork(NETWORK) + .withNetworkAliases("taskmanager") + .withEnv("FLINK_PROPERTIES", properties) + .dependsOn(jobManager) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + Startables.deepStart(Stream.of(jobManager)).join(); + Startables.deepStart(Stream.of(taskManager)).join(); + copySeaTunnelStarter(jobManager); + LOG.info("Flink containers are started."); + } + + protected List getFlinkProperties() { + return DEFAULT_FLINK_PROPERTIES; + } + + @AfterAll + public void close() { + if (taskManager != null) { + taskManager.stop(); + } + if (jobManager != null) { + jobManager.stop(); + } + } + + @Override + protected List getExtraStartShellCommands() { + return Collections.emptyList(); + } + + public Container.ExecResult executeSeaTunnelFlinkJob(String confFile) throws IOException, InterruptedException { + return executeJob(jobManager, confFile); + } +} diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractSparkContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractSparkContainer.java new file mode 100644 index 00000000000..fc16846c1db --- /dev/null +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractSparkContainer.java @@ -0,0 +1,87 @@ +/* + * 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.seatunnel.e2e.common; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Stream; + +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class AbstractSparkContainer extends AbstractContainer { + private static final Logger LOG = LoggerFactory.getLogger(AbstractSparkContainer.class); + + private static final String SPARK_SEATUNNEL_HOME = "/tmp/spark/seatunnel"; + private static final String DEFAULT_DOCKER_IMAGE = "bitnami/spark:2.4.3"; + public static final Network NETWORK = Network.newNetwork(); + + protected GenericContainer master; + + @Override + protected String getDockerImage() { + return DEFAULT_DOCKER_IMAGE; + } + + @Override + protected String getSeaTunnelHomeInContainer() { + return SPARK_SEATUNNEL_HOME; + } + + @BeforeAll + public void before() { + master = new GenericContainer<>(getDockerImage()) + .withNetwork(NETWORK) + .withNetworkAliases("spark-master") + .withExposedPorts() + .withEnv("SPARK_MODE", "master") + .withLogConsumer(new Slf4jLogConsumer(LOG)); + // In most case we can just use standalone mode to execute a spark job, if we want to use cluster mode, we need to + // start a worker. + Startables.deepStart(Stream.of(master)).join(); + copySeaTunnelStarter(master); + LOG.info("Spark container started"); + } + + @AfterAll + public void close() { + if (master != null) { + master.stop(); + } + } + + @Override + protected List getExtraStartShellCommands() { + return Arrays.asList("--master local", + "--deploy-mode client"); + } + + public Container.ExecResult executeSeaTunnelSparkJob(String confFile) throws IOException, InterruptedException { + return executeJob(master, confFile); + } +} diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/ContainerUtil.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/ContainerUtil.java new file mode 100644 index 00000000000..237e30f0911 --- /dev/null +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/ContainerUtil.java @@ -0,0 +1,165 @@ +/* + * 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.seatunnel.e2e.common; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigResolveOptions; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.utility.MountableFile; + +import java.io.File; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; + +public final class ContainerUtil { + + public static final String PLUGIN_MAPPING_FILE = "plugin-mapping.properties"; + + /** + * An error occurs when the user is not a submodule of seatunnel-e2e. + */ + public static final String PROJECT_ROOT_PATH = System.getProperty("user.dir").split("/seatunnel-e2e/")[0]; + + public static void copyConnectorJarToContainer(GenericContainer container, + String confFile, + String connectorsRootPath, + String connectorPrefix, + String connectorType, + String seatunnelHome) { + Config jobConfig = getConfig(getConfigFile(confFile)); + Config connectorsMapping = getConfig(new File(PROJECT_ROOT_PATH + File.separator + PLUGIN_MAPPING_FILE)); + if (!connectorsMapping.hasPath(connectorType) || connectorsMapping.getConfig(connectorType).isEmpty()) { + return; + } + Config connectors = connectorsMapping.getConfig(connectorType); + Set connectorNames = getConnectors(jobConfig, connectors, "source"); + connectorNames.addAll(getConnectors(jobConfig, connectors, "sink")); + File module = new File(PROJECT_ROOT_PATH + File.separator + connectorsRootPath); + + List connectorFiles = getConnectorFiles(module, connectorNames, connectorPrefix); + connectorFiles.forEach(jar -> + container.copyFileToContainer( + MountableFile.forHostPath(jar.getAbsolutePath()), + Paths.get(Paths.get(seatunnelHome, "connectors").toString(), connectorType, jar.getName()).toString())); + } + + public static String copyConfigFileToContainer(GenericContainer container, String confFile) { + final String targetConfInContainer = Paths.get("/tmp", confFile).toString(); + container.copyFileToContainer(MountableFile.forHostPath(getConfigFile(confFile).getAbsolutePath()), targetConfInContainer); + return targetConfInContainer; + } + + public static void copySeaTunnelStarter(GenericContainer container, + String startModuleName, + String startModulePath, + String seatunnelHomeInContainer, + String startShellName) { + final String startJarName = startModuleName + ".jar"; + // copy lib + final String startJarPath = startModulePath + File.separator + "target" + File.separator + startJarName; + container.copyFileToContainer( + MountableFile.forHostPath(startJarPath), + Paths.get(Paths.get(seatunnelHomeInContainer, "lib").toString(), startJarName).toString()); + + // copy bin + final String startBinPath = startModulePath + File.separator + "/src/main/bin/" + startShellName; + container.copyFileToContainer( + MountableFile.forHostPath(startBinPath), + Paths.get(Paths.get(seatunnelHomeInContainer, "bin").toString(), startShellName).toString()); + + // copy plugin-mapping.properties + container.copyFileToContainer( + MountableFile.forHostPath(PROJECT_ROOT_PATH + "/plugin-mapping.properties"), + Paths.get(Paths.get(seatunnelHomeInContainer, "connectors").toString(), PLUGIN_MAPPING_FILE).toString()); + } + + public static String adaptPathForWin(String path) { + // Running IT use cases under Windows requires replacing \ with / + return path == null ? "" : path.replaceAll("\\\\", "/"); + } + + private static List getConnectorFiles(File currentModule, Set connectorNames, String connectorPrefix) { + List connectorFiles = new ArrayList<>(); + for (File file : Objects.requireNonNull(currentModule.listFiles())) { + getConnectorFiles(file, connectorNames, connectorPrefix, connectorFiles); + } + return connectorFiles; + } + + private static void getConnectorFiles(File currentModule, Set connectorNames, String connectorPrefix, List connectors) { + if (currentModule.isFile() || connectorNames.size() == connectors.size()) { + return; + } + if (connectorNames.contains(currentModule.getName())) { + File targetPath = new File(currentModule.getAbsolutePath() + File.separator + "target"); + for (File file : Objects.requireNonNull(targetPath.listFiles())) { + if (file.getName().startsWith(currentModule.getName()) && !file.getName().endsWith("javadoc.jar")) { + connectors.add(file); + return; + } + } + } + + if (currentModule.getName().startsWith(connectorPrefix)) { + for (File file : Objects.requireNonNull(currentModule.listFiles())) { + getConnectorFiles(file, connectorNames, connectorPrefix, connectors); + } + } + } + + private static Set getConnectors(Config jobConfig, Config connectorsMap, String pluginType) { + List connectorConfigList = jobConfig.getConfigList(pluginType); + Map connectors = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); + ReadonlyConfig.fromConfig(connectorsMap.getConfig(pluginType)).toMap(connectors); + return connectorConfigList.stream() + .map(config -> config.getString("plugin_name")) + .filter(connectors::containsKey) + .map(connectors::get) + .collect(Collectors.toSet()); + } + + public static Path getCurrentModulePath() { + return Paths.get(System.getProperty("user.dir")); + } + + private static File getConfigFile(String confFile) { + File file = new File(getCurrentModulePath() + "/src/test/resources" + confFile); + if (file.exists()) { + return file; + } + throw new IllegalArgumentException(confFile + " doesn't exist"); + } + + private static Config getConfig(File file) { + return ConfigFactory + .parseFile(file) + .resolve(ConfigResolveOptions.defaults().setAllowUnresolved(true)) + .resolveWith(ConfigFactory.systemProperties(), ConfigResolveOptions.defaults().setAllowUnresolved(true)); + } +} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/pom.xml index 57a12fa6cca..d2f48c42009 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/pom.xml @@ -25,9 +25,14 @@ connector-flink-e2e-base - - 2.4 - + + + org.apache.seatunnel + seatunnel-e2e-common + ${project.version} + test-jar + + diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java index 4fe612a09c0..6ecaa77cbb9 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java @@ -17,152 +17,42 @@ package org.apache.seatunnel.e2e.flink; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.Container; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.Network; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.MountableFile; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import java.util.stream.Stream; +import org.apache.seatunnel.e2e.common.AbstractFlinkContainer; /** * This class is the base class of FlinkEnvironment test for new seatunnel connector API. * The before method will create a Flink cluster, and after method will close the Flink cluster. * You can use {@link FlinkContainer#executeSeaTunnelFlinkJob} to submit a seatunnel config and run a seatunnel job. */ -public abstract class FlinkContainer { - - private static final Logger LOG = LoggerFactory.getLogger(FlinkContainer.class); - - private static final String FLINK_DOCKER_IMAGE = "tyrantlucifer/flink:1.13.6-scala_2.11_hadoop27"; - protected static final Network NETWORK = Network.newNetwork(); - - protected GenericContainer jobManager; - protected GenericContainer taskManager; - private static final Path PROJECT_ROOT_PATH = Paths.get(System.getProperty("user.dir")).getParent().getParent().getParent(); - private static final String SEATUNNEL_FLINK_BIN = "start-seatunnel-flink-new-connector.sh"; - private static final String SEATUNNEL_FLINK_JAR = "seatunnel-flink-starter.jar"; - private static final String PLUGIN_MAPPING_FILE = "plugin-mapping.properties"; - private static final String SEATUNNEL_HOME = "/tmp/flink/seatunnel"; - private static final String SEATUNNEL_BIN = Paths.get(SEATUNNEL_HOME, "bin").toString(); - private static final String SEATUNNEL_LIB = Paths.get(SEATUNNEL_HOME, "lib").toString(); - private static final String SEATUNNEL_CONNECTORS = Paths.get(SEATUNNEL_HOME, "connectors").toString(); - - private static final int WAIT_FLINK_JOB_SUBMIT = 5000; - - private static final String FLINK_PROPERTIES = String.join( - "\n", - Arrays.asList( - "jobmanager.rpc.address: jobmanager", - "taskmanager.numberOfTaskSlots: 10", - "parallelism.default: 4", - "env.java.opts: -Doracle.jdbc.timezoneAsRegion=false")); - - @BeforeEach - public void before() { - jobManager = new GenericContainer<>(FLINK_DOCKER_IMAGE) - .withCommand("jobmanager") - .withNetwork(NETWORK) - .withNetworkAliases("jobmanager") - .withExposedPorts() - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - .withLogConsumer(new Slf4jLogConsumer(LOG)); +public abstract class FlinkContainer extends AbstractFlinkContainer { - taskManager = - new GenericContainer<>(FLINK_DOCKER_IMAGE) - .withCommand("taskmanager") - .withNetwork(NETWORK) - .withNetworkAliases("taskmanager") - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - .dependsOn(jobManager) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - - Startables.deepStart(Stream.of(jobManager)).join(); - Startables.deepStart(Stream.of(taskManager)).join(); - copySeaTunnelFlinkFile(); - LOG.info("Flink containers are started."); + @Override + protected String getDockerImage() { + return "tyrantlucifer/flink:1.13.6-scala_2.11_hadoop27"; } - @AfterEach - public void close() { - if (taskManager != null) { - taskManager.stop(); - } - if (jobManager != null) { - jobManager.stop(); - } + @Override + protected String getStartModulePath() { + return "seatunnel-flink-starter"; } - public Container.ExecResult executeSeaTunnelFlinkJob(String confFile) throws IOException, InterruptedException { - final String confPath = getResource(confFile); - if (!new File(confPath).exists()) { - throw new IllegalArgumentException(confFile + " doesn't exist"); - } - final String targetConfInContainer = Paths.get("/tmp", confFile).toString(); - jobManager.copyFileToContainer(MountableFile.forHostPath(confPath), targetConfInContainer); - - // Running IT use cases under Windows requires replacing \ with / - String conf = targetConfInContainer.replaceAll("\\\\", "/"); - String binPath = Paths.get(SEATUNNEL_HOME, "bin", SEATUNNEL_FLINK_BIN).toString().replaceAll("\\\\", "/"); - final List command = new ArrayList<>(); - command.add(binPath); - command.add("--config " + conf); - - Container.ExecResult execResult = jobManager.execInContainer("bash", "-c", String.join(" ", command)); - LOG.info(execResult.getStdout()); - LOG.error(execResult.getStderr()); - // wait job start - Thread.sleep(WAIT_FLINK_JOB_SUBMIT); - return execResult; + @Override + protected String getStartShellName() { + return "start-seatunnel-flink-new-connector.sh"; } - protected void copySeaTunnelFlinkFile() { - // copy lib - String seatunnelCoreFlinkJarPath = PROJECT_ROOT_PATH - + "/seatunnel-core/seatunnel-flink-starter/target/" + SEATUNNEL_FLINK_JAR; - jobManager.copyFileToContainer( - MountableFile.forHostPath(seatunnelCoreFlinkJarPath), - Paths.get(SEATUNNEL_LIB, SEATUNNEL_FLINK_JAR).toString()); - - // copy bin - String seatunnelFlinkBinPath = PROJECT_ROOT_PATH + "/seatunnel-core/seatunnel-flink-starter/src/main/bin/" + SEATUNNEL_FLINK_BIN; - jobManager.copyFileToContainer( - MountableFile.forHostPath(seatunnelFlinkBinPath), - Paths.get(SEATUNNEL_BIN, SEATUNNEL_FLINK_BIN).toString()); - - // copy connectors - File jars = new File(PROJECT_ROOT_PATH + - "/seatunnel-connectors-v2-dist/target/lib"); - Arrays.stream(Objects.requireNonNull(jars.listFiles(f -> f.getName().startsWith("connector-")))) - .forEach(jar -> - jobManager.copyFileToContainer( - MountableFile.forHostPath(jar.getAbsolutePath()), - getConnectorPath(jar.getName()))); - - // copy plugin-mapping.properties - jobManager.copyFileToContainer( - MountableFile.forHostPath(PROJECT_ROOT_PATH + "/plugin-mapping.properties"), - Paths.get(SEATUNNEL_CONNECTORS, PLUGIN_MAPPING_FILE).toString()); + @Override + protected String getConnectorType() { + return "seatunnel"; } - private String getResource(String confFile) { - return System.getProperty("user.dir") + "/src/test/resources" + confFile; + @Override + protected String getConnectorModulePath() { + return "seatunnel-connectors-v2"; } - private String getConnectorPath(String fileName) { - return Paths.get(SEATUNNEL_CONNECTORS, "seatunnel", fileName).toString(); + @Override + protected String getConnectorNamePrefix() { + return "connector-"; } } diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml index 4f5f037dd7d..045579f3701 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml @@ -41,7 +41,7 @@ org.apache.seatunnel - seatunnel-core-flink + seatunnel-flink-starter ${project.version} test diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/pom.xml b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/pom.xml index 32f26486dfa..7968e8526bd 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/pom.xml @@ -25,10 +25,14 @@ seatunnel-connector-flink-e2e-base - - 2.4 - - + + + org.apache.seatunnel + seatunnel-e2e-common + ${project.version} + test-jar + + diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java index f3308dead78..a1ec702690e 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java @@ -17,152 +17,37 @@ package org.apache.seatunnel.e2e.flink; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.Container; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.Network; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.MountableFile; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import java.util.stream.Stream; +import org.apache.seatunnel.e2e.common.AbstractFlinkContainer; /** * This class is the base class of FlinkEnvironment test. * The before method will create a Flink cluster, and after method will close the Flink cluster. * You can use {@link FlinkContainer#executeSeaTunnelFlinkJob} to submit a seatunnel config and run a seatunnel job. */ -public abstract class FlinkContainer { - - private static final Logger LOG = LoggerFactory.getLogger(FlinkContainer.class); - - private static final String FLINK_DOCKER_IMAGE = "flink:1.13.6-scala_2.11"; - protected static final Network NETWORK = Network.newNetwork(); - - protected GenericContainer jobManager; - protected GenericContainer taskManager; - private static final Path PROJECT_ROOT_PATH = Paths.get(System.getProperty("user.dir")).getParent().getParent().getParent(); - private static final String SEATUNNEL_FLINK_BIN = "start-seatunnel-flink.sh"; - private static final String SEATUNNEL_FLINK_JAR = "seatunnel-core-flink.jar"; - private static final String PLUGIN_MAPPING_FILE = "plugin-mapping.properties"; - private static final String SEATUNNEL_HOME = "/tmp/flink/seatunnel"; - private static final String SEATUNNEL_BIN = Paths.get(SEATUNNEL_HOME, "bin").toString(); - private static final String SEATUNNEL_LIB = Paths.get(SEATUNNEL_HOME, "lib").toString(); - private static final String SEATUNNEL_CONNECTORS = Paths.get(SEATUNNEL_HOME, "connectors").toString(); - - private static final int WAIT_FLINK_JOB_SUBMIT = 5000; - - private static final String FLINK_PROPERTIES = String.join( - "\n", - Arrays.asList( - "jobmanager.rpc.address: jobmanager", - "taskmanager.numberOfTaskSlots: 10", - "parallelism.default: 4", - "env.java.opts: -Doracle.jdbc.timezoneAsRegion=false")); - - @BeforeEach - public void before() { - jobManager = new GenericContainer<>(FLINK_DOCKER_IMAGE) - .withCommand("jobmanager") - .withNetwork(NETWORK) - .withNetworkAliases("jobmanager") - .withExposedPorts() - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - - taskManager = - new GenericContainer<>(FLINK_DOCKER_IMAGE) - .withCommand("taskmanager") - .withNetwork(NETWORK) - .withNetworkAliases("taskmanager") - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - .dependsOn(jobManager) - .withLogConsumer(new Slf4jLogConsumer(LOG)); +public abstract class FlinkContainer extends AbstractFlinkContainer { - Startables.deepStart(Stream.of(jobManager)).join(); - Startables.deepStart(Stream.of(taskManager)).join(); - copySeaTunnelFlinkFile(); - LOG.info("Flink containers are started."); + @Override + protected String getStartModulePath() { + return "seatunnel-core-flink"; } - @AfterEach - public void close() { - if (taskManager != null) { - taskManager.stop(); - } - if (jobManager != null) { - jobManager.stop(); - } + @Override + protected String getStartShellName() { + return "start-seatunnel-flink.sh"; } - public Container.ExecResult executeSeaTunnelFlinkJob(String confFile) throws IOException, InterruptedException { - final String confPath = getResource(confFile); - if (!new File(confPath).exists()) { - throw new IllegalArgumentException(confFile + " doesn't exist"); - } - final String targetConfInContainer = Paths.get("/tmp", confFile).toString(); - jobManager.copyFileToContainer(MountableFile.forHostPath(confPath), targetConfInContainer); - - // Running IT use cases under Windows requires replacing \ with / - String conf = targetConfInContainer.replaceAll("\\\\", "/"); - final List command = new ArrayList<>(); - command.add(Paths.get(SEATUNNEL_HOME, "bin/start-seatunnel-flink.sh").toString()); - command.add("--config " + conf); - - Container.ExecResult execResult = jobManager.execInContainer("bash", "-c", String.join(" ", command)); - LOG.info(execResult.getStdout()); - LOG.error(execResult.getStderr()); - // wait job start - Thread.sleep(WAIT_FLINK_JOB_SUBMIT); - return execResult; - } - - protected void copySeaTunnelFlinkFile() { - // copy lib - String seatunnelCoreFlinkJarPath = PROJECT_ROOT_PATH - + "/seatunnel-core/seatunnel-core-flink/target/seatunnel-core-flink.jar"; - jobManager.copyFileToContainer( - MountableFile.forHostPath(seatunnelCoreFlinkJarPath), - Paths.get(SEATUNNEL_LIB, SEATUNNEL_FLINK_JAR).toString()); - - // copy bin - String seatunnelFlinkBinPath = PROJECT_ROOT_PATH + "/seatunnel-core/seatunnel-core-flink/src/main/bin/start-seatunnel-flink.sh"; - jobManager.copyFileToContainer( - MountableFile.forHostPath(seatunnelFlinkBinPath), - Paths.get(SEATUNNEL_BIN, SEATUNNEL_FLINK_BIN).toString()); - - // copy connectors - File jars = new File(PROJECT_ROOT_PATH + - "/seatunnel-connectors/seatunnel-connectors-flink-dist/target/lib"); - Arrays.stream(Objects.requireNonNull(jars.listFiles(f -> f.getName().startsWith("seatunnel-connector-flink")))) - .forEach(jar -> - jobManager.copyFileToContainer( - MountableFile.forHostPath(jar.getAbsolutePath()), - getConnectorPath(jar.getName()))); - - // copy plugin-mapping.properties - jobManager.copyFileToContainer( - MountableFile.forHostPath(PROJECT_ROOT_PATH + "/plugin-mapping.properties"), - Paths.get(SEATUNNEL_CONNECTORS, PLUGIN_MAPPING_FILE).toString()); + @Override + protected String getConnectorType() { + return "seatunnel"; } - private String getResource(String confFile) { - return System.getProperty("user.dir") + "/src/test/resources" + confFile; + @Override + protected String getConnectorModulePath() { + return "seatunnel-connectors/seatunnel-connectors-flink"; } - private String getConnectorPath(String fileName) { - return Paths.get(SEATUNNEL_CONNECTORS, "flink", fileName).toString(); + @Override + protected String getConnectorNamePrefix() { + return "seatunnel-connector-flink-"; } - } diff --git a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/pom.xml b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/pom.xml index 2cd8cacdb0c..5de92cec77d 100644 --- a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/pom.xml @@ -25,9 +25,14 @@ setunnel-connector-flink-sql-e2e-base - - 2.4 - + + + org.apache.seatunnel + seatunnel-e2e-common + ${project.version} + test-jar + + diff --git a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java index 082500856ca..82a7159c17c 100644 --- a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java @@ -17,128 +17,37 @@ package org.apache.seatunnel.e2e.flink.sql; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.Container; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.Network; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.MountableFile; - -import java.io.File; -import java.io.IOException; -import java.net.URISyntaxException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Stream; +import org.apache.seatunnel.e2e.common.AbstractFlinkContainer; /** * This class is the base class of FlinkEnvironment test. * The before method will create a Flink cluster, and after method will close the Flink cluster. - * You can use {@link FlinkContainer#executeSeaTunnelFlinkSqlJob(String)} to submit a seatunnel config and run a seatunnel job. + * You can use {@link FlinkContainer#executeSeaTunnelFlinkJob(String)} to submit a seatunnel config and run a seatunnel job. */ -public abstract class FlinkContainer { - - private static final Logger LOG = LoggerFactory.getLogger(FlinkContainer.class); - - private static final String FLINK_DOCKER_IMAGE = "flink:1.13.6-scala_2.11"; - protected static final Network NETWORK = Network.newNetwork(); - - protected GenericContainer jobManager; - protected GenericContainer taskManager; - private static final Path PROJECT_ROOT_PATH = Paths.get(System.getProperty("user.dir")).getParent().getParent().getParent(); - private static final String SEATUNNEL_FLINK_BIN = "start-seatunnel-sql.sh"; - private static final String SEATUNNEL_FLINK_SQL_JAR = "seatunnel-core-flink-sql.jar"; - private static final String SEATUNNEL_HOME = "/tmp/flink/seatunnel"; - private static final String SEATUNNEL_BIN = Paths.get(SEATUNNEL_HOME, "bin", SEATUNNEL_FLINK_BIN).toString(); - private static final String FLINK_JAR_PATH = Paths.get(SEATUNNEL_HOME, "lib", SEATUNNEL_FLINK_SQL_JAR).toString(); - - private static final int WAIT_FLINK_JOB_SUBMIT = 5000; - - private static final String FLINK_PROPERTIES = String.join( - "\n", - Arrays.asList( - "jobmanager.rpc.address: jobmanager", - "taskmanager.numberOfTaskSlots: 10", - "parallelism.default: 4", - "env.java.opts: -Doracle.jdbc.timezoneAsRegion=false")); +public abstract class FlinkContainer extends AbstractFlinkContainer { - @BeforeEach - public void before() { - jobManager = new GenericContainer<>(FLINK_DOCKER_IMAGE) - .withCommand("jobmanager") - .withNetwork(NETWORK) - .withNetworkAliases("jobmanager") - .withExposedPorts() - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - - taskManager = - new GenericContainer<>(FLINK_DOCKER_IMAGE) - .withCommand("taskmanager") - .withNetwork(NETWORK) - .withNetworkAliases("taskmanager") - .withEnv("FLINK_PROPERTIES", FLINK_PROPERTIES) - .dependsOn(jobManager) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - - Startables.deepStart(Stream.of(jobManager)).join(); - Startables.deepStart(Stream.of(taskManager)).join(); - copySeaTunnelFlinkFile(); - LOG.info("Flink containers are started."); + @Override + protected String getStartModulePath() { + return "seatunnel-core-flink-sql"; } - @AfterEach - public void close() { - if (taskManager != null) { - taskManager.stop(); - } - if (jobManager != null) { - jobManager.stop(); - } + @Override + protected String getStartShellName() { + return "start-seatunnel-sql.sh"; } - public Container.ExecResult executeSeaTunnelFlinkSqlJob(String confFile) - throws IOException, InterruptedException, URISyntaxException { - final String confPath = Paths.get(FlinkContainer.class.getResource(confFile).toURI()).toString(); - if (!new File(confPath).exists()) { - throw new IllegalArgumentException(confFile + " doesn't exist"); - } - final String targetConfInContainer = Paths.get("/tmp", confFile).toString(); - jobManager.copyFileToContainer(MountableFile.forHostPath(confPath), targetConfInContainer); - - // Running IT use cases under Windows requires replacing \ with / - String conf = targetConfInContainer.replaceAll("\\\\", "/"); - final List command = new ArrayList<>(); - command.add(Paths.get(SEATUNNEL_HOME, "bin/start-seatunnel-sql.sh").toString()); - command.add("--config " + conf); - - Container.ExecResult execResult = jobManager.execInContainer("bash", "-c", String.join(" ", command)); - LOG.info(execResult.getStdout()); - LOG.error(execResult.getStderr()); - // wait job start - Thread.sleep(WAIT_FLINK_JOB_SUBMIT); - return execResult; + @Override + protected String getConnectorType() { + return "flink-sql"; } - protected void copySeaTunnelFlinkFile() { - // copy lib - String seatunnelCoreFlinkJarPath = PROJECT_ROOT_PATH + "/seatunnel-core/seatunnel-core-flink-sql/target/" + SEATUNNEL_FLINK_SQL_JAR; - jobManager.copyFileToContainer( - MountableFile.forHostPath(seatunnelCoreFlinkJarPath), - FLINK_JAR_PATH); - - // copy bin - String seatunnelFlinkBinPath = PROJECT_ROOT_PATH + "/seatunnel-core/seatunnel-core-flink-sql/src/main/bin/start-seatunnel-sql.sh"; - jobManager.copyFileToContainer( - MountableFile.forHostPath(seatunnelFlinkBinPath), - Paths.get(SEATUNNEL_BIN).toString()); + @Override + protected String getConnectorModulePath() { + return "seatunnel-connectors/seatunnel-connectors-flink-sql"; } + @Override + protected String getConnectorNamePrefix() { + return "flink-sql-connector-"; + } } diff --git a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-fake-e2e/src/test/java/org/apache/seatunnel/e2e/flink/sql/fake/DatagenToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-fake-e2e/src/test/java/org/apache/seatunnel/e2e/flink/sql/fake/DatagenToConsoleIT.java index 05c5eb09c4a..b9f57920c68 100644 --- a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-fake-e2e/src/test/java/org/apache/seatunnel/e2e/flink/sql/fake/DatagenToConsoleIT.java +++ b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-fake-e2e/src/test/java/org/apache/seatunnel/e2e/flink/sql/fake/DatagenToConsoleIT.java @@ -24,14 +24,13 @@ import org.testcontainers.containers.Container; import java.io.IOException; -import java.net.URISyntaxException; public class DatagenToConsoleIT extends FlinkContainer { @Test - public void testDatagenToConsole() throws IOException, URISyntaxException, InterruptedException { + public void testDatagenToConsole() throws IOException, InterruptedException { final String configFile = "/fake/flink.sql.conf"; - Container.ExecResult execResult = executeSeaTunnelFlinkSqlJob(configFile); + Container.ExecResult execResult = executeSeaTunnelFlinkJob(configFile); Assertions.assertEquals(0, execResult.getExitCode()); } } diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/pom.xml index cbc8495c594..fcb3569ed31 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/pom.xml @@ -25,9 +25,14 @@ connector-spark-e2e-base - - 2.4 - + + + org.apache.seatunnel + seatunnel-e2e-common + ${project.version} + test-jar + + diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java index 2f74483f006..7ad7c69f593 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java @@ -17,145 +17,36 @@ package org.apache.seatunnel.e2e.spark; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.Container; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.Network; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.MountableFile; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; -import java.util.stream.Stream; +import org.apache.seatunnel.e2e.common.AbstractSparkContainer; /** * This class is the base class of SparkEnvironment test. The before method will create a Spark master, and after method will close the Spark master. * You can use {@link SparkContainer#executeSeaTunnelSparkJob} to submit a seatunnel conf and a seatunnel spark job. */ -public abstract class SparkContainer { - - private static final Logger LOG = LoggerFactory.getLogger(SparkContainer.class); - - private static final String SPARK_DOCKER_IMAGE = "bitnami/spark:2.4.3"; - public static final Network NETWORK = Network.newNetwork(); - - protected GenericContainer master; - private static final Path PROJECT_ROOT_PATH = Paths.get(System.getProperty("user.dir")).getParent().getParent().getParent(); - private static final String SEATUNNEL_SPARK_BIN = "start-seatunnel-spark-new-connector.sh"; - private static final String SEATUNNEL_SPARK_JAR = "seatunnel-spark-starter.jar"; - private static final String PLUGIN_MAPPING_FILE = "plugin-mapping.properties"; - private static final String SEATUNNEL_HOME = "/tmp/spark/seatunnel"; - private static final String SEATUNNEL_BIN = Paths.get(SEATUNNEL_HOME, "bin").toString(); - private static final String SPARK_JAR_PATH = Paths.get(SEATUNNEL_HOME, "lib", SEATUNNEL_SPARK_JAR).toString(); - private static final String CONNECTORS_PATH = Paths.get(SEATUNNEL_HOME, "connectors").toString(); - - private static final int WAIT_SPARK_JOB_SUBMIT = 5000; - - @BeforeEach - public void before() { - master = new GenericContainer<>(SPARK_DOCKER_IMAGE) - .withNetwork(NETWORK) - .withNetworkAliases("spark-master") - .withExposedPorts() - .withEnv("SPARK_MODE", "master") - .withLogConsumer(new Slf4jLogConsumer(LOG)); - // In most case we can just use standalone mode to execute a spark job, if we want to use cluster mode, we need to - // start a worker. - - Startables.deepStart(Stream.of(master)).join(); - copySeaTunnelSparkFile(); - LOG.info("Spark container started"); - } - - @AfterEach - public void close() { - if (master != null) { - master.stop(); - } - } - - public Container.ExecResult executeSeaTunnelSparkJob(String confFile) throws IOException, InterruptedException { - final String confPath = getResource(confFile); - if (!new File(confPath).exists()) { - throw new IllegalArgumentException(confFile + " doesn't exist"); - } - final String targetConfInContainer = Paths.get("/tmp", confFile).toString(); - master.copyFileToContainer(MountableFile.forHostPath(confPath), targetConfInContainer); - - // TODO: use start-seatunnel-spark.sh to run the spark job. Need to modified the SparkStarter can find the seatunnel-core-spark.jar. - final List command = new ArrayList<>(); - String sparkBinPath = Paths.get(SEATUNNEL_HOME, "bin", SEATUNNEL_SPARK_BIN).toString(); - command.add(adaptPathForWin(sparkBinPath)); - command.add("--master"); - command.add("local"); - command.add("--deploy-mode"); - command.add("client"); - command.add("--config " + adaptPathForWin(targetConfInContainer)); - - Container.ExecResult execResult = master.execInContainer("bash", "-c", String.join(" ", command)); - LOG.info(execResult.getStdout()); - LOG.error(execResult.getStderr()); - // wait job start - Thread.sleep(WAIT_SPARK_JOB_SUBMIT); - return execResult; - } - - protected void copySeaTunnelSparkFile() { - // copy lib - String seatunnelCoreSparkJarPath = Paths.get(PROJECT_ROOT_PATH.toString(), - "seatunnel-core", "seatunnel-spark-starter", "target", SEATUNNEL_SPARK_JAR).toString(); - master.copyFileToContainer(MountableFile.forHostPath(seatunnelCoreSparkJarPath), SPARK_JAR_PATH); - - // copy bin - String seatunnelSparkBinPath = Paths.get(PROJECT_ROOT_PATH.toString(), - "seatunnel-core", "seatunnel-spark-starter", "src", "main", "bin", SEATUNNEL_SPARK_BIN).toString(); - master.copyFileToContainer( - MountableFile.forHostPath(seatunnelSparkBinPath), - Paths.get(SEATUNNEL_BIN, SEATUNNEL_SPARK_BIN).toString()); - - // copy connectors - getConnectorJarFiles() - .forEach(jar -> - master.copyFileToContainer( - MountableFile.forHostPath(jar.getAbsolutePath()), - getConnectorPath(jar.getName()))); +public abstract class SparkContainer extends AbstractSparkContainer { - // copy plugin-mapping.properties - master.copyFileToContainer( - MountableFile.forHostPath(PROJECT_ROOT_PATH + "/plugin-mapping.properties"), - Paths.get(CONNECTORS_PATH, PLUGIN_MAPPING_FILE).toString()); + @Override + protected String getStartModulePath() { + return "seatunnel-spark-starter"; } - private String getResource(String confFile) { - return System.getProperty("user.dir") + "/src/test/resources" + confFile; + @Override + protected String getStartShellName() { + return "start-seatunnel-spark-new-connector.sh"; } - private String getConnectorPath(String fileName) { - return Paths.get(CONNECTORS_PATH, "seatunnel", fileName).toString(); + @Override + protected String getConnectorType() { + return "seatunnel"; } - private List getConnectorJarFiles() { - File jars = new File(PROJECT_ROOT_PATH + - "/seatunnel-connectors-v2-dist/target/lib"); - return Arrays.stream( - Objects.requireNonNull( - jars.listFiles( - f -> f.getName().contains("connector-")))) - .collect(Collectors.toList()); + @Override + protected String getConnectorModulePath() { + return "seatunnel-connectors-v2"; } - private String adaptPathForWin(String path) { - return path == null ? "" : path.replaceAll("\\\\", "/"); + @Override + protected String getConnectorNamePrefix() { + return "connector-"; } } diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml index a7a084bac14..b5aef0ec786 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml @@ -39,7 +39,7 @@ org.apache.seatunnel - seatunnel-core-spark + seatunnel-spark-starter ${project.version} test diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/pom.xml b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/pom.xml index 7fa7dfaa035..2ff76f7db64 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/pom.xml @@ -25,9 +25,14 @@ seatunnel-connector-spark-e2e-base - - 2.4 - + + + org.apache.seatunnel + seatunnel-e2e-common + ${project.version} + test-jar + + diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java index 6dca627ac0c..8811c06a490 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java @@ -17,141 +17,36 @@ package org.apache.seatunnel.e2e.spark; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.Container; -import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.Network; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.MountableFile; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; -import java.util.stream.Stream; +import org.apache.seatunnel.e2e.common.AbstractSparkContainer; /** * This class is the base class of SparkEnvironment test. The before method will create a Spark master, and after method will close the Spark master. * You can use {@link SparkContainer#executeSeaTunnelSparkJob} to submit a seatunnel conf and a seatunnel spark job. */ -public abstract class SparkContainer { - - private static final Logger LOG = LoggerFactory.getLogger(SparkContainer.class); - - private static final String SPARK_DOCKER_IMAGE = "bitnami/spark:2.4.3"; - public static final Network NETWORK = Network.newNetwork(); - - protected GenericContainer master; - private static final Path PROJECT_ROOT_PATH = Paths.get(System.getProperty("user.dir")).getParent().getParent().getParent(); - private static final String SEATUNNEL_SPARK_BIN = "start-seatunnel-spark.sh"; - private static final String SEATUNNEL_SPARK_JAR = "seatunnel-core-spark.jar"; - private static final String PLUGIN_MAPPING_FILE = "plugin-mapping.properties"; - private static final String SEATUNNEL_HOME = "/tmp/spark/seatunnel"; - private static final String SEATUNNEL_BIN = Paths.get(SEATUNNEL_HOME, "bin").toString(); - private static final String SPARK_JAR_PATH = Paths.get(SEATUNNEL_HOME, "lib", SEATUNNEL_SPARK_JAR).toString(); - private static final String CONNECTORS_PATH = Paths.get(SEATUNNEL_HOME, "connectors").toString(); - - private static final int WAIT_SPARK_JOB_SUBMIT = 5000; - - @BeforeEach - public void before() { - master = new GenericContainer<>(SPARK_DOCKER_IMAGE) - .withNetwork(NETWORK) - .withNetworkAliases("spark-master") - .withExposedPorts() - .withEnv("SPARK_MODE", "master") - .withLogConsumer(new Slf4jLogConsumer(LOG)); - // In most case we can just use standalone mode to execute a spark job, if we want to use cluster mode, we need to - // start a worker. - - Startables.deepStart(Stream.of(master)).join(); - copySeaTunnelSparkFile(); - LOG.info("Spark container started"); - } +public abstract class SparkContainer extends AbstractSparkContainer { - @AfterEach - public void close() { - if (master != null) { - master.stop(); - } + @Override + protected String getStartModulePath() { + return "seatunnel-core-spark"; } - public Container.ExecResult executeSeaTunnelSparkJob(String confFile) throws IOException, InterruptedException { - final String confPath = getResource(confFile); - if (!new File(confPath).exists()) { - throw new IllegalArgumentException(confFile + " doesn't exist"); - } - final String targetConfInContainer = Paths.get("/tmp", confFile).toString(); - master.copyFileToContainer(MountableFile.forHostPath(confPath), targetConfInContainer); - - // TODO: use start-seatunnel-spark.sh to run the spark job. Need to modified the SparkStarter can find the seatunnel-core-spark.jar. - // Running IT use cases under Windows requires replacing \ with / - String conf = targetConfInContainer.replaceAll("\\\\", "/"); - final List command = new ArrayList<>(); - command.add(Paths.get(SEATUNNEL_HOME, "bin/start-seatunnel-spark.sh").toString()); - command.add("--master"); - command.add("local"); - command.add("--deploy-mode"); - command.add("client"); - command.add("--config " + conf); - - Container.ExecResult execResult = master.execInContainer("bash", "-c", String.join(" ", command)); - LOG.info(execResult.getStdout()); - LOG.error(execResult.getStderr()); - // wait job start - Thread.sleep(WAIT_SPARK_JOB_SUBMIT); - return execResult; - } - - protected void copySeaTunnelSparkFile() { - // copy lib - String seatunnelCoreSparkJarPath = PROJECT_ROOT_PATH - + "/seatunnel-core/seatunnel-core-spark/target/seatunnel-core-spark.jar"; - master.copyFileToContainer(MountableFile.forHostPath(seatunnelCoreSparkJarPath), SPARK_JAR_PATH); - - // copy bin - String seatunnelFlinkBinPath = PROJECT_ROOT_PATH + "/seatunnel-core/seatunnel-core-spark/src/main/bin/start-seatunnel-spark.sh"; - master.copyFileToContainer( - MountableFile.forHostPath(seatunnelFlinkBinPath), - Paths.get(SEATUNNEL_BIN, SEATUNNEL_SPARK_BIN).toString()); - - // copy connectors - getConnectorJarFiles() - .forEach(jar -> - master.copyFileToContainer( - MountableFile.forHostPath(jar.getAbsolutePath()), - getConnectorPath(jar.getName()))); - - // copy plugin-mapping.properties - master.copyFileToContainer( - MountableFile.forHostPath(PROJECT_ROOT_PATH + "/plugin-mapping.properties"), - Paths.get(CONNECTORS_PATH, PLUGIN_MAPPING_FILE).toString()); + @Override + protected String getStartShellName() { + return "start-seatunnel-spark.sh"; } - private String getResource(String confFile) { - return System.getProperty("user.dir") + "/src/test/resources" + confFile; + @Override + protected String getConnectorType() { + return "spark"; } - private String getConnectorPath(String fileName) { - return Paths.get(CONNECTORS_PATH, "spark", fileName).toString(); + @Override + protected String getConnectorModulePath() { + return "seatunnel-connectors/seatunnel-connectors-spark"; } - private List getConnectorJarFiles() { - File jars = new File(PROJECT_ROOT_PATH + - "/seatunnel-connectors/seatunnel-connectors-spark-dist/target/lib"); - return Arrays.stream( - Objects.requireNonNull( - jars.listFiles( - f -> f.getName().contains("seatunnel-connector-spark")))) - .collect(Collectors.toList()); + @Override + protected String getConnectorNamePrefix() { + return "seatunnel-connector-spark-"; } } From 7f9f202cbf967e5342206b32b70d1f86522591f3 Mon Sep 17 00:00:00 2001 From: Zongwen Li Date: Fri, 9 Sep 2022 23:53:00 +0800 Subject: [PATCH 20/44] [Improve][build] The e2e module don't depend on the connector*-dist module (#2702) * [Improve][build] The e2e module don't depend on the connector*-dist module * fix connector type error * fix flink sql --- .../connector-assert-flink-e2e/pom.xml | 8 ++++++++ .../connector-datahub-flink-e2e/pom.xml | 8 ++++++++ .../connector-fake-flink-e2e/pom.xml | 8 ++++++++ .../connector-file-flink-e2e/pom.xml | 8 ++++++++ .../connector-iotdb-flink-e2e/pom.xml | 8 ++++++++ .../connector-jdbc-flink-e2e/pom.xml | 16 ++++++++++++++++ .../connector-mongodb-flink-e2e/pom.xml | 8 ++++++++ .../connector-redis-flink-e2e/pom.xml | 8 ++++++++ .../seatunnel-flink-connector-v2-e2e/pom.xml | 12 ------------ seatunnel-e2e/seatunnel-flink-e2e/pom.xml | 12 ------------ .../seatunnel-connector-flink-assert-e2e/pom.xml | 8 ++++++++ .../pom.xml | 8 ++++++++ .../seatunnel/e2e/flink/FlinkContainer.java | 2 +- .../seatunnel-connector-flink-fake-e2e/pom.xml | 8 ++++++++ .../seatunnel-connector-flink-file-e2e/pom.xml | 8 ++++++++ .../seatunnel-connector-flink-http-e2e/pom.xml | 8 ++++++++ seatunnel-e2e/seatunnel-flink-sql-e2e/pom.xml | 12 ------------ .../seatunnel/e2e/flink/sql/FlinkContainer.java | 11 +++++++++++ .../pom.xml | 1 + .../connector-datahub-spark-e2e/pom.xml | 8 ++++++++ .../connector-fake-spark-e2e/pom.xml | 8 ++++++++ .../connector-file-spark-e2e/pom.xml | 8 ++++++++ .../connector-iotdb-spark-e2e/pom.xml | 8 ++++++++ .../connector-jdbc-spark-e2e/pom.xml | 4 ++++ .../connector-redis-spark-e2e/pom.xml | 8 ++++++++ .../seatunnel-spark-connector-v2-e2e/pom.xml | 12 ------------ seatunnel-e2e/seatunnel-spark-e2e/pom.xml | 12 ------------ .../seatunnel-connector-spark-fake-e2e/pom.xml | 8 ++++++++ .../seatunnel-connector-spark-http-e2e/pom.xml | 6 ++++++ .../seatunnel-connector-spark-jdbc-e2e/pom.xml | 14 ++++++++++++++ 30 files changed, 197 insertions(+), 61 deletions(-) diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-assert-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-assert-flink-e2e/pom.xml index 37de9280a0a..7db3cb830bb 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-assert-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-assert-flink-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + org.apache.seatunnel connector-assert diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-datahub-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-datahub-flink-e2e/pom.xml index 2adf5557e45..79dc07e22a4 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-datahub-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-datahub-flink-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + org.apache.seatunnel connector-datahub diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-fake-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-fake-flink-e2e/pom.xml index 299c2dddc4c..ffb3d91b22d 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-fake-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-fake-flink-e2e/pom.xml @@ -34,12 +34,20 @@ test-jar test + + org.apache.seatunnel connector-fake ${project.version} test + + org.apache.seatunnel + connector-console + ${project.version} + test + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/pom.xml index 70e91ec79ce..df018a71c52 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + org.apache.seatunnel connector-file-local diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iotdb-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iotdb-flink-e2e/pom.xml index 30c1ae671d9..975febb372a 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iotdb-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iotdb-flink-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + org.apache.seatunnel connector-iotdb diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/pom.xml index 4b4d1d9981e..eb9ed148f86 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/pom.xml @@ -34,12 +34,28 @@ test-jar test + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-console + ${project.version} + test + org.apache.seatunnel connector-jdbc ${project.version} test + + org.testcontainers postgresql diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/pom.xml index a49496c68f4..b7c2ef0a621 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/pom.xml @@ -35,11 +35,19 @@ test-jar test + + org.apache.seatunnel connector-mongodb ${project.version} test + + org.apache.seatunnel + connector-console + ${project.version} + test + diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-redis-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-redis-flink-e2e/pom.xml index a1b314bd1bb..d92a1b3b55b 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-redis-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-redis-flink-e2e/pom.xml @@ -34,12 +34,20 @@ test-jar test + + org.apache.seatunnel connector-redis ${project.version} test + + org.apache.seatunnel + connector-assert + ${project.version} + test + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml index 045579f3701..61d7911a548 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml @@ -45,18 +45,6 @@ ${project.version} test - - org.apache.seatunnel - seatunnel-connectors-v2-dist - ${project.version} - test - - - * - * - - - org.awaitility awaitility diff --git a/seatunnel-e2e/seatunnel-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-e2e/pom.xml index 2ca2e18613a..fd6116d2faa 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-e2e/pom.xml @@ -42,18 +42,6 @@ ${project.version} test - - org.apache.seatunnel - seatunnel-connectors-flink-dist - ${project.version} - test - - - * - * - - - \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-assert-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-assert-e2e/pom.xml index fd262b6f3e1..358771f045b 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-assert-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-assert-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + seatunnel-connector-flink-fake + ${project.version} + test + org.apache.seatunnel seatunnel-connector-flink-assert diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-clickhouse-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-clickhouse-e2e/pom.xml index 87ef96e073d..cfa0378627c 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-clickhouse-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-clickhouse-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + seatunnel-connector-flink-fake + ${project.version} + test + org.apache.seatunnel seatunnel-connector-flink-clickhouse diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java index a1ec702690e..ab62a623d01 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java @@ -38,7 +38,7 @@ protected String getStartShellName() { @Override protected String getConnectorType() { - return "seatunnel"; + return "flink"; } @Override diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-fake-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-fake-e2e/pom.xml index 873d6c65f9b..1d406819675 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-fake-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-fake-e2e/pom.xml @@ -34,12 +34,20 @@ test-jar test + + org.apache.seatunnel seatunnel-connector-flink-fake ${project.version} test + + org.apache.seatunnel + seatunnel-connector-flink-console + ${project.version} + test + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-file-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-file-e2e/pom.xml index 1bc0ebca3e1..fd39c309247 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-file-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-file-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + seatunnel-connector-flink-fake + ${project.version} + test + org.apache.seatunnel seatunnel-connector-flink-file diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-http-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-http-e2e/pom.xml index 000b29c2c44..c4ff4e7cfbf 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-http-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-http-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + seatunnel-connector-flink-console + ${project.version} + test + org.apache.seatunnel seatunnel-connector-flink-http diff --git a/seatunnel-e2e/seatunnel-flink-sql-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-sql-e2e/pom.xml index ffd5e49192d..02795a7d26d 100644 --- a/seatunnel-e2e/seatunnel-flink-sql-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-sql-e2e/pom.xml @@ -38,17 +38,5 @@ ${project.version} test - - org.apache.seatunnel - seatunnel-connectors-flink-sql-dist - ${project.version} - test - - - * - * - - - \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java index 82a7159c17c..e195b4aa9c2 100644 --- a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java @@ -17,8 +17,14 @@ package org.apache.seatunnel.e2e.flink.sql; +import static org.apache.seatunnel.e2e.common.ContainerUtil.copyConfigFileToContainer; + import org.apache.seatunnel.e2e.common.AbstractFlinkContainer; +import org.testcontainers.containers.Container; + +import java.io.IOException; + /** * This class is the base class of FlinkEnvironment test. * The before method will create a Flink cluster, and after method will close the Flink cluster. @@ -50,4 +56,9 @@ protected String getConnectorModulePath() { protected String getConnectorNamePrefix() { return "flink-sql-connector-"; } + + public Container.ExecResult executeSeaTunnelFlinkJob(String confFile) throws IOException, InterruptedException { + final String confInContainerPath = copyConfigFileToContainer(jobManager, confFile); + return executeCommand(jobManager, confInContainerPath); + } } diff --git a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-fake-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-fake-e2e/pom.xml index 9eda3de27cf..24b8b010da8 100644 --- a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-fake-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-fake-e2e/pom.xml @@ -34,6 +34,7 @@ test-jar test + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-datahub-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-datahub-spark-e2e/pom.xml index eb5d46f6993..79c3916e787 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-datahub-spark-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-datahub-spark-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + org.apache.seatunnel connector-datahub diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-fake-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-fake-spark-e2e/pom.xml index 8f2b4e6f133..1f9e6d844fe 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-fake-spark-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-fake-spark-e2e/pom.xml @@ -34,12 +34,20 @@ test-jar test + + org.apache.seatunnel connector-fake ${project.version} test + + org.apache.seatunnel + connector-console + ${project.version} + test + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/pom.xml index 7a310d19c93..0ab6f3efd72 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + org.apache.seatunnel connector-file-local diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iotdb-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iotdb-spark-e2e/pom.xml index 171f5a27a69..2c4e8e4e547 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iotdb-spark-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iotdb-spark-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + org.apache.seatunnel connector-iotdb diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/pom.xml index a5242ef72c4..bd0a341fea9 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/pom.xml @@ -34,12 +34,16 @@ test-jar test + + org.apache.seatunnel connector-jdbc ${project.version} test + + mysql mysql-connector-java diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-redis-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-redis-spark-e2e/pom.xml index b5538ed1be7..25212814ad8 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-redis-spark-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-redis-spark-e2e/pom.xml @@ -34,6 +34,14 @@ test-jar test + + + + org.apache.seatunnel + connector-assert + ${project.version} + test + org.apache.seatunnel connector-redis diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml index b5aef0ec786..898bf04c930 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml @@ -43,18 +43,6 @@ ${project.version} test - - org.apache.seatunnel - seatunnel-connectors-v2-dist - ${project.version} - test - - - * - * - - - org.awaitility diff --git a/seatunnel-e2e/seatunnel-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-e2e/pom.xml index 5a80812f232..86c903a84a3 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-e2e/pom.xml @@ -40,18 +40,6 @@ ${project.version} test - - org.apache.seatunnel - seatunnel-connectors-spark-dist - ${project.version} - test - - - * - * - - - \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-fake-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-fake-e2e/pom.xml index 4dabac2fb81..562db71d605 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-fake-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-fake-e2e/pom.xml @@ -34,12 +34,20 @@ test-jar test + + org.apache.seatunnel seatunnel-connector-spark-fake ${project.version} test + + org.apache.seatunnel + seatunnel-connector-spark-console + ${project.version} + test + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-http-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-http-e2e/pom.xml index d859d86eed3..98a7191c712 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-http-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-http-e2e/pom.xml @@ -40,6 +40,12 @@ ${project.version} test + + org.apache.seatunnel + seatunnel-connector-spark-console + ${project.version} + test + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/pom.xml index 6e1b2aec5f9..b0669c9a0ee 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/pom.xml @@ -34,6 +34,20 @@ test-jar test + + + + org.apache.seatunnel + seatunnel-connector-spark-fake + ${project.version} + test + + + org.apache.seatunnel + seatunnel-connector-spark-console + ${project.version} + test + org.apache.seatunnel seatunnel-connector-spark-jdbc From 8b43e93551edfaeb9cb8d1a445540c6db1453dad Mon Sep 17 00:00:00 2001 From: Zongwen Li Date: Sat, 10 Sep 2022 12:39:29 +0800 Subject: [PATCH 21/44] [Improve][build] Change the way releases are packaged (#2703) * [build] Change the packing mode of seatunnel connectors * change the activation mode Use '!' can cause 'true' failed * simple starter code * fix starter jar output file name --- pom.xml | 10 + seatunnel-dist/pom.xml | 533 ++++++++++++++++-- .../src/main/assembly/assembly-bin-ci.xml | 181 +++--- .../src/main/assembly/assembly-bin.xml | 163 +++--- 4 files changed, 632 insertions(+), 255 deletions(-) diff --git a/pom.xml b/pom.xml index 1a904df22f1..f47e0a13125 100644 --- a/pom.xml +++ b/pom.xml @@ -95,6 +95,10 @@ all true + + release + false + seatunnel-connectors-v2 @@ -105,6 +109,12 @@ release + + + release + true + + diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 58c3b67c881..03c85791381 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -35,114 +35,535 @@ seatunnel-dist pom + + + + maven-assembly-plugin + + + bin + package + + single + + + + src/main/assembly/assembly-bin-ci.xml + + true + + + + src + package + + single + + + + src/main/assembly/assembly-src.xml + + true + + + + + + + apache-seatunnel-incubating-${project.version} + + - all + seatunnel true + + release + false + + org.apache.seatunnel - seatunnel-connectors-v2-dist + seatunnel-flink-starter ${project.version} + provided org.apache.seatunnel - seatunnel-connectors-spark-dist + seatunnel-spark-starter ${project.version} + provided + org.apache.seatunnel - seatunnel-connectors-flink-dist + connector-fake ${project.version} + provided - - + + org.apache.seatunnel + connector-console + ${project.version} + provided + + + org.apache.seatunnel + connector-assert + ${project.version} + provided + + + org.apache.seatunnel + connector-kafka + ${project.version} + provided + + + org.apache.seatunnel + connector-http-base + ${project.version} + provided + + + org.apache.seatunnel + connector-http-feishu + ${project.version} + provided + + + org.apache.seatunnel + connector-http-wechat + ${project.version} + provided + + + org.apache.seatunnel + connector-jdbc + ${project.version} + provided + + + org.apache.seatunnel + connector-socket + ${project.version} + provided + + + org.apache.seatunnel + connector-clickhouse + ${project.version} + provided + + + org.apache.seatunnel + connector-pulsar + ${project.version} + provided + + + org.apache.seatunnel + connector-hive + ${project.version} + provided + + + org.apache.seatunnel + connector-file-hadoop + ${project.version} + provided + + + org.apache.seatunnel + connector-file-local + ${project.version} + provided + + + org.apache.seatunnel + connector-file-oss + ${project.version} + provided + + + org.apache.seatunnel + connector-file-ftp + ${project.version} + provided + + + org.apache.seatunnel + connector-hudi + ${project.version} + provided + + + org.apache.seatunnel + connector-dingtalk + ${project.version} + provided + + + org.apache.seatunnel + connector-kudu + ${project.version} + provided + + + org.apache.seatunnel + connector-email + ${project.version} + provided + + + org.apache.seatunnel + connector-elasticsearch + ${project.version} + provided + + + org.apache.seatunnel + connector-iotdb + ${project.version} + provided + + + org.apache.seatunnel + connector-neo4j + ${project.version} + provided + + + org.apache.seatunnel + connector-redis + ${project.version} + provided + + + org.apache.seatunnel + connector-datahub + ${project.version} + provided + + + org.apache.seatunnel + connector-sentry + ${project.version} + provided + + + org.apache.seatunnel + connector-mongodb + ${project.version} + provided + + + + + flink-v1 + + true + + release + true + + + + org.apache.seatunnel seatunnel-core-flink ${project.version} + provided - + + + org.apache.seatunnel + seatunnel-connector-flink-kafka + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-console + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-jdbc + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-fake + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-socket + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-file + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-elasticsearch6 + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-elasticsearch7 + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-doris + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-clickhouse + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-http + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-flink-assert + ${project.version} + provided + + + + + flink-sql + + true + + release + true + + + + org.apache.seatunnel seatunnel-core-flink-sql ${project.version} + provided - + org.apache.seatunnel - seatunnel-core-spark + flink-sql-connector-jdbc ${project.version} + provided - org.apache.seatunnel - seatunnel-flink-starter + flink-sql-connector-kafka ${project.version} + provided - org.apache.seatunnel - seatunnel-spark-starter + flink-sql-connector-elasticsearch-6 + ${project.version} + provided + + + org.apache.seatunnel + flink-sql-connector-elasticsearch-7 ${project.version} + provided - - - - maven-assembly-plugin - - - bin - package - - single - - - - - src/main/assembly/assembly-bin-ci.xml - - true - - - - - src - package - - single - - - - src/main/assembly/assembly-src.xml - - true - - - - - - - apache-seatunnel-incubating-${project.version} - - release + spark-v1 + + true + + release + true + + + + + org.apache.seatunnel + seatunnel-core-spark + ${project.version} + provided + + + + org.apache.seatunnel + seatunnel-connector-spark-kafka + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-fake + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-file + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-socket + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-jdbc + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-elasticsearch + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-hive + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-phoenix + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-redis + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-mongodb + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-kudu + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-email + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-console + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-clickhouse + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-hbase + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-hudi + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-doris + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-tidb + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-neo4j + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-iceberg + ${project.version} + provided + + + org.apache.seatunnel + seatunnel-connector-spark-feishu + ${project.version} + provided + org.apache.seatunnel - seatunnel-connectors-spark-dist + seatunnel-connector-spark-http ${project.version} + provided org.apache.seatunnel - seatunnel-connectors-flink-dist + seatunnel-connector-spark-webhook ${project.version} + provided + + + release + + + release + true + + @@ -154,7 +575,6 @@ single - src/main/assembly/assembly-bin.xml @@ -176,7 +596,6 @@ true - diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index 1f15cbc08a7..2e016d3a3b7 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -43,16 +43,8 @@ plugins/** - - ../seatunnel-core/seatunnel-core-flink/target - - seatunnel-core-flink*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - + + ../seatunnel-core/seatunnel-core-flink/src/main/bin /bin @@ -63,131 +55,122 @@ /bin 0755 - - ../seatunnel-core/seatunnel-core-flink-sql/target - - seatunnel-core-flink-sql*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - ../seatunnel-core/seatunnel-core-spark/src/main/bin /bin 0755 - - ../seatunnel-core/seatunnel-core-spark/target - - seatunnel-core-spark*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - ../seatunnel-core/seatunnel-flink-starter/src/main/bin /bin 0755 - - ../seatunnel-core/seatunnel-flink-starter/target - - seatunnel-flink-starter*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - ../seatunnel-core/seatunnel-spark-starter/src/main/bin /bin 0755 + - ../seatunnel-core/seatunnel-spark-starter/target + ${project.build.directory}/bin + /bin - seatunnel-spark-starter*.jar + * - - %regex[.*((javadoc)|(sources))\.jar] - - /lib + 0755 - + - ../seatunnel-connectors/seatunnel-connectors-flink-dist/target/lib - - seatunnel-connector-flink*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /connectors/flink + release-docs + . + - ../seatunnel-connectors/seatunnel-connectors-flink-sql-dist/target/lib + ${basedir}/.././ - flink-sql-connector*.jar + DISCLAIMER - - %regex[.*((javadoc)|(sources))\.jar] - - /connectors/flink-sql + . - - ../seatunnel-connectors/seatunnel-connectors-spark-dist/target/lib + + + + + ../plugin-mapping.properties + /connectors + + + + + + false + true + false - seatunnel-connector-spark*.jar + + org.apache.seatunnel:seatunnel-flink-starter:jar + + org.apache.seatunnel:seatunnel-spark-starter:jar + + org.apache.seatunnel:seatunnel-core-flink:jar + + org.apache.seatunnel:seatunnel-core-flink-sql:jar + + org.apache.seatunnel:seatunnel-core-spark:jar - - %regex[.*((javadoc)|(sources))\.jar] - - /connectors/spark - - - ../seatunnel-connectors-v2-dist/target/lib + ${artifact.file.name} + /lib + provided + + + + + + false + true + false - connector-*.jar + org.apache.seatunnel:connector-*:jar - %regex[.*((javadoc)|(sources))\.jar] - connector-common*.jar + org.apache.seatunnel:connector-common + org.apache.seatunnel:connector-*-base /connectors/seatunnel - - - ../ + provided + + + + false + true + false - plugin-mapping.properties + org.apache.seatunnel:seatunnel-connector-flink-*:jar - /connectors - - - ${project.build.directory}/bin - /bin + /connectors/flink + provided + + + + false + true + false - * + org.apache.seatunnel:flink-sql-connector-*:jar - 0755 - - - - release-docs - . - - - - ${basedir}/.././ + /connectors/flink-sql + provided + + + + false + true + false - DISCLAIMER + org.apache.seatunnel:seatunnel-connector-spark-*:jar - . - - - + /connectors/spark + provided + + diff --git a/seatunnel-dist/src/main/assembly/assembly-bin.xml b/seatunnel-dist/src/main/assembly/assembly-bin.xml index d68a147da07..125e1a1765f 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin.xml @@ -43,16 +43,7 @@ plugins/** - - ../seatunnel-core/seatunnel-core-flink/target - - seatunnel-core-flink*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - + ../seatunnel-core/seatunnel-core-flink/src/main/bin /bin @@ -63,99 +54,12 @@ /bin 0755 - - ../seatunnel-core/seatunnel-core-flink-sql/target - - seatunnel-core-flink-sql*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - ../seatunnel-core/seatunnel-core-spark/src/main/bin /bin 0755 - - ../seatunnel-core/seatunnel-core-spark/target - - seatunnel-core-spark*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - - - ../seatunnel-core/seatunnel-flink-starter/src/main/bin - /bin - 0755 - - - ../seatunnel-core/seatunnel-flink-starter/target - - seatunnel-flink-starter*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - - - ../seatunnel-core/seatunnel-spark-starter/src/main/bin - /bin - 0755 - - - ../seatunnel-core/seatunnel-spark-starter/target - - seatunnel-spark-starter*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /lib - - - - ../seatunnel-connectors/seatunnel-connectors-flink-dist/target/lib - - seatunnel-connector-flink*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /connectors/flink - - - ../seatunnel-connectors/seatunnel-connectors-flink-sql-dist/target/lib - - flink-sql-connector*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /connectors/flink-sql - - - ../seatunnel-connectors/seatunnel-connectors-spark-dist/target/lib - - seatunnel-connector-spark*.jar - - - %regex[.*((javadoc)|(sources))\.jar] - - /connectors/spark - - - ../ - - plugin-mapping.properties - - /connectors - + ${project.build.directory}/bin /bin @@ -177,6 +81,67 @@ . - + + + + ../plugin-mapping.properties + /connectors + + + + + + + false + true + false + + + org.apache.seatunnel:seatunnel-core-flink:jar + + org.apache.seatunnel:seatunnel-core-flink-sql:jar + + org.apache.seatunnel:seatunnel-core-spark:jar + + ${artifact.file.name} + /lib + provided + + + + + + false + true + false + + org.apache.seatunnel:seatunnel-connector-flink-*:jar + + /connectors/flink + provided + + + + false + true + false + + org.apache.seatunnel:flink-sql-connector-*:jar + + /connectors/flink-sql + provided + + + + false + true + false + + org.apache.seatunnel:seatunnel-connector-spark-*:jar + + /connectors/spark + provided + + From 0e05db444d3aa7ab71d94bd3015c0ba86035a51b Mon Sep 17 00:00:00 2001 From: TyrantLucifer Date: Sat, 10 Sep 2022 17:04:20 +0800 Subject: [PATCH 22/44] [Hotfix][Connector-V2-e2e] Fix bash interpreter from sh to bash (#2710) --- .../src/main/bin/start-seatunnel-flink-new-connector.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh b/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh index 9dc9187d805..24f277d427a 100755 --- a/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh +++ b/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh @@ -1,4 +1,4 @@ -#!/bin/sh +#!/bin/bash # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with From a49673655b026b3d92e12d40f1a531870ae52992 Mon Sep 17 00:00:00 2001 From: songjianet <1778651752@qq.com> Date: Sat, 10 Sep 2022 20:35:20 +0800 Subject: [PATCH 23/44] [Feat][UI] Add themes in the project. (#2688) --- seatunnel-ui/package.json | 52 +- seatunnel-ui/pnpm-lock.yaml | 1393 +++++++++++----------- seatunnel-ui/src/themes/modules/light.ts | 16 +- 3 files changed, 734 insertions(+), 727 deletions(-) diff --git a/seatunnel-ui/package.json b/seatunnel-ui/package.json index cffb2094e90..afa6db692c1 100644 --- a/seatunnel-ui/package.json +++ b/seatunnel-ui/package.json @@ -9,46 +9,46 @@ "prettier": "prettier --write \"src/**/*.{vue,ts,tsx}\"" }, "dependencies": { - "@vueuse/core": "^8.7.5", - "autoprefixer": "^10.4.7", + "@vueuse/core": "^9.2.0", + "autoprefixer": "^10.4.8", "axios": "^0.27.2", - "date-fns": "^2.28.0", - "date-fns-tz": "^1.3.5", + "date-fns": "^2.29.2", + "date-fns-tz": "^1.3.7", "echarts": "^5.3.3", "lodash": "^4.17.21", - "monaco-editor": "^0.33.0", - "naive-ui": "^2.30.7", + "monaco-editor": "^0.34.0", + "naive-ui": "2.30.7", "nprogress": "^0.2.0", - "pinia": "^2.0.14", - "pinia-plugin-persistedstate": "^1.6.1", - "postcss": "^8.4.14", - "tailwindcss": "^3.1.6", + "pinia": "^2.0.22", + "pinia-plugin-persistedstate": "^2.1.1", + "postcss": "^8.4.16", + "tailwindcss": "^3.1.8", "vfonts": "^0.0.3", - "vue": "^3.2.37", - "vue-i18n": "^9.1.10", - "vue-router": "^4.0.16" + "vue": "^3.2.38", + "vue-i18n": "^9.2.2", + "vue-router": "^4.1.5" }, "devDependencies": { - "@types/lodash": "^4.14.182", - "@types/node": "^18.0.0", + "@types/lodash": "^4.14.184", + "@types/node": "^18.7.15", "@types/nprogress": "^0.2.0", - "@typescript-eslint/eslint-plugin": "^5.30.0", - "@typescript-eslint/parser": "^5.30.0", + "@typescript-eslint/eslint-plugin": "^5.36.2", + "@typescript-eslint/parser": "^5.36.2", "@vicons/antd": "^0.12.0", - "@vitejs/plugin-vue": "^2.3.3", - "@vitejs/plugin-vue-jsx": "^1.3.10", + "@vitejs/plugin-vue": "^3.1.0", + "@vitejs/plugin-vue-jsx": "^2.0.1", "dart-sass": "^1.25.0", - "eslint": "^8.18.0", + "eslint": "^8.23.0", "eslint-config-prettier": "^8.5.0", - "eslint-plugin-prettier": "^4.1.0", - "eslint-plugin-vue": "^9.1.1", + "eslint-plugin-prettier": "^4.2.1", + "eslint-plugin-vue": "^9.4.0", "prettier": "^2.7.1", - "sass": "^1.53.0", + "sass": "^1.54.8", "sass-loader": "^13.0.2", - "typescript": "^4.7.4", + "typescript": "^4.8.2", "typescript-plugin-css-modules": "^3.4.0", - "vite": "^2.9.13", + "vite": "^3.1.0", "vite-plugin-compression": "^0.5.1", - "vue-tsc": "^0.38.2" + "vue-tsc": "^0.40.9" } } diff --git a/seatunnel-ui/pnpm-lock.yaml b/seatunnel-ui/pnpm-lock.yaml index f91e0ec67ab..8ad56e5db3c 100644 --- a/seatunnel-ui/pnpm-lock.yaml +++ b/seatunnel-ui/pnpm-lock.yaml @@ -18,88 +18,88 @@ lockfileVersion: 5.4 specifiers: - '@types/lodash': ^4.14.182 - '@types/node': ^18.0.0 + '@types/lodash': ^4.14.184 + '@types/node': ^18.7.15 '@types/nprogress': ^0.2.0 - '@typescript-eslint/eslint-plugin': ^5.30.0 - '@typescript-eslint/parser': ^5.30.0 + '@typescript-eslint/eslint-plugin': ^5.36.2 + '@typescript-eslint/parser': ^5.36.2 '@vicons/antd': ^0.12.0 - '@vitejs/plugin-vue': ^2.3.3 - '@vitejs/plugin-vue-jsx': ^1.3.10 - '@vueuse/core': ^8.7.5 - autoprefixer: ^10.4.7 + '@vitejs/plugin-vue': ^3.1.0 + '@vitejs/plugin-vue-jsx': ^2.0.1 + '@vueuse/core': ^9.2.0 + autoprefixer: ^10.4.8 axios: ^0.27.2 dart-sass: ^1.25.0 - date-fns: ^2.28.0 - date-fns-tz: ^1.3.5 + date-fns: ^2.29.2 + date-fns-tz: ^1.3.7 echarts: ^5.3.3 - eslint: ^8.18.0 + eslint: ^8.23.0 eslint-config-prettier: ^8.5.0 - eslint-plugin-prettier: ^4.1.0 - eslint-plugin-vue: ^9.1.1 + eslint-plugin-prettier: ^4.2.1 + eslint-plugin-vue: ^9.4.0 lodash: ^4.17.21 - monaco-editor: ^0.33.0 - naive-ui: ^2.30.7 + monaco-editor: ^0.34.0 + naive-ui: 2.30.7 nprogress: ^0.2.0 - pinia: ^2.0.14 - pinia-plugin-persistedstate: ^1.6.1 - postcss: ^8.4.14 + pinia: ^2.0.22 + pinia-plugin-persistedstate: ^2.1.1 + postcss: ^8.4.16 prettier: ^2.7.1 - sass: ^1.53.0 + sass: ^1.54.8 sass-loader: ^13.0.2 - tailwindcss: ^3.1.6 - typescript: ^4.7.4 + tailwindcss: ^3.1.8 + typescript: ^4.8.2 typescript-plugin-css-modules: ^3.4.0 vfonts: ^0.0.3 - vite: ^2.9.13 + vite: ^3.1.0 vite-plugin-compression: ^0.5.1 - vue: ^3.2.37 - vue-i18n: ^9.1.10 - vue-router: ^4.0.16 - vue-tsc: ^0.38.2 + vue: ^3.2.38 + vue-i18n: ^9.2.2 + vue-router: ^4.1.5 + vue-tsc: ^0.40.9 dependencies: - '@vueuse/core': 8.7.5_vue@3.2.37 - autoprefixer: 10.4.7_postcss@8.4.14 + '@vueuse/core': 9.2.0_vue@3.2.38 + autoprefixer: 10.4.8_postcss@8.4.16 axios: 0.27.2 - date-fns: 2.28.0 - date-fns-tz: 1.3.5_date-fns@2.28.0 + date-fns: 2.29.2 + date-fns-tz: 1.3.7_date-fns@2.29.2 echarts: 5.3.3 lodash: 4.17.21 - monaco-editor: 0.33.0 - naive-ui: 2.30.7_vue@3.2.37 + monaco-editor: 0.34.0 + naive-ui: 2.30.7_vue@3.2.38 nprogress: 0.2.0 - pinia: 2.0.14_j6bzmzd4ujpabbp5objtwxyjp4 - pinia-plugin-persistedstate: 1.6.1_pinia@2.0.14 - postcss: 8.4.14 - tailwindcss: 3.1.6 + pinia: 2.0.22_pj7ch6rmow6odq73xb5hfvge3q + pinia-plugin-persistedstate: 2.1.1_pinia@2.0.22 + postcss: 8.4.16 + tailwindcss: 3.1.8 vfonts: 0.0.3 - vue: 3.2.37 - vue-i18n: 9.1.10_vue@3.2.37 - vue-router: 4.0.16_vue@3.2.37 + vue: 3.2.38 + vue-i18n: 9.2.2_vue@3.2.38 + vue-router: 4.1.5_vue@3.2.38 devDependencies: - '@types/lodash': 4.14.182 - '@types/node': 18.0.0 + '@types/lodash': 4.14.184 + '@types/node': 18.7.15 '@types/nprogress': 0.2.0 - '@typescript-eslint/eslint-plugin': 5.30.0_5mtqsiui4sk53pmkx7i7ue45wm - '@typescript-eslint/parser': 5.30.0_b5e7v2qnwxfo6hmiq56u52mz3e + '@typescript-eslint/eslint-plugin': 5.36.2_iurrlxgqcgk5svigzxakafpeuu + '@typescript-eslint/parser': 5.36.2_yqf6kl63nyoq5megxukfnom5rm '@vicons/antd': 0.12.0 - '@vitejs/plugin-vue': 2.3.3_vite@2.9.13+vue@3.2.37 - '@vitejs/plugin-vue-jsx': 1.3.10 + '@vitejs/plugin-vue': 3.1.0_vite@3.1.0+vue@3.2.38 + '@vitejs/plugin-vue-jsx': 2.0.1_vite@3.1.0+vue@3.2.38 dart-sass: 1.25.0 - eslint: 8.18.0 - eslint-config-prettier: 8.5.0_eslint@8.18.0 - eslint-plugin-prettier: 4.1.0_xu6ewijrtliw5q5lksq5uixwby - eslint-plugin-vue: 9.1.1_eslint@8.18.0 + eslint: 8.23.0 + eslint-config-prettier: 8.5.0_eslint@8.23.0 + eslint-plugin-prettier: 4.2.1_tgumt6uwl2md3n6uqnggd6wvce + eslint-plugin-vue: 9.4.0_eslint@8.23.0 prettier: 2.7.1 - sass: 1.53.0 - sass-loader: 13.0.2_sass@1.53.0 - typescript: 4.7.4 - typescript-plugin-css-modules: 3.4.0_typescript@4.7.4 - vite: 2.9.13_sass@1.53.0 - vite-plugin-compression: 0.5.1_vite@2.9.13 - vue-tsc: 0.38.2_typescript@4.7.4 + sass: 1.54.8 + sass-loader: 13.0.2_sass@1.54.8 + typescript: 4.8.2 + typescript-plugin-css-modules: 3.4.0_typescript@4.8.2 + vite: 3.1.0_sass@1.54.8 + vite-plugin-compression: 0.5.1_vite@3.1.0 + vue-tsc: 0.40.9_typescript@4.8.2 packages: @@ -108,7 +108,7 @@ packages: engines: {node: '>=6.0.0'} dependencies: '@jridgewell/gen-mapping': 0.1.1 - '@jridgewell/trace-mapping': 0.3.14 + '@jridgewell/trace-mapping': 0.3.15 dev: true /@babel/code-frame/7.18.6: @@ -118,25 +118,25 @@ packages: '@babel/highlight': 7.18.6 dev: true - /@babel/compat-data/7.18.6: - resolution: {integrity: sha512-tzulrgDT0QD6U7BJ4TKVk2SDDg7wlP39P9yAx1RfLy7vP/7rsDRlWVfbWxElslu56+r7QOhB2NSDsabYYruoZQ==} + /@babel/compat-data/7.19.0: + resolution: {integrity: sha512-y5rqgTTPTmaF5e2nVhOxw+Ur9HDJLsWb6U/KpgUzRZEdPfE6VOubXBKLdbcUTijzRptednSBDQbYZBOSqJxpJw==} engines: {node: '>=6.9.0'} dev: true - /@babel/core/7.18.6: - resolution: {integrity: sha512-cQbWBpxcbbs/IUredIPkHiAGULLV8iwgNRMFzvbhEXISp4f3rUUXE5+TIw6KwUWUR3DwyI6gmBRnmAtYaWehwQ==} + /@babel/core/7.19.0: + resolution: {integrity: sha512-reM4+U7B9ss148rh2n1Qs9ASS+w94irYXga7c2jaQv9RVzpS7Mv1a9rnYYwuDa45G+DkORt9g6An2k/V4d9LbQ==} engines: {node: '>=6.9.0'} dependencies: '@ampproject/remapping': 2.2.0 '@babel/code-frame': 7.18.6 - '@babel/generator': 7.18.7 - '@babel/helper-compilation-targets': 7.18.6_@babel+core@7.18.6 - '@babel/helper-module-transforms': 7.18.6 - '@babel/helpers': 7.18.6 - '@babel/parser': 7.18.6 - '@babel/template': 7.18.6 - '@babel/traverse': 7.18.6 - '@babel/types': 7.18.7 + '@babel/generator': 7.19.0 + '@babel/helper-compilation-targets': 7.19.0_@babel+core@7.19.0 + '@babel/helper-module-transforms': 7.19.0 + '@babel/helpers': 7.19.0 + '@babel/parser': 7.19.0 + '@babel/template': 7.18.10 + '@babel/traverse': 7.19.0 + '@babel/types': 7.19.0 convert-source-map: 1.8.0 debug: 4.3.4 gensync: 1.0.0-beta.2 @@ -146,11 +146,11 @@ packages: - supports-color dev: true - /@babel/generator/7.18.7: - resolution: {integrity: sha512-shck+7VLlY72a2w9c3zYWuE1pwOKEiQHV7GTUbSnhyl5eu3i04t30tBY82ZRWrDfo3gkakCFtevExnxbkf2a3A==} + /@babel/generator/7.19.0: + resolution: {integrity: sha512-S1ahxf1gZ2dpoiFgA+ohK9DIpz50bJ0CWs7Zlzb54Z4sG8qmdIrGrVqmy1sAtTVRb+9CU6U8VqT9L0Zj7hxHVg==} engines: {node: '>=6.9.0'} dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 '@jridgewell/gen-mapping': 0.3.2 jsesc: 2.5.2 dev: true @@ -159,86 +159,86 @@ packages: resolution: {integrity: sha512-duORpUiYrEpzKIop6iNbjnwKLAKnJ47csTyRACyEmWj0QdUrm5aqNJGHSSEQSUAvNW0ojX0dOmK9dZduvkfeXA==} engines: {node: '>=6.9.0'} dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 dev: true - /@babel/helper-compilation-targets/7.18.6_@babel+core@7.18.6: - resolution: {integrity: sha512-vFjbfhNCzqdeAtZflUFrG5YIFqGTqsctrtkZ1D/NB0mDW9TwW3GmmUepYY4G9wCET5rY5ugz4OGTcLd614IzQg==} + /@babel/helper-compilation-targets/7.19.0_@babel+core@7.19.0: + resolution: {integrity: sha512-Ai5bNWXIvwDvWM7njqsG3feMlL9hCVQsPYXodsZyLwshYkZVJt59Gftau4VrE8S9IT9asd2uSP1hG6wCNw+sXA==} engines: {node: '>=6.9.0'} peerDependencies: '@babel/core': ^7.0.0 dependencies: - '@babel/compat-data': 7.18.6 - '@babel/core': 7.18.6 + '@babel/compat-data': 7.19.0 + '@babel/core': 7.19.0 '@babel/helper-validator-option': 7.18.6 - browserslist: 4.21.1 + browserslist: 4.21.3 semver: 6.3.0 dev: true - /@babel/helper-create-class-features-plugin/7.18.6_@babel+core@7.18.6: - resolution: {integrity: sha512-YfDzdnoxHGV8CzqHGyCbFvXg5QESPFkXlHtvdCkesLjjVMT2Adxe4FGUR5ChIb3DxSaXO12iIOCWoXdsUVwnqw==} + /@babel/helper-create-class-features-plugin/7.19.0_@babel+core@7.19.0: + resolution: {integrity: sha512-NRz8DwF4jT3UfrmUoZjd0Uph9HQnP30t7Ash+weACcyNkiYTywpIjDBgReJMKgr+n86sn2nPVVmJ28Dm053Kqw==} engines: {node: '>=6.9.0'} peerDependencies: '@babel/core': ^7.0.0 dependencies: - '@babel/core': 7.18.6 + '@babel/core': 7.19.0 '@babel/helper-annotate-as-pure': 7.18.6 - '@babel/helper-environment-visitor': 7.18.6 - '@babel/helper-function-name': 7.18.6 - '@babel/helper-member-expression-to-functions': 7.18.6 + '@babel/helper-environment-visitor': 7.18.9 + '@babel/helper-function-name': 7.19.0 + '@babel/helper-member-expression-to-functions': 7.18.9 '@babel/helper-optimise-call-expression': 7.18.6 - '@babel/helper-replace-supers': 7.18.6 + '@babel/helper-replace-supers': 7.18.9 '@babel/helper-split-export-declaration': 7.18.6 transitivePeerDependencies: - supports-color dev: true - /@babel/helper-environment-visitor/7.18.6: - resolution: {integrity: sha512-8n6gSfn2baOY+qlp+VSzsosjCVGFqWKmDF0cCWOybh52Dw3SEyoWR1KrhMJASjLwIEkkAufZ0xvr+SxLHSpy2Q==} + /@babel/helper-environment-visitor/7.18.9: + resolution: {integrity: sha512-3r/aACDJ3fhQ/EVgFy0hpj8oHyHpQc+LPtJoY9SzTThAsStm4Ptegq92vqKoE3vD706ZVFWITnMnxucw+S9Ipg==} engines: {node: '>=6.9.0'} dev: true - /@babel/helper-function-name/7.18.6: - resolution: {integrity: sha512-0mWMxV1aC97dhjCah5U5Ua7668r5ZmSC2DLfH2EZnf9c3/dHZKiFa5pRLMH5tjSl471tY6496ZWk/kjNONBxhw==} + /@babel/helper-function-name/7.19.0: + resolution: {integrity: sha512-WAwHBINyrpqywkUH0nTnNgI5ina5TFn85HKS0pbPDfxFfhyR/aNQEn4hGi1P1JyT//I0t4OgXUlofzWILRvS5w==} engines: {node: '>=6.9.0'} dependencies: - '@babel/template': 7.18.6 - '@babel/types': 7.18.7 + '@babel/template': 7.18.10 + '@babel/types': 7.19.0 dev: true /@babel/helper-hoist-variables/7.18.6: resolution: {integrity: sha512-UlJQPkFqFULIcyW5sbzgbkxn2FKRgwWiRexcuaR8RNJRy8+LLveqPjwZV/bwrLZCN0eUHD/x8D0heK1ozuoo6Q==} engines: {node: '>=6.9.0'} dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 dev: true - /@babel/helper-member-expression-to-functions/7.18.6: - resolution: {integrity: sha512-CeHxqwwipekotzPDUuJOfIMtcIHBuc7WAzLmTYWctVigqS5RktNMQ5bEwQSuGewzYnCtTWa3BARXeiLxDTv+Ng==} + /@babel/helper-member-expression-to-functions/7.18.9: + resolution: {integrity: sha512-RxifAh2ZoVU67PyKIO4AMi1wTenGfMR/O/ae0CCRqwgBAt5v7xjdtRw7UoSbsreKrQn5t7r89eruK/9JjYHuDg==} engines: {node: '>=6.9.0'} dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 dev: true /@babel/helper-module-imports/7.18.6: resolution: {integrity: sha512-0NFvs3VkuSYbFi1x2Vd6tKrywq+z/cLeYC/RJNFrIX/30Bf5aiGYbtvGXolEktzJH8o5E5KJ3tT+nkxuuZFVlA==} engines: {node: '>=6.9.0'} dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 dev: true - /@babel/helper-module-transforms/7.18.6: - resolution: {integrity: sha512-L//phhB4al5uucwzlimruukHB3jRd5JGClwRMD/ROrVjXfLqovYnvQrK/JK36WYyVwGGO7OD3kMyVTjx+WVPhw==} + /@babel/helper-module-transforms/7.19.0: + resolution: {integrity: sha512-3HBZ377Fe14RbLIA+ac3sY4PTgpxHVkFrESaWhoI5PuyXPBBX8+C34qblV9G89ZtycGJCmCI/Ut+VUDK4bltNQ==} engines: {node: '>=6.9.0'} dependencies: - '@babel/helper-environment-visitor': 7.18.6 + '@babel/helper-environment-visitor': 7.18.9 '@babel/helper-module-imports': 7.18.6 '@babel/helper-simple-access': 7.18.6 '@babel/helper-split-export-declaration': 7.18.6 '@babel/helper-validator-identifier': 7.18.6 - '@babel/template': 7.18.6 - '@babel/traverse': 7.18.6 - '@babel/types': 7.18.7 + '@babel/template': 7.18.10 + '@babel/traverse': 7.19.0 + '@babel/types': 7.19.0 transitivePeerDependencies: - supports-color dev: true @@ -247,23 +247,23 @@ packages: resolution: {integrity: sha512-HP59oD9/fEHQkdcbgFCnbmgH5vIQTJbxh2yf+CdM89/glUNnuzr87Q8GIjGEnOktTROemO0Pe0iPAYbqZuOUiA==} engines: {node: '>=6.9.0'} dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 dev: true - /@babel/helper-plugin-utils/7.18.6: - resolution: {integrity: sha512-gvZnm1YAAxh13eJdkb9EWHBnF3eAub3XTLCZEehHT2kWxiKVRL64+ae5Y6Ivne0mVHmMYKT+xWgZO+gQhuLUBg==} + /@babel/helper-plugin-utils/7.19.0: + resolution: {integrity: sha512-40Ryx7I8mT+0gaNxm8JGTZFUITNqdLAgdg0hXzeVZxVD6nFsdhQvip6v8dqkRHzsz1VFpFAaOCHNn0vKBL7Czw==} engines: {node: '>=6.9.0'} dev: true - /@babel/helper-replace-supers/7.18.6: - resolution: {integrity: sha512-fTf7zoXnUGl9gF25fXCWE26t7Tvtyn6H4hkLSYhATwJvw2uYxd3aoXplMSe0g9XbwK7bmxNes7+FGO0rB/xC0g==} + /@babel/helper-replace-supers/7.18.9: + resolution: {integrity: sha512-dNsWibVI4lNT6HiuOIBr1oyxo40HvIVmbwPUm3XZ7wMh4k2WxrxTqZwSqw/eEmXDS9np0ey5M2bz9tBmO9c+YQ==} engines: {node: '>=6.9.0'} dependencies: - '@babel/helper-environment-visitor': 7.18.6 - '@babel/helper-member-expression-to-functions': 7.18.6 + '@babel/helper-environment-visitor': 7.18.9 + '@babel/helper-member-expression-to-functions': 7.18.9 '@babel/helper-optimise-call-expression': 7.18.6 - '@babel/traverse': 7.18.6 - '@babel/types': 7.18.7 + '@babel/traverse': 7.19.0 + '@babel/types': 7.19.0 transitivePeerDependencies: - supports-color dev: true @@ -272,16 +272,20 @@ packages: resolution: {integrity: sha512-iNpIgTgyAvDQpDj76POqg+YEt8fPxx3yaNBg3S30dxNKm2SWfYhD0TGrK/Eu9wHpUW63VQU894TsTg+GLbUa1g==} engines: {node: '>=6.9.0'} dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 dev: true /@babel/helper-split-export-declaration/7.18.6: resolution: {integrity: sha512-bde1etTx6ZyTmobl9LLMMQsaizFVZrquTEHOqKeQESMKo4PlObf+8+JA25ZsIpZhT/WEd39+vOdLXAFG/nELpA==} engines: {node: '>=6.9.0'} dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 dev: true + /@babel/helper-string-parser/7.18.10: + resolution: {integrity: sha512-XtIfWmeNY3i4t7t4D2t02q50HvqHybPqW2ki1kosnvWCwuCMeo81Jf0gwr85jy/neUdg5XDdeFE/80DXiO+njw==} + engines: {node: '>=6.9.0'} + /@babel/helper-validator-identifier/7.18.6: resolution: {integrity: sha512-MmetCkz9ej86nJQV+sFCxoGGrUbU3q02kgLciwkrt9QqEB7cP39oKEY0PakknEO0Gu20SskMRi+AYZ3b1TpN9g==} engines: {node: '>=6.9.0'} @@ -291,13 +295,13 @@ packages: engines: {node: '>=6.9.0'} dev: true - /@babel/helpers/7.18.6: - resolution: {integrity: sha512-vzSiiqbQOghPngUYt/zWGvK3LAsPhz55vc9XNN0xAl2gV4ieShI2OQli5duxWHD+72PZPTKAcfcZDE1Cwc5zsQ==} + /@babel/helpers/7.19.0: + resolution: {integrity: sha512-DRBCKGwIEdqY3+rPJgG/dKfQy9+08rHIAJx8q2p+HSWP87s2HCrQmaAMMyMll2kIXKCW0cO1RdQskx15Xakftg==} engines: {node: '>=6.9.0'} dependencies: - '@babel/template': 7.18.6 - '@babel/traverse': 7.18.6 - '@babel/types': 7.18.7 + '@babel/template': 7.18.10 + '@babel/traverse': 7.19.0 + '@babel/types': 7.19.0 transitivePeerDependencies: - supports-color dev: true @@ -311,118 +315,128 @@ packages: js-tokens: 4.0.0 dev: true - /@babel/parser/7.18.6: - resolution: {integrity: sha512-uQVSa9jJUe/G/304lXspfWVpKpK4euFLgGiMQFOCpM/bgcAdeoHwi/OQz23O9GK2osz26ZiXRRV9aV+Yl1O8tw==} + /@babel/parser/7.19.0: + resolution: {integrity: sha512-74bEXKX2h+8rrfQUfsBfuZZHzsEs6Eql4pqy/T4Nn6Y9wNPggQOqD6z6pn5Bl8ZfysKouFZT/UXEH94ummEeQw==} engines: {node: '>=6.0.0'} hasBin: true dependencies: - '@babel/types': 7.18.7 + '@babel/types': 7.19.0 - /@babel/plugin-syntax-import-meta/7.10.4_@babel+core@7.18.6: + /@babel/plugin-syntax-import-meta/7.10.4_@babel+core@7.19.0: resolution: {integrity: sha512-Yqfm+XDx0+Prh3VSeEQCPU81yC+JWZ2pDPFSS4ZdpfZhp4MkFMaDC1UqseovEKwSUpnIL7+vK+Clp7bfh0iD7g==} peerDependencies: '@babel/core': ^7.0.0-0 dependencies: - '@babel/core': 7.18.6 - '@babel/helper-plugin-utils': 7.18.6 + '@babel/core': 7.19.0 + '@babel/helper-plugin-utils': 7.19.0 dev: true - /@babel/plugin-syntax-jsx/7.18.6_@babel+core@7.18.6: + /@babel/plugin-syntax-jsx/7.18.6_@babel+core@7.19.0: resolution: {integrity: sha512-6mmljtAedFGTWu2p/8WIORGwy+61PLgOMPOdazc7YoJ9ZCWUyFy3A6CpPkRKLKD1ToAesxX8KGEViAiLo9N+7Q==} engines: {node: '>=6.9.0'} peerDependencies: '@babel/core': ^7.0.0-0 dependencies: - '@babel/core': 7.18.6 - '@babel/helper-plugin-utils': 7.18.6 + '@babel/core': 7.19.0 + '@babel/helper-plugin-utils': 7.19.0 dev: true - /@babel/plugin-syntax-typescript/7.18.6_@babel+core@7.18.6: + /@babel/plugin-syntax-typescript/7.18.6_@babel+core@7.19.0: resolution: {integrity: sha512-mAWAuq4rvOepWCBid55JuRNvpTNf2UGVgoz4JV0fXEKolsVZDzsa4NqCef758WZJj/GDu0gVGItjKFiClTAmZA==} engines: {node: '>=6.9.0'} peerDependencies: '@babel/core': ^7.0.0-0 dependencies: - '@babel/core': 7.18.6 - '@babel/helper-plugin-utils': 7.18.6 + '@babel/core': 7.19.0 + '@babel/helper-plugin-utils': 7.19.0 dev: true - /@babel/plugin-transform-typescript/7.18.6_@babel+core@7.18.6: - resolution: {integrity: sha512-ijHNhzIrLj5lQCnI6aaNVRtGVuUZhOXFLRVFs7lLrkXTHip4FKty5oAuQdk4tywG0/WjXmjTfQCWmuzrvFer1w==} + /@babel/plugin-transform-typescript/7.19.0_@babel+core@7.19.0: + resolution: {integrity: sha512-DOOIywxPpkQHXijXv+s9MDAyZcLp12oYRl3CMWZ6u7TjSoCBq/KqHR/nNFR3+i2xqheZxoF0H2XyL7B6xeSRuA==} engines: {node: '>=6.9.0'} peerDependencies: '@babel/core': ^7.0.0-0 dependencies: - '@babel/core': 7.18.6 - '@babel/helper-create-class-features-plugin': 7.18.6_@babel+core@7.18.6 - '@babel/helper-plugin-utils': 7.18.6 - '@babel/plugin-syntax-typescript': 7.18.6_@babel+core@7.18.6 + '@babel/core': 7.19.0 + '@babel/helper-create-class-features-plugin': 7.19.0_@babel+core@7.19.0 + '@babel/helper-plugin-utils': 7.19.0 + '@babel/plugin-syntax-typescript': 7.18.6_@babel+core@7.19.0 transitivePeerDependencies: - supports-color dev: true - /@babel/template/7.18.6: - resolution: {integrity: sha512-JoDWzPe+wgBsTTgdnIma3iHNFC7YVJoPssVBDjiHfNlyt4YcunDtcDOUmfVDfCK5MfdsaIoX9PkijPhjH3nYUw==} + /@babel/template/7.18.10: + resolution: {integrity: sha512-TI+rCtooWHr3QJ27kJxfjutghu44DLnasDMwpDqCXVTal9RLp3RSYNh4NdBrRP2cQAoG9A8juOQl6P6oZG4JxA==} engines: {node: '>=6.9.0'} dependencies: '@babel/code-frame': 7.18.6 - '@babel/parser': 7.18.6 - '@babel/types': 7.18.7 + '@babel/parser': 7.19.0 + '@babel/types': 7.19.0 dev: true - /@babel/traverse/7.18.6: - resolution: {integrity: sha512-zS/OKyqmD7lslOtFqbscH6gMLFYOfG1YPqCKfAW5KrTeolKqvB8UelR49Fpr6y93kYkW2Ik00mT1LOGiAGvizw==} + /@babel/traverse/7.19.0: + resolution: {integrity: sha512-4pKpFRDh+utd2mbRC8JLnlsMUii3PMHjpL6a0SZ4NMZy7YFP9aXORxEhdMVOc9CpWtDF09IkciQLEhK7Ml7gRA==} engines: {node: '>=6.9.0'} dependencies: '@babel/code-frame': 7.18.6 - '@babel/generator': 7.18.7 - '@babel/helper-environment-visitor': 7.18.6 - '@babel/helper-function-name': 7.18.6 + '@babel/generator': 7.19.0 + '@babel/helper-environment-visitor': 7.18.9 + '@babel/helper-function-name': 7.19.0 '@babel/helper-hoist-variables': 7.18.6 '@babel/helper-split-export-declaration': 7.18.6 - '@babel/parser': 7.18.6 - '@babel/types': 7.18.7 + '@babel/parser': 7.19.0 + '@babel/types': 7.19.0 debug: 4.3.4 globals: 11.12.0 transitivePeerDependencies: - supports-color dev: true - /@babel/types/7.18.7: - resolution: {integrity: sha512-QG3yxTcTIBoAcQmkCs+wAPYZhu7Dk9rXKacINfNbdJDNERTbLQbHGyVG8q/YGMPeCJRIhSY0+fTc5+xuh6WPSQ==} + /@babel/types/7.19.0: + resolution: {integrity: sha512-YuGopBq3ke25BVSiS6fgF49Ul9gH1x70Bcr6bqRLjWCkcX8Hre1/5+z+IiWOIerRMSSEfGZVB9z9kyq7wVs9YA==} engines: {node: '>=6.9.0'} dependencies: + '@babel/helper-string-parser': 7.18.10 '@babel/helper-validator-identifier': 7.18.6 to-fast-properties: 2.0.0 - /@css-render/plugin-bem/0.15.10_css-render@0.15.10: - resolution: {integrity: sha512-V7b08sM2PWJlXI7BJiVIa0Sg30H3u/jHay4AclNXfF2yRFwwb4ZJjggsMfzwj3WSihAdNf2WTqvOU5qsOD80Dg==} + /@css-render/plugin-bem/0.15.11_css-render@0.15.11: + resolution: {integrity: sha512-Bn8qadYPIz5DhZ4obTGHOJzeziQH6kY0+Fk5AEvwuuy378SLwwvXuuoechLjBHcgKkPCM03Oo4dDSGP/6NMdyw==} peerDependencies: - css-render: ~0.15.10 + css-render: ~0.15.11 dependencies: - css-render: 0.15.10 + css-render: 0.15.11 dev: false - /@css-render/vue3-ssr/0.15.10_vue@3.2.37: - resolution: {integrity: sha512-keGKnkB2nyVGoA8GezMKNsmvTGXEzgLOGGlgshwOTSEzd1dsROyZ2m/khJ9jV5zbzDM4rWeAWbWF0zwHemsJcw==} + /@css-render/vue3-ssr/0.15.11_vue@3.2.38: + resolution: {integrity: sha512-n+SuqLPbY30FUTM8slX75OaEG+c8XlTOFrAklekX2XQGvBbz9XdBE6hTEgGlV5kPcTMqTJeCG7Vzhs9/29VC7w==} peerDependencies: vue: ^3.0.11 dependencies: - vue: 3.2.37 + vue: 3.2.38 dev: false /@emotion/hash/0.8.0: resolution: {integrity: sha512-kBJtf7PH6aWwZ6fka3zQ0p6SBYzx4fl1LoZXE2RrnYST9Xljm7WfKJrU4g/Xr3Beg72MLrp1AWNUmuYJTL7Cow==} dev: false - /@eslint/eslintrc/1.3.0: - resolution: {integrity: sha512-UWW0TMTmk2d7hLcWD1/e2g5HDM/HQ3csaLSqXCfqwh4uNDuNqlaKWXmEsL4Cs41Z0KnILNvwbHAah3C2yt06kw==} + /@esbuild/linux-loong64/0.15.7: + resolution: {integrity: sha512-IKznSJOsVUuyt7cDzzSZyqBEcZe+7WlBqTVXiF1OXP/4Nm387ToaXZ0fyLwI1iBlI/bzpxVq411QE2/Bt2XWWw==} + engines: {node: '>=12'} + cpu: [loong64] + os: [linux] + requiresBuild: true + dev: true + optional: true + + /@eslint/eslintrc/1.3.1: + resolution: {integrity: sha512-OhSY22oQQdw3zgPOOwdoj01l/Dzl1Z+xyUP33tkSN+aqyEhymJCcPHyXt+ylW8FSe0TfRC2VG+ROQOapD0aZSQ==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dependencies: ajv: 6.12.6 debug: 4.3.4 - espree: 9.3.2 - globals: 13.15.0 + espree: 9.4.0 + globals: 13.17.0 ignore: 5.2.0 import-fresh: 3.3.0 js-yaml: 4.1.0 @@ -432,8 +446,8 @@ packages: - supports-color dev: true - /@humanwhocodes/config-array/0.9.5: - resolution: {integrity: sha512-ObyMyWxZiCu/yTisA7uzx81s40xR2fD5Cg/2Kq7G02ajkNubJf6BopgDTmDyc3U7sXpNKM8cYOw7s7Tyr+DnCw==} + /@humanwhocodes/config-array/0.10.4: + resolution: {integrity: sha512-mXAIHxZT3Vcpg83opl1wGlVZ9xydbfZO3r5YfRSH6Gpp2J/PfdBP0wbDa2sO6/qRbcalpoevVyW6A/fI6LfeMw==} engines: {node: '>=10.10.0'} dependencies: '@humanwhocodes/object-schema': 1.2.1 @@ -443,64 +457,55 @@ packages: - supports-color dev: true + /@humanwhocodes/gitignore-to-minimatch/1.0.2: + resolution: {integrity: sha512-rSqmMJDdLFUsyxR6FMtD00nfQKKLFb1kv+qBbOVKqErvloEIJLo5bDTJTQNTYgeyp78JsA7u/NPi5jT1GR/MuA==} + dev: true + + /@humanwhocodes/module-importer/1.0.1: + resolution: {integrity: sha512-bxveV4V8v5Yb4ncFTT3rPSgZBOpCkjfK0y4oVVVJwIuDVBRMDXrPyXRL988i5ap9m9bnyEEjWfm5WkBmtffLfA==} + engines: {node: '>=12.22'} + dev: true + /@humanwhocodes/object-schema/1.2.1: resolution: {integrity: sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==} dev: true - /@intlify/core-base/9.1.10: - resolution: {integrity: sha512-So9CNUavB/IsZ+zBmk2Cv6McQp6vc2wbGi1S0XQmJ8Vz+UFcNn9MFXAe9gY67PreIHrbLsLxDD0cwo1qsxM1Nw==} - engines: {node: '>= 10'} + /@intlify/core-base/9.2.2: + resolution: {integrity: sha512-JjUpQtNfn+joMbrXvpR4hTF8iJQ2sEFzzK3KIESOx+f+uwIjgw20igOyaIdhfsVVBCds8ZM64MoeNSx+PHQMkA==} + engines: {node: '>= 14'} dependencies: - '@intlify/devtools-if': 9.1.10 - '@intlify/message-compiler': 9.1.10 - '@intlify/message-resolver': 9.1.10 - '@intlify/runtime': 9.1.10 - '@intlify/shared': 9.1.10 - '@intlify/vue-devtools': 9.1.10 + '@intlify/devtools-if': 9.2.2 + '@intlify/message-compiler': 9.2.2 + '@intlify/shared': 9.2.2 + '@intlify/vue-devtools': 9.2.2 dev: false - /@intlify/devtools-if/9.1.10: - resolution: {integrity: sha512-SHaKoYu6sog3+Q8js1y3oXLywuogbH1sKuc7NSYkN3GElvXSBaMoCzW+we0ZSFqj/6c7vTNLg9nQ6rxhKqYwnQ==} - engines: {node: '>= 10'} + /@intlify/devtools-if/9.2.2: + resolution: {integrity: sha512-4ttr/FNO29w+kBbU7HZ/U0Lzuh2cRDhP8UlWOtV9ERcjHzuyXVZmjyleESK6eVP60tGC9QtQW9yZE+JeRhDHkg==} + engines: {node: '>= 14'} dependencies: - '@intlify/shared': 9.1.10 + '@intlify/shared': 9.2.2 dev: false - /@intlify/message-compiler/9.1.10: - resolution: {integrity: sha512-+JiJpXff/XTb0EadYwdxOyRTB0hXNd4n1HaJ/a4yuV960uRmPXaklJsedW0LNdcptd/hYUZtCkI7Lc9J5C1gxg==} - engines: {node: '>= 10'} + /@intlify/message-compiler/9.2.2: + resolution: {integrity: sha512-IUrQW7byAKN2fMBe8z6sK6riG1pue95e5jfokn8hA5Q3Bqy4MBJ5lJAofUsawQJYHeoPJ7svMDyBaVJ4d0GTtA==} + engines: {node: '>= 14'} dependencies: - '@intlify/message-resolver': 9.1.10 - '@intlify/shared': 9.1.10 + '@intlify/shared': 9.2.2 source-map: 0.6.1 dev: false - /@intlify/message-resolver/9.1.10: - resolution: {integrity: sha512-5YixMG/M05m0cn9+gOzd4EZQTFRUu8RGhzxJbR1DWN21x/Z3bJ8QpDYj6hC4FwBj5uKsRfKpJQ3Xqg98KWoA+w==} - engines: {node: '>= 10'} - dev: false - - /@intlify/runtime/9.1.10: - resolution: {integrity: sha512-7QsuByNzpe3Gfmhwq6hzgXcMPpxz8Zxb/XFI6s9lQdPLPe5Lgw4U1ovRPZTOs6Y2hwitR3j/HD8BJNGWpJnOFA==} - engines: {node: '>= 10'} - dependencies: - '@intlify/message-compiler': 9.1.10 - '@intlify/message-resolver': 9.1.10 - '@intlify/shared': 9.1.10 - dev: false - - /@intlify/shared/9.1.10: - resolution: {integrity: sha512-Om54xJeo1Vw+K1+wHYyXngE8cAbrxZHpWjYzMR9wCkqbhGtRV5VLhVc214Ze2YatPrWlS2WSMOWXR8JktX/IgA==} - engines: {node: '>= 10'} + /@intlify/shared/9.2.2: + resolution: {integrity: sha512-wRwTpsslgZS5HNyM7uDQYZtxnbI12aGiBZURX3BTR9RFIKKRWpllTsgzHWvj3HKm3Y2Sh5LPC1r0PDCKEhVn9Q==} + engines: {node: '>= 14'} dev: false - /@intlify/vue-devtools/9.1.10: - resolution: {integrity: sha512-5l3qYARVbkWAkagLu1XbDUWRJSL8br1Dj60wgMaKB0+HswVsrR6LloYZTg7ozyvM621V6+zsmwzbQxbVQyrytQ==} - engines: {node: '>= 10'} + /@intlify/vue-devtools/9.2.2: + resolution: {integrity: sha512-+dUyqyCHWHb/UcvY1MlIpO87munedm3Gn6E9WWYdWrMuYLcoIoOEVDWSS8xSwtlPU+kA+MEQTP6Q1iI/ocusJg==} + engines: {node: '>= 14'} dependencies: - '@intlify/message-resolver': 9.1.10 - '@intlify/runtime': 9.1.10 - '@intlify/shared': 9.1.10 + '@intlify/core-base': 9.2.2 + '@intlify/shared': 9.2.2 dev: false /@jridgewell/gen-mapping/0.1.1: @@ -517,11 +522,11 @@ packages: dependencies: '@jridgewell/set-array': 1.1.2 '@jridgewell/sourcemap-codec': 1.4.14 - '@jridgewell/trace-mapping': 0.3.14 + '@jridgewell/trace-mapping': 0.3.15 dev: true - /@jridgewell/resolve-uri/3.0.8: - resolution: {integrity: sha512-YK5G9LaddzGbcucK4c8h5tWFmMPBvRZ/uyWmN1/SbBdIvqGUdWGkJ5BAaccgs6XbzVLsqbPJrBSFwKv3kT9i7w==} + /@jridgewell/resolve-uri/3.1.0: + resolution: {integrity: sha512-F2msla3tad+Mfht5cJq7LSXcdudKTWCVYUgw6pLFOOHSTtZlj6SWNYAp+AhuqLmWdBO2X5hPrLcu8cVP8fy28w==} engines: {node: '>=6.0.0'} dev: true @@ -534,15 +539,15 @@ packages: resolution: {integrity: sha512-XPSJHWmi394fuUuzDnGz1wiKqWfo1yXecHQMRf2l6hztTO+nPru658AyDngaBe7isIxEkRsPR3FZh+s7iVa4Uw==} dev: true - /@jridgewell/trace-mapping/0.3.14: - resolution: {integrity: sha512-bJWEfQ9lPTvm3SneWwRFVLzrh6nhjwqw7TUFFBEMzwvg7t7PCDenf2lDwqo4NQXzdpgBXyFgDWnQA+2vkruksQ==} + /@jridgewell/trace-mapping/0.3.15: + resolution: {integrity: sha512-oWZNOULl+UbhsgB51uuZzglikfIKSUBO/M9W2OfEjn7cmqoAiCgmv9lyACTUacZwBz0ITnJ2NqjU8Tx0DHL88g==} dependencies: - '@jridgewell/resolve-uri': 3.0.8 + '@jridgewell/resolve-uri': 3.1.0 '@jridgewell/sourcemap-codec': 1.4.14 dev: true - /@juggle/resize-observer/3.3.1: - resolution: {integrity: sha512-zMM9Ds+SawiUkakS7y94Ymqx+S0ORzpG3frZirN3l+UlXUmSUR7hF4wxCVqW+ei94JzV5kt0uXBcoOEAuiydrw==} + /@juggle/resize-observer/3.4.0: + resolution: {integrity: sha512-dfLbk+PwWvFzSxwk3n5ySL0hfBog779o8h68wK/7/APo/7cgyWp5jcXockbxdk5kFRkbeXWm4Fbi9FrdN381sA==} dev: false /@nodelib/fs.scandir/2.1.5: @@ -563,21 +568,6 @@ packages: '@nodelib/fs.scandir': 2.1.5 fastq: 1.13.0 - /@rollup/pluginutils/4.2.1: - resolution: {integrity: sha512-iKnFXr7NkdZAIHiIWE+BX5ULi/ucVFYWD6TbAV+rZctiRTY2PL6tsIKhoIOaoskiWAkgu+VsbXgUVDNLHf+InQ==} - engines: {node: '>= 8.0.0'} - dependencies: - estree-walker: 2.0.2 - picomatch: 2.3.1 - dev: true - - /@types/jest/27.5.2: - resolution: {integrity: sha512-mpT8LJJ4CMeeahobofYWIjFo0xonRS/HfxnVEPMPFSQdGUt1uHCnoPT7Zhb+sjDU2wz0oKV0OLUR0WzrHNgfeA==} - dependencies: - jest-matcher-utils: 27.5.1 - pretty-format: 27.5.1 - dev: false - /@types/json-schema/7.0.11: resolution: {integrity: sha512-wOuvG1SN4Us4rez+tylwwwCV1psiNVOkJeM3AUWUNWg/jDQY2+HE/444y5gc+jBmRqASOm2Oeh5c1axHobwRKQ==} dev: true @@ -589,30 +579,30 @@ packages: /@types/lodash-es/4.17.6: resolution: {integrity: sha512-R+zTeVUKDdfoRxpAryaQNRKk3105Rrgx2CFRClIgRGaqDTdjsm8h6IYA8ir584W3ePzkZfst5xIgDwYrlh9HLg==} dependencies: - '@types/lodash': 4.14.182 + '@types/lodash': 4.14.184 dev: false - /@types/lodash/4.14.182: - resolution: {integrity: sha512-/THyiqyQAP9AfARo4pF+aCGcyiQ94tX/Is2I7HofNRqoYLgN1PBoOWu2/zTA5zMxzP5EFutMtWtGAFRKUe961Q==} + /@types/lodash/4.14.184: + resolution: {integrity: sha512-RoZphVtHbxPZizt4IcILciSWiC6dcn+eZ8oX9IWEYfDMcocdd42f7NPI6fQj+6zI8y4E0L7gu2pcZKLGTRaV9Q==} /@types/node/17.0.45: resolution: {integrity: sha512-w+tIMs3rq2afQdsPJlODhoUEKzFP1ayaoyl1CcnwtIlsVe7K7bA1NGm4s3PraqTLlXnbIN84zuBlxBWo1u9BLw==} dev: false - /@types/node/18.0.0: - resolution: {integrity: sha512-cHlGmko4gWLVI27cGJntjs/Sj8th9aYwplmZFwmmgYQQvL5NUsgVJG7OddLvNfLqYS31KFN0s3qlaD9qCaxACA==} + /@types/node/18.7.15: + resolution: {integrity: sha512-XnjpaI8Bgc3eBag2Aw4t2Uj/49lLBSStHWfqKvIuXD7FIrZyMLWp8KuAFHAqxMZYTF9l08N1ctUn9YNybZJVmQ==} dev: true /@types/nprogress/0.2.0: resolution: {integrity: sha512-1cYJrqq9GezNFPsWTZpFut/d4CjpZqA0vhqDUPFWYKF1oIyBz5qnoYMzR+0C/T96t3ebLAC1SSnwrVOm5/j74A==} dev: true - /@types/web-bluetooth/0.0.14: - resolution: {integrity: sha512-5d2RhCard1nQUC3aHcq/gHzWYO6K0WJmAbjO7mQJgCQKtZpgXxv1rOM6O/dBDhDYYVutk1sciOgNSe+5YyfM8A==} + /@types/web-bluetooth/0.0.15: + resolution: {integrity: sha512-w7hEHXnPMEZ+4nGKl/KDRVpxkwYxYExuHOYXyzIzCDzEZ9ZCGMAewulr9IqJu2LR4N37fcnb1XVeuZ09qgOxhA==} dev: false - /@typescript-eslint/eslint-plugin/5.30.0_5mtqsiui4sk53pmkx7i7ue45wm: - resolution: {integrity: sha512-lvhRJ2pGe2V9MEU46ELTdiHgiAFZPKtLhiU5wlnaYpMc2+c1R8fh8i80ZAa665drvjHKUJyRRGg3gEm1If54ow==} + /@typescript-eslint/eslint-plugin/5.36.2_iurrlxgqcgk5svigzxakafpeuu: + resolution: {integrity: sha512-OwwR8LRwSnI98tdc2z7mJYgY60gf7I9ZfGjN5EjCwwns9bdTuQfAXcsjSB2wSQ/TVNYSGKf4kzVXbNGaZvwiXw==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} peerDependencies: '@typescript-eslint/parser': ^5.0.0 @@ -622,24 +612,24 @@ packages: typescript: optional: true dependencies: - '@typescript-eslint/parser': 5.30.0_b5e7v2qnwxfo6hmiq56u52mz3e - '@typescript-eslint/scope-manager': 5.30.0 - '@typescript-eslint/type-utils': 5.30.0_b5e7v2qnwxfo6hmiq56u52mz3e - '@typescript-eslint/utils': 5.30.0_b5e7v2qnwxfo6hmiq56u52mz3e + '@typescript-eslint/parser': 5.36.2_yqf6kl63nyoq5megxukfnom5rm + '@typescript-eslint/scope-manager': 5.36.2 + '@typescript-eslint/type-utils': 5.36.2_yqf6kl63nyoq5megxukfnom5rm + '@typescript-eslint/utils': 5.36.2_yqf6kl63nyoq5megxukfnom5rm debug: 4.3.4 - eslint: 8.18.0 + eslint: 8.23.0 functional-red-black-tree: 1.0.1 ignore: 5.2.0 regexpp: 3.2.0 semver: 7.3.7 - tsutils: 3.21.0_typescript@4.7.4 - typescript: 4.7.4 + tsutils: 3.21.0_typescript@4.8.2 + typescript: 4.8.2 transitivePeerDependencies: - supports-color dev: true - /@typescript-eslint/parser/5.30.0_b5e7v2qnwxfo6hmiq56u52mz3e: - resolution: {integrity: sha512-2oYYUws5o2liX6SrFQ5RB88+PuRymaM2EU02/9Ppoyu70vllPnHVO7ioxDdq/ypXHA277R04SVjxvwI8HmZpzA==} + /@typescript-eslint/parser/5.36.2_yqf6kl63nyoq5megxukfnom5rm: + resolution: {integrity: sha512-qS/Kb0yzy8sR0idFspI9Z6+t7mqk/oRjnAYfewG+VN73opAUvmYL3oPIMmgOX6CnQS6gmVIXGshlb5RY/R22pA==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} peerDependencies: eslint: ^6.0.0 || ^7.0.0 || ^8.0.0 @@ -648,26 +638,26 @@ packages: typescript: optional: true dependencies: - '@typescript-eslint/scope-manager': 5.30.0 - '@typescript-eslint/types': 5.30.0 - '@typescript-eslint/typescript-estree': 5.30.0_typescript@4.7.4 + '@typescript-eslint/scope-manager': 5.36.2 + '@typescript-eslint/types': 5.36.2 + '@typescript-eslint/typescript-estree': 5.36.2_typescript@4.8.2 debug: 4.3.4 - eslint: 8.18.0 - typescript: 4.7.4 + eslint: 8.23.0 + typescript: 4.8.2 transitivePeerDependencies: - supports-color dev: true - /@typescript-eslint/scope-manager/5.30.0: - resolution: {integrity: sha512-3TZxvlQcK5fhTBw5solQucWSJvonXf5yua5nx8OqK94hxdrT7/6W3/CS42MLd/f1BmlmmbGEgQcTHHCktUX5bQ==} + /@typescript-eslint/scope-manager/5.36.2: + resolution: {integrity: sha512-cNNP51L8SkIFSfce8B1NSUBTJTu2Ts4nWeWbFrdaqjmn9yKrAaJUBHkyTZc0cL06OFHpb+JZq5AUHROS398Orw==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dependencies: - '@typescript-eslint/types': 5.30.0 - '@typescript-eslint/visitor-keys': 5.30.0 + '@typescript-eslint/types': 5.36.2 + '@typescript-eslint/visitor-keys': 5.36.2 dev: true - /@typescript-eslint/type-utils/5.30.0_b5e7v2qnwxfo6hmiq56u52mz3e: - resolution: {integrity: sha512-GF8JZbZqSS+azehzlv/lmQQ3EU3VfWYzCczdZjJRxSEeXDQkqFhCBgFhallLDbPwQOEQ4MHpiPfkjKk7zlmeNg==} + /@typescript-eslint/type-utils/5.36.2_yqf6kl63nyoq5megxukfnom5rm: + resolution: {integrity: sha512-rPQtS5rfijUWLouhy6UmyNquKDPhQjKsaKH0WnY6hl/07lasj8gPaH2UD8xWkePn6SC+jW2i9c2DZVDnL+Dokw==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} peerDependencies: eslint: '*' @@ -676,22 +666,23 @@ packages: typescript: optional: true dependencies: - '@typescript-eslint/utils': 5.30.0_b5e7v2qnwxfo6hmiq56u52mz3e + '@typescript-eslint/typescript-estree': 5.36.2_typescript@4.8.2 + '@typescript-eslint/utils': 5.36.2_yqf6kl63nyoq5megxukfnom5rm debug: 4.3.4 - eslint: 8.18.0 - tsutils: 3.21.0_typescript@4.7.4 - typescript: 4.7.4 + eslint: 8.23.0 + tsutils: 3.21.0_typescript@4.8.2 + typescript: 4.8.2 transitivePeerDependencies: - supports-color dev: true - /@typescript-eslint/types/5.30.0: - resolution: {integrity: sha512-vfqcBrsRNWw/LBXyncMF/KrUTYYzzygCSsVqlZ1qGu1QtGs6vMkt3US0VNSQ05grXi5Yadp3qv5XZdYLjpp8ag==} + /@typescript-eslint/types/5.36.2: + resolution: {integrity: sha512-9OJSvvwuF1L5eS2EQgFUbECb99F0mwq501w0H0EkYULkhFa19Qq7WFbycdw1PexAc929asupbZcgjVIe6OK/XQ==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dev: true - /@typescript-eslint/typescript-estree/5.30.0_typescript@4.7.4: - resolution: {integrity: sha512-hDEawogreZB4n1zoqcrrtg/wPyyiCxmhPLpZ6kmWfKF5M5G0clRLaEexpuWr31fZ42F96SlD/5xCt1bT5Qm4Nw==} + /@typescript-eslint/typescript-estree/5.36.2_typescript@4.8.2: + resolution: {integrity: sha512-8fyH+RfbKc0mTspfuEjlfqA4YywcwQK2Amcf6TDOwaRLg7Vwdu4bZzyvBZp4bjt1RRjQ5MDnOZahxMrt2l5v9w==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} peerDependencies: typescript: '*' @@ -699,41 +690,41 @@ packages: typescript: optional: true dependencies: - '@typescript-eslint/types': 5.30.0 - '@typescript-eslint/visitor-keys': 5.30.0 + '@typescript-eslint/types': 5.36.2 + '@typescript-eslint/visitor-keys': 5.36.2 debug: 4.3.4 globby: 11.1.0 is-glob: 4.0.3 semver: 7.3.7 - tsutils: 3.21.0_typescript@4.7.4 - typescript: 4.7.4 + tsutils: 3.21.0_typescript@4.8.2 + typescript: 4.8.2 transitivePeerDependencies: - supports-color dev: true - /@typescript-eslint/utils/5.30.0_b5e7v2qnwxfo6hmiq56u52mz3e: - resolution: {integrity: sha512-0bIgOgZflLKIcZsWvfklsaQTM3ZUbmtH0rJ1hKyV3raoUYyeZwcjQ8ZUJTzS7KnhNcsVT1Rxs7zeeMHEhGlltw==} + /@typescript-eslint/utils/5.36.2_yqf6kl63nyoq5megxukfnom5rm: + resolution: {integrity: sha512-uNcopWonEITX96v9pefk9DC1bWMdkweeSsewJ6GeC7L6j2t0SJywisgkr9wUTtXk90fi2Eljj90HSHm3OGdGRg==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} peerDependencies: eslint: ^6.0.0 || ^7.0.0 || ^8.0.0 dependencies: '@types/json-schema': 7.0.11 - '@typescript-eslint/scope-manager': 5.30.0 - '@typescript-eslint/types': 5.30.0 - '@typescript-eslint/typescript-estree': 5.30.0_typescript@4.7.4 - eslint: 8.18.0 + '@typescript-eslint/scope-manager': 5.36.2 + '@typescript-eslint/types': 5.36.2 + '@typescript-eslint/typescript-estree': 5.36.2_typescript@4.8.2 + eslint: 8.23.0 eslint-scope: 5.1.1 - eslint-utils: 3.0.0_eslint@8.18.0 + eslint-utils: 3.0.0_eslint@8.23.0 transitivePeerDependencies: - supports-color - typescript dev: true - /@typescript-eslint/visitor-keys/5.30.0: - resolution: {integrity: sha512-6WcIeRk2DQ3pHKxU1Ni0qMXJkjO/zLjBymlYBy/53qxe7yjEFSvzKLDToJjURUhSl2Fzhkl4SMXQoETauF74cw==} + /@typescript-eslint/visitor-keys/5.36.2: + resolution: {integrity: sha512-BtRvSR6dEdrNt7Net2/XDjbYKU5Ml6GqJgVfXT0CxTCJlnIqK7rAGreuWKMT2t8cFUT2Msv5oxw0GMRD7T5J7A==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dependencies: - '@typescript-eslint/types': 5.30.0 + '@typescript-eslint/types': 5.36.2 eslint-visitor-keys: 3.3.0 dev: true @@ -741,73 +732,84 @@ packages: resolution: {integrity: sha512-C0p6aO1EmGG1QHrqgUWQS1No20934OdWSRQshM5NIDK5H1On6tC26U0hT6Rmp40KfUsvhvX5YW8BoWJdNFifPg==} dev: true - /@vitejs/plugin-vue-jsx/1.3.10: - resolution: {integrity: sha512-Cf5zznh4yNMiEMBfTOztaDVDmK1XXfgxClzOSUVUc8WAmHzogrCUeM8B05ABzuGtg0D1amfng+mUmSIOFGP3Pw==} - engines: {node: '>=12.0.0'} + /@vitejs/plugin-vue-jsx/2.0.1_vite@3.1.0+vue@3.2.38: + resolution: {integrity: sha512-lmiR1k9+lrF7LMczO0pxtQ8mOn6XeppJDHxnpxkJQpT5SiKz4SKhKdeNstXaTNuR8qZhUo5X0pJlcocn72Y4Jg==} + engines: {node: ^14.18.0 || >=16.0.0} + peerDependencies: + vite: ^3.0.0 + vue: ^3.0.0 dependencies: - '@babel/core': 7.18.6 - '@babel/plugin-syntax-import-meta': 7.10.4_@babel+core@7.18.6 - '@babel/plugin-transform-typescript': 7.18.6_@babel+core@7.18.6 - '@rollup/pluginutils': 4.2.1 - '@vue/babel-plugin-jsx': 1.1.1_@babel+core@7.18.6 - hash-sum: 2.0.0 + '@babel/core': 7.19.0 + '@babel/plugin-syntax-import-meta': 7.10.4_@babel+core@7.19.0 + '@babel/plugin-transform-typescript': 7.19.0_@babel+core@7.19.0 + '@vue/babel-plugin-jsx': 1.1.1_@babel+core@7.19.0 + vite: 3.1.0_sass@1.54.8 + vue: 3.2.38 transitivePeerDependencies: - supports-color dev: true - /@vitejs/plugin-vue/2.3.3_vite@2.9.13+vue@3.2.37: - resolution: {integrity: sha512-SmQLDyhz+6lGJhPELsBdzXGc+AcaT8stgkbiTFGpXPe8Tl1tJaBw1A6pxDqDuRsVkD8uscrkx3hA7QDOoKYtyw==} - engines: {node: '>=12.0.0'} + /@vitejs/plugin-vue/3.1.0_vite@3.1.0+vue@3.2.38: + resolution: {integrity: sha512-fmxtHPjSOEIRg6vHYDaem+97iwCUg/uSIaTzp98lhELt2ISOQuDo2hbkBdXod0g15IhfPMQmAxh4heUks2zvDA==} + engines: {node: ^14.18.0 || >=16.0.0} peerDependencies: - vite: ^2.5.10 + vite: ^3.0.0 vue: ^3.2.25 dependencies: - vite: 2.9.13_sass@1.53.0 - vue: 3.2.37 + vite: 3.1.0_sass@1.54.8 + vue: 3.2.38 dev: true - /@volar/code-gen/0.38.2: - resolution: {integrity: sha512-H81I6d7rZB7teqL+zhK/Xz1v0/kKkUwkB0Aq6b4+BTCqcJeiZkoWxd0gFhrhWTnUoqiM83lhoTGo2vkvx5YagQ==} + /@volar/code-gen/0.40.9: + resolution: {integrity: sha512-SlLwXi1XzZG9ta4MQ6a2cdbtVYA4ibZFIK1Eb51Ii05m1AvrYvdOsLXOOC6ezRK2bws9dPEKbW0eYYnCoMjM3w==} dependencies: - '@volar/source-map': 0.38.2 + '@volar/source-map': 0.40.9 dev: true - /@volar/source-map/0.38.2: - resolution: {integrity: sha512-DWcYbYt9SPwk0r4VmXk1F0v4X5+hCqH1JRkAWSeJymQyXCQ2OQDEbY2PF12a7y2qn4FUBD2gOba2TynAqI8ZFQ==} + /@volar/source-map/0.40.9: + resolution: {integrity: sha512-hJcedV2DXVfsu2fBKRuXX4s3WgQYIVtzQSGbfba5/+krY5joZzegnccJJPj/3TIo6P+1/p9CFi10xxP16JUGwQ==} + dependencies: + '@vue/reactivity': 3.2.38 dev: true - /@volar/vue-code-gen/0.38.2: - resolution: {integrity: sha512-whLunD6phSGWBUHZKdTxeglrpzQu26ii8CRVapFdjfyMaVhQ7ESNeIAhkTVyg2ovOPc0PiDYPQEPzfWAADIWog==} + /@volar/typescript-faster/0.40.9: + resolution: {integrity: sha512-27kVrBwXX/qRLnD/MqsXUYRiYDLjfgwDsdMXKN7Nq7acMvYWmImY9k71x6hyDb/97iKvHFmgxbBlgN3SDcTcxg==} dependencies: - '@volar/code-gen': 0.38.2 - '@volar/source-map': 0.38.2 - '@vue/compiler-core': 3.2.37 - '@vue/compiler-dom': 3.2.37 - '@vue/shared': 3.2.37 + semver: 7.3.7 dev: true - /@volar/vue-typescript/0.38.2: - resolution: {integrity: sha512-5IKvSK2m5yUmH6iu/tNScVlvJGuiHawTfSmjxaMs+/tod25WeK37LEdf+pdKtlJ30bYTQmmkAuEfG01QvvBRGQ==} + /@volar/vue-language-core/0.40.9: + resolution: {integrity: sha512-Vmx2l7IOl2NCg1RT+CYJowUG2MiAZOZ9OkBVPWZN+GYFcE6W90UDbGJyO64zSqlw/fYO0ERPRvxSBX0IULiigQ==} dependencies: - '@volar/code-gen': 0.38.2 - '@volar/source-map': 0.38.2 - '@volar/vue-code-gen': 0.38.2 - '@vue/compiler-sfc': 3.2.37 - '@vue/reactivity': 3.2.37 + '@volar/code-gen': 0.40.9 + '@volar/source-map': 0.40.9 + '@vue/compiler-core': 3.2.38 + '@vue/compiler-dom': 3.2.38 + '@vue/compiler-sfc': 3.2.38 + '@vue/reactivity': 3.2.38 + '@vue/shared': 3.2.38 + dev: true + + /@volar/vue-typescript/0.40.9: + resolution: {integrity: sha512-2k+7B5WGGOWIr0vBNpe7PzkOCRnrbkRpBLpeQJ7TSprnE4oi3OzVDcxDBXIGhW1+ccMJcBcASv2LrvVALbQKZA==} + dependencies: + '@volar/code-gen': 0.40.9 + '@volar/typescript-faster': 0.40.9 + '@volar/vue-language-core': 0.40.9 dev: true /@vue/babel-helper-vue-transform-on/1.0.2: resolution: {integrity: sha512-hz4R8tS5jMn8lDq6iD+yWL6XNB699pGIVLk7WSJnn1dbpjaazsjZQkieJoRX6gW5zpYSCFqQ7jUquPNY65tQYA==} dev: true - /@vue/babel-plugin-jsx/1.1.1_@babel+core@7.18.6: + /@vue/babel-plugin-jsx/1.1.1_@babel+core@7.19.0: resolution: {integrity: sha512-j2uVfZjnB5+zkcbc/zsOc0fSNGCMMjaEXP52wdwdIfn0qjFfEYpYZBFKFg+HHnQeJCVrjOeO0YxgaL7DMrym9w==} dependencies: '@babel/helper-module-imports': 7.18.6 - '@babel/plugin-syntax-jsx': 7.18.6_@babel+core@7.18.6 - '@babel/template': 7.18.6 - '@babel/traverse': 7.18.6 - '@babel/types': 7.18.7 + '@babel/plugin-syntax-jsx': 7.18.6_@babel+core@7.19.0 + '@babel/template': 7.18.10 + '@babel/traverse': 7.19.0 + '@babel/types': 7.19.0 '@vue/babel-helper-vue-transform-on': 1.0.2 camelcase: 6.3.0 html-tags: 3.2.0 @@ -817,126 +819,114 @@ packages: - supports-color dev: true - /@vue/compiler-core/3.2.37: - resolution: {integrity: sha512-81KhEjo7YAOh0vQJoSmAD68wLfYqJvoiD4ulyedzF+OEk/bk6/hx3fTNVfuzugIIaTrOx4PGx6pAiBRe5e9Zmg==} + /@vue/compiler-core/3.2.38: + resolution: {integrity: sha512-/FsvnSu7Z+lkd/8KXMa4yYNUiqQrI22135gfsQYVGuh5tqEgOB0XqrUdb/KnCLa5+TmQLPwvyUnKMyCpu+SX3Q==} dependencies: - '@babel/parser': 7.18.6 - '@vue/shared': 3.2.37 + '@babel/parser': 7.19.0 + '@vue/shared': 3.2.38 estree-walker: 2.0.2 source-map: 0.6.1 - /@vue/compiler-dom/3.2.37: - resolution: {integrity: sha512-yxJLH167fucHKxaqXpYk7x8z7mMEnXOw3G2q62FTkmsvNxu4FQSu5+3UMb+L7fjKa26DEzhrmCxAgFLLIzVfqQ==} + /@vue/compiler-dom/3.2.38: + resolution: {integrity: sha512-zqX4FgUbw56kzHlgYuEEJR8mefFiiyR3u96498+zWPsLeh1WKvgIReoNE+U7gG8bCUdvsrJ0JRmev0Ky6n2O0g==} dependencies: - '@vue/compiler-core': 3.2.37 - '@vue/shared': 3.2.37 + '@vue/compiler-core': 3.2.38 + '@vue/shared': 3.2.38 - /@vue/compiler-sfc/3.2.37: - resolution: {integrity: sha512-+7i/2+9LYlpqDv+KTtWhOZH+pa8/HnX/905MdVmAcI/mPQOBwkHHIzrsEsucyOIZQYMkXUiTkmZq5am/NyXKkg==} + /@vue/compiler-sfc/3.2.38: + resolution: {integrity: sha512-KZjrW32KloMYtTcHAFuw3CqsyWc5X6seb8KbkANSWt3Cz9p2qA8c1GJpSkksFP9ABb6an0FLCFl46ZFXx3kKpg==} dependencies: - '@babel/parser': 7.18.6 - '@vue/compiler-core': 3.2.37 - '@vue/compiler-dom': 3.2.37 - '@vue/compiler-ssr': 3.2.37 - '@vue/reactivity-transform': 3.2.37 - '@vue/shared': 3.2.37 + '@babel/parser': 7.19.0 + '@vue/compiler-core': 3.2.38 + '@vue/compiler-dom': 3.2.38 + '@vue/compiler-ssr': 3.2.38 + '@vue/reactivity-transform': 3.2.38 + '@vue/shared': 3.2.38 estree-walker: 2.0.2 magic-string: 0.25.9 - postcss: 8.4.14 + postcss: 8.4.16 source-map: 0.6.1 - /@vue/compiler-ssr/3.2.37: - resolution: {integrity: sha512-7mQJD7HdXxQjktmsWp/J67lThEIcxLemz1Vb5I6rYJHR5vI+lON3nPGOH3ubmbvYGt8xEUaAr1j7/tIFWiEOqw==} + /@vue/compiler-ssr/3.2.38: + resolution: {integrity: sha512-bm9jOeyv1H3UskNm4S6IfueKjUNFmi2kRweFIGnqaGkkRePjwEcfCVqyS3roe7HvF4ugsEkhf4+kIvDhip6XzQ==} dependencies: - '@vue/compiler-dom': 3.2.37 - '@vue/shared': 3.2.37 + '@vue/compiler-dom': 3.2.38 + '@vue/shared': 3.2.38 - /@vue/devtools-api/6.1.4: - resolution: {integrity: sha512-IiA0SvDrJEgXvVxjNkHPFfDx6SXw0b/TUkqMcDZWNg9fnCAHbTpoo59YfJ9QLFkwa3raau5vSlRVzMSLDnfdtQ==} + /@vue/devtools-api/6.2.1: + resolution: {integrity: sha512-OEgAMeQXvCoJ+1x8WyQuVZzFo0wcyCmUR3baRVLmKBo1LmYZWMlRiXlux5jd0fqVJu6PfDbOrZItVqUEzLobeQ==} dev: false - /@vue/reactivity-transform/3.2.37: - resolution: {integrity: sha512-IWopkKEb+8qpu/1eMKVeXrK0NLw9HicGviJzhJDEyfxTR9e1WtpnnbYkJWurX6WwoFP0sz10xQg8yL8lgskAZg==} + /@vue/reactivity-transform/3.2.38: + resolution: {integrity: sha512-3SD3Jmi1yXrDwiNJqQ6fs1x61WsDLqVk4NyKVz78mkaIRh6d3IqtRnptgRfXn+Fzf+m6B1KxBYWq1APj6h4qeA==} dependencies: - '@babel/parser': 7.18.6 - '@vue/compiler-core': 3.2.37 - '@vue/shared': 3.2.37 + '@babel/parser': 7.19.0 + '@vue/compiler-core': 3.2.38 + '@vue/shared': 3.2.38 estree-walker: 2.0.2 magic-string: 0.25.9 - /@vue/reactivity/3.2.37: - resolution: {integrity: sha512-/7WRafBOshOc6m3F7plwzPeCu/RCVv9uMpOwa/5PiY1Zz+WLVRWiy0MYKwmg19KBdGtFWsmZ4cD+LOdVPcs52A==} + /@vue/reactivity/3.2.38: + resolution: {integrity: sha512-6L4myYcH9HG2M25co7/BSo0skKFHpAN8PhkNPM4xRVkyGl1K5M3Jx4rp5bsYhvYze2K4+l+pioN4e6ZwFLUVtw==} dependencies: - '@vue/shared': 3.2.37 + '@vue/shared': 3.2.38 - /@vue/runtime-core/3.2.37: - resolution: {integrity: sha512-JPcd9kFyEdXLl/i0ClS7lwgcs0QpUAWj+SKX2ZC3ANKi1U4DOtiEr6cRqFXsPwY5u1L9fAjkinIdB8Rz3FoYNQ==} + /@vue/runtime-core/3.2.38: + resolution: {integrity: sha512-kk0qiSiXUU/IKxZw31824rxmFzrLr3TL6ZcbrxWTKivadoKupdlzbQM4SlGo4MU6Zzrqv4fzyUasTU1jDoEnzg==} dependencies: - '@vue/reactivity': 3.2.37 - '@vue/shared': 3.2.37 + '@vue/reactivity': 3.2.38 + '@vue/shared': 3.2.38 - /@vue/runtime-dom/3.2.37: - resolution: {integrity: sha512-HimKdh9BepShW6YozwRKAYjYQWg9mQn63RGEiSswMbW+ssIht1MILYlVGkAGGQbkhSh31PCdoUcfiu4apXJoPw==} + /@vue/runtime-dom/3.2.38: + resolution: {integrity: sha512-4PKAb/ck2TjxdMSzMsnHViOrrwpudk4/A56uZjhzvusoEU9xqa5dygksbzYepdZeB5NqtRw5fRhWIiQlRVK45A==} dependencies: - '@vue/runtime-core': 3.2.37 - '@vue/shared': 3.2.37 + '@vue/runtime-core': 3.2.38 + '@vue/shared': 3.2.38 csstype: 2.6.20 - /@vue/server-renderer/3.2.37_vue@3.2.37: - resolution: {integrity: sha512-kLITEJvaYgZQ2h47hIzPh2K3jG8c1zCVbp/o/bzQOyvzaKiCquKS7AaioPI28GNxIsE/zSx+EwWYsNxDCX95MA==} + /@vue/server-renderer/3.2.38_vue@3.2.38: + resolution: {integrity: sha512-pg+JanpbOZ5kEfOZzO2bt02YHd+ELhYP8zPeLU1H0e7lg079NtuuSB8fjLdn58c4Ou8UQ6C1/P+528nXnLPAhA==} peerDependencies: - vue: 3.2.37 + vue: 3.2.38 dependencies: - '@vue/compiler-ssr': 3.2.37 - '@vue/shared': 3.2.37 - vue: 3.2.37 + '@vue/compiler-ssr': 3.2.38 + '@vue/shared': 3.2.38 + vue: 3.2.38 - /@vue/shared/3.2.37: - resolution: {integrity: sha512-4rSJemR2NQIo9Klm1vabqWjD8rs/ZaJSzMxkMNeJS6lHiUjjUeYFbooN19NgFjztubEKh3WlZUeOLVdbbUWHsw==} + /@vue/shared/3.2.38: + resolution: {integrity: sha512-dTyhTIRmGXBjxJE+skC8tTWCGLCVc4wQgRRLt8+O9p5ewBAjoBwtCAkLPrtToSr1xltoe3st21Pv953aOZ7alg==} - /@vueuse/core/8.7.5_vue@3.2.37: - resolution: {integrity: sha512-tqgzeZGoZcXzoit4kOGLWJibDMLp0vdm6ZO41SSUQhkhtrPhAg6dbIEPiahhUu6sZAmSYvVrZgEr5aKD51nrLA==} - peerDependencies: - '@vue/composition-api': ^1.1.0 - vue: ^2.6.0 || ^3.2.0 - peerDependenciesMeta: - '@vue/composition-api': - optional: true - vue: - optional: true + /@vueuse/core/9.2.0_vue@3.2.38: + resolution: {integrity: sha512-/MZ6qpz6uSyaXrtoeBWQzAKRG3N7CvfVWvQxiM3ei3Xe5ydOjjtVbo7lGl9p8dECV93j7W8s63A8H0kFLpLyxg==} dependencies: - '@types/web-bluetooth': 0.0.14 - '@vueuse/metadata': 8.7.5 - '@vueuse/shared': 8.7.5_vue@3.2.37 - vue: 3.2.37 - vue-demi: 0.13.1_vue@3.2.37 + '@types/web-bluetooth': 0.0.15 + '@vueuse/metadata': 9.2.0 + '@vueuse/shared': 9.2.0_vue@3.2.38 + vue-demi: 0.13.11_vue@3.2.38 + transitivePeerDependencies: + - '@vue/composition-api' + - vue dev: false - /@vueuse/metadata/8.7.5: - resolution: {integrity: sha512-emJZKRQSaEnVqmlu39NpNp8iaW+bPC2kWykWoWOZMSlO/0QVEmO/rt8A5VhOEJTKLX3vwTevqbiRy9WJRwVOQg==} + /@vueuse/metadata/9.2.0: + resolution: {integrity: sha512-exN4KE6iquxDCdt72BgEhb3tlOpECtD61AUdXnUqBTIUCl70x1Ar/QXo3bYcvxmdMS2/peQyfeTzBjRTpvL5xw==} dev: false - /@vueuse/shared/8.7.5_vue@3.2.37: - resolution: {integrity: sha512-THXPvMBFmg6Gf6AwRn/EdTh2mhqwjGsB2Yfp374LNQSQVKRHtnJ0I42bsZTn7nuEliBxqUrGQm/lN6qUHmhJLw==} - peerDependencies: - '@vue/composition-api': ^1.1.0 - vue: ^2.6.0 || ^3.2.0 - peerDependenciesMeta: - '@vue/composition-api': - optional: true - vue: - optional: true + /@vueuse/shared/9.2.0_vue@3.2.38: + resolution: {integrity: sha512-NnRp/noSWuXW0dKhZK5D0YLrDi0nmZ18UeEgwXQq7Ul5TTP93lcNnKjrHtd68j2xFB/l59yPGFlCryL692bnrA==} dependencies: - vue: 3.2.37 - vue-demi: 0.13.1_vue@3.2.37 + vue-demi: 0.13.11_vue@3.2.38 + transitivePeerDependencies: + - '@vue/composition-api' + - vue dev: false - /acorn-jsx/5.3.2_acorn@8.7.1: + /acorn-jsx/5.3.2_acorn@8.8.0: resolution: {integrity: sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ==} peerDependencies: acorn: ^6.0.0 || ^7.0.0 || ^8.0.0 dependencies: - acorn: 8.7.1 + acorn: 8.8.0 dev: true /acorn-node/1.8.2: @@ -958,8 +948,8 @@ packages: hasBin: true dev: false - /acorn/8.7.1: - resolution: {integrity: sha512-Xx54uLJQZ19lKygFXOWsscKUbsBZW0CPykPhVQdhIeIwrbPmJzqeASDInc8nKBnp/JT6igTs82qPXz069H8I/A==} + /acorn/8.8.0: + resolution: {integrity: sha512-QOxyigPVrpZ2GXT+PFyZTl6TtOFc5egxHIP9IlQ+RbupQuX4RkT/Bee4/kQuC02Xkzg84JcT7oLYtDIQxp+v7w==} engines: {node: '>=0.4.0'} hasBin: true dev: true @@ -976,6 +966,7 @@ packages: /ansi-regex/5.0.1: resolution: {integrity: sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==} engines: {node: '>=8'} + dev: true /ansi-styles/3.2.1: resolution: {integrity: sha512-VT0ZI6kZRdTh8YyJw3SMbYm/u+NqfsAxEpWO0Pf9sq8/e94WxxOpPKx9FR1FlyCtOVDNOQ+8ntlqFxiRc+r5qA==} @@ -989,11 +980,7 @@ packages: engines: {node: '>=8'} dependencies: color-convert: 2.0.1 - - /ansi-styles/5.2.0: - resolution: {integrity: sha512-Cxwpt2SfTzTtXcfOlzGEee8O+c+MmUgGrNiBcXnuWxuFJHe6a5Hz7qwhwe5OgaSYI0IJvkLqWX1ASG+cJOkEiA==} - engines: {node: '>=10'} - dev: false + dev: true /anymatch/3.1.2: resolution: {integrity: sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg==} @@ -1029,19 +1016,19 @@ packages: hasBin: true dev: true - /autoprefixer/10.4.7_postcss@8.4.14: - resolution: {integrity: sha512-ypHju4Y2Oav95SipEcCcI5J7CGPuvz8oat7sUtYj3ClK44bldfvtvcxK6IEK++7rqB7YchDGzweZIBG+SD0ZAA==} + /autoprefixer/10.4.8_postcss@8.4.16: + resolution: {integrity: sha512-75Jr6Q/XpTqEf6D2ltS5uMewJIx5irCU1oBYJrWjFenq/m12WRRrz6g15L1EIoYvPLXTbEry7rDOwrcYNj77xw==} engines: {node: ^10 || ^12 || >=14} hasBin: true peerDependencies: postcss: ^8.1.0 dependencies: - browserslist: 4.21.1 - caniuse-lite: 1.0.30001361 + browserslist: 4.21.3 + caniuse-lite: 1.0.30001390 fraction.js: 4.2.0 normalize-range: 0.1.2 picocolors: 1.0.0 - postcss: 8.4.14 + postcss: 8.4.16 postcss-value-parser: 4.2.0 dev: false @@ -1083,15 +1070,15 @@ packages: dependencies: fill-range: 7.0.1 - /browserslist/4.21.1: - resolution: {integrity: sha512-Nq8MFCSrnJXSc88yliwlzQe3qNe3VntIjhsArW9IJOEPSHNx23FalwApUVbzAWABLhYJJ7y8AynWI/XM8OdfjQ==} + /browserslist/4.21.3: + resolution: {integrity: sha512-898rgRXLAyRkM1GryrrBHGkqA5hlpkV5MhtZwg9QXeiyLUYs2k00Un05aX5l2/yJIOObYKOpS2JNo8nJDE7fWQ==} engines: {node: ^6 || ^7 || ^8 || ^9 || ^10 || ^11 || ^12 || >=13.7} hasBin: true dependencies: - caniuse-lite: 1.0.30001361 - electron-to-chromium: 1.4.173 - node-releases: 2.0.5 - update-browserslist-db: 1.0.4_browserslist@4.21.1 + caniuse-lite: 1.0.30001390 + electron-to-chromium: 1.4.242 + node-releases: 2.0.6 + update-browserslist-db: 1.0.7_browserslist@4.21.3 /callsites/3.1.0: resolution: {integrity: sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ==} @@ -1108,8 +1095,8 @@ packages: engines: {node: '>=10'} dev: true - /caniuse-lite/1.0.30001361: - resolution: {integrity: sha512-ybhCrjNtkFji1/Wto6SSJKkWk6kZgVQsDq5QI83SafsF6FXv2JB4df9eEdH6g8sdGgqTXrFLjAxqBGgYoU3azQ==} + /caniuse-lite/1.0.30001390: + resolution: {integrity: sha512-sS4CaUM+/+vqQUlCvCJ2WtDlV81aWtHhqeEVkLokVJJa3ViN4zDxAGfq9R8i1m90uGHxo99cy10Od+lvn3hf0g==} /chalk/2.4.2: resolution: {integrity: sha512-Mti+f9lpJNcwF4tWV8/OrTTtF1gZi+f8FqlyAdouralcFWFQWF2+NgCHShjkCb+IFBLq9buZwE1xckQU4peSuQ==} @@ -1126,6 +1113,7 @@ packages: dependencies: ansi-styles: 4.3.0 supports-color: 7.2.0 + dev: true /chokidar/3.5.3: resolution: {integrity: sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==} @@ -1152,6 +1140,7 @@ packages: engines: {node: '>=7.0.0'} dependencies: color-name: 1.1.4 + dev: true /color-name/1.1.3: resolution: {integrity: sha512-72fSenhMw2HZMTVHeCA9KCmpEIbzWiQsjN+BHcBbS9vr1mtt+vJjPdksIBNUmKAW8TFUDPJK5SUU3QhE9NEXDw==} @@ -1198,8 +1187,8 @@ packages: css: 2.2.4 dev: true - /css-render/0.15.10: - resolution: {integrity: sha512-6j5acvm81sXTHJiF47FNNICtDpF74YoWk1xEK3qQvdqgW6vc+OXrPqflL6m8f5GE6XuFYrbACNEd17kraCSBAQ==} + /css-render/0.15.11: + resolution: {integrity: sha512-hnLrHPUndVUTF5nmNPRey6hpixK02IPUGdEsm2xRjvJuewToyrVFx9Nmai8rgfVzhTFo5SJVh2PHAtzaIV8JKw==} dependencies: '@emotion/hash': 0.8.0 '@types/node': 17.0.45 @@ -1243,16 +1232,16 @@ packages: chokidar: 3.5.3 dev: true - /date-fns-tz/1.3.5_date-fns@2.28.0: - resolution: {integrity: sha512-SNhl/fWe7i2HoIB9ejLZhEEJ6ZtRRpOBbzizFrq11K2/iceS9Nk7fPN2VTYVOMgFB9u0f3eidSC4n1xaRONW2A==} + /date-fns-tz/1.3.7_date-fns@2.29.2: + resolution: {integrity: sha512-1t1b8zyJo+UI8aR+g3iqr5fkUHWpd58VBx8J/ZSQ+w7YrGlw80Ag4sA86qkfCXRBLmMc4I2US+aPMd4uKvwj5g==} peerDependencies: date-fns: '>=2.0.0' dependencies: - date-fns: 2.28.0 + date-fns: 2.29.2 dev: false - /date-fns/2.28.0: - resolution: {integrity: sha512-8d35hViGYx/QH0icHYCeLmsLmMUheMmTyV9Fcm6gvNwdw31yXXH+O85sOBJ+OLnLQMKZowvpKb6FgMIQjcpvQw==} + /date-fns/2.29.2: + resolution: {integrity: sha512-0VNbwmWJDS/G3ySwFSJA3ayhbURMTJLtwM2DTxf9CWondCnh6DTNlO9JgRSq6ibf4eD0lfMJNBxUdEAHHix+bA==} engines: {node: '>=0.11'} dev: false @@ -1323,11 +1312,6 @@ packages: resolution: {integrity: sha512-gxtyfqMg7GKyhQmb056K7M3xszy/myH8w+B4RT+QXBQsvAOdc3XymqDDPHx1BgPgsdAA5SIifona89YtRATDzw==} dev: false - /diff-sequences/27.5.1: - resolution: {integrity: sha512-k1gCAXAsNgLwEL+Y8Wvl+M6oEFj5bgazfZULpS5CneoPPXRaCCW7dm+q21Ky2VEE5X+VeRDBVg1Pcvvsr4TtNQ==} - engines: {node: ^10.13.0 || ^12.13.0 || ^14.15.0 || >=15.0.0} - dev: false - /dir-glob/3.0.1: resolution: {integrity: sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA==} engines: {node: '>=8'} @@ -1358,8 +1342,8 @@ packages: zrender: 5.3.2 dev: false - /electron-to-chromium/1.4.173: - resolution: {integrity: sha512-Qo3LnVW6JRNhD32viSdPebxKI7K+3WeBDjU1+Q2yZS83zAh8C2LyPpzTimlciv6U74KpY9n/0ESAhUByRke0jw==} + /electron-to-chromium/1.4.242: + resolution: {integrity: sha512-nPdgMWtjjWGCtreW/2adkrB2jyHjClo9PtVhR6rW+oxa4E4Wom642Tn+5LslHP3XPL5MCpkn5/UEY60EXylNeQ==} /emojis-list/2.1.0: resolution: {integrity: sha512-knHEZMgs8BB+MInokmNTg/OyPlAddghe1YBgNwJBc5zsJi/uyIcXoSDsL/W9ymOsBoBGdPIHXYJ9+qKFwRwDng==} @@ -1375,8 +1359,8 @@ packages: dev: true optional: true - /esbuild-android-64/0.14.47: - resolution: {integrity: sha512-R13Bd9+tqLVFndncMHssZrPWe6/0Kpv2/dt4aA69soX4PRxlzsVpCvoJeFE8sOEoeVEiBkI0myjlkDodXlHa0g==} + /esbuild-android-64/0.15.7: + resolution: {integrity: sha512-p7rCvdsldhxQr3YHxptf1Jcd86dlhvc3EQmQJaZzzuAxefO9PvcI0GLOa5nCWem1AJ8iMRu9w0r5TG8pHmbi9w==} engines: {node: '>=12'} cpu: [x64] os: [android] @@ -1384,8 +1368,8 @@ packages: dev: true optional: true - /esbuild-android-arm64/0.14.47: - resolution: {integrity: sha512-OkwOjj7ts4lBp/TL6hdd8HftIzOy/pdtbrNA4+0oVWgGG64HrdVzAF5gxtJufAPOsEjkyh1oIYvKAUinKKQRSQ==} + /esbuild-android-arm64/0.15.7: + resolution: {integrity: sha512-L775l9ynJT7rVqRM5vo+9w5g2ysbOCfsdLV4CWanTZ1k/9Jb3IYlQ06VCI1edhcosTYJRECQFJa3eAvkx72eyQ==} engines: {node: '>=12'} cpu: [arm64] os: [android] @@ -1393,8 +1377,8 @@ packages: dev: true optional: true - /esbuild-darwin-64/0.14.47: - resolution: {integrity: sha512-R6oaW0y5/u6Eccti/TS6c/2c1xYTb1izwK3gajJwi4vIfNs1s8B1dQzI1UiC9T61YovOQVuePDcfqHLT3mUZJA==} + /esbuild-darwin-64/0.15.7: + resolution: {integrity: sha512-KGPt3r1c9ww009t2xLB6Vk0YyNOXh7hbjZ3EecHoVDxgtbUlYstMPDaReimKe6eOEfyY4hBEEeTvKwPsiH5WZg==} engines: {node: '>=12'} cpu: [x64] os: [darwin] @@ -1402,8 +1386,8 @@ packages: dev: true optional: true - /esbuild-darwin-arm64/0.14.47: - resolution: {integrity: sha512-seCmearlQyvdvM/noz1L9+qblC5vcBrhUaOoLEDDoLInF/VQ9IkobGiLlyTPYP5dW1YD4LXhtBgOyevoIHGGnw==} + /esbuild-darwin-arm64/0.15.7: + resolution: {integrity: sha512-kBIHvtVqbSGajN88lYMnR3aIleH3ABZLLFLxwL2stiuIGAjGlQW741NxVTpUHQXUmPzxi6POqc9npkXa8AcSZQ==} engines: {node: '>=12'} cpu: [arm64] os: [darwin] @@ -1411,8 +1395,8 @@ packages: dev: true optional: true - /esbuild-freebsd-64/0.14.47: - resolution: {integrity: sha512-ZH8K2Q8/Ux5kXXvQMDsJcxvkIwut69KVrYQhza/ptkW50DC089bCVrJZZ3sKzIoOx+YPTrmsZvqeZERjyYrlvQ==} + /esbuild-freebsd-64/0.15.7: + resolution: {integrity: sha512-hESZB91qDLV5MEwNxzMxPfbjAhOmtfsr9Wnuci7pY6TtEh4UDuevmGmkUIjX/b+e/k4tcNBMf7SRQ2mdNuK/HQ==} engines: {node: '>=12'} cpu: [x64] os: [freebsd] @@ -1420,8 +1404,8 @@ packages: dev: true optional: true - /esbuild-freebsd-arm64/0.14.47: - resolution: {integrity: sha512-ZJMQAJQsIOhn3XTm7MPQfCzEu5b9STNC+s90zMWe2afy9EwnHV7Ov7ohEMv2lyWlc2pjqLW8QJnz2r0KZmeAEQ==} + /esbuild-freebsd-arm64/0.15.7: + resolution: {integrity: sha512-dLFR0ChH5t+b3J8w0fVKGvtwSLWCv7GYT2Y2jFGulF1L5HftQLzVGN+6pi1SivuiVSmTh28FwUhi9PwQicXI6Q==} engines: {node: '>=12'} cpu: [arm64] os: [freebsd] @@ -1429,8 +1413,8 @@ packages: dev: true optional: true - /esbuild-linux-32/0.14.47: - resolution: {integrity: sha512-FxZOCKoEDPRYvq300lsWCTv1kcHgiiZfNrPtEhFAiqD7QZaXrad8LxyJ8fXGcWzIFzRiYZVtB3ttvITBvAFhKw==} + /esbuild-linux-32/0.15.7: + resolution: {integrity: sha512-v3gT/LsONGUZcjbt2swrMjwxo32NJzk+7sAgtxhGx1+ZmOFaTRXBAi1PPfgpeo/J//Un2jIKm/I+qqeo4caJvg==} engines: {node: '>=12'} cpu: [ia32] os: [linux] @@ -1438,8 +1422,8 @@ packages: dev: true optional: true - /esbuild-linux-64/0.14.47: - resolution: {integrity: sha512-nFNOk9vWVfvWYF9YNYksZptgQAdstnDCMtR6m42l5Wfugbzu11VpMCY9XrD4yFxvPo9zmzcoUL/88y0lfJZJJw==} + /esbuild-linux-64/0.15.7: + resolution: {integrity: sha512-LxXEfLAKwOVmm1yecpMmWERBshl+Kv5YJ/1KnyAr6HRHFW8cxOEsEfisD3sVl/RvHyW//lhYUVSuy9jGEfIRAQ==} engines: {node: '>=12'} cpu: [x64] os: [linux] @@ -1447,8 +1431,8 @@ packages: dev: true optional: true - /esbuild-linux-arm/0.14.47: - resolution: {integrity: sha512-ZGE1Bqg/gPRXrBpgpvH81tQHpiaGxa8c9Rx/XOylkIl2ypLuOcawXEAo8ls+5DFCcRGt/o3sV+PzpAFZobOsmA==} + /esbuild-linux-arm/0.15.7: + resolution: {integrity: sha512-JKgAHtMR5f75wJTeuNQbyznZZa+pjiUHV7sRZp42UNdyXC6TiUYMW/8z8yIBAr2Fpad8hM1royZKQisqPABPvQ==} engines: {node: '>=12'} cpu: [arm] os: [linux] @@ -1456,8 +1440,8 @@ packages: dev: true optional: true - /esbuild-linux-arm64/0.14.47: - resolution: {integrity: sha512-ywfme6HVrhWcevzmsufjd4iT3PxTfCX9HOdxA7Hd+/ZM23Y9nXeb+vG6AyA6jgq/JovkcqRHcL9XwRNpWG6XRw==} + /esbuild-linux-arm64/0.15.7: + resolution: {integrity: sha512-P3cfhudpzWDkglutWgXcT2S7Ft7o2e3YDMrP1n0z2dlbUZghUkKCyaWw0zhp4KxEEzt/E7lmrtRu/pGWnwb9vw==} engines: {node: '>=12'} cpu: [arm64] os: [linux] @@ -1465,8 +1449,8 @@ packages: dev: true optional: true - /esbuild-linux-mips64le/0.14.47: - resolution: {integrity: sha512-mg3D8YndZ1LvUiEdDYR3OsmeyAew4MA/dvaEJxvyygahWmpv1SlEEnhEZlhPokjsUMfRagzsEF/d/2XF+kTQGg==} + /esbuild-linux-mips64le/0.15.7: + resolution: {integrity: sha512-T7XKuxl0VpeFLCJXub6U+iybiqh0kM/bWOTb4qcPyDDwNVhLUiPcGdG2/0S7F93czUZOKP57YiLV8YQewgLHKw==} engines: {node: '>=12'} cpu: [mips64el] os: [linux] @@ -1474,8 +1458,8 @@ packages: dev: true optional: true - /esbuild-linux-ppc64le/0.14.47: - resolution: {integrity: sha512-WER+f3+szmnZiWoK6AsrTKGoJoErG2LlauSmk73LEZFQ/iWC+KhhDsOkn1xBUpzXWsxN9THmQFltLoaFEH8F8w==} + /esbuild-linux-ppc64le/0.15.7: + resolution: {integrity: sha512-6mGuC19WpFN7NYbecMIJjeQgvDb5aMuvyk0PDYBJrqAEMkTwg3Z98kEKuCm6THHRnrgsdr7bp4SruSAxEM4eJw==} engines: {node: '>=12'} cpu: [ppc64] os: [linux] @@ -1483,8 +1467,8 @@ packages: dev: true optional: true - /esbuild-linux-riscv64/0.14.47: - resolution: {integrity: sha512-1fI6bP3A3rvI9BsaaXbMoaOjLE3lVkJtLxsgLHqlBhLlBVY7UqffWBvkrX/9zfPhhVMd9ZRFiaqXnB1T7BsL2g==} + /esbuild-linux-riscv64/0.15.7: + resolution: {integrity: sha512-uUJsezbswAYo/X7OU/P+PuL/EI9WzxsEQXDekfwpQ23uGiooxqoLFAPmXPcRAt941vjlY9jtITEEikWMBr+F/g==} engines: {node: '>=12'} cpu: [riscv64] os: [linux] @@ -1492,8 +1476,8 @@ packages: dev: true optional: true - /esbuild-linux-s390x/0.14.47: - resolution: {integrity: sha512-eZrWzy0xFAhki1CWRGnhsHVz7IlSKX6yT2tj2Eg8lhAwlRE5E96Hsb0M1mPSE1dHGpt1QVwwVivXIAacF/G6mw==} + /esbuild-linux-s390x/0.15.7: + resolution: {integrity: sha512-+tO+xOyTNMc34rXlSxK7aCwJgvQyffqEM5MMdNDEeMU3ss0S6wKvbBOQfgd5jRPblfwJ6b+bKiz0g5nABpY0QQ==} engines: {node: '>=12'} cpu: [s390x] os: [linux] @@ -1501,8 +1485,8 @@ packages: dev: true optional: true - /esbuild-netbsd-64/0.14.47: - resolution: {integrity: sha512-Qjdjr+KQQVH5Q2Q1r6HBYswFTToPpss3gqCiSw2Fpq/ua8+eXSQyAMG+UvULPqXceOwpnPo4smyZyHdlkcPppQ==} + /esbuild-netbsd-64/0.15.7: + resolution: {integrity: sha512-yVc4Wz+Pu3cP5hzm5kIygNPrjar/v5WCSoRmIjCPWfBVJkZNb5brEGKUlf+0Y759D48BCWa0WHrWXaNy0DULTQ==} engines: {node: '>=12'} cpu: [x64] os: [netbsd] @@ -1510,8 +1494,8 @@ packages: dev: true optional: true - /esbuild-openbsd-64/0.14.47: - resolution: {integrity: sha512-QpgN8ofL7B9z8g5zZqJE+eFvD1LehRlxr25PBkjyyasakm4599iroUpaj96rdqRlO2ShuyqwJdr+oNqWwTUmQw==} + /esbuild-openbsd-64/0.15.7: + resolution: {integrity: sha512-GsimbwC4FSR4lN3wf8XmTQ+r8/0YSQo21rWDL0XFFhLHKlzEA4SsT1Tl8bPYu00IU6UWSJ+b3fG/8SB69rcuEQ==} engines: {node: '>=12'} cpu: [x64] os: [openbsd] @@ -1519,8 +1503,8 @@ packages: dev: true optional: true - /esbuild-sunos-64/0.14.47: - resolution: {integrity: sha512-uOeSgLUwukLioAJOiGYm3kNl+1wJjgJA8R671GYgcPgCx7QR73zfvYqXFFcIO93/nBdIbt5hd8RItqbbf3HtAQ==} + /esbuild-sunos-64/0.15.7: + resolution: {integrity: sha512-8CDI1aL/ts0mDGbWzjEOGKXnU7p3rDzggHSBtVryQzkSOsjCHRVe0iFYUuhczlxU1R3LN/E7HgUO4NXzGGP/Ag==} engines: {node: '>=12'} cpu: [x64] os: [sunos] @@ -1528,8 +1512,8 @@ packages: dev: true optional: true - /esbuild-windows-32/0.14.47: - resolution: {integrity: sha512-H0fWsLTp2WBfKLBgwYT4OTfFly4Im/8B5f3ojDv1Kx//kiubVY0IQunP2Koc/fr/0wI7hj3IiBDbSrmKlrNgLQ==} + /esbuild-windows-32/0.15.7: + resolution: {integrity: sha512-cOnKXUEPS8EGCzRSFa1x6NQjGhGsFlVgjhqGEbLTPsA7x4RRYiy2RKoArNUU4iR2vHmzqS5Gr84MEumO/wxYKA==} engines: {node: '>=12'} cpu: [ia32] os: [win32] @@ -1537,8 +1521,8 @@ packages: dev: true optional: true - /esbuild-windows-64/0.14.47: - resolution: {integrity: sha512-/Pk5jIEH34T68r8PweKRi77W49KwanZ8X6lr3vDAtOlH5EumPE4pBHqkCUdELanvsT14yMXLQ/C/8XPi1pAtkQ==} + /esbuild-windows-64/0.15.7: + resolution: {integrity: sha512-7MI08Ec2sTIDv+zH6StNBKO+2hGUYIT42GmFyW6MBBWWtJhTcQLinKS6ldIN1d52MXIbiJ6nXyCJ+LpL4jBm3Q==} engines: {node: '>=12'} cpu: [x64] os: [win32] @@ -1546,8 +1530,8 @@ packages: dev: true optional: true - /esbuild-windows-arm64/0.14.47: - resolution: {integrity: sha512-HFSW2lnp62fl86/qPQlqw6asIwCnEsEoNIL1h2uVMgakddf+vUuMcCbtUY1i8sst7KkgHrVKCJQB33YhhOweCQ==} + /esbuild-windows-arm64/0.15.7: + resolution: {integrity: sha512-R06nmqBlWjKHddhRJYlqDd3Fabx9LFdKcjoOy08YLimwmsswlFBJV4rXzZCxz/b7ZJXvrZgj8DDv1ewE9+StMw==} engines: {node: '>=12'} cpu: [arm64] os: [win32] @@ -1555,32 +1539,33 @@ packages: dev: true optional: true - /esbuild/0.14.47: - resolution: {integrity: sha512-wI4ZiIfFxpkuxB8ju4MHrGwGLyp1+awEHAHVpx6w7a+1pmYIq8T9FGEVVwFo0iFierDoMj++Xq69GXWYn2EiwA==} + /esbuild/0.15.7: + resolution: {integrity: sha512-7V8tzllIbAQV1M4QoE52ImKu8hT/NLGlGXkiDsbEU5PS6K8Mn09ZnYoS+dcmHxOS9CRsV4IRAMdT3I67IyUNXw==} engines: {node: '>=12'} hasBin: true requiresBuild: true optionalDependencies: - esbuild-android-64: 0.14.47 - esbuild-android-arm64: 0.14.47 - esbuild-darwin-64: 0.14.47 - esbuild-darwin-arm64: 0.14.47 - esbuild-freebsd-64: 0.14.47 - esbuild-freebsd-arm64: 0.14.47 - esbuild-linux-32: 0.14.47 - esbuild-linux-64: 0.14.47 - esbuild-linux-arm: 0.14.47 - esbuild-linux-arm64: 0.14.47 - esbuild-linux-mips64le: 0.14.47 - esbuild-linux-ppc64le: 0.14.47 - esbuild-linux-riscv64: 0.14.47 - esbuild-linux-s390x: 0.14.47 - esbuild-netbsd-64: 0.14.47 - esbuild-openbsd-64: 0.14.47 - esbuild-sunos-64: 0.14.47 - esbuild-windows-32: 0.14.47 - esbuild-windows-64: 0.14.47 - esbuild-windows-arm64: 0.14.47 + '@esbuild/linux-loong64': 0.15.7 + esbuild-android-64: 0.15.7 + esbuild-android-arm64: 0.15.7 + esbuild-darwin-64: 0.15.7 + esbuild-darwin-arm64: 0.15.7 + esbuild-freebsd-64: 0.15.7 + esbuild-freebsd-arm64: 0.15.7 + esbuild-linux-32: 0.15.7 + esbuild-linux-64: 0.15.7 + esbuild-linux-arm: 0.15.7 + esbuild-linux-arm64: 0.15.7 + esbuild-linux-mips64le: 0.15.7 + esbuild-linux-ppc64le: 0.15.7 + esbuild-linux-riscv64: 0.15.7 + esbuild-linux-s390x: 0.15.7 + esbuild-netbsd-64: 0.15.7 + esbuild-openbsd-64: 0.15.7 + esbuild-sunos-64: 0.15.7 + esbuild-windows-32: 0.15.7 + esbuild-windows-64: 0.15.7 + esbuild-windows-arm64: 0.15.7 dev: true /escalade/3.1.1: @@ -1597,17 +1582,17 @@ packages: engines: {node: '>=10'} dev: true - /eslint-config-prettier/8.5.0_eslint@8.18.0: + /eslint-config-prettier/8.5.0_eslint@8.23.0: resolution: {integrity: sha512-obmWKLUNCnhtQRKc+tmnYuQl0pFU1ibYJQ5BGhTVB08bHe9wC8qUeG7c08dj9XX+AuPj1YSGSQIHl1pnDHZR0Q==} hasBin: true peerDependencies: eslint: '>=7.0.0' dependencies: - eslint: 8.18.0 + eslint: 8.23.0 dev: true - /eslint-plugin-prettier/4.1.0_xu6ewijrtliw5q5lksq5uixwby: - resolution: {integrity: sha512-A3AXIEfTnq3D5qDFjWJdQ9c4BLhw/TqhSR+6+SVaoPJBAWciFEuJiNQh275OnjRrAi7yssZzuWBRw66VG2g6UA==} + /eslint-plugin-prettier/4.2.1_tgumt6uwl2md3n6uqnggd6wvce: + resolution: {integrity: sha512-f/0rXLXUt0oFYs8ra4w49wYZBG5GKZpAYsJSm6rnYL5uVDjd+zowwMwVZHnAjf4edNrKpCDYfXDgmRE/Ak7QyQ==} engines: {node: '>=12.0.0'} peerDependencies: eslint: '>=7.28.0' @@ -1617,25 +1602,25 @@ packages: eslint-config-prettier: optional: true dependencies: - eslint: 8.18.0 - eslint-config-prettier: 8.5.0_eslint@8.18.0 + eslint: 8.23.0 + eslint-config-prettier: 8.5.0_eslint@8.23.0 prettier: 2.7.1 prettier-linter-helpers: 1.0.0 dev: true - /eslint-plugin-vue/9.1.1_eslint@8.18.0: - resolution: {integrity: sha512-W9n5PB1X2jzC7CK6riG0oAcxjmKrjTF6+keL1rni8n57DZeilx/Fulz+IRJK3lYseLNAygN0I62L7DvioW40Tw==} + /eslint-plugin-vue/9.4.0_eslint@8.23.0: + resolution: {integrity: sha512-Nzz2QIJ8FG+rtJaqT/7/ru5ie2XgT9KCudkbN0y3uFYhQ41nuHEaboLAiqwMcK006hZPQv/rVMRhUIwEGhIvfQ==} engines: {node: ^14.17.0 || >=16.0.0} peerDependencies: eslint: ^6.2.0 || ^7.0.0 || ^8.0.0 dependencies: - eslint: 8.18.0 - eslint-utils: 3.0.0_eslint@8.18.0 + eslint: 8.23.0 + eslint-utils: 3.0.0_eslint@8.23.0 natural-compare: 1.4.0 nth-check: 2.1.1 postcss-selector-parser: 6.0.10 semver: 7.3.7 - vue-eslint-parser: 9.0.3_eslint@8.18.0 + vue-eslint-parser: 9.0.3_eslint@8.23.0 xml-name-validator: 4.0.0 transitivePeerDependencies: - supports-color @@ -1657,13 +1642,13 @@ packages: estraverse: 5.3.0 dev: true - /eslint-utils/3.0.0_eslint@8.18.0: + /eslint-utils/3.0.0_eslint@8.23.0: resolution: {integrity: sha512-uuQC43IGctw68pJA1RgbQS8/NP7rch6Cwd4j3ZBtgo4/8Flj4eGE7ZYSZRN3iq5pVUv6GPdW5Z1RFleo84uLDA==} engines: {node: ^10.0.0 || ^12.0.0 || >= 14.0.0} peerDependencies: eslint: '>=5' dependencies: - eslint: 8.18.0 + eslint: 8.23.0 eslint-visitor-keys: 2.1.0 dev: true @@ -1677,13 +1662,15 @@ packages: engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dev: true - /eslint/8.18.0: - resolution: {integrity: sha512-As1EfFMVk7Xc6/CvhssHUjsAQSkpfXvUGMFC3ce8JDe6WvqCgRrLOBQbVpsBFr1X1V+RACOadnzVvcUS5ni2bA==} + /eslint/8.23.0: + resolution: {integrity: sha512-pBG/XOn0MsJcKcTRLr27S5HpzQo4kLr+HjLQIyK4EiCsijDl/TB+h5uEuJU6bQ8Edvwz1XWOjpaP2qgnXGpTcA==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} hasBin: true dependencies: - '@eslint/eslintrc': 1.3.0 - '@humanwhocodes/config-array': 0.9.5 + '@eslint/eslintrc': 1.3.1 + '@humanwhocodes/config-array': 0.10.4 + '@humanwhocodes/gitignore-to-minimatch': 1.0.2 + '@humanwhocodes/module-importer': 1.0.1 ajv: 6.12.6 chalk: 4.1.2 cross-spawn: 7.0.3 @@ -1691,16 +1678,19 @@ packages: doctrine: 3.0.0 escape-string-regexp: 4.0.0 eslint-scope: 7.1.1 - eslint-utils: 3.0.0_eslint@8.18.0 + eslint-utils: 3.0.0_eslint@8.23.0 eslint-visitor-keys: 3.3.0 - espree: 9.3.2 + espree: 9.4.0 esquery: 1.4.0 esutils: 2.0.3 fast-deep-equal: 3.1.3 file-entry-cache: 6.0.1 + find-up: 5.0.0 functional-red-black-tree: 1.0.1 glob-parent: 6.0.2 - globals: 13.15.0 + globals: 13.17.0 + globby: 11.1.0 + grapheme-splitter: 1.0.4 ignore: 5.2.0 import-fresh: 3.3.0 imurmurhash: 0.1.4 @@ -1716,17 +1706,16 @@ packages: strip-ansi: 6.0.1 strip-json-comments: 3.1.1 text-table: 0.2.0 - v8-compile-cache: 2.3.0 transitivePeerDependencies: - supports-color dev: true - /espree/9.3.2: - resolution: {integrity: sha512-D211tC7ZwouTIuY5x9XnS0E9sWNChB7IYKX/Xp5eQj3nFXhqmiUDB9q27y76oFl8jTg3pXcQx/bpxMfs3CIZbA==} + /espree/9.4.0: + resolution: {integrity: sha512-DQmnRpLj7f6TgN/NYb0MTzJXL+vJF9h3pHy4JhCIs3zwcgez8xmGg3sXHcEO97BrmO2OSvCwMdfdlyl+E9KjOw==} engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0} dependencies: - acorn: 8.7.1 - acorn-jsx: 5.3.2_acorn@8.7.1 + acorn: 8.8.0 + acorn-jsx: 5.3.2_acorn@8.8.0 eslint-visitor-keys: 3.3.0 dev: true @@ -1762,8 +1751,8 @@ packages: engines: {node: '>=0.10.0'} dev: true - /evtd/0.2.3: - resolution: {integrity: sha512-tmiT1YUVqFjTY+BSBOAskL83xNx41iUfpvKP6Gcd/xMHjg3mnER98jXGXJyKnxCG19uPc6EhZiUC+MUyvoqCtw==} + /evtd/0.2.4: + resolution: {integrity: sha512-qaeGN5bx63s/AXgQo8gj6fBkxge+OoLddLniox5qtLAEY5HSnuSlISXVPxnSae1dWblvTh4/HoMIB+mbMsvZzw==} dev: false /fast-deep-equal/3.1.3: @@ -1814,16 +1803,24 @@ packages: dependencies: to-regex-range: 5.0.1 + /find-up/5.0.0: + resolution: {integrity: sha512-78/PXT1wlLLDgTzDs7sjq9hzz0vXD+zn+7wypEe4fXQxCmdmqfGsEPQxmiCSQI3ajFV91bVSsvNtrJRiW6nGng==} + engines: {node: '>=10'} + dependencies: + locate-path: 6.0.0 + path-exists: 4.0.0 + dev: true + /flat-cache/3.0.4: resolution: {integrity: sha512-dm9s5Pw7Jc0GvMYbshN6zchCA9RgQlzzEZX3vylR9IqFfS8XciblUXOKfW6SiuJ0e13eDYZoZV5wdrev7P3Nwg==} engines: {node: ^10.12.0 || >=12.0.0} dependencies: - flatted: 3.2.6 + flatted: 3.2.7 rimraf: 3.0.2 dev: true - /flatted/3.2.6: - resolution: {integrity: sha512-0sQoMh9s0BYsm+12Huy/rkKxVu4R1+r96YX5cG44rHV0pQ6iC3Q+mkoMFaGWObMFYQxCVT+ssG1ksneA2MI9KQ==} + /flatted/3.2.7: + resolution: {integrity: sha512-5nqDSxl8nn5BSNxyR3n4I6eDmbolI6WT+QqR547RwxQapgjQBmtktdP+HTBb/a/zLsbzERTONyUB5pefh5TtjQ==} dev: true /follow-redirects/1.15.1: @@ -1915,8 +1912,8 @@ packages: engines: {node: '>=4'} dev: true - /globals/13.15.0: - resolution: {integrity: sha512-bpzcOlgDhMG070Av0Vy5Owklpv1I6+j96GhUI7Rh7IzDCKLzboflLrrfqMu8NquDbiR4EOQk7XzJwqVJxicxog==} + /globals/13.17.0: + resolution: {integrity: sha512-1C+6nQRb1GwGMKm2dH/E7enFAMxGTmGI7/dEdhy/DNelv85w9B72t3uc5frtMNXIbzrarJJ/lTCjcaZwbLJmyw==} engines: {node: '>=8'} dependencies: type-fest: 0.20.2 @@ -1938,6 +1935,10 @@ packages: resolution: {integrity: sha512-9ByhssR2fPVsNZj478qUUbKfmL0+t5BDVyjShtyZZLiK7ZDAArFFfopyOTj0M05wE2tJPisA4iTnnXl2YoPvOA==} dev: true + /grapheme-splitter/1.0.4: + resolution: {integrity: sha512-bzh50DW9kTPM00T8y4o8vQg89Di9oLJVLW/KaOGIXJWP/iqCN6WKYkbNOF04vFLJhwcpYUh9ydh/+5vpOqV4YQ==} + dev: true + /has-flag/3.0.0: resolution: {integrity: sha512-sKJf1+ceQBr4SMkvQnBDNDtf4TXpVhVGateu0t918bl30FnbE2m4vNLX+VWe/dpjlb+HugGYzW7uQXH98HPEYw==} engines: {node: '>=4'} @@ -1946,6 +1947,7 @@ packages: /has-flag/4.0.0: resolution: {integrity: sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==} engines: {node: '>=8'} + dev: true /has/1.0.3: resolution: {integrity: sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw==} @@ -1953,12 +1955,8 @@ packages: dependencies: function-bind: 1.1.1 - /hash-sum/2.0.0: - resolution: {integrity: sha512-WdZTbAByD+pHfl/g9QSsBIIwy8IT+EsPiKDs0KNX+zSHhdDLFKdZu0BQHljvO+0QI/BasbMSUa8wYNCZTvhslg==} - dev: true - - /highlight.js/11.5.1: - resolution: {integrity: sha512-LKzHqnxr4CrD2YsNoIf/o5nJ09j4yi/GcH5BnYz9UnVpZdS4ucMgvP61TDty5xJcFGRjnH4DpujkS9bHT3hq0Q==} + /highlight.js/11.6.0: + resolution: {integrity: sha512-ig1eqDzJaB0pqEvlPVIpSSyMaO92bH1N2rJpLMN/nX396wTpDA4Eq0uK+7I/2XG17pFaaKE0kjV/XPeGt7Evjw==} engines: {node: '>=12.0.0'} dev: false @@ -1981,13 +1979,13 @@ packages: postcss: 6.0.23 dev: true - /icss-utils/5.1.0_postcss@8.4.14: + /icss-utils/5.1.0_postcss@8.4.16: resolution: {integrity: sha512-soFhflCVWLfRNOPU3iv5Z9VUdT44xFRbzjLsEzSr5AQmgqPMTHdU3PMT1Cf1ssx8fLNJDA1juftYl+PUcv3MqA==} engines: {node: ^10 || ^12 || >= 14} peerDependencies: postcss: ^8.1.0 dependencies: - postcss: 8.4.14 + postcss: 8.4.16 dev: true /ignore/5.2.0: @@ -2037,8 +2035,8 @@ packages: dependencies: binary-extensions: 2.2.0 - /is-core-module/2.9.0: - resolution: {integrity: sha512-+5FPy5PnwmO3lvfMb0AsoPaBG+5KHUI0wYFXOtYPnVVVspTFUuMZNfNaNVRt3FZadstu2c8x23vykRW/NBoU6A==} + /is-core-module/2.10.0: + resolution: {integrity: sha512-Erxj2n/LDAZ7H8WNJXd9tw38GYM3dv8rk8Zcs+jJuxYTW7sozH+SS8NtrSjVL1/vpLvWi1hxy96IzjJ3EHTJJg==} dependencies: has: 1.0.3 @@ -2064,31 +2062,6 @@ packages: resolution: {integrity: sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==} dev: true - /jest-diff/27.5.1: - resolution: {integrity: sha512-m0NvkX55LDt9T4mctTEgnZk3fmEg3NRYutvMPWM/0iPnkFj2wIeF45O1718cMSOFO1vINkqmxqD8vE37uTEbqw==} - engines: {node: ^10.13.0 || ^12.13.0 || ^14.15.0 || >=15.0.0} - dependencies: - chalk: 4.1.2 - diff-sequences: 27.5.1 - jest-get-type: 27.5.1 - pretty-format: 27.5.1 - dev: false - - /jest-get-type/27.5.1: - resolution: {integrity: sha512-2KY95ksYSaK7DMBWQn6dQz3kqAf3BB64y2udeG+hv4KfSOb9qwcYQstTJc1KCbsix+wLZWZYN8t7nwX3GOBLRw==} - engines: {node: ^10.13.0 || ^12.13.0 || ^14.15.0 || >=15.0.0} - dev: false - - /jest-matcher-utils/27.5.1: - resolution: {integrity: sha512-z2uTx/T6LBaCoNWNFWwChLBKYxTMcGBRjAt+2SbP929/Fflb9aa5LGma654Rz8z9HLxsrUaYzxE9T/EFIL/PAw==} - engines: {node: ^10.13.0 || ^12.13.0 || ^14.15.0 || >=15.0.0} - dependencies: - chalk: 4.1.2 - jest-diff: 27.5.1 - jest-get-type: 27.5.1 - pretty-format: 27.5.1 - dev: false - /js-tokens/4.0.0: resolution: {integrity: sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ==} dev: true @@ -2173,8 +2146,8 @@ packages: type-check: 0.4.0 dev: true - /lilconfig/2.0.5: - resolution: {integrity: sha512-xaYmXZtTHPAw5m+xLN8ab9C+3a8YmV3asNSPOATITbtwrfbwaLJj8h66H1WMIpALCkqsIzK3h7oQ+PdX+LQ9Eg==} + /lilconfig/2.0.6: + resolution: {integrity: sha512-9JROoBW7pobfsx+Sq2JsASvCo6Pfo6WWoUW79HuB1BCoBXD4PLWJPqDF6fNj67pqBYTbAHkE57M1kS/+L1neOg==} engines: {node: '>=10'} /loader-utils/0.2.17: @@ -2186,6 +2159,13 @@ packages: object-assign: 4.1.1 dev: true + /locate-path/6.0.0: + resolution: {integrity: sha512-iPZK6eYjbxRu3uB4/WZ3EsEIMJFMqAoopl3R+zuq0UjcAm/MO6KCweDgPfP3elTztoKP3KtnVHxTn2NHBSDVUw==} + engines: {node: '>=10'} + dependencies: + p-locate: 5.0.0 + dev: true + /lodash-es/4.17.21: resolution: {integrity: sha512-mKnC+QJ9pWVzv+C4/U3rRsHapFfHvQFoFB92e52xeyGMcX6/OlIl78je1u8vePzYZSkkogMPJ2yjxxsb89cxyw==} dev: false @@ -2269,8 +2249,8 @@ packages: hasBin: true dev: true - /monaco-editor/0.33.0: - resolution: {integrity: sha512-VcRWPSLIUEgQJQIE0pVT8FcGBIgFoxz7jtqctE+IiCxWugD0DwgyQBcZBhdSrdMC84eumoqMZsGl2GTreOzwqw==} + /monaco-editor/0.34.0: + resolution: {integrity: sha512-VF+S5zG8wxfinLKLrWcl4WUizMx+LeJrG4PM/M78OhcwocpV0jiyhX/pG6Q9jIOhrb/ckYi6nHnaR5OojlOZCQ==} dev: false /ms/2.0.0: @@ -2286,29 +2266,29 @@ packages: dev: true optional: true - /naive-ui/2.30.7_vue@3.2.37: + /naive-ui/2.30.7_vue@3.2.38: resolution: {integrity: sha512-5F/dcSP5nnbOGuwZl0TJ6PciuyAatEllcTC+Q/aaxJS/N2iEpysonyWNMruBCQn2vHoASTK5i/KqaHdxM9aL1Q==} peerDependencies: vue: ^3.0.0 dependencies: - '@css-render/plugin-bem': 0.15.10_css-render@0.15.10 - '@css-render/vue3-ssr': 0.15.10_vue@3.2.37 - '@types/lodash': 4.14.182 + '@css-render/plugin-bem': 0.15.11_css-render@0.15.11 + '@css-render/vue3-ssr': 0.15.11_vue@3.2.38 + '@types/lodash': 4.14.184 '@types/lodash-es': 4.17.6 async-validator: 4.2.5 - css-render: 0.15.10 - date-fns: 2.28.0 - date-fns-tz: 1.3.5_date-fns@2.28.0 - evtd: 0.2.3 - highlight.js: 11.5.1 + css-render: 0.15.11 + date-fns: 2.29.2 + date-fns-tz: 1.3.7_date-fns@2.29.2 + evtd: 0.2.4 + highlight.js: 11.6.0 lodash: 4.17.21 lodash-es: 4.17.21 - seemly: 0.3.4 + seemly: 0.3.6 treemate: 0.3.11 - vdirs: 0.1.8_vue@3.2.37 - vooks: 0.2.12_vue@3.2.37 - vue: 3.2.37 - vueuc: 0.4.45_vue@3.2.37 + vdirs: 0.1.8_vue@3.2.38 + vooks: 0.2.12_vue@3.2.38 + vue: 3.2.38 + vueuc: 0.4.48_vue@3.2.38 dev: false /nanoid/3.3.4: @@ -2338,8 +2318,8 @@ packages: resolution: {integrity: sha512-Yd3UES5mWCSqR+qNT93S3UoYUkqAZ9lLg8a7g9rimsWmYGK8cVToA4/sF3RrshdyV3sAGMXVUmpMYOw+dLpOuw==} dev: true - /node-releases/2.0.5: - resolution: {integrity: sha512-U9h1NLROZTq9uE1SNffn6WuPDg8icmi3ns4rEl/oTfIle4iLjTliCzgTsbaIFMq/Xn078/lfY/BL0GWZ+psK4Q==} + /node-releases/2.0.6: + resolution: {integrity: sha512-PiVXnNuFm5+iYkLBNeq5211hvO38y63T0i2KKh2KnUs3RpzJ+JtODFjkD8yjLwnDkTYF1eKXheUwdssR+NRZdg==} /normalize-path/3.0.0: resolution: {integrity: sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==} @@ -2388,6 +2368,20 @@ packages: word-wrap: 1.2.3 dev: true + /p-limit/3.1.0: + resolution: {integrity: sha512-TYOanM3wGwNGsZN2cVTYPArw454xnXj5qmWF1bEoAc4+cU/ol7GVh7odevjp1FNHduHc3KZMcFduxU5Xc6uJRQ==} + engines: {node: '>=10'} + dependencies: + yocto-queue: 0.1.0 + dev: true + + /p-locate/5.0.0: + resolution: {integrity: sha512-LaNjtRWUBY++zB5nE/NwcaoMylSPk+S+ZHNB1TzdbMJMny6dynpAGt7X/tl/QYq3TIeE6nxHppbo2LGymrG5Pw==} + engines: {node: '>=10'} + dependencies: + p-limit: 3.1.0 + dev: true + /parent-module/1.0.1: resolution: {integrity: sha512-GQ2EWRpQV8/o+Aw8YqtfZZPfNRWZYkbidE9k5rpl/hC3vtHHBfGm2Ifi6qWV+coDGkrUKZAxE3Lot5kcsRlh+g==} engines: {node: '>=6'} @@ -2400,6 +2394,11 @@ packages: engines: {node: '>= 0.10'} dev: true + /path-exists/4.0.0: + resolution: {integrity: sha512-ak9Qy5Q7jYb2Wwcey5Fpvg2KoAc/ZIhLSLOSBmRmygPsGwkVVt0fZa0qrtMz+m6tJTAHfZQ8FnmB4MG4LWy7/w==} + engines: {node: '>=8'} + dev: true + /path-is-absolute/1.0.1: resolution: {integrity: sha512-AVbw3UJ2e9bq64vSaS9Am0fje1Pa8pbGqTTsmXfaIiMpnr5DlDhfJOuLj9Sf95ZPVDAUerDfEk88MPmPe7UCQg==} engines: {node: '>=0.10.0'} @@ -2436,16 +2435,19 @@ packages: dev: true optional: true - /pinia-plugin-persistedstate/1.6.1_pinia@2.0.14: - resolution: {integrity: sha512-4re0ZO5KgqDnSNWX2RhJVIsIYvH+uaEGTN+UOx34mb7fVoE8A3CfB9QhbMT3Zn7ZSBXTfT2cgI/w7cLq2zNQMw==} + /pinia-plugin-persistedstate/2.1.1_pinia@2.0.22: + resolution: {integrity: sha512-HUgsU5IRtM75eAQiIqzT3p1oPEuYH1/B2ipTMU++yE+FV0LkHaBswdKXs0RMWYCmugO8s62oxLTh/N1dLNp+5A==} peerDependencies: pinia: ^2.0.0 + peerDependenciesMeta: + pinia: + optional: true dependencies: - pinia: 2.0.14_j6bzmzd4ujpabbp5objtwxyjp4 + pinia: 2.0.22_pj7ch6rmow6odq73xb5hfvge3q dev: false - /pinia/2.0.14_j6bzmzd4ujpabbp5objtwxyjp4: - resolution: {integrity: sha512-0nPuZR4TetT/WcLN+feMSjWJku3SQU7dBbXC6uw+R6FLQJCsg+/0pzXyD82T1FmAYe0lsx+jnEDQ1BLgkRKlxA==} + /pinia/2.0.22_pj7ch6rmow6odq73xb5hfvge3q: + resolution: {integrity: sha512-u+b8/BC+tmvo3ACbYO2w5NfxHWFOjvvw9DQnyT0dW8aUMCPRQT5QnfZ5R5W2MzZBMTeZRMQI7V/QFbafmM9QHw==} peerDependencies: '@vue/composition-api': ^1.4.0 typescript: '>=4.4.4' @@ -2456,10 +2458,10 @@ packages: typescript: optional: true dependencies: - '@vue/devtools-api': 6.1.4 - typescript: 4.7.4 - vue: 3.2.37 - vue-demi: 0.13.1_vue@3.2.37 + '@vue/devtools-api': 6.2.1 + typescript: 4.8.2 + vue: 3.2.38 + vue-demi: 0.13.11_vue@3.2.38 dev: false /postcss-filter-plugins/3.0.1: @@ -2486,29 +2488,29 @@ packages: postcss: 6.0.23 dev: true - /postcss-import/14.1.0_postcss@8.4.14: + /postcss-import/14.1.0_postcss@8.4.16: resolution: {integrity: sha512-flwI+Vgm4SElObFVPpTIT7SU7R3qk2L7PyduMcokiaVKuWv9d/U+Gm/QAd8NDLuykTWTkcrjOeD2Pp1rMeBTGw==} engines: {node: '>=10.0.0'} peerDependencies: postcss: ^8.0.0 dependencies: - postcss: 8.4.14 + postcss: 8.4.16 postcss-value-parser: 4.2.0 read-cache: 1.0.0 resolve: 1.22.1 dev: false - /postcss-js/4.0.0_postcss@8.4.14: + /postcss-js/4.0.0_postcss@8.4.16: resolution: {integrity: sha512-77QESFBwgX4irogGVPgQ5s07vLvFqWr228qZY+w6lW599cRlK/HmnlivnnVUxkjHnCu4J16PDMHcH+e+2HbvTQ==} engines: {node: ^12 || ^14 || >= 16} peerDependencies: postcss: ^8.3.3 dependencies: camelcase-css: 2.0.1 - postcss: 8.4.14 + postcss: 8.4.16 dev: false - /postcss-load-config/3.1.4_postcss@8.4.14: + /postcss-load-config/3.1.4_postcss@8.4.16: resolution: {integrity: sha512-6DiM4E7v4coTE4uzA8U//WhtPwyhiim3eyjEMFCnUpzbrkK9wJHgKDT2mR+HbtSrd/NubVaYTOpSpjUl8NQeRg==} engines: {node: '>= 10'} peerDependencies: @@ -2520,17 +2522,17 @@ packages: ts-node: optional: true dependencies: - lilconfig: 2.0.5 - postcss: 8.4.14 + lilconfig: 2.0.6 + postcss: 8.4.16 yaml: 1.10.2 - /postcss-nested/5.0.6_postcss@8.4.14: + /postcss-nested/5.0.6_postcss@8.4.16: resolution: {integrity: sha512-rKqm2Fk0KbA8Vt3AdGN0FB9OBOMDVajMG6ZCf/GoHgdxUJ4sBFp0A/uMIRm+MJUdo33YXEtjqIz8u7DAp8B7DA==} engines: {node: '>=12.0'} peerDependencies: postcss: ^8.2.14 dependencies: - postcss: 8.4.14 + postcss: 8.4.16 postcss-selector-parser: 6.0.10 dev: false @@ -2558,8 +2560,8 @@ packages: supports-color: 5.5.0 dev: true - /postcss/8.4.14: - resolution: {integrity: sha512-E398TUmfAYFPBSdzgeieK2Y1+1cpdxJx8yXbK/m57nRhKSmk1GB2tO4lbLBtlkfPQTDKfe4Xqv1ASWPpayPEig==} + /postcss/8.4.16: + resolution: {integrity: sha512-ipHE1XBvKzm5xI7hiHCZJCSugxvsdq2mPnsq5+UF+VHCjiBvtDrlxJfMBToWaP9D5XlgNmcFGqoHmUn0EYEaRQ==} engines: {node: ^10 || ^12 || >=14} dependencies: nanoid: 3.3.4 @@ -2584,15 +2586,6 @@ packages: hasBin: true dev: true - /pretty-format/27.5.1: - resolution: {integrity: sha512-Qb1gy5OrP5+zDf2Bvnzdl3jsTf1qXVMazbvCoKhtKqVs4/YK4ozX4gKQJJVyNe+cajNPn0KoC0MC3FUmaHWEmQ==} - engines: {node: ^10.13.0 || ^12.13.0 || ^14.15.0 || >=15.0.0} - dependencies: - ansi-regex: 5.0.1 - ansi-styles: 5.2.0 - react-is: 17.0.2 - dev: false - /prr/1.0.1: resolution: {integrity: sha512-yPw4Sng1gWghHQWj0B3ZggWUm4qVbPwPFcRG8KyxiU7J2OHFSoEHKS+EZ3fv5l1t9CyCiop6l/ZYeWbrgoQejw==} dev: true @@ -2611,10 +2604,6 @@ packages: engines: {node: '>=10'} dev: false - /react-is/17.0.2: - resolution: {integrity: sha512-w2GsyukL62IJnlaff/nRegPQR94C/XXamvMWmSHRJ4y7Ts/4ocGRmTHvOs8PSE6pB3dWOrD/nueuU5sduBsQ4w==} - dev: false - /read-cache/1.0.0: resolution: {integrity: sha512-Owdv/Ft7IjOgm/i0xvNDZ1LrRANRfew4b2prF3OWMQLxLfu3bS8FVhCsrSCMK4lR56Y9ya+AThoTpDCTxCmpRA==} dependencies: @@ -2650,7 +2639,7 @@ packages: resolution: {integrity: sha512-nBpuuYuY5jFsli/JIs1oldw6fOQCBioohqWZg/2hiaOybXOft4lonv85uDOKXdf8rhyK159cxU5cDcK/NKk8zw==} hasBin: true dependencies: - is-core-module: 2.9.0 + is-core-module: 2.10.0 path-parse: 1.0.7 supports-preserve-symlinks-flag: 1.0.0 @@ -2665,8 +2654,8 @@ packages: glob: 7.2.3 dev: true - /rollup/2.75.7: - resolution: {integrity: sha512-VSE1iy0eaAYNCxEXaleThdFXqZJ42qDBatAwrfnPlENEZ8erQ+0LYX4JXOLPceWfZpV1VtZwZ3dFCuOZiSyFtQ==} + /rollup/2.78.1: + resolution: {integrity: sha512-VeeCgtGi4P+o9hIg+xz4qQpRl6R401LWEXBmxYKOV4zlF82lyhgh2hTZnheFUbANE8l2A41F458iwj2vEYaXJg==} engines: {node: '>=10.0.0'} hasBin: true optionalDependencies: @@ -2686,7 +2675,7 @@ packages: resolution: {integrity: sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==} dev: true - /sass-loader/13.0.2_sass@1.53.0: + /sass-loader/13.0.2_sass@1.54.8: resolution: {integrity: sha512-BbiqbVmbfJaWVeOOAu2o7DhYWtcNmTfvroVgFXa6k2hHheMxNAeDHLNoDy/Q5aoaVlz0LH+MbMktKwm9vN/j8Q==} engines: {node: '>= 14.15.0'} peerDependencies: @@ -2707,11 +2696,11 @@ packages: dependencies: klona: 2.0.5 neo-async: 2.6.2 - sass: 1.53.0 + sass: 1.54.8 dev: true - /sass/1.53.0: - resolution: {integrity: sha512-zb/oMirbKhUgRQ0/GFz8TSAwRq2IlR29vOUJZOx0l8sV+CkHUfHa4u5nqrG+1VceZp7Jfj59SVW9ogdhTvJDcQ==} + /sass/1.54.8: + resolution: {integrity: sha512-ib4JhLRRgbg6QVy6bsv5uJxnJMTS2soVcCp9Y88Extyy13A8vV0G1fAwujOzmNkFQbR3LvedudAMbtuNRPbQww==} engines: {node: '>=12.0.0'} hasBin: true dependencies: @@ -2724,10 +2713,8 @@ packages: resolution: {integrity: sha512-NqVDv9TpANUjFm0N8uM5GxL36UgKi9/atZw+x7YFnQ8ckwFGKrl4xX4yWtrey3UJm5nP1kUbnYgLopqWNSRhWw==} dev: true - /seemly/0.3.4: - resolution: {integrity: sha512-/crL+UfbtYd6NdLdnf58xaABfeSRt5uKE4N1SFQAW7nIznOrJQchAaslJHxV8/iAfV0LWInrmxJBFdDb0c2/sw==} - dependencies: - '@types/jest': 27.5.2 + /seemly/0.3.6: + resolution: {integrity: sha512-lEV5VB8BUKTo/AfktXJcy+JeXns26ylbMkIUco8CYREsQijuz4mrXres2Q+vMLdwkuLxJdIPQ8IlCIxLYm71Yw==} dev: false /semver/5.7.1: @@ -2843,6 +2830,7 @@ packages: engines: {node: '>=8'} dependencies: has-flag: 4.0.0 + dev: true /supports-preserve-symlinks-flag/1.0.0: resolution: {integrity: sha512-ot0WnXS9fgdkgIcePe6RHNk1WA8+muPa6cSjeR3V8K27q9BB1rTE3R1p7Hv0z1ZyAc8s6Vvv8DIyWf681MAt0w==} @@ -2852,8 +2840,8 @@ packages: resolution: {integrity: sha512-ovssysQTa+luh7A5Weu3Rta6FJlFBBbInjOh722LIt6klpU2/HtdUbszju/G4devcvk8PGt7FCLv5wftu3THUA==} dev: true - /tailwindcss/3.1.6: - resolution: {integrity: sha512-7skAOY56erZAFQssT1xkpk+kWt2NrO45kORlxFPXUt3CiGsVPhH1smuH5XoDH6sGPXLyBv+zgCKA2HWBsgCytg==} + /tailwindcss/3.1.8: + resolution: {integrity: sha512-YSneUCZSFDYMwk+TGq8qYFdCA3yfBRdBlS7txSq0LUmzyeqRe3a8fBQzbz9M3WS/iFT4BNf/nmw9mEzrnSaC0g==} engines: {node: '>=12.13.0'} hasBin: true dependencies: @@ -2866,15 +2854,15 @@ packages: fast-glob: 3.2.11 glob-parent: 6.0.2 is-glob: 4.0.3 - lilconfig: 2.0.5 + lilconfig: 2.0.6 normalize-path: 3.0.0 object-hash: 3.0.0 picocolors: 1.0.0 - postcss: 8.4.14 - postcss-import: 14.1.0_postcss@8.4.14 - postcss-js: 4.0.0_postcss@8.4.14 - postcss-load-config: 3.1.4_postcss@8.4.14 - postcss-nested: 5.0.6_postcss@8.4.14 + postcss: 8.4.16 + postcss-import: 14.1.0_postcss@8.4.16 + postcss-js: 4.0.0_postcss@8.4.16 + postcss-load-config: 3.1.4_postcss@8.4.16 + postcss-nested: 5.0.6_postcss@8.4.16 postcss-selector-parser: 6.0.10 postcss-value-parser: 4.2.0 quick-lru: 5.1.1 @@ -2922,14 +2910,14 @@ packages: resolution: {integrity: sha512-d6xOpEDfsi2CZVlPQzGeux8XMwLT9hssAsaPYExaQMuYskwb+x1x7J371tWlbBdWHroy99KnVB6qIkUbs5X3UQ==} dev: true - /tsutils/3.21.0_typescript@4.7.4: + /tsutils/3.21.0_typescript@4.8.2: resolution: {integrity: sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA==} engines: {node: '>= 6'} peerDependencies: typescript: '>=2.8.0 || >= 3.2.0-dev || >= 3.3.0-dev || >= 3.4.0-dev || >= 3.5.0-dev || >= 3.6.0-dev || >= 3.6.0-beta || >= 3.7.0-dev || >= 3.7.0-beta' dependencies: tslib: 1.14.1 - typescript: 4.7.4 + typescript: 4.8.2 dev: true /type-check/0.4.0: @@ -2944,32 +2932,32 @@ packages: engines: {node: '>=10'} dev: true - /typescript-plugin-css-modules/3.4.0_typescript@4.7.4: + /typescript-plugin-css-modules/3.4.0_typescript@4.8.2: resolution: {integrity: sha512-2MdjfSg4MGex1csCWRUwKD+MpgnvcvLLr9bSAMemU/QYGqBsXdez0cc06H/fFhLtRoKJjXg6PSTur3Gy1Umhpw==} peerDependencies: typescript: '>=3.0.0' dependencies: dotenv: 10.0.0 - icss-utils: 5.1.0_postcss@8.4.14 + icss-utils: 5.1.0_postcss@8.4.16 less: 4.1.3 lodash.camelcase: 4.3.0 - postcss: 8.4.14 + postcss: 8.4.16 postcss-filter-plugins: 3.0.1 postcss-icss-keyframes: 0.2.1 postcss-icss-selectors: 2.0.3 - postcss-load-config: 3.1.4_postcss@8.4.14 + postcss-load-config: 3.1.4_postcss@8.4.16 reserved-words: 0.1.2 - sass: 1.53.0 + sass: 1.54.8 stylus: 0.54.8 tsconfig-paths: 3.14.1 - typescript: 4.7.4 + typescript: 4.8.2 transitivePeerDependencies: - supports-color - ts-node dev: true - /typescript/4.7.4: - resolution: {integrity: sha512-C0WQT0gezHuw6AdY1M2jxUO83Rjf0HP7Sk1DtXj6j1EwkQNZrHAg2XPWlq62oqEhYvONq5pkC2Y9oPljWToLmQ==} + /typescript/4.8.2: + resolution: {integrity: sha512-C0I1UsrrDHo2fYI5oaCGbSejwX4ch+9Y5jTQELvovfmFkK3HHSZJB8MSJcWLmCUBzQBchCrZ9rMRV6GuNrvGtw==} engines: {node: '>=4.2.0'} hasBin: true @@ -2978,13 +2966,13 @@ packages: engines: {node: '>= 10.0.0'} dev: true - /update-browserslist-db/1.0.4_browserslist@4.21.1: - resolution: {integrity: sha512-jnmO2BEGUjsMOe/Fg9u0oczOe/ppIDZPebzccl1yDWGLFP16Pa1/RM5wEoKYPG2zstNcDuAStejyxsOuKINdGA==} + /update-browserslist-db/1.0.7_browserslist@4.21.3: + resolution: {integrity: sha512-iN/XYesmZ2RmmWAiI4Z5rq0YqSiv0brj9Ce9CfhNE4xIW2h+MFxcgkxIzZ+ShkFPUkjU3gQ+3oypadD3RAMtrg==} hasBin: true peerDependencies: browserslist: '>= 4.21.0' dependencies: - browserslist: 4.21.1 + browserslist: 4.21.3 escalade: 3.1.1 picocolors: 1.0.0 @@ -3002,24 +2990,20 @@ packages: /util-deprecate/1.0.2: resolution: {integrity: sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==} - /v8-compile-cache/2.3.0: - resolution: {integrity: sha512-l8lCEmLcLYZh4nbunNZvQCJc5pv7+RCwa8q/LdUx8u7lsWvPDKmpodJAJNwkAhJC//dFY48KuIEmjtd4RViDrA==} - dev: true - - /vdirs/0.1.8_vue@3.2.37: + /vdirs/0.1.8_vue@3.2.38: resolution: {integrity: sha512-H9V1zGRLQZg9b+GdMk8MXDN2Lva0zx72MPahDKc30v+DtwKjfyOSXWRIX4t2mhDubM1H09gPhWeth/BJWPHGUw==} peerDependencies: vue: ^3.0.11 dependencies: - evtd: 0.2.3 - vue: 3.2.37 + evtd: 0.2.4 + vue: 3.2.38 dev: false /vfonts/0.0.3: resolution: {integrity: sha512-nguyw8L6Un8eelg1vQ31vIU2ESxqid7EYmy8V+MDeMaHBqaRSkg3dTBToC1PR00D89UzS/SLkfYPnx0Wf23IQQ==} dev: false - /vite-plugin-compression/0.5.1_vite@2.9.13: + /vite-plugin-compression/0.5.1_vite@3.1.0: resolution: {integrity: sha512-5QJKBDc+gNYVqL/skgFAP81Yuzo9R+EAf19d+EtsMF/i8kFUpNi3J/H01QD3Oo8zBQn+NzoCIFkpPLynoOzaJg==} peerDependencies: vite: '>=2.0.0' @@ -3027,19 +3011,20 @@ packages: chalk: 4.1.2 debug: 4.3.4 fs-extra: 10.1.0 - vite: 2.9.13_sass@1.53.0 + vite: 3.1.0_sass@1.54.8 transitivePeerDependencies: - supports-color dev: true - /vite/2.9.13_sass@1.53.0: - resolution: {integrity: sha512-AsOBAaT0AD7Mhe8DuK+/kE4aWYFMx/i0ZNi98hJclxb4e0OhQcZYUrvLjIaQ8e59Ui7txcvKMiJC1yftqpQoDw==} - engines: {node: '>=12.2.0'} + /vite/3.1.0_sass@1.54.8: + resolution: {integrity: sha512-YBg3dUicDpDWFCGttmvMbVyS9ydjntwEjwXRj2KBFwSB8SxmGcudo1yb8FW5+M/G86aS8x828ujnzUVdsLjs9g==} + engines: {node: ^14.18.0 || >=16.0.0} hasBin: true peerDependencies: less: '*' sass: '*' stylus: '*' + terser: ^5.4.0 peerDependenciesMeta: less: optional: true @@ -3047,27 +3032,29 @@ packages: optional: true stylus: optional: true + terser: + optional: true dependencies: - esbuild: 0.14.47 - postcss: 8.4.14 + esbuild: 0.15.7 + postcss: 8.4.16 resolve: 1.22.1 - rollup: 2.75.7 - sass: 1.53.0 + rollup: 2.78.1 + sass: 1.54.8 optionalDependencies: fsevents: 2.3.2 dev: true - /vooks/0.2.12_vue@3.2.37: + /vooks/0.2.12_vue@3.2.38: resolution: {integrity: sha512-iox0I3RZzxtKlcgYaStQYKEzWWGAduMmq+jS7OrNdQo1FgGfPMubGL3uGHOU9n97NIvfFDBGnpSvkWyb/NSn/Q==} peerDependencies: vue: ^3.0.0 dependencies: - evtd: 0.2.3 - vue: 3.2.37 + evtd: 0.2.4 + vue: 3.2.38 dev: false - /vue-demi/0.13.1_vue@3.2.37: - resolution: {integrity: sha512-xmkJ56koG3ptpLnpgmIzk9/4nFf4CqduSJbUM0OdPoU87NwRuZ6x49OLhjSa/fC15fV+5CbEnrxU4oyE022svg==} + /vue-demi/0.13.11_vue@3.2.38: + resolution: {integrity: sha512-IR8HoEEGM65YY3ZJYAjMlKygDQn25D5ajNFNoKh9RSDMQtlzCxtfQjdQgv9jjK+m3377SsJXY8ysq8kLCZL25A==} engines: {node: '>=12'} hasBin: true requiresBuild: true @@ -3078,20 +3065,20 @@ packages: '@vue/composition-api': optional: true dependencies: - vue: 3.2.37 + vue: 3.2.38 dev: false - /vue-eslint-parser/9.0.3_eslint@8.18.0: + /vue-eslint-parser/9.0.3_eslint@8.23.0: resolution: {integrity: sha512-yL+ZDb+9T0ELG4VIFo/2anAOz8SvBdlqEnQnvJ3M7Scq56DvtjY0VY88bByRZB0D4J0u8olBcfrXTVONXsh4og==} engines: {node: ^14.17.0 || >=16.0.0} peerDependencies: eslint: '>=6.0.0' dependencies: debug: 4.3.4 - eslint: 8.18.0 + eslint: 8.23.0 eslint-scope: 7.1.1 eslint-visitor-keys: 3.3.0 - espree: 9.3.2 + espree: 9.4.0 esquery: 1.4.0 lodash: 4.17.21 semver: 7.3.7 @@ -3099,60 +3086,61 @@ packages: - supports-color dev: true - /vue-i18n/9.1.10_vue@3.2.37: - resolution: {integrity: sha512-jpr7gV5KPk4n+sSPdpZT8Qx3XzTcNDWffRlHV/cT2NUyEf+sEgTTmLvnBAibjOFJ0zsUyZlVTAWH5DDnYep+1g==} - engines: {node: '>= 10'} + /vue-i18n/9.2.2_vue@3.2.38: + resolution: {integrity: sha512-yswpwtj89rTBhegUAv9Mu37LNznyu3NpyLQmozF3i1hYOhwpG8RjcjIFIIfnu+2MDZJGSZPXaKWvnQA71Yv9TQ==} + engines: {node: '>= 14'} peerDependencies: vue: ^3.0.0 dependencies: - '@intlify/core-base': 9.1.10 - '@intlify/shared': 9.1.10 - '@intlify/vue-devtools': 9.1.10 - '@vue/devtools-api': 6.1.4 - vue: 3.2.37 + '@intlify/core-base': 9.2.2 + '@intlify/shared': 9.2.2 + '@intlify/vue-devtools': 9.2.2 + '@vue/devtools-api': 6.2.1 + vue: 3.2.38 dev: false - /vue-router/4.0.16_vue@3.2.37: - resolution: {integrity: sha512-JcO7cb8QJLBWE+DfxGUL3xUDOae/8nhM1KVdnudadTAORbuxIC/xAydC5Zr/VLHUDQi1ppuTF5/rjBGzgzrJNA==} + /vue-router/4.1.5_vue@3.2.38: + resolution: {integrity: sha512-IsvoF5D2GQ/EGTs/Th4NQms9gd2NSqV+yylxIyp/OYp8xOwxmU8Kj/74E9DTSYAyH5LX7idVUngN3JSj1X4xcQ==} peerDependencies: vue: ^3.2.0 dependencies: - '@vue/devtools-api': 6.1.4 - vue: 3.2.37 + '@vue/devtools-api': 6.2.1 + vue: 3.2.38 dev: false - /vue-tsc/0.38.2_typescript@4.7.4: - resolution: {integrity: sha512-+OMmpw9BZC9khul3I1HGtWchv7BCiaM7NvfdilVAiOFkjnivIoaW6jJm6YPQJaEPouePtpkDUWovyzgNxWdDsw==} + /vue-tsc/0.40.9_typescript@4.8.2: + resolution: {integrity: sha512-GnfwngCgbUvFgs+vaPesrJB76yoX1W/DSQZqoQ+pArjZ9+EFCFkqMpihE1D8W5p/tgTCAAPr/3Sfz/jtTiYGaA==} hasBin: true peerDependencies: typescript: '*' dependencies: - '@volar/vue-typescript': 0.38.2 - typescript: 4.7.4 + '@volar/vue-language-core': 0.40.9 + '@volar/vue-typescript': 0.40.9 + typescript: 4.8.2 dev: true - /vue/3.2.37: - resolution: {integrity: sha512-bOKEZxrm8Eh+fveCqS1/NkG/n6aMidsI6hahas7pa0w/l7jkbssJVsRhVDs07IdDq7h9KHswZOgItnwJAgtVtQ==} + /vue/3.2.38: + resolution: {integrity: sha512-hHrScEFSmDAWL0cwO4B6WO7D3sALZPbfuThDsGBebthrNlDxdJZpGR3WB87VbjpPh96mep1+KzukYEhpHDFa8Q==} dependencies: - '@vue/compiler-dom': 3.2.37 - '@vue/compiler-sfc': 3.2.37 - '@vue/runtime-dom': 3.2.37 - '@vue/server-renderer': 3.2.37_vue@3.2.37 - '@vue/shared': 3.2.37 + '@vue/compiler-dom': 3.2.38 + '@vue/compiler-sfc': 3.2.38 + '@vue/runtime-dom': 3.2.38 + '@vue/server-renderer': 3.2.38_vue@3.2.38 + '@vue/shared': 3.2.38 - /vueuc/0.4.45_vue@3.2.37: - resolution: {integrity: sha512-TdiDHn9TkJyStKJYzGGiZlekasdoYTial47UAMwQ5fuVp/5GAeJKvFk4UMtSrDfECsNncfNJ6TNveVBAq+IvrQ==} + /vueuc/0.4.48_vue@3.2.38: + resolution: {integrity: sha512-dQTBLxCzfaPuzD3c4/dIxAULtnyY+xwdotCRFUDgf0DJiwuR3tI+txJ9K8uJKmaHwc1JDUVqhRAj9Jd/pvInWg==} peerDependencies: vue: ^3.0.11 dependencies: - '@css-render/vue3-ssr': 0.15.10_vue@3.2.37 - '@juggle/resize-observer': 3.3.1 - css-render: 0.15.10 - evtd: 0.2.3 - seemly: 0.3.4 - vdirs: 0.1.8_vue@3.2.37 - vooks: 0.2.12_vue@3.2.37 - vue: 3.2.37 + '@css-render/vue3-ssr': 0.15.11_vue@3.2.38 + '@juggle/resize-observer': 3.4.0 + css-render: 0.15.11 + evtd: 0.2.4 + seemly: 0.3.6 + vdirs: 0.1.8_vue@3.2.38 + vooks: 0.2.12_vue@3.2.38 + vue: 3.2.38 dev: false /which/2.0.2: @@ -3190,6 +3178,11 @@ packages: resolution: {integrity: sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg==} engines: {node: '>= 6'} + /yocto-queue/0.1.0: + resolution: {integrity: sha512-rVksvsnNCdJ/ohGc6xgPwyN8eheCxsiLM8mxuE/t/mOVqJewPuO1miLpTHQiRgTKCLexL4MeAFVagts7HmNZ2Q==} + engines: {node: '>=10'} + dev: true + /zrender/5.3.2: resolution: {integrity: sha512-8IiYdfwHj2rx0UeIGZGGU4WEVSDEdeVCaIg/fomejg1Xu6OifAL1GVzIPHg2D+MyUkbNgPWji90t0a8IDk+39w==} dependencies: diff --git a/seatunnel-ui/src/themes/modules/light.ts b/seatunnel-ui/src/themes/modules/light.ts index d691a04ecf3..4986ed2c7b8 100644 --- a/seatunnel-ui/src/themes/modules/light.ts +++ b/seatunnel-ui/src/themes/modules/light.ts @@ -15,6 +15,20 @@ * limitations under the License. */ -const light = {} +import type { GlobalThemeOverrides } from 'naive-ui' + +const light: GlobalThemeOverrides = { + common: { + primaryColor: '#1890ff', + primaryColorHover: '#40a9ff', + primaryColorPressed: '#096dd9', + primaryColorSuppl: '#1890ff', + + infoColor: '#1890ff', + successColor: '#52c41a', + warningColor: '#faad14', + errorColor: '#ff4d4f' + } +} export default light From 51c28a3387fa99ebea9118034cd0d99d3c82578a Mon Sep 17 00:00:00 2001 From: ChunFu Wu <319355703@qq.com> Date: Sun, 11 Sep 2022 16:33:08 +0800 Subject: [PATCH 24/44] [Feature][Connector-V2] Add mongodb connecter sink (#2694) * [Feature][Connector-V2] Add mongodb connecter sink * Add license header * Add spark mongodb sink e2e * Add spark mongodb sink e2e * Fix * Fix * Fix --- docs/en/connector-v2/sink/MongoDB.md | 46 ++++++ plugin-mapping.properties | 1 + .../seatunnel/mongodb/sink/MongodbSink.java | 80 ++++++++++ .../mongodb/sink/MongodbSinkWriter.java | 72 +++++++++ .../v2/mongodb/FakeSourceToMongodbIT.java | 137 ++++++++++++++++++ .../resources/mongodb/fake_to_mongodb.conf | 74 ++++++++++ .../connector-mongodb-spark-e2e/pom.xml | 55 +++++++ .../v2/mongodb/FakeSourceToMongodbIT.java | 137 ++++++++++++++++++ .../src/test/resources/log4j.properties | 23 +++ .../resources/mongodb/fake_to_mongodb.conf | 70 +++++++++ .../seatunnel-spark-connector-v2-e2e/pom.xml | 1 + 11 files changed, 696 insertions(+) create mode 100644 docs/en/connector-v2/sink/MongoDB.md create mode 100644 seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSink.java create mode 100644 seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkWriter.java create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/FakeSourceToMongodbIT.java create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/resources/mongodb/fake_to_mongodb.conf create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/mongodb/FakeSourceToMongodbIT.java create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/resources/log4j.properties create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/resources/mongodb/fake_to_mongodb.conf diff --git a/docs/en/connector-v2/sink/MongoDB.md b/docs/en/connector-v2/sink/MongoDB.md new file mode 100644 index 00000000000..2768aa03c98 --- /dev/null +++ b/docs/en/connector-v2/sink/MongoDB.md @@ -0,0 +1,46 @@ +# MongoDb + +> MongoDB sink connector + +## Description + +Write data to `MongoDB` + +## Key features + +- [x] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [ ] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [schema projection](../../concept/connector-v2-features.md) +- [ ] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) + +## Options + +| name | type | required | default value | +|------------| ------ |----------| ------------- | +| uri | string | yes | - | +| database | string | yes | - | +| collection | string | yes | - | + +### uri [string] + +uri to write to mongoDB + +### database [string] + +database to write to mongoDB + +### collection [string] + +collection to write to mongoDB + +## Example + +```bash +mongodb { + uri = "mongodb://username:password@127.0.0.1:27017/mypost?retryWrites=true&writeConcern=majority" + database = "mydatabase" + collection = "mycollection" +} +``` diff --git a/plugin-mapping.properties b/plugin-mapping.properties index e90f025348c..cde36d782a4 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -127,4 +127,5 @@ seatunnel.sink.Redis = connector-redis seatunnel.sink.DataHub = connector-datahub seatunnel.sink.Sentry = connector-sentry seatunnel.source.MongoDB = connector-mongodb +seatunnel.sink.MongoDB = connector-mongodb diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSink.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSink.java new file mode 100644 index 00000000000..e3abeed0bcb --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSink.java @@ -0,0 +1,80 @@ +/* + * 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.seatunnel.connectors.seatunnel.mongodb.sink; + +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.COLLECTION; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.DATABASE; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.URI; + +import org.apache.seatunnel.api.common.PrepareFailException; +import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.config.CheckConfigUtil; +import org.apache.seatunnel.common.config.CheckResult; +import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbParameters; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigBeanFactory; + +import com.google.auto.service.AutoService; + +import java.io.IOException; + +@AutoService(SeaTunnelSink.class) +public class MongodbSink extends AbstractSimpleSink { + + private SeaTunnelRowType rowType; + + private MongodbParameters params; + + @Override + public String getPluginName() { + return "MongoDB"; + } + + @Override + public void prepare(Config config) throws PrepareFailException { + CheckResult result = CheckConfigUtil.checkAllExists(config, URI, DATABASE, COLLECTION); + if (!result.isSuccess()) { + throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg()); + } + + this.params = ConfigBeanFactory.create(config, MongodbParameters.class); + } + + @Override + public void setTypeInfo(SeaTunnelRowType rowType) { + this.rowType = rowType; + } + + @Override + public SeaTunnelDataType getConsumedType() { + return rowType; + } + + @Override + public AbstractSinkWriter createWriter(SinkWriter.Context context) throws IOException { + return new MongodbSinkWriter(rowType, params); + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkWriter.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkWriter.java new file mode 100644 index 00000000000..a1000931cdb --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkWriter.java @@ -0,0 +1,72 @@ +/* + * 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.seatunnel.connectors.seatunnel.mongodb.sink; + +import org.apache.seatunnel.api.serialization.SerializationSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbParameters; +import org.apache.seatunnel.format.json.JsonSerializationSchema; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCollection; +import org.bson.Document; + +import java.io.IOException; + +public class MongodbSinkWriter extends AbstractSinkWriter { + + private final SeaTunnelRowType rowType; + + private final SerializationSchema serializationSchema; + + private MongoClient client; + + private final String database; + + private final String collection; + + private final MongoCollection mongoCollection; + + public MongodbSinkWriter(SeaTunnelRowType rowType, MongodbParameters params) { + this.rowType = rowType; + this.database = params.getDatabase(); + this.collection = params.getCollection(); + this.client = MongoClients.create(params.getUri()); + this.mongoCollection = this.client.getDatabase(database).getCollection(collection); + this.serializationSchema = new JsonSerializationSchema(rowType); + } + + @Override + public void write(SeaTunnelRow rows) throws IOException { + byte[] serialize = serializationSchema.serialize(rows); + String content = new String(serialize); + + Document doc = Document.parse(content); + mongoCollection.insertOne(doc); + } + + @Override + public void close() throws IOException { + if (client != null) { + client.close(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/FakeSourceToMongodbIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/FakeSourceToMongodbIT.java new file mode 100644 index 00000000000..b052ee00f53 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/FakeSourceToMongodbIT.java @@ -0,0 +1,137 @@ +/* + * 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.seatunnel.e2e.flink.v2.mongodb; + +import static java.net.HttpURLConnection.HTTP_OK; +import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED; + +import org.apache.seatunnel.e2e.flink.FlinkContainer; + +import com.google.common.collect.Lists; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCursor; +import lombok.extern.slf4j.Slf4j; +import org.bson.Document; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +@Slf4j +public class FakeSourceToMongodbIT extends FlinkContainer { + + private static final String MONGODB_IMAGE = "mongo:latest"; + + private static final String MONGODB_CONTAINER_HOST = "flink_e2e_mongodb_sink"; + + private static final String MONGODB_HOST = "localhost"; + + private static final int MONGODB_PORT = 27017; + + private static final String MONGODB_DATABASE = "test_db"; + + private static final String MONGODB_COLLECTION = "test_table"; + + private static final String MONGODB_URI = String.format("mongodb://%s:%d/%s", MONGODB_HOST, MONGODB_PORT, MONGODB_DATABASE); + + private MongoClient client; + + private GenericContainer mongodbContainer; + + @BeforeEach + public void startMongoContainer() { + DockerImageName imageName = DockerImageName.parse(MONGODB_IMAGE); + mongodbContainer = new GenericContainer<>(imageName) + .withNetwork(NETWORK) + .withNetworkAliases(MONGODB_CONTAINER_HOST) + .withExposedPorts(MONGODB_PORT) + .waitingFor(new HttpWaitStrategy() + .forPort(MONGODB_PORT) + .forStatusCodeMatching(response -> response == HTTP_OK || response == HTTP_UNAUTHORIZED) + .withStartupTimeout(Duration.ofMinutes(2))) + .withLogConsumer(new Slf4jLogConsumer(log)); + mongodbContainer.setPortBindings(Lists.newArrayList(String.format("%s:%s", MONGODB_PORT, MONGODB_PORT))); + Startables.deepStart(Stream.of(mongodbContainer)).join(); + log.info("Mongodb container started"); + Awaitility.given().ignoreExceptions() + .await() + .atMost(30, TimeUnit.SECONDS) + .untilAsserted(this::initConnection); + } + + public void initConnection() { + client = MongoClients.create(MONGODB_URI); + } + + @Test + public void testMongodbSink() throws IOException, InterruptedException { + Container.ExecResult execResult = executeSeaTunnelFlinkJob("/mongodb/fake_to_mongodb.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + + List> list = new ArrayList<>(); + try (MongoCursor mongoCursor = client.getDatabase(MONGODB_DATABASE) + .getCollection(MONGODB_COLLECTION) + .find() + .iterator() + ) { + while (mongoCursor.hasNext()) { + Document doc = mongoCursor.next(); + HashMap map = new HashMap<>(doc.size()); + Set> entries = doc.entrySet(); + for (Map.Entry entry : entries) { + String key = entry.getKey(); + Object value = entry.getValue(); + map.put(key, value); + } + log.info("Document ===>>>: " + map); + list.add(map); + } + } + + Assertions.assertEquals(10, list.size()); + } + + @AfterEach + public void close() { + super.close(); + if (client != null) { + client.close(); + } + if (mongodbContainer != null) { + mongodbContainer.close(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/resources/mongodb/fake_to_mongodb.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/resources/mongodb/fake_to_mongodb.conf new file mode 100644 index 00000000000..482f92ee76e --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/resources/mongodb/fake_to_mongodb.conf @@ -0,0 +1,74 @@ +# +# 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. +# + +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set flink configuration here + execution.parallelism = 1 + #job.mode = "BATCH" + #job.mode = "STREAMING" + #execution.checkpoint.interval = 10000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_decimal = "decimal(30, 8)" + c_bytes = bytes + c_date = date + c_time = time + c_timestamp = timestamp + } + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/Fake +} + +transform { + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/transform/Sql +} + +sink { + MongoDB { + uri = "mongodb://flink_e2e_mongodb_sink:27017/test_db?retryWrites=true&writeConcern=majority" + database = "test_db" + collection = "test_table" + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/sink/MongoDB +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/pom.xml new file mode 100644 index 00000000000..f10c0de4d47 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/pom.xml @@ -0,0 +1,55 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-spark-connector-v2-e2e + ${revision} + + + connector-mongodb-spark-e2e + + + + org.apache.seatunnel + connector-spark-e2e-base + ${project.version} + tests + test-jar + test + + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-mongodb + ${project.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/mongodb/FakeSourceToMongodbIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/mongodb/FakeSourceToMongodbIT.java new file mode 100644 index 00000000000..a26307ca017 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/mongodb/FakeSourceToMongodbIT.java @@ -0,0 +1,137 @@ +/* + * 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.seatunnel.e2e.spark.v2.mongodb; + +import static java.net.HttpURLConnection.HTTP_OK; +import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED; + +import org.apache.seatunnel.e2e.spark.SparkContainer; + +import com.google.common.collect.Lists; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCursor; +import lombok.extern.slf4j.Slf4j; +import org.bson.Document; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +@Slf4j +public class FakeSourceToMongodbIT extends SparkContainer { + + private static final String MONGODB_IMAGE = "mongo:latest"; + + private static final String MONGODB_CONTAINER_HOST = "spark_e2e_mongodb_sink"; + + private static final String MONGODB_HOST = "localhost"; + + private static final int MONGODB_PORT = 27017; + + private static final String MONGODB_DATABASE = "test_db"; + + private static final String MONGODB_COLLECTION = "test_table"; + + private static final String MONGODB_URI = String.format("mongodb://%s:%d/%s", MONGODB_HOST, MONGODB_PORT, MONGODB_DATABASE); + + private MongoClient client; + + private GenericContainer mongodbContainer; + + @BeforeEach + public void startMongoContainer() { + DockerImageName imageName = DockerImageName.parse(MONGODB_IMAGE); + mongodbContainer = new GenericContainer<>(imageName) + .withNetwork(NETWORK) + .withNetworkAliases(MONGODB_CONTAINER_HOST) + .withExposedPorts(MONGODB_PORT) + .waitingFor(new HttpWaitStrategy() + .forPort(MONGODB_PORT) + .forStatusCodeMatching(response -> response == HTTP_OK || response == HTTP_UNAUTHORIZED) + .withStartupTimeout(Duration.ofMinutes(2))) + .withLogConsumer(new Slf4jLogConsumer(log)); + mongodbContainer.setPortBindings(Lists.newArrayList(String.format("%s:%s", MONGODB_PORT, MONGODB_PORT))); + Startables.deepStart(Stream.of(mongodbContainer)).join(); + log.info("Mongodb container started"); + Awaitility.given().ignoreExceptions() + .await() + .atMost(30, TimeUnit.SECONDS) + .untilAsserted(this::initConnection); + } + + public void initConnection() { + client = MongoClients.create(MONGODB_URI); + } + + @Test + public void testMongodbSink() throws IOException, InterruptedException { + Container.ExecResult execResult = executeSeaTunnelSparkJob("/mongodb/fake_to_mongodb.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + + List> list = new ArrayList<>(); + try (MongoCursor mongoCursor = client.getDatabase(MONGODB_DATABASE) + .getCollection(MONGODB_COLLECTION) + .find() + .iterator() + ) { + while (mongoCursor.hasNext()) { + Document doc = mongoCursor.next(); + HashMap map = new HashMap<>(doc.size()); + Set> entries = doc.entrySet(); + for (Map.Entry entry : entries) { + String key = entry.getKey(); + Object value = entry.getValue(); + map.put(key, value); + } + log.info("Document ===>>>: " + map); + list.add(map); + } + } + + Assertions.assertEquals(10, list.size()); + } + + @AfterEach + public void close() { + super.close(); + if (client != null) { + client.close(); + } + if (mongodbContainer != null) { + mongodbContainer.close(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/resources/log4j.properties b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/resources/log4j.properties new file mode 100644 index 00000000000..89ed3ad31e5 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/resources/log4j.properties @@ -0,0 +1,23 @@ +# +# 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. +# + +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/resources/mongodb/fake_to_mongodb.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/resources/mongodb/fake_to_mongodb.conf new file mode 100644 index 00000000000..0c978615e91 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-mongodb-spark-e2e/src/test/resources/mongodb/fake_to_mongodb.conf @@ -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. +# + +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set spark configuration here + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + FakeSource { + result_table_name = "fake" + schema = { + fields { + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_decimal = "decimal(30, 8)" + c_bytes = bytes + } + } + } + + # If you would like to get more information about how to configure seatunnel and see full list of source plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/source/Fake +} + +transform { + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/transform/Sql +} + +sink { + MongoDB { + uri = "mongodb://spark_e2e_mongodb_sink:27017/test_db?retryWrites=true&writeConcern=majority" + database = "test_db" + collection = "test_table" + } + + # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, + # please go to https://seatunnel.apache.org/docs/connector-v2/sink/MongoDB +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml index 898bf04c930..c1fd6c512b1 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml @@ -34,6 +34,7 @@ connector-iotdb-spark-e2e connector-jdbc-spark-e2e connector-redis-spark-e2e + connector-mongodb-spark-e2e From 9e82ad2594de330acfb1d48653af9227f52fb565 Mon Sep 17 00:00:00 2001 From: mans2singh Date: Sun, 11 Sep 2022 08:27:47 -0400 Subject: [PATCH 25/44] [hotfix][transform][replace] Corrected syntax (#2716) --- docs/en/transform/replace.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/transform/replace.md b/docs/en/transform/replace.md index 11499a0e1bf..1bf57fc7f69 100644 --- a/docs/en/transform/replace.md +++ b/docs/en/transform/replace.md @@ -69,7 +69,7 @@ Use `Replace` as udf in sql. Replace { fields = "_replaced" pattern = "([^ ]*) ([^ ]*)" - replacement = "$2 + replacement = "$2" isRegex = true replaceFirst = true } From 3ea7a429b9de19298d272ec777e52b5f8ba149a7 Mon Sep 17 00:00:00 2001 From: TyrantLucifer Date: Sun, 11 Sep 2022 20:48:20 +0800 Subject: [PATCH 26/44] [Bug][Connector-V2-spark-examples] Fix jackson dependency conflict (#2711) * [Bug][Connector-V2-e2e] Fix jackson dependency conflict * [Bug][Connector-V2-spark-examples] Fix jackson dependency conflict --- .../seatunnel-spark-connector-v2-example/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml b/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml index 90dbe5c17d1..abb515daf15 100644 --- a/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml +++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml @@ -31,8 +31,15 @@ compile + 2.6.7 + + com.fasterxml.jackson.core + jackson-databind + ${spark.2.4.0.jackson.version} + + org.apache.seatunnel seatunnel-spark-starter From b3392cb9d1c2769fa3e235c39eb8c60c992c304d Mon Sep 17 00:00:00 2001 From: Laglangyue <35491928+laglangyue@users.noreply.github.com> Date: Sun, 11 Sep 2022 22:39:01 +0800 Subject: [PATCH 27/44] [Bug] Fix path split exception in win10,not check file existed (#2715) * [Bug]fix path split exception in win10,not check file existed * improve code Co-authored-by: laglangyue --- .../seatunnel/e2e/common/AbstractContainer.java | 16 +++++++--------- .../e2e/common/AbstractSparkContainer.java | 3 ++- .../seatunnel/e2e/common/ContainerUtil.java | 11 +++++++++-- .../seatunnel/e2e/flink/FlinkContainer.java | 2 +- .../seatunnel/e2e/flink/FlinkContainer.java | 2 +- .../seatunnel/e2e/flink/sql/FlinkContainer.java | 2 +- .../seatunnel/e2e/spark/SparkContainer.java | 2 +- .../seatunnel/e2e/spark/SparkContainer.java | 2 +- 8 files changed, 23 insertions(+), 17 deletions(-) diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractContainer.java index f4fb9d1169f..71e93ab9200 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractContainer.java @@ -34,6 +34,7 @@ import java.util.List; public abstract class AbstractContainer { + protected static final Logger LOG = LoggerFactory.getLogger(AbstractContainer.class); protected static final String START_ROOT_MODULE_NAME = "seatunnel-core"; @@ -42,15 +43,14 @@ public abstract class AbstractContainer { protected final String startModuleFullPath; public AbstractContainer() { - String[] modules = getStartModulePath().split(File.separator); - this.startModuleName = modules[modules.length - 1]; - this.startModuleFullPath = PROJECT_ROOT_PATH + File.separator + - START_ROOT_MODULE_NAME + File.separator + getStartModulePath(); + this.startModuleName = getStartModuleName(); + this.startModuleFullPath = PROJECT_ROOT_PATH + File.separator + START_ROOT_MODULE_NAME + File.separator + this.startModuleName; + ContainerUtil.checkPathExist(startModuleFullPath); } protected abstract String getDockerImage(); - protected abstract String getStartModulePath(); + protected abstract String getStartModuleName(); protected abstract String getStartShellName(); @@ -65,11 +65,9 @@ public AbstractContainer() { protected abstract List getExtraStartShellCommands(); protected void copySeaTunnelStarter(GenericContainer container) { - String[] modules = getStartModulePath().split(File.separator); - final String startModuleName = modules[modules.length - 1]; ContainerUtil.copySeaTunnelStarter(container, - startModuleName, - PROJECT_ROOT_PATH + File.separator + START_ROOT_MODULE_NAME + File.separator + getStartModulePath(), + this.startModuleName, + this.startModuleFullPath, getSeaTunnelHomeInContainer(), getStartShellName()); } diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractSparkContainer.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractSparkContainer.java index fc16846c1db..6cb49b5a075 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractSparkContainer.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/AbstractSparkContainer.java @@ -35,6 +35,7 @@ @TestInstance(TestInstance.Lifecycle.PER_CLASS) public abstract class AbstractSparkContainer extends AbstractContainer { + private static final Logger LOG = LoggerFactory.getLogger(AbstractSparkContainer.class); private static final String SPARK_SEATUNNEL_HOME = "/tmp/spark/seatunnel"; @@ -78,7 +79,7 @@ public void close() { @Override protected List getExtraStartShellCommands() { return Arrays.asList("--master local", - "--deploy-mode client"); + "--deploy-mode client"); } public Container.ExecResult executeSeaTunnelSparkJob(String confFile) throws IOException, InterruptedException { diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/ContainerUtil.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/ContainerUtil.java index 237e30f0911..764839b15c9 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/ContainerUtil.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/ContainerUtil.java @@ -23,6 +23,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; import org.apache.seatunnel.shade.com.typesafe.config.ConfigResolveOptions; +import org.junit.jupiter.api.Assertions; import org.testcontainers.containers.GenericContainer; import org.testcontainers.utility.MountableFile; @@ -44,7 +45,7 @@ public final class ContainerUtil { /** * An error occurs when the user is not a submodule of seatunnel-e2e. */ - public static final String PROJECT_ROOT_PATH = System.getProperty("user.dir").split("/seatunnel-e2e/")[0]; + public static final String PROJECT_ROOT_PATH = Paths.get(System.getProperty("user.dir")).getParent().getParent().getParent().toString(); public static void copyConnectorJarToContainer(GenericContainer container, String confFile, @@ -83,12 +84,14 @@ public static void copySeaTunnelStarter(GenericContainer container, final String startJarName = startModuleName + ".jar"; // copy lib final String startJarPath = startModulePath + File.separator + "target" + File.separator + startJarName; + checkPathExist(startJarPath); container.copyFileToContainer( MountableFile.forHostPath(startJarPath), Paths.get(Paths.get(seatunnelHomeInContainer, "lib").toString(), startJarName).toString()); // copy bin - final String startBinPath = startModulePath + File.separator + "/src/main/bin/" + startShellName; + final String startBinPath = startModulePath + File.separator + "src/main/bin/" + startShellName; + checkPathExist(startBinPath); container.copyFileToContainer( MountableFile.forHostPath(startBinPath), Paths.get(Paths.get(seatunnelHomeInContainer, "bin").toString(), startShellName).toString()); @@ -162,4 +165,8 @@ private static Config getConfig(File file) { .resolve(ConfigResolveOptions.defaults().setAllowUnresolved(true)) .resolveWith(ConfigFactory.systemProperties(), ConfigResolveOptions.defaults().setAllowUnresolved(true)); } + + public static void checkPathExist(String path) { + Assertions.assertTrue(new File(path).exists(), path + "must exist"); + } } diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java index 6ecaa77cbb9..f6f02cce752 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java @@ -32,7 +32,7 @@ protected String getDockerImage() { } @Override - protected String getStartModulePath() { + protected String getStartModuleName() { return "seatunnel-flink-starter"; } diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java index ab62a623d01..78703279900 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java @@ -27,7 +27,7 @@ public abstract class FlinkContainer extends AbstractFlinkContainer { @Override - protected String getStartModulePath() { + protected String getStartModuleName() { return "seatunnel-core-flink"; } diff --git a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java index e195b4aa9c2..e92c23a1ad4 100644 --- a/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-sql-e2e/setunnel-connector-flink-sql-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/sql/FlinkContainer.java @@ -33,7 +33,7 @@ public abstract class FlinkContainer extends AbstractFlinkContainer { @Override - protected String getStartModulePath() { + protected String getStartModuleName() { return "seatunnel-core-flink-sql"; } diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java index 7ad7c69f593..a26271655c9 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java @@ -26,7 +26,7 @@ public abstract class SparkContainer extends AbstractSparkContainer { @Override - protected String getStartModulePath() { + protected String getStartModuleName() { return "seatunnel-spark-starter"; } diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java index 8811c06a490..56c07c4fe23 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java @@ -26,7 +26,7 @@ public abstract class SparkContainer extends AbstractSparkContainer { @Override - protected String getStartModulePath() { + protected String getStartModuleName() { return "seatunnel-core-spark"; } From 9d0b5d2d48ac36fd2d8f2bb722064951bd35bd57 Mon Sep 17 00:00:00 2001 From: Zongwen Li Date: Mon, 12 Sep 2022 10:52:52 +0800 Subject: [PATCH 28/44] [Improve][build] Dependency management split (#2686) --- pom.xml | 84 ---------------------------------------- seatunnel-server/pom.xml | 81 +++++++++++++++++++++++++++++++++++++- 2 files changed, 80 insertions(+), 85 deletions(-) diff --git a/pom.xml b/pom.xml index f47e0a13125..e33d00bc2c0 100644 --- a/pom.xml +++ b/pom.xml @@ -196,18 +196,8 @@ 3.0.0 org.apache.seatunnel.shade 1.1.8.3 - 2.6.8 - 5.3.20 - 2.2.2 - 1.2.9 - 2.6.1 - 1.5.10 - 6.2.2.Final - 1.14.3 - 1.3.2 3.10.0 4.2.0 - 0.10.7 @@ -341,62 +331,6 @@ ${commons.logging.version} - - - org.springframework.boot - spring-boot-starter-web - ${spring-boot.version} - - - - org.springframework.boot - spring-boot-starter-jetty - ${spring-boot.version} - - - - org.mybatis.spring.boot - mybatis-spring-boot-starter - ${mybatis-spring-boot-starter.version} - - - - com.alibaba - druid-spring-boot-starter - ${druid-spring-boot-starter.version} - - - - io.springfox - springfox-swagger2 - ${springfox-swagger.version} - - - - io.springfox - springfox-swagger-ui - ${springfox-swagger.version} - - - - io.swagger - swagger-annotations - ${swagger-annotations.version} - - - - org.hibernate.validator - hibernate-validator - ${hibernate.validator.version} - - - - - org.jsoup - jsoup - ${jsoup.version} - - org.checkerframework checker-qual @@ -409,24 +343,6 @@ ${awaitility.version} test - - - io.jsonwebtoken - jjwt-api - ${jwt.version} - - - io.jsonwebtoken - jjwt-impl - ${jwt.version} - runtime - - - io.jsonwebtoken - jjwt-jackson - ${jwt.version} - runtime - diff --git a/seatunnel-server/pom.xml b/seatunnel-server/pom.xml index 1438fcd5b2d..193b5af88db 100644 --- a/seatunnel-server/pom.xml +++ b/seatunnel-server/pom.xml @@ -31,9 +31,28 @@ seatunnel-scheduler seatunnel-server-common - + + + 2.6.8 + 5.3.20 + 2.2.2 + 1.2.9 + 2.6.1 + 1.5.10 + 6.2.2.Final + 1.3.2 + 1.14.3 + 0.10.7 + + + + + org.springframework.boot + spring-boot-starter-web + ${spring-boot.version} + org.springframework.boot spring-boot-starter-jetty @@ -44,6 +63,66 @@ spring-boot-starter-aop ${spring-boot.version} + + com.alibaba + druid-spring-boot-starter + ${druid-spring-boot-starter.version} + + + + + org.mybatis.spring.boot + mybatis-spring-boot-starter + ${mybatis-spring-boot-starter.version} + + + org.hibernate.validator + hibernate-validator + ${hibernate.validator.version} + + + + + io.springfox + springfox-swagger2 + ${springfox-swagger.version} + + + io.springfox + springfox-swagger-ui + ${springfox-swagger.version} + + + io.swagger + swagger-annotations + ${swagger-annotations.version} + + + + + io.jsonwebtoken + jjwt-api + ${jwt.version} + + + io.jsonwebtoken + jjwt-impl + ${jwt.version} + runtime + + + io.jsonwebtoken + jjwt-jackson + ${jwt.version} + runtime + + + + + org.jsoup + jsoup + ${jsoup.version} + From 7e2bb647d8da4fa0cc139cc5aeb196b67695abaa Mon Sep 17 00:00:00 2001 From: Zongwen Li Date: Tue, 13 Sep 2022 10:04:42 +0800 Subject: [PATCH 29/44] [Improve][build] delete connectors*-dist modules (#2709) --- pom.xml | 1 - seatunnel-connectors-v2-dist/pom.xml | 193 ----------------- seatunnel-connectors/pom.xml | 3 - .../seatunnel-connectors-flink-dist/pom.xml | 125 ----------- .../pom.xml | 76 ------- .../seatunnel-connectors-spark-dist/pom.xml | 196 ------------------ 6 files changed, 594 deletions(-) delete mode 100644 seatunnel-connectors-v2-dist/pom.xml delete mode 100644 seatunnel-connectors/seatunnel-connectors-flink-dist/pom.xml delete mode 100644 seatunnel-connectors/seatunnel-connectors-flink-sql-dist/pom.xml delete mode 100644 seatunnel-connectors/seatunnel-connectors-spark-dist/pom.xml diff --git a/pom.xml b/pom.xml index e33d00bc2c0..6f3c5414d1b 100644 --- a/pom.xml +++ b/pom.xml @@ -102,7 +102,6 @@ seatunnel-connectors-v2 - seatunnel-connectors-v2-dist seatunnel-examples seatunnel-e2e diff --git a/seatunnel-connectors-v2-dist/pom.xml b/seatunnel-connectors-v2-dist/pom.xml deleted file mode 100644 index 25bec4a20aa..00000000000 --- a/seatunnel-connectors-v2-dist/pom.xml +++ /dev/null @@ -1,193 +0,0 @@ - - - - - - seatunnel - org.apache.seatunnel - ${revision} - - 4.0.0 - - seatunnel-connectors-v2-dist - - - - org.apache.seatunnel - connector-fake - ${project.version} - - - org.apache.seatunnel - connector-console - ${project.version} - - - org.apache.seatunnel - connector-assert - ${project.version} - - - org.apache.seatunnel - connector-kafka - ${project.version} - - - org.apache.seatunnel - connector-http-base - ${project.version} - - - org.apache.seatunnel - connector-http-feishu - ${project.version} - - - org.apache.seatunnel - connector-http-wechat - ${project.version} - - - org.apache.seatunnel - connector-jdbc - ${project.version} - - - org.apache.seatunnel - connector-socket - ${project.version} - - - org.apache.seatunnel - connector-clickhouse - ${project.version} - - - org.apache.seatunnel - connector-pulsar - ${project.version} - - - org.apache.seatunnel - connector-hive - ${project.version} - - - org.apache.seatunnel - connector-file-hadoop - ${project.version} - - - org.apache.seatunnel - connector-file-local - ${project.version} - - - org.apache.seatunnel - connector-file-oss - ${project.version} - - - org.apache.seatunnel - connector-file-ftp - ${project.version} - - - org.apache.seatunnel - connector-hudi - ${project.version} - - - org.apache.seatunnel - connector-dingtalk - ${project.version} - - - org.apache.seatunnel - connector-kudu - ${project.version} - - - org.apache.seatunnel - connector-email - ${project.version} - - - org.apache.seatunnel - connector-elasticsearch - ${project.version} - - - org.apache.seatunnel - connector-iotdb - ${project.version} - - - org.apache.seatunnel - connector-neo4j - ${project.version} - - - org.apache.seatunnel - connector-redis - ${project.version} - - - org.apache.seatunnel - connector-datahub - ${project.version} - - - org.apache.seatunnel - connector-sentry - ${project.version} - - - org.apache.seatunnel - connector-mongodb - ${project.version} - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - copy-connector - package - - copy-dependencies - - - jar - jar - ${project.build.directory}/lib - - - - - - - diff --git a/seatunnel-connectors/pom.xml b/seatunnel-connectors/pom.xml index ad996224da6..5ba37e3cf52 100644 --- a/seatunnel-connectors/pom.xml +++ b/seatunnel-connectors/pom.xml @@ -32,11 +32,8 @@ seatunnel-connectors-flink - seatunnel-connectors-flink-dist seatunnel-connectors-spark - seatunnel-connectors-spark-dist seatunnel-connectors-flink-sql - seatunnel-connectors-flink-sql-dist diff --git a/seatunnel-connectors/seatunnel-connectors-flink-dist/pom.xml b/seatunnel-connectors/seatunnel-connectors-flink-dist/pom.xml deleted file mode 100644 index 71c43622c59..00000000000 --- a/seatunnel-connectors/seatunnel-connectors-flink-dist/pom.xml +++ /dev/null @@ -1,125 +0,0 @@ - - - - - seatunnel-connectors - org.apache.seatunnel - ${revision} - - 4.0.0 - - seatunnel-connectors-flink-dist - - - - org.apache.seatunnel - seatunnel-connector-flink-kafka - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-console - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-jdbc - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-fake - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-socket - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-file - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-elasticsearch6 - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-elasticsearch7 - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-doris - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-clickhouse - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-http - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-flink-assert - ${project.version} - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - copy-connector - package - - copy-dependencies - - - jar - jar - ${project.build.directory}/lib - - - - - - - \ No newline at end of file diff --git a/seatunnel-connectors/seatunnel-connectors-flink-sql-dist/pom.xml b/seatunnel-connectors/seatunnel-connectors-flink-sql-dist/pom.xml deleted file mode 100644 index 4c74d4c45c6..00000000000 --- a/seatunnel-connectors/seatunnel-connectors-flink-sql-dist/pom.xml +++ /dev/null @@ -1,76 +0,0 @@ - - - - - seatunnel-connectors - org.apache.seatunnel - ${revision} - - 4.0.0 - - seatunnel-connectors-flink-sql-dist - - - - org.apache.seatunnel - flink-sql-connector-jdbc - ${project.version} - - - - org.apache.seatunnel - flink-sql-connector-kafka - ${project.version} - - - - org.apache.seatunnel - flink-sql-connector-elasticsearch-6 - ${project.version} - - - - org.apache.seatunnel - flink-sql-connector-elasticsearch-7 - ${project.version} - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - copy-connector - package - - copy-dependencies - - - jar - jar - ${project.build.directory}/lib - - - - - - - \ No newline at end of file diff --git a/seatunnel-connectors/seatunnel-connectors-spark-dist/pom.xml b/seatunnel-connectors/seatunnel-connectors-spark-dist/pom.xml deleted file mode 100644 index 934c35026f9..00000000000 --- a/seatunnel-connectors/seatunnel-connectors-spark-dist/pom.xml +++ /dev/null @@ -1,196 +0,0 @@ - - - - - seatunnel-connectors - org.apache.seatunnel - ${revision} - - 4.0.0 - - seatunnel-connectors-spark-dist - - - - org.apache.seatunnel - seatunnel-connector-spark-kafka - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-fake - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-file - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-socket - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-jdbc - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-elasticsearch - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-hive - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-phoenix - ${project.version} - - - org.apache.thrift - libthrift - - - - - - org.apache.seatunnel - seatunnel-connector-spark-redis - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-mongodb - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-kudu - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-email - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-console - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-clickhouse - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-hbase - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-hudi - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-doris - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-tidb - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-neo4j - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-iceberg - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-feishu - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-http - ${project.version} - - - - org.apache.seatunnel - seatunnel-connector-spark-webhook - ${project.version} - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - copy-connector - package - - copy-dependencies - - - jar - jar - ${project.build.directory}/lib - - - - - - - \ No newline at end of file From 5bd96a5fc786fc91becf9142b4a78b404ea9acb2 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Tue, 13 Sep 2022 11:58:22 +0800 Subject: [PATCH 30/44] [Improve][e2e] Improved e2e start sleep (#2677) * [Improve][e2e] Improved e2e start sleep * update * update * fix 2675 --- seatunnel-e2e/pom.xml | 5 +++++ .../e2e/flink/v2/iotdb/FakeSourceToIoTDBIT.java | 8 +++++--- .../e2e/flink/v2/jdbc/FakeSourceToJdbcIT.java | 11 +++++++++-- .../seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java | 2 +- .../seatunnel/e2e/flink/v2/jdbc/JdbcGreenplumIT.java | 10 ++++++---- .../e2e/flink/v2/jdbc/JdbcSourceToConsoleIT.java | 11 +++++++++-- .../flink/v2/mongodb/MongodbSourceToConsoleIT.java | 5 +++-- .../apache/seatunnel/e2e/flink/v2/redis/RedisIT.java | 10 ++++++---- .../seatunnel-flink-connector-v2-e2e/pom.xml | 5 ----- .../flink/clickhouse/FakeSourceToClickhouseIT.java | 11 +++++++++-- .../e2e/spark/v2/iotdb/FakeSourceToIoTDBIT.java | 8 +++++--- .../seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java | 2 +- .../seatunnel/e2e/spark/v2/jdbc/JdbcGreenplumIT.java | 10 ++++++---- .../apache/seatunnel/e2e/spark/v2/redis/RedisIT.java | 10 ++++++---- .../seatunnel-spark-connector-v2-e2e/pom.xml | 6 ------ .../seatunnel/e2e/spark/jdbc/FakeSourceToJdbcIT.java | 11 +++++++++-- .../e2e/spark/jdbc/JdbcSourceToConsoleIT.java | 11 +++++++++-- 17 files changed, 89 insertions(+), 47 deletions(-) diff --git a/seatunnel-e2e/pom.xml b/seatunnel-e2e/pom.xml index c8bb6374e78..a800ca3e1a4 100644 --- a/seatunnel-e2e/pom.xml +++ b/seatunnel-e2e/pom.xml @@ -56,6 +56,11 @@ ${junit4.version} test + + org.awaitility + awaitility + test + diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iotdb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iotdb/FakeSourceToIoTDBIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iotdb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iotdb/FakeSourceToIoTDBIT.java index b126831cc47..704d23228ac 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iotdb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iotdb/FakeSourceToIoTDBIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iotdb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iotdb/FakeSourceToIoTDBIT.java @@ -70,9 +70,11 @@ public void startIoTDBContainer() throws Exception { // wait for IoTDB fully start session = createSession(); given().ignoreExceptions() - .await() - .atMost(30, TimeUnit.SECONDS) - .untilAsserted(() -> session.open()); + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(30, TimeUnit.SECONDS) + .untilAsserted(() -> session.open()); initIoTDBTimeseries(); } diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/FakeSourceToJdbcIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/FakeSourceToJdbcIT.java index 9bc241bf477..f837815719e 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/FakeSourceToJdbcIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/FakeSourceToJdbcIT.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.e2e.flink.v2.jdbc; +import static org.awaitility.Awaitility.given; + import org.apache.seatunnel.e2e.flink.FlinkContainer; import com.google.common.collect.Lists; @@ -39,6 +41,7 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.stream.Stream; public class FakeSourceToJdbcIT extends FlinkContainer { @@ -54,9 +57,13 @@ public void startPostgreSqlContainer() throws InterruptedException, ClassNotFoun .withLogConsumer(new Slf4jLogConsumer(LOGGER)); Startables.deepStart(Stream.of(psl)).join(); LOGGER.info("PostgreSql container started"); - Thread.sleep(5000L); Class.forName(psl.getDriverClassName()); - initializeJdbcTable(); + given().ignoreExceptions() + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> initializeJdbcTable()); } private void initializeJdbcTable() { diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java index b88aab09350..861200526e6 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcDmdbIT.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.e2e.flink.v2.jdbc; -import static org.testcontainers.shaded.org.awaitility.Awaitility.given; +import static org.awaitility.Awaitility.given; import org.apache.seatunnel.e2e.flink.FlinkContainer; diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcGreenplumIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcGreenplumIT.java index 715441032e1..ac436650165 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcGreenplumIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcGreenplumIT.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.e2e.flink.v2.jdbc; -import static org.testcontainers.shaded.org.awaitility.Awaitility.given; +import static org.awaitility.Awaitility.given; import org.apache.seatunnel.e2e.flink.FlinkContainer; @@ -76,9 +76,11 @@ public void startGreenplumContainer() throws ClassNotFoundException, SQLExceptio // wait for Greenplum fully start Class.forName(GREENPLUM_DRIVER); given().ignoreExceptions() - .await() - .atMost(180, TimeUnit.SECONDS) - .untilAsserted(() -> initializeJdbcConnection()); + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(180, TimeUnit.SECONDS) + .untilAsserted(() -> initializeJdbcConnection()); initializeJdbcTable(); batchInsertData(); } diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcSourceToConsoleIT.java index 14eba119d06..bee52537353 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcSourceToConsoleIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-jdbc-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/jdbc/JdbcSourceToConsoleIT.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.e2e.flink.v2.jdbc; +import static org.awaitility.Awaitility.given; + import org.apache.seatunnel.e2e.flink.FlinkContainer; import org.junit.jupiter.api.AfterEach; @@ -37,6 +39,7 @@ import java.sql.PreparedStatement; import java.sql.SQLException; import java.sql.Statement; +import java.util.concurrent.TimeUnit; import java.util.stream.Stream; public class JdbcSourceToConsoleIT extends FlinkContainer { @@ -52,9 +55,13 @@ public void startPostgreSqlContainer() throws InterruptedException, ClassNotFoun .withLogConsumer(new Slf4jLogConsumer(LOGGER)); Startables.deepStart(Stream.of(psl)).join(); LOGGER.info("PostgreSql container started"); - Thread.sleep(5000L); Class.forName(psl.getDriverClassName()); - initializeJdbcTable(); + given().ignoreExceptions() + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> initializeJdbcTable()); batchInsertData(); } diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/MongodbSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/MongodbSourceToConsoleIT.java index 1c5dc90ceae..2ce0d42413e 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/MongodbSourceToConsoleIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/MongodbSourceToConsoleIT.java @@ -26,6 +26,7 @@ import com.mongodb.client.MongoClients; import com.mongodb.client.MongoCollection; import lombok.extern.slf4j.Slf4j; +import org.awaitility.Awaitility; import org.bson.Document; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -36,7 +37,6 @@ import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; import org.testcontainers.lifecycle.Startables; -import org.testcontainers.shaded.org.awaitility.Awaitility; import org.testcontainers.utility.DockerImageName; import java.io.IOException; @@ -77,7 +77,8 @@ public void startMongoContainer() { Startables.deepStart(Stream.of(mongodbContainer)).join(); log.info("Mongodb container started"); Awaitility.given().ignoreExceptions() - .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) .atMost(180, TimeUnit.SECONDS) .untilAsserted(this::initConnection); this.generateTestData(); diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-redis-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/redis/RedisIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-redis-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/redis/RedisIT.java index 9fd929d2f71..de2e8950fc8 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-redis-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/redis/RedisIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-redis-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/redis/RedisIT.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.e2e.flink.v2.redis; -import static org.testcontainers.shaded.org.awaitility.Awaitility.given; +import static org.awaitility.Awaitility.given; import org.apache.seatunnel.e2e.flink.FlinkContainer; @@ -56,9 +56,11 @@ public void startRedisContainer() { Startables.deepStart(Stream.of(redisContainer)).join(); log.info("Redis container started"); given().ignoreExceptions() - .await() - .atMost(180, TimeUnit.SECONDS) - .untilAsserted(this::initJedis); + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(180, TimeUnit.SECONDS) + .untilAsserted(this::initJedis); this.generateTestData(); } diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml index 61d7911a548..f9f46ffa507 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml @@ -45,10 +45,5 @@ ${project.version} test - - org.awaitility - awaitility - test - \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-clickhouse-e2e/src/test/java/org/apache/seatunnel/e2e/flink/clickhouse/FakeSourceToClickhouseIT.java b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-clickhouse-e2e/src/test/java/org/apache/seatunnel/e2e/flink/clickhouse/FakeSourceToClickhouseIT.java index 7340f9b5322..4790c2bb7b7 100644 --- a/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-clickhouse-e2e/src/test/java/org/apache/seatunnel/e2e/flink/clickhouse/FakeSourceToClickhouseIT.java +++ b/seatunnel-e2e/seatunnel-flink-e2e/seatunnel-connector-flink-clickhouse-e2e/src/test/java/org/apache/seatunnel/e2e/flink/clickhouse/FakeSourceToClickhouseIT.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.e2e.flink.clickhouse; +import static org.awaitility.Awaitility.given; + import org.apache.seatunnel.e2e.flink.FlinkContainer; import com.google.common.collect.Lists; @@ -39,6 +41,7 @@ import java.sql.SQLException; import java.util.List; import java.util.Properties; +import java.util.concurrent.TimeUnit; import java.util.stream.Stream; public class FakeSourceToClickhouseIT extends FlinkContainer { @@ -59,9 +62,13 @@ public void startClickhouseContainer() throws InterruptedException { Startables.deepStart(Stream.of(clickhouseServer)).join(); LOGGER.info("Clickhouse container started"); // wait for clickhouse fully start - Thread.sleep(5000L); dataSource = createDatasource(); - initializeClickhouseTable(); + given().ignoreExceptions() + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> initializeClickhouseTable()); } /** diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iotdb-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/iotdb/FakeSourceToIoTDBIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iotdb-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/iotdb/FakeSourceToIoTDBIT.java index 66b044fb044..40a9a8ce6b1 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iotdb-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/iotdb/FakeSourceToIoTDBIT.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iotdb-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/iotdb/FakeSourceToIoTDBIT.java @@ -69,9 +69,11 @@ public void startIoTDBContainer() throws Exception { // wait for IoTDB fully start session = createSession(); given().ignoreExceptions() - .await() - .atMost(30, TimeUnit.SECONDS) - .untilAsserted(() -> session.open()); + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(30, TimeUnit.SECONDS) + .untilAsserted(() -> session.open()); initIoTDBTimeseries(); } diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java index d1eb5043d9e..9845c3c20fe 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcDmdbIT.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.e2e.spark.v2.jdbc; -import static org.testcontainers.shaded.org.awaitility.Awaitility.given; +import static org.awaitility.Awaitility.given; import org.apache.seatunnel.e2e.spark.SparkContainer; diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcGreenplumIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcGreenplumIT.java index 7910a0dde5d..b11910eeef1 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcGreenplumIT.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-jdbc-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/jdbc/JdbcGreenplumIT.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.e2e.spark.v2.jdbc; -import static org.testcontainers.shaded.org.awaitility.Awaitility.given; +import static org.awaitility.Awaitility.given; import org.apache.seatunnel.e2e.spark.SparkContainer; @@ -76,9 +76,11 @@ public void startGreenplumContainer() throws ClassNotFoundException, SQLExceptio // wait for Greenplum fully start Class.forName(GREENPLUM_DRIVER); given().ignoreExceptions() - .await() - .atMost(180, TimeUnit.SECONDS) - .untilAsserted(() -> initializeJdbcConnection()); + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(180, TimeUnit.SECONDS) + .untilAsserted(() -> initializeJdbcConnection()); initializeJdbcTable(); batchInsertData(); } diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-redis-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/redis/RedisIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-redis-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/redis/RedisIT.java index 00b84531746..7efdd438a47 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-redis-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/redis/RedisIT.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-redis-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/redis/RedisIT.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.e2e.spark.v2.redis; -import static org.testcontainers.shaded.org.awaitility.Awaitility.given; +import static org.awaitility.Awaitility.given; import org.apache.seatunnel.e2e.spark.SparkContainer; @@ -56,9 +56,11 @@ public void startRedisContainer() { Startables.deepStart(Stream.of(redisContainer)).join(); log.info("Redis container started"); given().ignoreExceptions() - .await() - .atMost(180, TimeUnit.SECONDS) - .untilAsserted(this::initJedis); + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(180, TimeUnit.SECONDS) + .untilAsserted(this::initJedis); this.generateTestData(); } diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml index c1fd6c512b1..cc5cc2bafa3 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml @@ -44,12 +44,6 @@ ${project.version} test - - - org.awaitility - awaitility - test - \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/src/test/java/org/apache/seatunnel/e2e/spark/jdbc/FakeSourceToJdbcIT.java b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/src/test/java/org/apache/seatunnel/e2e/spark/jdbc/FakeSourceToJdbcIT.java index 061de4e1ab2..c2d2a142661 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/src/test/java/org/apache/seatunnel/e2e/spark/jdbc/FakeSourceToJdbcIT.java +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/src/test/java/org/apache/seatunnel/e2e/spark/jdbc/FakeSourceToJdbcIT.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.e2e.spark.jdbc; +import static org.awaitility.Awaitility.given; + import org.apache.seatunnel.e2e.spark.SparkContainer; import com.google.common.collect.Lists; @@ -39,6 +41,7 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.stream.Stream; public class FakeSourceToJdbcIT extends SparkContainer { @@ -54,9 +57,13 @@ public void startPostgreSqlContainer() throws InterruptedException, ClassNotFoun .withLogConsumer(new Slf4jLogConsumer(LOGGER)); Startables.deepStart(Stream.of(psl)).join(); LOGGER.info("PostgreSql container started"); - Thread.sleep(5000L); Class.forName(psl.getDriverClassName()); - initializeJdbcTable(); + given().ignoreExceptions() + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> initializeJdbcTable()); } private void initializeJdbcTable() { diff --git a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/src/test/java/org/apache/seatunnel/e2e/spark/jdbc/JdbcSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/src/test/java/org/apache/seatunnel/e2e/spark/jdbc/JdbcSourceToConsoleIT.java index f430070197d..cf9be5e9ea6 100644 --- a/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/src/test/java/org/apache/seatunnel/e2e/spark/jdbc/JdbcSourceToConsoleIT.java +++ b/seatunnel-e2e/seatunnel-spark-e2e/seatunnel-connector-spark-jdbc-e2e/src/test/java/org/apache/seatunnel/e2e/spark/jdbc/JdbcSourceToConsoleIT.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.e2e.spark.jdbc; +import static org.awaitility.Awaitility.given; + import org.apache.seatunnel.e2e.spark.SparkContainer; import com.google.common.collect.Lists; @@ -38,6 +40,7 @@ import java.sql.PreparedStatement; import java.sql.SQLException; import java.sql.Statement; +import java.util.concurrent.TimeUnit; import java.util.stream.Stream; public class JdbcSourceToConsoleIT extends SparkContainer { @@ -54,9 +57,13 @@ public void startPostgreSqlContainer() throws InterruptedException, ClassNotFoun psl.setPortBindings(Lists.newArrayList("33306:3306")); Startables.deepStart(Stream.of(psl)).join(); LOGGER.info("PostgreSql container started"); - Thread.sleep(5000L); Class.forName(psl.getDriverClassName()); - initializeJdbcTable(); + given().ignoreExceptions() + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> initializeJdbcTable()); batchInsertData(); } From b38321ecf051c0fb53f8f693962e5a9a181841ee Mon Sep 17 00:00:00 2001 From: dijie Date: Wed, 14 Sep 2022 15:20:56 +0800 Subject: [PATCH 31/44] [Improve][seatunnel-server] Update api style. (#2723) --- .../app/controller/ScriptController.java | 84 ++++++++++++------- .../app/controller/TaskController.java | 60 +++++++++---- .../app/controller/UserController.java | 58 ++++++------- .../request/script/AddEmptyScriptReq.java | 2 +- .../script/UpdateScriptContentReq.java | 7 +- .../request/script/UpdateScriptParamReq.java | 4 +- .../app/domain/request/task/ExecuteReq.java | 14 +++- .../domain/request/user/UpdateUserReq.java | 4 +- .../seatunnel/app/service/ITaskService.java | 2 +- .../app/service/impl/ScriptServiceImpl.java | 4 +- .../app/service/impl/TaskServiceImpl.java | 2 +- .../app/service/impl/UserServiceImpl.java | 2 +- 12 files changed, 156 insertions(+), 87 deletions(-) diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/ScriptController.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/ScriptController.java index 01a5e173272..f983cbdbdb9 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/ScriptController.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/ScriptController.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.app.controller; +import org.apache.seatunnel.app.aspect.UserId; import org.apache.seatunnel.app.common.Result; import org.apache.seatunnel.app.domain.request.script.AddEmptyScriptReq; import org.apache.seatunnel.app.domain.request.script.PublishScriptReq; @@ -32,14 +33,18 @@ import io.swagger.annotations.ApiImplicitParam; import io.swagger.annotations.ApiImplicitParams; import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; import org.springframework.web.bind.annotation.DeleteMapping; import org.springframework.web.bind.annotation.GetMapping; +import org.springframework.web.bind.annotation.PatchMapping; +import org.springframework.web.bind.annotation.PathVariable; import org.springframework.web.bind.annotation.PostMapping; import org.springframework.web.bind.annotation.PutMapping; import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestParam; import org.springframework.web.bind.annotation.RestController; +import springfox.documentation.annotations.ApiIgnore; import javax.annotation.Resource; import javax.validation.constraints.NotNull; @@ -52,63 +57,84 @@ public class ScriptController { @Resource private IScriptService iScriptService; - @PostMapping("/script") + @PostMapping @ApiOperation(value = "add an empty script", httpMethod = "POST") - public Result addEmptyScript(@RequestBody @NotNull AddEmptyScriptReq addEmptyScriptReq) { + public Result addEmptyScript(@RequestBody @NotNull AddEmptyScriptReq addEmptyScriptReq, + @ApiIgnore @UserId Integer operatorId) { + addEmptyScriptReq.setCreatorId(operatorId); return Result.success(iScriptService.addEmptyScript(addEmptyScriptReq)); } - @PutMapping("/scriptContent") + @PutMapping("/{scriptId}/content") @ApiOperation(value = "update script", httpMethod = "PUT") - public Result updateScriptContent(@RequestBody @NotNull UpdateScriptContentReq updateScriptContentReq) { - iScriptService.updateScriptContent(updateScriptContentReq); + public Result updateScriptContent(@ApiParam(value = "script id", required = true) @PathVariable(value = "scriptId") Integer scriptId, + @RequestBody @NotNull String content, + @ApiIgnore @UserId Integer operatorId) { + final UpdateScriptContentReq req = new UpdateScriptContentReq(); + req.setScriptId(scriptId); + req.setContent(content); + req.setMenderId(operatorId); + + iScriptService.updateScriptContent(req); return Result.success(); } - @DeleteMapping("/script") + @DeleteMapping("/{scriptId}") @ApiOperation(value = "delete script", httpMethod = "DELETE") - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", value = "script id", dataType = "Integer"), - }) - public Result delete(@RequestParam @NotNull Integer id) { - iScriptService.delete(id); + public Result delete(@ApiParam(value = "script id", required = true) @PathVariable(value = "scriptId") Integer scriptId) { + iScriptService.delete(scriptId); return Result.success(); } - @PostMapping("/list") - @ApiOperation(value = "script list", httpMethod = "POST") - public Result> list(@RequestBody @NotNull ScriptListReq scriptListReq) { - return Result.success(iScriptService.list(scriptListReq)); + @GetMapping + @ApiOperation(value = "script list", httpMethod = "GET") + public Result> list(@ApiParam(value = "script name") @RequestParam(required = false) String name, + @ApiParam(value = "script status") @RequestParam(required = false) Byte status, + @ApiParam(value = "page num", required = true) @RequestParam Integer pageNo, + @ApiParam(value = "page size", required = true) @RequestParam Integer pageSize) { + + final ScriptListReq req = new ScriptListReq(); + req.setName(name); + req.setStatus(status); + req.setPageNo(pageNo); + req.setPageSize(pageSize); + + return Result.success(iScriptService.list(req)); } - @GetMapping("/scriptContent") + @GetMapping("/{scriptId}/content") @ApiOperation(value = "fetch script content", httpMethod = "GET") - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", value = "script id", dataType = "Integer"), - }) - public Result fetchScriptContent(@RequestParam @NotNull Integer id) { - return Result.success(iScriptService.fetchScriptContent(id)); + public Result fetchScriptContent(@ApiParam(value = "script id", required = true) @PathVariable(value = "scriptId") Integer scriptId) { + return Result.success(iScriptService.fetchScriptContent(scriptId)); } - @PutMapping("/scriptParam") + @PutMapping("/{scriptId}/param") @ApiOperation(value = "update script param", httpMethod = "PUT") - public Result updateScriptParam(@RequestBody @NotNull UpdateScriptParamReq updateScriptParamReq) { + public Result updateScriptParam(@ApiParam(value = "script id", required = true) @PathVariable(value = "scriptId") Integer scriptId, + @RequestBody @NotNull UpdateScriptParamReq updateScriptParamReq) { + updateScriptParamReq.setScriptId(scriptId); iScriptService.updateScriptParam(updateScriptParamReq); return Result.success(); } - @GetMapping("/scriptParam") + @GetMapping("/{scriptId}/param") @ApiOperation(value = "fetch script param", httpMethod = "GET") @ApiImplicitParams({ @ApiImplicitParam(name = "id", value = "script id", dataType = "Integer"), }) - public Result> fetchScriptParam(@RequestParam @NotNull Integer id) { - return Result.success(iScriptService.fetchScriptParam(id)); + public Result> fetchScriptParam(@ApiParam(value = "script id", required = true) @PathVariable(value = "scriptId") Integer scriptId) { + return Result.success(iScriptService.fetchScriptParam(scriptId)); } - @PutMapping("/publish") - @ApiOperation(value = "publish script", httpMethod = "PUT") - public Result publish(@RequestBody @NotNull PublishScriptReq req) { + @PatchMapping("/{scriptId}/publish") + @ApiOperation(value = "publish script", httpMethod = "PATCH") + public Result publish(@ApiParam(value = "script id", required = true) @PathVariable(value = "scriptId") Integer scriptId, + @ApiIgnore @UserId Integer operatorId) { + + final PublishScriptReq req = new PublishScriptReq(); + req.setScriptId(scriptId); + req.setOperatorId(operatorId); + iScriptService.publishScript(req); return Result.success(); } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/TaskController.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/TaskController.java index 2c63edd36a3..5ca3ace8c66 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/TaskController.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/TaskController.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.app.controller; +import org.apache.seatunnel.app.aspect.UserId; import org.apache.seatunnel.app.common.Result; import org.apache.seatunnel.app.domain.request.task.ExecuteReq; import org.apache.seatunnel.app.domain.request.task.InstanceListReq; @@ -29,13 +30,16 @@ import org.apache.seatunnel.app.service.ITaskService; import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; import org.springframework.web.bind.annotation.GetMapping; +import org.springframework.web.bind.annotation.PatchMapping; +import org.springframework.web.bind.annotation.PathVariable; import org.springframework.web.bind.annotation.PostMapping; -import org.springframework.web.bind.annotation.PutMapping; import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RequestMapping; import org.springframework.web.bind.annotation.RequestParam; import org.springframework.web.bind.annotation.RestController; +import springfox.documentation.annotations.ApiIgnore; import javax.annotation.Resource; import javax.validation.constraints.NotNull; @@ -47,41 +51,65 @@ public class TaskController { @Resource private ITaskService iTaskService; - @PutMapping("/recycle") - @ApiOperation(value = "recycle script", httpMethod = "PUT") - Result recycle(@RequestBody @NotNull RecycleScriptReq req) { + @PatchMapping("/{scriptId}/recycle") + @ApiOperation(value = "recycle script", httpMethod = "PATCH") + Result recycle(@ApiParam(value = "script id", required = true) @PathVariable(value = "scriptId") Integer scriptId, + @ApiIgnore @UserId Integer operatorId) { + final RecycleScriptReq req = new RecycleScriptReq(); + req.setScriptId(scriptId); + req.setOperatorId(operatorId); + iTaskService.recycleScriptFromScheduler(req); return Result.success(); } - @GetMapping("/listJob") + @GetMapping("/job") @ApiOperation(value = "list job", httpMethod = "GET") - Result> listJob(@RequestBody @NotNull JobListReq req) { + Result> listJob(@ApiParam(value = "job name") @RequestParam(required = false) String name, + @ApiParam(value = "page num", required = true) @RequestParam Integer pageNo, + @ApiParam(value = "page size", required = true) @RequestParam Integer pageSize) { + final JobListReq req = new JobListReq(); + req.setName(name); + req.setPageNo(pageNo); + req.setPageSize(pageSize); + return Result.success(iTaskService.listJob(req)); } - @GetMapping("/listInstance") + @GetMapping("/instance") @ApiOperation(value = "list instance", httpMethod = "GET") - Result> listInstance(@RequestBody @NotNull InstanceListReq req) { + Result> listInstance(@ApiParam(value = "job name") @RequestParam(required = false) String name, + @ApiParam(value = "page num", required = true) @RequestParam Integer pageNo, + @ApiParam(value = "page size", required = true) @RequestParam Integer pageSize) { + final InstanceListReq req = new InstanceListReq(); + req.setName(name); + req.setPageNo(pageNo); + req.setPageSize(pageSize); + return Result.success(iTaskService.listInstance(req)); } - @PostMapping("/tmpExecute") - @ApiOperation(value = "execute script temporary", httpMethod = "GET") - Result tmpExecute(@RequestBody @NotNull ExecuteReq req) { + @PostMapping("/{scriptId}/execute") + @ApiOperation(value = "execute script temporary", httpMethod = "POST") + Result tmpExecute(@ApiParam(value = "script id", required = true) @PathVariable(value = "scriptId") Integer scriptId, + @RequestBody @NotNull ExecuteReq req, + @ApiIgnore @UserId Integer operatorId) { + req.setScriptId(scriptId); + req.setOperatorId(operatorId); + return Result.success(iTaskService.tmpExecute(req)); } - @GetMapping("/queryInstanceLog") + @GetMapping("/{taskInstanceId}") @ApiOperation(value = "query instance log", httpMethod = "GET") - Result queryInstanceLog(@RequestParam long taskInstanceId) { + Result queryInstanceLog(@ApiParam(value = "task instance id", required = true) @PathVariable(value = "taskInstanceId") Long taskInstanceId) { return Result.success(iTaskService.queryInstanceLog(taskInstanceId)); } - @PostMapping("/kill") + @PatchMapping("/{taskInstanceId}") @ApiOperation(value = "kill running instance", httpMethod = "POST") - Result kill(@RequestParam Long instanceId) { - iTaskService.kill(instanceId); + Result kill(@ApiParam(value = "task instance id", required = true) @PathVariable(value = "taskInstanceId") Long taskInstanceId) { + iTaskService.kill(taskInstanceId); return Result.success(); } } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/UserController.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/UserController.java index feb8b460c19..59a59ee85d6 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/UserController.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/controller/UserController.java @@ -27,11 +27,12 @@ import org.apache.seatunnel.app.domain.response.user.UserSimpleInfoRes; import org.apache.seatunnel.app.service.IUserService; -import io.swagger.annotations.ApiImplicitParam; -import io.swagger.annotations.ApiImplicitParams; import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; import org.springframework.web.bind.annotation.DeleteMapping; +import org.springframework.web.bind.annotation.GetMapping; import org.springframework.web.bind.annotation.PatchMapping; +import org.springframework.web.bind.annotation.PathVariable; import org.springframework.web.bind.annotation.PostMapping; import org.springframework.web.bind.annotation.PutMapping; import org.springframework.web.bind.annotation.RequestBody; @@ -49,52 +50,53 @@ public class UserController { @Resource private IUserService iUserService; - @PostMapping("/user") + @PostMapping @ApiOperation(value = "add user", httpMethod = "POST") public Result add(@RequestBody @NotNull AddUserReq addReq) { return Result.success(iUserService.add(addReq)); } - @PutMapping("/user") + @PutMapping("/{userId}") @ApiOperation(value = "update user", httpMethod = "PUT") - public Result update(@RequestBody @NotNull UpdateUserReq updateReq) { + public Result update(@ApiParam(value = "user id", required = true) @PathVariable(value = "userId") Integer userId, + @RequestBody @NotNull UpdateUserReq updateReq) { + updateReq.setUserId(userId); + iUserService.update(updateReq); return Result.success(); } - @DeleteMapping("/user") + @DeleteMapping("/{userId}") @ApiOperation(value = "delete user", httpMethod = "DELETE") - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", value = "user id", dataType = "Integer"), - }) - public Result delete(@RequestParam @NotNull Integer id) { - iUserService.delete(id); + public Result delete(@ApiParam(value = "user id", required = true) @PathVariable(value = "userId") Integer userId) { + iUserService.delete(userId); return Result.success(); } - @PostMapping("/list") - @ApiOperation(value = "user list", httpMethod = "POST") - public Result> list(@RequestBody @NotNull UserListReq userListReq) { - return Result.success(iUserService.list(userListReq)); + @GetMapping + @ApiOperation(value = "user list", httpMethod = "GET") + public Result> list(@ApiParam(value = "job name") @RequestParam(required = false) String name, + @ApiParam(value = "page num", required = true) @RequestParam Integer pageNo, + @ApiParam(value = "page size", required = true) @RequestParam Integer pageSize) { + final UserListReq req = new UserListReq(); + req.setName(name); + req.setPageNo(pageNo); + req.setPageSize(pageSize); + + return Result.success(iUserService.list(req)); } - @PutMapping("/enable") - @ApiOperation(value = "enable a user", httpMethod = "PUT") - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", value = "user id", dataType = "Integer"), - }) - public Result enable(@RequestParam @NotNull Integer id) { - iUserService.enable(id); + @PatchMapping("/{userId}/enable") + @ApiOperation(value = "enable a user", httpMethod = "PATCH") + public Result enable(@ApiParam(value = "user id", required = true) @PathVariable(value = "userId") Integer userId) { + iUserService.enable(userId); return Result.success(); } - @PutMapping("/disable") + @PutMapping("/{userId}/disable") @ApiOperation(value = "disable a user", httpMethod = "PUT") - @ApiImplicitParams({ - @ApiImplicitParam(name = "id", value = "user id", dataType = "Integer"), - }) - public Result disable(@RequestParam @NotNull Integer id) { - iUserService.disable(id); + public Result disable(@ApiParam(value = "user id", required = true) @PathVariable(value = "userId") Integer userId) { + iUserService.disable(userId); return Result.success(); } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/AddEmptyScriptReq.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/AddEmptyScriptReq.java index cae5913d364..4bd650d6dc5 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/AddEmptyScriptReq.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/AddEmptyScriptReq.java @@ -31,7 +31,7 @@ public class AddEmptyScriptReq { @ApiModelProperty(value = "script type", required = true, dataType = "Byte") @NotNull private Byte type; - @ApiModelProperty(value = "script creator id", required = true, dataType = "Integer") + @ApiModelProperty(value = "script creator id", required = true, dataType = "Integer", hidden = true) @NotNull private Integer creatorId; } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/UpdateScriptContentReq.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/UpdateScriptContentReq.java index 7ee1270e20c..5a70d69f3e9 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/UpdateScriptContentReq.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/UpdateScriptContentReq.java @@ -27,13 +27,12 @@ @Data @ApiModel(value = "updateScriptContentReq", description = "update an exist script content request") public class UpdateScriptContentReq { - @ApiModelProperty(value = "script id", required = true, dataType = "Integer") - @NotNull - private Integer id; + @ApiModelProperty(value = "script id", required = true, dataType = "Integer", hidden = true) + private Integer scriptId; @ApiModelProperty(value = "script id", required = true, dataType = "Integer") @NotBlank private String content; - @ApiModelProperty(value = "script mender id", required = true, dataType = "Integer") + @ApiModelProperty(value = "script mender id", required = true, dataType = "Integer", hidden = true) @NotNull private Integer menderId; } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/UpdateScriptParamReq.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/UpdateScriptParamReq.java index 9856378999d..df10570341d 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/UpdateScriptParamReq.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/script/UpdateScriptParamReq.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.app.domain.request.script; import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; import lombok.Data; import javax.validation.constraints.NotEmpty; @@ -27,8 +28,9 @@ @Data @ApiModel(value = "update", description = "update script param request") public class UpdateScriptParamReq { + @ApiModelProperty(value = "script id", required = true, dataType = "Integer", hidden = true) private Integer scriptId; + @ApiModelProperty(value = "script params", required = true, dataType = "Map") @NotEmpty private Map params; - } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/task/ExecuteReq.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/task/ExecuteReq.java index 0f127daa6c7..fd853d4354c 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/task/ExecuteReq.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/task/ExecuteReq.java @@ -17,19 +17,31 @@ package org.apache.seatunnel.app.domain.request.task; +import io.swagger.annotations.ApiModelProperty; import lombok.Data; +import javax.validation.constraints.NotEmpty; + import java.util.Date; import java.util.Map; @Data public class ExecuteReq { + @ApiModelProperty(value = "script id", required = true, dataType = "Integer", hidden = true) private Integer scriptId; + @ApiModelProperty(value = "execute content", required = true, dataType = "String") private String content; + @ApiModelProperty(value = "operator id", required = true, dataType = "Integer", hidden = true) private Integer operatorId; + @ApiModelProperty(value = "script params", required = true, dataType = "Map") + @NotEmpty private Map params; - private int executeType; + @ApiModelProperty(value = "execute type", required = true, dataType = "Integer", allowableValues = "0, 1, 2, 3") + private Integer executeType; + @ApiModelProperty(value = "start time", required = false, dataType = "Date", hidden = true) private Date startTime = new Date(); + @ApiModelProperty(value = "end time", required = false, dataType = "Date", hidden = true) private Date endTime = new Date(); + @ApiModelProperty(value = "parallelism number", required = false, dataType = "Integer", hidden = true) private Integer parallelismNum = 1; } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/user/UpdateUserReq.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/user/UpdateUserReq.java index 39f73e4414d..29fae94c3f4 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/user/UpdateUserReq.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/domain/request/user/UpdateUserReq.java @@ -26,7 +26,7 @@ @ApiModel(value = "updateUserReq", description = "update an exists user request") @Data public class UpdateUserReq extends BaseUserReq{ - @ApiModelProperty(value = "user id", required = true, dataType = "Integer") + @ApiModelProperty(value = "user id", required = true, dataType = "Integer", hidden = true) @NotNull - private Integer id; + private Integer userId; } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/ITaskService.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/ITaskService.java index f959b88e69c..a197f1e758e 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/ITaskService.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/ITaskService.java @@ -38,7 +38,7 @@ public interface ITaskService { InstanceSimpleInfoRes tmpExecute(ExecuteReq req); - InstanceLogRes queryInstanceLog(long instanceId); + InstanceLogRes queryInstanceLog(Long instanceId); void kill(Long instanceId); } diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/ScriptServiceImpl.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/ScriptServiceImpl.java index dfb07c2e000..6d654c974d2 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/ScriptServiceImpl.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/ScriptServiceImpl.java @@ -116,11 +116,11 @@ public void updateScriptContent(UpdateScriptContentReq updateScriptContentReq) { final String content = updateScriptContentReq.getContent(); final String contentMd5 = Strings.isNullOrEmpty(content) ? "" : Md5Utils.toMd5(content); - final boolean needSave = checkIfNeedSave(updateScriptContentReq.getId(), contentMd5); + final boolean needSave = checkIfNeedSave(updateScriptContentReq.getScriptId(), contentMd5); if (needSave){ final UpdateScriptContentDto dto = UpdateScriptContentDto.builder() - .id(updateScriptContentReq.getId()) + .id(updateScriptContentReq.getScriptId()) .content(content) .contentMd5(contentMd5) .menderId(updateScriptContentReq.getMenderId()) diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/TaskServiceImpl.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/TaskServiceImpl.java index 8efcee0f9ef..c687c93dbb3 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/TaskServiceImpl.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/TaskServiceImpl.java @@ -227,7 +227,7 @@ public InstanceSimpleInfoRes tmpExecute(ExecuteReq req) { } @Override - public InstanceLogRes queryInstanceLog(long instanceId) { + public InstanceLogRes queryInstanceLog(Long instanceId) { final InstanceLogDto dto = iInstanceService.queryInstanceLog(instanceId); return InstanceLogRes.builder() diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/UserServiceImpl.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/UserServiceImpl.java index 9b9607630b9..aa63f60c33b 100644 --- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/UserServiceImpl.java +++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/service/impl/UserServiceImpl.java @@ -85,7 +85,7 @@ public AddUserRes add(AddUserReq addReq) { @Override public void update(UpdateUserReq updateReq) { final UpdateUserDto dto = UpdateUserDto.builder() - .id(updateReq.getId()) + .id(updateReq.getUserId()) .username(updateReq.getUsername()) // encryption user's password .password(PasswordUtils.encryptWithSalt(defaultSalt, updateReq.getPassword())) From cbf82f755ca3f905b6b0fad8c2ab2d0d1c5bf36a Mon Sep 17 00:00:00 2001 From: ic4y <83933160+ic4y@users.noreply.github.com> Date: Wed, 14 Sep 2022 16:36:58 +0800 Subject: [PATCH 32/44] [DEV][Api] Replace SeaTunnelContext with JobContext and remove singleton pattern (#2706) * remove singleton pattern for SeaTunnelContext * rename SeaTunnelContext to JobContext and fix checkstyle * fix checkstyle * put the constructor on top --- ...{SeaTunnelContext.java => JobContext.java} | 16 ++++-------- .../seatunnel/api/sink/SeaTunnelSink.java | 4 +-- ...ntextAware.java => SeaTunnelJobAware.java} | 8 +++--- .../seatunnel/api/source/SeaTunnelSource.java | 2 +- .../seatunnel/assertion/sink/AssertSink.java | 7 ----- .../sink/client/ClickhouseSink.java | 6 ----- .../sink/file/ClickhouseFileSink.java | 7 ----- .../elasticsearch/client/EsRestClient.java | 26 ++++++++++--------- .../elasticsearch/config/SinkConfig.java | 6 ++--- .../elasticsearch/constant/BulkConfig.java | 2 ++ .../serialize/ElasticsearchRowSerializer.java | 19 +++++++------- .../serialize/SeaTunnelRowSerializer.java | 1 - .../index/impl/VariableIndexSerializer.java | 4 +-- .../serialize/type/IndexTypeSerializer.java | 1 - .../type/IndexTypeSerializerFactory.java | 6 +++-- .../type/impl/NotIndexTypeSerializer.java | 1 - .../elasticsearch/sink/ElasticsearchSink.java | 16 +++--------- .../sink/ElasticsearchSinkWriter.java | 7 ++--- .../elasticsearch/util/RegexUtils.java | 1 - .../seatunnel/fake/source/FakeSource.java | 10 +++---- .../seatunnel/file/sink/AbstractFileSink.java | 12 ++++----- .../seatunnel/file/sink/BaseFileSink.java | 10 +++---- .../seatunnel/hive/sink/HiveSink.java | 12 ++++----- .../seatunnel/hive/source/HiveSource.java | 8 ------ .../seatunnel/http/source/HttpSource.java | 10 +++---- .../seatunnel/hudi/source/HudiSource.java | 8 ------ .../seatunnel/iotdb/source/IoTDBSource.java | 4 +-- .../internal/xa/SemanticXidGenerator.java | 6 ++--- .../jdbc/internal/xa/XaGroupOps.java | 4 +-- .../jdbc/internal/xa/XaGroupOpsImpl.java | 4 +-- .../jdbc/internal/xa/XidGenerator.java | 6 ++--- .../jdbc/sink/JdbcExactlyOnceSinkWriter.java | 6 ++--- .../seatunnel/jdbc/sink/JdbcSink.java | 12 ++++----- .../seatunnel/jdbc/source/JdbcSource.java | 7 ----- .../seatunnel/kafka/source/KafkaSource.java | 10 +++---- .../seatunnel/kudu/source/KuduSource.java | 7 ----- .../mongodb/source/MongodbSource.java | 8 ------ .../seatunnel/redis/source/RedisSource.java | 7 ----- .../seatunnel/socket/source/SocketSource.java | 10 +++---- .../AbstractPluginExecuteProcessor.java | 4 +++ .../flink/execution/FlinkExecution.java | 11 ++++---- .../flink/execution/SinkExecuteProcessor.java | 7 ++--- .../execution/SourceExecuteProcessor.java | 9 ++++--- .../execution/TransformExecuteProcessor.java | 4 ++- .../AbstractPluginExecuteProcessor.java | 4 +++ .../spark/execution/SinkExecuteProcessor.java | 7 ++--- .../execution/SourceExecuteProcessor.java | 7 ++--- .../spark/execution/SparkExecution.java | 11 ++++---- .../execution/TransformExecuteProcessor.java | 4 ++- 49 files changed, 156 insertions(+), 213 deletions(-) rename seatunnel-api/src/main/java/org/apache/seatunnel/api/common/{SeaTunnelContext.java => JobContext.java} (84%) rename seatunnel-api/src/main/java/org/apache/seatunnel/api/source/{SeaTunnelContextAware.java => SeaTunnelJobAware.java} (83%) diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/SeaTunnelContext.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/JobContext.java similarity index 84% rename from seatunnel-api/src/main/java/org/apache/seatunnel/api/common/SeaTunnelContext.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/common/JobContext.java index b6152e44ffb..9e56de36aeb 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/SeaTunnelContext.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/JobContext.java @@ -28,14 +28,12 @@ import java.util.concurrent.ConcurrentHashMap; /** - * This class is used to store the context of the application. e.g. the table schema, catalog...etc. + * This class is used to store the context of the job. e.g. the table schema, catalog...etc. */ -public final class SeaTunnelContext implements Serializable { +public final class JobContext implements Serializable { private static final long serialVersionUID = -1L; - private static final SeaTunnelContext INSTANCE = new SeaTunnelContext(); - // tableName -> tableSchema private final Map tableSchemaMap = new ConcurrentHashMap<>(Common.COLLECTION_SIZE); @@ -43,8 +41,8 @@ public final class SeaTunnelContext implements Serializable { private final String jobId; - public static SeaTunnelContext getContext() { - return INSTANCE; + public JobContext() { + this.jobId = UUID.randomUUID().toString().replace("-", ""); } /** @@ -67,7 +65,7 @@ public Optional getSchema(String tableName) { return Optional.ofNullable(tableSchemaMap.get(tableName)); } - public SeaTunnelContext setJobMode(JobMode jobMode) { + public JobContext setJobMode(JobMode jobMode) { this.jobMode = jobMode; return this; } @@ -80,8 +78,4 @@ public String getJobId() { return this.jobId; } - private SeaTunnelContext() { - this.jobId = UUID.randomUUID().toString().replace("-", ""); - } - } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SeaTunnelSink.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SeaTunnelSink.java index 59517e409a6..e09d8110ae6 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SeaTunnelSink.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SeaTunnelSink.java @@ -20,7 +20,7 @@ import org.apache.seatunnel.api.common.PluginIdentifierInterface; import org.apache.seatunnel.api.common.SeaTunnelPluginLifeCycle; import org.apache.seatunnel.api.serialization.Serializer; -import org.apache.seatunnel.api.source.SeaTunnelContextAware; +import org.apache.seatunnel.api.source.SeaTunnelJobAware; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -43,7 +43,7 @@ * {@link SinkAggregatedCommitter} handle it, this class should implement interface {@link Serializable}. */ public interface SeaTunnelSink - extends Serializable, PluginIdentifierInterface, SeaTunnelPluginLifeCycle, SeaTunnelContextAware { + extends Serializable, PluginIdentifierInterface, SeaTunnelPluginLifeCycle, SeaTunnelJobAware { /** * Set the row type info of sink row data. This method will be automatically called by translation. diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelContextAware.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelJobAware.java similarity index 83% rename from seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelContextAware.java rename to seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelJobAware.java index 429f0515583..8eb90b04f3f 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelContextAware.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelJobAware.java @@ -17,14 +17,14 @@ package org.apache.seatunnel.api.source; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; /** - * This interface defines the runtime environment of the SeaTunnel application. + * This interface defines the runtime environment of the SeaTunnel job. */ -public interface SeaTunnelContextAware { +public interface SeaTunnelJobAware { - default void setSeaTunnelContext(SeaTunnelContext seaTunnelContext){ + default void setJobContext(JobContext jobContext){ // nothing } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelSource.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelSource.java index f93f4d3bf7f..6edac04a834 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelSource.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelSource.java @@ -34,7 +34,7 @@ * @param The type of checkpoint states. */ public interface SeaTunnelSource - extends Serializable, PluginIdentifierInterface, SeaTunnelPluginLifeCycle, SeaTunnelContextAware { + extends Serializable, PluginIdentifierInterface, SeaTunnelPluginLifeCycle, SeaTunnelJobAware { /** * Get the boundedness of this source. diff --git a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSink.java b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSink.java index dc0c397d646..11c17a1a423 100644 --- a/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSink.java +++ b/seatunnel-connectors-v2/connector-assert/src/main/java/org/apache/seatunnel/connectors/seatunnel/assertion/sink/AssertSink.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.assertion.sink; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -41,7 +40,6 @@ @AutoService(SeaTunnelSink.class) public class AssertSink extends AbstractSimpleSink { private static final String RULES = "rules"; - private SeaTunnelContext seaTunnelContext; private SeaTunnelRowType seaTunnelRowType; private List assertFieldRules; @@ -73,11 +71,6 @@ public void prepare(Config pluginConfig) { assertFieldRules = new AssertRuleParser().parseRules(configList); } - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } - @Override public String getPluginName() { return "Assert"; diff --git a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/client/ClickhouseSink.java b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/client/ClickhouseSink.java index c296ca03af1..3c2c1104df3 100644 --- a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/client/ClickhouseSink.java +++ b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/client/ClickhouseSink.java @@ -29,7 +29,6 @@ import static org.apache.seatunnel.connectors.seatunnel.clickhouse.config.Config.USERNAME; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.DefaultSerializer; import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.sink.SeaTunnelSink; @@ -67,7 +66,6 @@ @AutoService(SeaTunnelSink.class) public class ClickhouseSink implements SeaTunnelSink { - private SeaTunnelContext seaTunnelContext; private ReaderOption option; @Override @@ -194,8 +192,4 @@ public SeaTunnelDataType getConsumedType() { return this.option.getSeaTunnelRowType(); } - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } } diff --git a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/file/ClickhouseFileSink.java b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/file/ClickhouseFileSink.java index 62694c3fe23..319937ac19c 100644 --- a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/file/ClickhouseFileSink.java +++ b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/file/ClickhouseFileSink.java @@ -30,7 +30,6 @@ import static org.apache.seatunnel.connectors.seatunnel.clickhouse.config.Config.USERNAME; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -65,7 +64,6 @@ @AutoService(SeaTunnelSink.class) public class ClickhouseFileSink implements SeaTunnelSink { - private SeaTunnelContext seaTunnelContext; private FileReaderOption readerOption; @Override @@ -140,9 +138,4 @@ public SeaTunnelDataType getConsumedType() { public SinkWriter createWriter(SinkWriter.Context context) throws IOException { return new ClickhouseFileSinkWriter(readerOption, context); } - - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java index 5a2a3df0995..661d5b7ca6c 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/client/EsRestClient.java @@ -17,6 +17,10 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.client; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.BulkResponse; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.BulkElasticsearchException; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.GetElasticsearchVersionException; + import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.lang3.StringUtils; @@ -27,9 +31,6 @@ import org.apache.http.client.CredentialsProvider; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.util.EntityUtils; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.BulkResponse; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.BulkElasticsearchException; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.exception.GetElasticsearchVersionException; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.client.RestClient; @@ -40,13 +41,14 @@ public class EsRestClient { - private static EsRestClient esRestClient; - private static RestClient restClient; + private static EsRestClient ES_REST_CLIENT; + private static RestClient REST_CLIENT; private EsRestClient() { } + @SuppressWarnings("checkstyle:MagicNumber") private static RestClientBuilder getRestClientBuilder(List hosts, String username, String password) { HttpHost[] httpHosts = new HttpHost[hosts.size()]; for (int i = 0; i < hosts.size(); i++) { @@ -68,19 +70,19 @@ private static RestClientBuilder getRestClientBuilder(List hosts, String } public static EsRestClient getInstance(List hosts, String username, String password) { - if (restClient == null) { + if (REST_CLIENT == null) { RestClientBuilder restClientBuilder = getRestClientBuilder(hosts, username, password); - restClient = restClientBuilder.build(); - esRestClient = new EsRestClient(); + REST_CLIENT = restClientBuilder.build(); + ES_REST_CLIENT = new EsRestClient(); } - return esRestClient; + return ES_REST_CLIENT; } public BulkResponse bulk(String requestBody) { Request request = new Request("POST", "_bulk"); request.setJsonEntity(requestBody); try { - Response response = restClient.performRequest(request); + Response response = REST_CLIENT.performRequest(request); if (response == null) { throw new BulkElasticsearchException("bulk es Response is null"); } @@ -105,7 +107,7 @@ public BulkResponse bulk(String requestBody) { public static String getClusterVersion() { Request request = new Request("GET", "/"); try { - Response response = restClient.performRequest(request); + Response response = REST_CLIENT.performRequest(request); String result = EntityUtils.toString(response.getEntity()); ObjectMapper objectMapper = new ObjectMapper(); JsonNode jsonNode = objectMapper.readTree(result); @@ -117,7 +119,7 @@ public static String getClusterVersion() { } public void close() throws IOException { - restClient.close(); + REST_CLIENT.close(); } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java index f747fad8583..6dc753bb29b 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/config/SinkConfig.java @@ -35,11 +35,11 @@ public class SinkConfig { public static final String MAX_RETRY_SIZE = "max_retry_size"; - public static void setValue(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig){ - if(pluginConfig.hasPath(MAX_BATCH_SIZE)){ + public static void setValue(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig) { + if (pluginConfig.hasPath(MAX_BATCH_SIZE)) { BulkConfig.MAX_BATCH_SIZE = pluginConfig.getInt(MAX_BATCH_SIZE); } - if(pluginConfig.hasPath(MAX_RETRY_SIZE)){ + if (pluginConfig.hasPath(MAX_RETRY_SIZE)) { BulkConfig.MAX_RETRY_SIZE = pluginConfig.getInt(MAX_RETRY_SIZE); } } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java index b6108dc4790..dba8b8dd1ef 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/constant/BulkConfig.java @@ -27,11 +27,13 @@ public class BulkConfig { * once bulk es include max document size * {@link SinkConfig#MAX_BATCH_SIZE} */ + @SuppressWarnings("checkstyle:MagicNumber") public static int MAX_BATCH_SIZE = 10; /** * the max retry size of bulk es * {@link SinkConfig#MAX_RETRY_SIZE} */ + @SuppressWarnings("checkstyle:MagicNumber") public static int MAX_RETRY_SIZE = 3; } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java index 06c5581bbec..0e4c12b9f7c 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/ElasticsearchRowSerializer.java @@ -17,17 +17,18 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.IndexInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion; +import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.IndexInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.index.IndexSerializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.index.IndexSerializerFactory; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type.IndexTypeSerializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type.IndexTypeSerializerFactory; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + import java.util.HashMap; import java.util.Map; @@ -43,8 +44,8 @@ public class ElasticsearchRowSerializer implements SeaTunnelRowSerializer{ private final IndexTypeSerializer indexTypeSerializer; public ElasticsearchRowSerializer(ElasticsearchVersion elasticsearchVersion, IndexInfo indexInfo, SeaTunnelRowType seaTunnelRowType) { - this.indexTypeSerializer = IndexTypeSerializerFactory.getIndexTypeSerializer(elasticsearchVersion,indexInfo.getType()); - this.indexSerializer = IndexSerializerFactory.getIndexSerializer(indexInfo.getIndex(),seaTunnelRowType); + this.indexTypeSerializer = IndexTypeSerializerFactory.getIndexTypeSerializer(elasticsearchVersion, indexInfo.getType()); + this.indexSerializer = IndexSerializerFactory.getIndexSerializer(indexInfo.getIndex(), seaTunnelRowType); this.seaTunnelRowType = seaTunnelRowType; } @@ -59,13 +60,13 @@ public String serializeRow(SeaTunnelRow row){ StringBuilder sb = new StringBuilder(); - Map indexInner = new HashMap<>(); + Map indexInner = new HashMap<>(); String index = indexSerializer.serialize(row); - indexInner.put("_index",index); + indexInner.put("_index", index); indexTypeSerializer.fillType(indexInner); - Map> indexParam = new HashMap<>(); - indexParam.put("index",indexInner); + Map> indexParam = new HashMap<>(); + indexParam.put("index", indexInner); try { sb.append(objectMapper.writeValueAsString(indexParam)); sb.append("\n"); diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java index d1fbae8a4f6..53300f984f1 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/SeaTunnelRowSerializer.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize; - import org.apache.seatunnel.api.table.type.SeaTunnelRow; public interface SeaTunnelRowSerializer { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java index 7997632359a..2ddfc35c5ce 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/index/impl/VariableIndexSerializer.java @@ -33,7 +33,7 @@ public class VariableIndexSerializer implements IndexSerializer { private final String index; private final Map fieldIndexMap; - private final String NULL_DEFAULT = "null"; + private final String nullDefault = "null"; public VariableIndexSerializer(SeaTunnelRowType seaTunnelRowType, String index, List fieldNames) { this.index = index; @@ -61,7 +61,7 @@ public String serialize(SeaTunnelRow row) { private String getValue(int fieldIndex, SeaTunnelRow row) { Object valueObj = row.getField(fieldIndex); if (valueObj == null) { - return NULL_DEFAULT; + return nullDefault; } else { return valueObj.toString(); } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java index 3e528058ed1..7d0a395e015 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializer.java @@ -17,7 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type; - import java.util.Map; public interface IndexTypeSerializer { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java index 878257cb631..f70a54dc1c4 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/IndexTypeSerializerFactory.java @@ -17,12 +17,14 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type; +import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion.ES2; +import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion.ES5; +import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion.ES6; + import org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type.impl.NotIndexTypeSerializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.type.impl.RequiredIndexTypeSerializer; -import static org.apache.seatunnel.connectors.seatunnel.elasticsearch.constant.ElasticsearchVersion.*; - public class IndexTypeSerializerFactory { private static final String DEFAULT_TYPE = "st"; diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java index fa5afb5b85c..57bfe8116b7 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/serialize/type/impl/NotIndexTypeSerializer.java @@ -26,7 +26,6 @@ */ public class NotIndexTypeSerializer implements IndexTypeSerializer { - @Override public void fillType(Map indexInner) { diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java index a5eac83acd7..e0f7630cf9a 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java @@ -17,9 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.elasticsearch.sink; -import com.google.auto.service.AutoService; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -30,25 +28,25 @@ import org.apache.seatunnel.connectors.seatunnel.elasticsearch.state.ElasticsearchCommitInfo; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.state.ElasticsearchSinkState; -import java.util.Collections; +import com.google.auto.service.AutoService; +import java.util.Collections; @AutoService(SeaTunnelSink.class) public class ElasticsearchSink implements SeaTunnelSink { private org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig; - private SeaTunnelContext seaTunnelContext; private SeaTunnelRowType seaTunnelRowType; - @Override public String getPluginName() { return "Elasticsearch"; } @Override - public void prepare(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig) throws PrepareFailException { + public void prepare(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig) throws + PrepareFailException { this.pluginConfig = pluginConfig; SinkConfig.setValue(pluginConfig); } @@ -67,10 +65,4 @@ public SeaTunnelDataType getConsumedType() { public SinkWriter createWriter(SinkWriter.Context context) { return new ElasticsearchSinkWriter(context, seaTunnelRowType, pluginConfig, Collections.emptyList()); } - - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } - } diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java index 048f48f441d..d527d90d58c 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java @@ -30,7 +30,9 @@ import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.ElasticsearchRowSerializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.SeaTunnelRowSerializer; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.state.ElasticsearchCommitInfo; + import org.apache.seatunnel.shade.com.typesafe.config.Config; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +44,7 @@ /** * ElasticsearchSinkWriter is a sink writer that will write {@link SeaTunnelRow} to Elasticsearch. */ -public class ElasticsearchSinkWriter implements SinkWriter { +public class ElasticsearchSinkWriter implements SinkWriter { private final Context context; @@ -53,12 +55,11 @@ public class ElasticsearchSinkWriter implements SinkWrit private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchSinkWriter.class); - public ElasticsearchSinkWriter( Context context, SeaTunnelRowType seaTunnelRowType, Config pluginConfig, - List elasticsearchStates) { + List elasticsearchStates) { this.context = context; IndexInfo indexInfo = new IndexInfo(pluginConfig); diff --git a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java index 9ccc413ff03..097eca897ae 100644 --- a/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java +++ b/seatunnel-connectors-v2/connector-elasticsearch/src/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/util/RegexUtils.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.List; import java.util.regex.Matcher; - import java.util.regex.Pattern; public class RegexUtils { diff --git a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSource.java b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSource.java index 0200362d0ab..fd9387f2d1e 100644 --- a/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSource.java +++ b/seatunnel-connectors-v2/connector-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSource.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.fake.source; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.type.SeaTunnelRow; @@ -36,12 +36,12 @@ public class FakeSource extends AbstractSingleSplitSource { private Config pluginConfig; - private SeaTunnelContext seaTunnelContext; + private JobContext jobContext; private SeaTunnelSchema schema; @Override public Boundedness getBoundedness() { - return JobMode.BATCH.equals(seaTunnelContext.getJobMode()) ? Boundedness.BOUNDED : Boundedness.UNBOUNDED; + return JobMode.BATCH.equals(jobContext.getJobMode()) ? Boundedness.BOUNDED : Boundedness.UNBOUNDED; } @Override @@ -67,7 +67,7 @@ public void prepare(Config pluginConfig) { } @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java index 77b72f004ca..84827b7173d 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.file.sink; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.DefaultSerializer; import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.sink.SeaTunnelSink; @@ -47,7 +47,7 @@ public abstract class AbstractFileSink implements SeaTunnelSink createWriter(SinkWriter.Context context) throws IOException { - if (!seaTunnelContext.getJobMode().equals(JobMode.BATCH) && this.getSinkConfig().getSaveMode().equals(SaveMode.OVERWRITE)) { + if (!jobContext.getJobMode().equals(JobMode.BATCH) && this.getSinkConfig().getSaveMode().equals(SaveMode.OVERWRITE)) { throw new RuntimeException("only batch job can overwrite mode"); } @@ -104,9 +104,9 @@ public SinkWriter restoreWriter(Sin } @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - this.jobId = seaTunnelContext.getJobId(); + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; + this.jobId = jobContext.getJobId(); } @Override diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/BaseFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/BaseFileSink.java index e5bc67ffa57..0e5187e5b41 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/BaseFileSink.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/BaseFileSink.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.file.sink; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.DefaultSerializer; import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.sink.SeaTunnelSink; @@ -48,13 +48,13 @@ public abstract class BaseFileSink implements SeaTunnelSink createWriter(SinkWriter.Context context) throws IOException { - if (!seaTunnelContext.getJobMode().equals(JobMode.BATCH) && hiveSinkConfig.getTextFileSinkConfig().getSaveMode().equals(SaveMode.OVERWRITE)) { + if (!jobContext.getJobMode().equals(JobMode.BATCH) && hiveSinkConfig.getTextFileSinkConfig().getSaveMode().equals(SaveMode.OVERWRITE)) { throw new RuntimeException("only batch job can overwrite hive table"); } @@ -96,9 +96,9 @@ public SinkWriter restoreWriter(Sin } @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - this.jobId = seaTunnelContext.getJobId(); + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; + this.jobId = jobContext.getJobId(); } @Override diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java index 039d9b4c745..1f1a8c3fbb1 100644 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java @@ -21,7 +21,6 @@ import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceReader; @@ -47,8 +46,6 @@ @AutoService(SeaTunnelSource.class) public class HiveSource implements SeaTunnelSource { - private SeaTunnelContext seaTunnelContext; - private SeaTunnelRowType typeInfo; private ReadStrategy readStrategy; @@ -85,11 +82,6 @@ public void prepare(Config pluginConfig) { } } - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } - @Override public SeaTunnelDataType getProducedType() { return this.typeInfo; diff --git a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java index cd52b19a609..dff630b1219 100644 --- a/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java +++ b/seatunnel-connectors-v2/connector-http/connector-http-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/http/source/HttpSource.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.http.source; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; @@ -45,7 +45,7 @@ public class HttpSource extends AbstractSingleSplitSource { protected final HttpParameter httpParameter = new HttpParameter(); protected SeaTunnelRowType rowType; - protected SeaTunnelContext seaTunnelContext; + protected JobContext jobContext; protected DeserializationSchema deserializationSchema; @Override @@ -55,7 +55,7 @@ public String getPluginName() { @Override public Boundedness getBoundedness() { - return JobMode.BATCH.equals(seaTunnelContext.getJobMode()) ? Boundedness.BOUNDED : Boundedness.UNBOUNDED; + return JobMode.BATCH.equals(jobContext.getJobMode()) ? Boundedness.BOUNDED : Boundedness.UNBOUNDED; } @Override @@ -84,8 +84,8 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; } @Override diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java index 2ca69d784e7..f1cbb619a3d 100644 --- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java +++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java @@ -25,7 +25,6 @@ import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.USE_KERBEROS; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceReader; @@ -48,8 +47,6 @@ @AutoService(SeaTunnelSource.class) public class HudiSource implements SeaTunnelSource { - private SeaTunnelContext seaTunnelContext; - private SeaTunnelRowType typeInfo; private String filePath; @@ -103,11 +100,6 @@ public void prepare(Config pluginConfig) { } - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } - @Override public SeaTunnelDataType getProducedType() { return this.typeInfo; diff --git a/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSource.java b/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSource.java index a7c05259219..668d2861d66 100644 --- a/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSource.java +++ b/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/source/IoTDBSource.java @@ -21,8 +21,8 @@ import static org.apache.seatunnel.connectors.seatunnel.iotdb.config.SourceConfig.NODE_URLS; import static org.apache.seatunnel.connectors.seatunnel.iotdb.config.SourceConfig.PORT; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceReader; @@ -46,7 +46,7 @@ @AutoService(SeaTunnelSource.class) public class IoTDBSource implements SeaTunnelSource { - private SeaTunnelContext seaTunnelContext; + private JobContext jobContext; private SeaTunnelRowType typeInfo; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/SemanticXidGenerator.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/SemanticXidGenerator.java index 3d2a82b3d13..4f51c31a550 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/SemanticXidGenerator.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/SemanticXidGenerator.java @@ -19,7 +19,7 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.sink.SinkWriter; import javax.transaction.xa.Xid; @@ -63,7 +63,7 @@ public void open() { } @Override - public Xid generateXid(SeaTunnelContext context, SinkWriter.Context sinkContext, long checkpointId) { + public Xid generateXid(JobContext context, SinkWriter.Context sinkContext, long checkpointId) { byte[] jobIdBytes = context.getJobId().getBytes(); checkArgument(jobIdBytes.length <= JOB_ID_BYTES); System.arraycopy(jobIdBytes, 0, gtridBuffer, 0, JOB_ID_BYTES); @@ -75,7 +75,7 @@ public Xid generateXid(SeaTunnelContext context, SinkWriter.Context sinkContext, } @Override - public boolean belongsToSubtask(Xid xid, SeaTunnelContext context, SinkWriter.Context sinkContext) { + public boolean belongsToSubtask(Xid xid, JobContext context, SinkWriter.Context sinkContext) { if (xid.getFormatId() != FORMAT_ID) { return false; } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XaGroupOps.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XaGroupOps.java index e37e6b05a47..11ebf063395 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XaGroupOps.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XaGroupOps.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.xa; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.XidInfo; @@ -38,6 +38,6 @@ public GroupXaOperationResult commit( GroupXaOperationResult failAndRollback(Collection xids); - void recoverAndRollback(SeaTunnelContext context, SinkWriter.Context sinkContext, XidGenerator xidGenerator, Xid excludeXid); + void recoverAndRollback(JobContext context, SinkWriter.Context sinkContext, XidGenerator xidGenerator, Xid excludeXid); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XaGroupOpsImpl.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XaGroupOpsImpl.java index 05ecce1603e..ff2012fd757 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XaGroupOpsImpl.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XaGroupOpsImpl.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.xa; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.XidInfo; @@ -112,7 +112,7 @@ public GroupXaOperationResult failAndRollback(Collection xids) } @Override - public void recoverAndRollback(SeaTunnelContext context, SinkWriter.Context sinkContext, XidGenerator xidGenerator, Xid excludeXid) { + public void recoverAndRollback(JobContext context, SinkWriter.Context sinkContext, XidGenerator xidGenerator, Xid excludeXid) { Collection recovered = xaFacade.recover(); recovered.remove(excludeXid); if (recovered.isEmpty()) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XidGenerator.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XidGenerator.java index a801750540f..de3ef005916 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XidGenerator.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/XidGenerator.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.xa; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.sink.SinkWriter; import javax.transaction.xa.Xid; @@ -31,14 +31,14 @@ public interface XidGenerator extends Serializable, AutoCloseable { - Xid generateXid(SeaTunnelContext context, SinkWriter.Context sinkContext, long checkpointId); + Xid generateXid(JobContext context, SinkWriter.Context sinkContext, long checkpointId); default void open() {} /** * @return true if the provided transaction belongs to this subtask */ - boolean belongsToSubtask(Xid xid, SeaTunnelContext context, SinkWriter.Context sinkContext); + boolean belongsToSubtask(Xid xid, JobContext context, SinkWriter.Context sinkContext); @Override default void close() {} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcExactlyOnceSinkWriter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcExactlyOnceSinkWriter.java index b4527a9ff59..20461db9f7d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcExactlyOnceSinkWriter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcExactlyOnceSinkWriter.java @@ -20,7 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSinkOptions; @@ -53,7 +53,7 @@ public class JdbcExactlyOnceSinkWriter private final SinkWriter.Context sinkcontext; - private final SeaTunnelContext context; + private final JobContext context; private final List recoverStates; @@ -72,7 +72,7 @@ public class JdbcExactlyOnceSinkWriter public JdbcExactlyOnceSinkWriter( SinkWriter.Context sinkcontext, - SeaTunnelContext context, + JobContext context, JdbcStatementBuilder statementBuilder, JdbcSinkOptions jdbcSinkOptions, List states) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java index 672303f8c9b..97a63b980e5 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.sink; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.DefaultSerializer; import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.sink.SeaTunnelSink; @@ -51,7 +51,7 @@ public class JdbcSink private SeaTunnelRowType seaTunnelRowType; - private SeaTunnelContext seaTunnelContext; + private JobContext jobContext; private JdbcSinkOptions jdbcSinkOptions; @@ -76,7 +76,7 @@ public SinkWriter createWriter(SinkWriter. if (jdbcSinkOptions.isExactlyOnce()) { sinkWriter = new JdbcExactlyOnceSinkWriter( context, - seaTunnelContext, + jobContext, statementBuilder, jdbcSinkOptions, new ArrayList<>() @@ -98,7 +98,7 @@ public SinkWriter restoreWriter(SinkWriter JdbcStatementBuilder statementBuilder = (st, row) -> JdbcUtils.setRecordToStatement(st, null, row); return new JdbcExactlyOnceSinkWriter( context, - seaTunnelContext, + jobContext, statementBuilder, jdbcSinkOptions, states @@ -132,8 +132,8 @@ public Optional> getAggregatedCommitInfoSer } @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java index 2717436f194..a8d0766f3ea 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.source; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; @@ -57,7 +56,6 @@ public class JdbcSource implements SeaTunnelSource { protected static final Logger LOG = LoggerFactory.getLogger(JdbcSource.class); - private SeaTunnelContext seaTunnelContext; private JdbcSourceOptions jdbcSourceOptions; private SeaTunnelRowType typeInfo; @@ -96,11 +94,6 @@ public void prepare(Config pluginConfig) throws PrepareFailException { ); } - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } - @Override public Boundedness getBoundedness() { return Boundedness.BOUNDED; diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java index a4d534e6775..62314b87a63 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java @@ -23,8 +23,8 @@ import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PATTERN; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TOPIC; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceReader; @@ -53,11 +53,11 @@ public class KafkaSource implements SeaTunnelSource restoreEnumerat } @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; } } diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSource.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSource.java index 65b004a6a36..c10796a21ed 100644 --- a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSource.java +++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSource.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.connectors.seatunnel.kudu.source; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.DefaultSerializer; import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.source.Boundedness; @@ -53,7 +52,6 @@ public class KuduSource implements SeaTunnelSource { private static final Logger LOGGER = LoggerFactory.getLogger(KuduSource.class); - private SeaTunnelContext seaTunnelContext; private SeaTunnelRowType rowTypeInfo; private KuduInputFormat kuduInputFormat; private PartitionParameter partitionParameter; @@ -167,11 +165,6 @@ private PartitionParameter initPartitionParameter(KuduClient kuduClient, String return new PartitionParameter(keyColumn, Long.parseLong(minKey + ""), Long.parseLong(maxKey + "")); } - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } - public SeaTunnelRowType getSeaTunnelRowType(List columnSchemaList) { ArrayList> seaTunnelDataTypes = new ArrayList<>(); ArrayList fieldNames = new ArrayList<>(); diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java index 0770975690f..45c79048bea 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java @@ -25,7 +25,6 @@ import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.URI; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; @@ -50,8 +49,6 @@ @AutoService(SeaTunnelSource.class) public class MongodbSource extends AbstractSingleSplitSource { - private SeaTunnelContext seaTunnelContext; - private SeaTunnelRowType rowType; private MongodbParameters params; @@ -91,11 +88,6 @@ public void prepare(Config config) throws PrepareFailException { } } - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } - @Override public Boundedness getBoundedness() { return Boundedness.BOUNDED; diff --git a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java index d860e6af59b..407d32222d3 100644 --- a/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java +++ b/seatunnel-connectors-v2/connector-redis/src/main/java/org/apache/seatunnel/connectors/seatunnel/redis/source/RedisSource.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.connectors.seatunnel.redis.source; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.serialization.DeserializationSchema; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; @@ -43,7 +42,6 @@ @AutoService(SeaTunnelSource.class) public class RedisSource extends AbstractSingleSplitSource { private final RedisParameters redisParameters = new RedisParameters(); - private SeaTunnelContext seaTunnelContext; private SeaTunnelRowType seaTunnelRowType; private DeserializationSchema deserializationSchema; @@ -87,11 +85,6 @@ public SeaTunnelDataType getProducedType() { return seaTunnelRowType; } - @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; - } - @Override public AbstractSingleSplitReader createReader(SingleSplitReaderContext readerContext) throws Exception { return new RedisSourceReader(redisParameters, readerContext, deserializationSchema); diff --git a/seatunnel-connectors-v2/connector-socket/src/main/java/org/apache/seatunnel/connectors/seatunnel/socket/source/SocketSource.java b/seatunnel-connectors-v2/connector-socket/src/main/java/org/apache/seatunnel/connectors/seatunnel/socket/source/SocketSource.java index a237679d434..f97e00e93dd 100644 --- a/seatunnel-connectors-v2/connector-socket/src/main/java/org/apache/seatunnel/connectors/seatunnel/socket/source/SocketSource.java +++ b/seatunnel-connectors-v2/connector-socket/src/main/java/org/apache/seatunnel/connectors/seatunnel/socket/source/SocketSource.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.socket.source; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelContext; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.type.BasicType; @@ -38,11 +38,11 @@ @AutoService(SeaTunnelSource.class) public class SocketSource extends AbstractSingleSplitSource { private SocketSourceParameter parameter; - private SeaTunnelContext seaTunnelContext; + private JobContext jobContext; @Override public Boundedness getBoundedness() { - return JobMode.BATCH.equals(seaTunnelContext.getJobMode()) ? Boundedness.BOUNDED : Boundedness.UNBOUNDED; + return JobMode.BATCH.equals(jobContext.getJobMode()) ? Boundedness.BOUNDED : Boundedness.UNBOUNDED; } @Override @@ -56,8 +56,8 @@ public void prepare(Config pluginConfig) throws PrepareFailException { } @Override - public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { - this.seaTunnelContext = seaTunnelContext; + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; } @Override diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractPluginExecuteProcessor.java index 5e3178bfb81..1b680b9e8b6 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractPluginExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractPluginExecuteProcessor.java @@ -20,6 +20,7 @@ import static org.apache.seatunnel.apis.base.plugin.Plugin.RESULT_TABLE_NAME; import static org.apache.seatunnel.apis.base.plugin.Plugin.SOURCE_TABLE_NAME; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.common.utils.ReflectionUtils; import org.apache.seatunnel.flink.FlinkEnvironment; import org.apache.seatunnel.flink.util.TableUtil; @@ -41,6 +42,7 @@ public abstract class AbstractPluginExecuteProcessor implements PluginExecute protected final FlinkEnvironment flinkEnvironment; protected final List pluginConfigs; + protected final JobContext jobContext; protected final List plugins; protected static final String ENGINE_TYPE = "seatunnel"; protected static final String PLUGIN_NAME = "plugin_name"; @@ -57,8 +59,10 @@ public abstract class AbstractPluginExecuteProcessor implements PluginExecute }; protected AbstractPluginExecuteProcessor(FlinkEnvironment flinkEnvironment, + JobContext jobContext, List pluginConfigs) { this.flinkEnvironment = flinkEnvironment; + this.jobContext = jobContext; this.pluginConfigs = pluginConfigs; this.plugins = initializePlugins(pluginConfigs); } diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java index df452dd855a..e6f5719fa0e 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.core.starter.flink.execution; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.core.starter.config.EngineType; import org.apache.seatunnel.core.starter.config.EnvironmentFactory; @@ -55,10 +55,11 @@ public class FlinkExecution implements TaskExecution { public FlinkExecution(Config config) { this.config = config; this.flinkEnvironment = new EnvironmentFactory(config, EngineType.FLINK).getEnvironment(); - SeaTunnelContext.getContext().setJobMode(flinkEnvironment.getJobMode()); - this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(flinkEnvironment, config.getConfigList("source")); - this.transformPluginExecuteProcessor = new TransformExecuteProcessor(flinkEnvironment, config.getConfigList("transform")); - this.sinkPluginExecuteProcessor = new SinkExecuteProcessor(flinkEnvironment, config.getConfigList("sink")); + JobContext jobContext = new JobContext(); + jobContext.setJobMode(flinkEnvironment.getJobMode()); + this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList("source")); + this.transformPluginExecuteProcessor = new TransformExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList("transform")); + this.sinkPluginExecuteProcessor = new SinkExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList("sink")); registerPlugin(); } diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java index 8de5d422dcd..6a06516d597 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.core.starter.flink.execution; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -46,8 +46,9 @@ public class SinkExecuteProcessor extends AbstractPluginExecuteProcessor pluginConfigs) { - super(flinkEnvironment, pluginConfigs); + super(flinkEnvironment, jobContext, pluginConfigs); } @Override @@ -60,7 +61,7 @@ protected List seaTunnelSink = sinkPluginDiscovery.createPluginInstance(pluginIdentifier); seaTunnelSink.prepare(sinkConfig); - seaTunnelSink.setSeaTunnelContext(SeaTunnelContext.getContext()); + seaTunnelSink.setJobContext(jobContext); return seaTunnelSink; }).distinct().collect(Collectors.toList()); flinkEnvironment.registerPlugin(pluginJars); diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java index a1b31836fd0..1dc378a835e 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java @@ -19,7 +19,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.common.constants.JobMode; @@ -53,8 +53,9 @@ public class SourceExecuteProcessor extends AbstractPluginExecuteProcessor sourceConfigs) { - super(flinkEnvironment, sourceConfigs); + super(flinkEnvironment, jobContext, sourceConfigs); } @Override @@ -110,8 +111,8 @@ protected List initializePlugins(List pluginC jars.addAll(sourcePluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier))); SeaTunnelSource seaTunnelSource = sourcePluginDiscovery.createPluginInstance(pluginIdentifier); seaTunnelSource.prepare(sourceConfig); - seaTunnelSource.setSeaTunnelContext(SeaTunnelContext.getContext()); - if (SeaTunnelContext.getContext().getJobMode() == JobMode.BATCH + seaTunnelSource.setJobContext(jobContext); + if (jobContext.getJobMode() == JobMode.BATCH && seaTunnelSource.getBoundedness() == org.apache.seatunnel.api.source.Boundedness.UNBOUNDED) { throw new UnsupportedOperationException(String.format("'%s' source don't support off-line job.", seaTunnelSource.getPluginName())); } diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java index d077f6d3a44..62d65fbcdac 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.core.starter.flink.execution; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; import org.apache.seatunnel.flink.FlinkEnvironment; import org.apache.seatunnel.flink.stream.FlinkStreamTransform; @@ -39,8 +40,9 @@ public class TransformExecuteProcessor extends AbstractPluginExecuteProcessor pluginConfigs) { - super(flinkEnvironment, pluginConfigs); + super(flinkEnvironment, jobContext, pluginConfigs); } @Override diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/AbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/AbstractPluginExecuteProcessor.java index 0dad6f680b1..af087c0c4ae 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/AbstractPluginExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/AbstractPluginExecuteProcessor.java @@ -20,6 +20,7 @@ import static org.apache.seatunnel.apis.base.plugin.Plugin.RESULT_TABLE_NAME; import static org.apache.seatunnel.apis.base.plugin.Plugin.SOURCE_TABLE_NAME; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.spark.SparkEnvironment; import org.apache.seatunnel.shade.com.typesafe.config.Config; @@ -34,13 +35,16 @@ public abstract class AbstractPluginExecuteProcessor implements PluginExecute protected final SparkEnvironment sparkEnvironment; protected final List pluginConfigs; + protected final JobContext jobContext; protected final List plugins; protected static final String ENGINE_TYPE = "seatunnel"; protected static final String PLUGIN_NAME = "plugin_name"; protected AbstractPluginExecuteProcessor(SparkEnvironment sparkEnvironment, + JobContext jobContext, List pluginConfigs) { this.sparkEnvironment = sparkEnvironment; + this.jobContext = jobContext; this.pluginConfigs = pluginConfigs; this.plugins = initializePlugins(pluginConfigs); } diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index 45ab010124a..fbcecd7c820 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.core.starter.spark.execution; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; @@ -43,8 +43,9 @@ public class SinkExecuteProcessor extends AbstractPluginExecuteProcessor pluginConfigs) { - super(sparkEnvironment, pluginConfigs); + super(sparkEnvironment, jobContext, pluginConfigs); } @Override @@ -56,7 +57,7 @@ protected SinkExecuteProcessor(SparkEnvironment sparkEnvironment, pluginJars.addAll(sinkPluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier))); SeaTunnelSink seaTunnelSink = sinkPluginDiscovery.createPluginInstance(pluginIdentifier); seaTunnelSink.prepare(sinkConfig); - seaTunnelSink.setSeaTunnelContext(SeaTunnelContext.getContext()); + seaTunnelSink.setJobContext(jobContext); return seaTunnelSink; }).distinct().collect(Collectors.toList()); sparkEnvironment.registerPlugin(pluginJars); diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java index 8dfcb4ef488..f6eafe67c6e 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.core.starter.spark.execution; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.common.utils.SerializationUtils; @@ -44,8 +44,9 @@ public class SourceExecuteProcessor extends AbstractPluginExecuteProcessor sourceConfigs) { - super(sparkEnvironment, sourceConfigs); + super(sparkEnvironment, jobContext, sourceConfigs); } @Override @@ -75,7 +76,7 @@ public List> execute(List> upstreamDataStreams) { jars.addAll(sourcePluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier))); SeaTunnelSource seaTunnelSource = sourcePluginDiscovery.createPluginInstance(pluginIdentifier); seaTunnelSource.prepare(sourceConfig); - seaTunnelSource.setSeaTunnelContext(SeaTunnelContext.getContext()); + seaTunnelSource.setJobContext(jobContext); sources.add(seaTunnelSource); } sparkEnvironment.registerPlugin(new ArrayList<>(jars)); diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java index 31193e0cffb..c1d0c9c22cc 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.core.starter.spark.execution; -import org.apache.seatunnel.api.common.SeaTunnelContext; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.core.starter.config.EngineType; import org.apache.seatunnel.core.starter.config.EnvironmentFactory; @@ -47,10 +47,11 @@ public class SparkExecution { public SparkExecution(Config config) { this.config = config; this.sparkEnvironment = (SparkEnvironment) new EnvironmentFactory<>(config, EngineType.SPARK).getEnvironment(); - SeaTunnelContext.getContext().setJobMode(sparkEnvironment.getJobMode()); - this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(sparkEnvironment, config.getConfigList(Constants.SOURCE)); - this.transformPluginExecuteProcessor = new TransformExecuteProcessor(sparkEnvironment, config.getConfigList(Constants.TRANSFORM)); - this.sinkPluginExecuteProcessor = new SinkExecuteProcessor(sparkEnvironment, config.getConfigList(Constants.SINK)); + JobContext jobContext = new JobContext(); + jobContext.setJobMode(sparkEnvironment.getJobMode()); + this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(sparkEnvironment, jobContext, config.getConfigList(Constants.SOURCE)); + this.transformPluginExecuteProcessor = new TransformExecuteProcessor(sparkEnvironment, jobContext, config.getConfigList(Constants.TRANSFORM)); + this.sinkPluginExecuteProcessor = new SinkExecuteProcessor(sparkEnvironment, jobContext, config.getConfigList(Constants.SINK)); } public void execute() throws TaskExecuteException { diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java index 5668457b768..66d5e1ee654 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.core.starter.spark.execution; +import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; import org.apache.seatunnel.plugin.discovery.PluginIdentifier; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSparkTransformPluginDiscovery; @@ -39,8 +40,9 @@ public class TransformExecuteProcessor extends AbstractPluginExecuteProcessor pluginConfigs) { - super(sparkEnvironment, pluginConfigs); + super(sparkEnvironment, jobContext, pluginConfigs); } @Override From 547a6be2d4ff6b30ce66117100d0e9fca203ddb7 Mon Sep 17 00:00:00 2001 From: Hisoka Date: Wed, 14 Sep 2022 18:50:02 +0800 Subject: [PATCH 33/44] [Starter] [Core] Modify connector v2 starter (#2726) * [Starter] [Core] Modify connector v2 starter and remove spark/flink in seatunnel-core-starter --- pom.xml | 8 ++ seatunnel-apis/seatunnel-api-base/pom.xml | 5 - seatunnel-core/seatunnel-core-starter/pom.xml | 10 +- .../config/AbstractExecutionContext.java | 92 ------------------- .../starter/config/EnvironmentFactory.java | 26 ++---- .../core/starter/config/ExecutionFactory.java | 84 ----------------- .../seatunnel-flink-starter/pom.xml | 6 -- ... => start-seatunnel-flink-connector-v2.sh} | 0 .../core/starter/flink/FlinkStarter.java | 4 +- .../core/starter/flink/SeatunnelFlink.java | 4 +- .../flink/config/FlinkEnvironmentFactory.java | 35 +++++++ ...FlinkJobType.java => StarterConstant.java} | 14 +-- .../flink/execution/FlinkExecution.java | 15 ++- .../seatunnel-spark-starter/pom.xml | 6 -- ... => start-seatunnel-spark-connector-v2.sh} | 2 +- .../core/starter/spark/SeatunnelSpark.java | 3 +- .../core/starter/spark/SparkStarter.java | 3 +- .../spark/config/SparkApiConfigChecker.java | 1 + ...ment.java => SparkEnvironmentFactory.java} | 45 ++------- .../starter/spark/config/StarterConstant.java | 24 +++++ .../spark/execution/SparkExecution.java | 8 +- seatunnel-dist/release-docs/LICENSE | 5 - .../seatunnel/e2e/flink/FlinkContainer.java | 2 +- .../seatunnel/e2e/spark/SparkContainer.java | 2 +- tools/dependencies/known-dependencies.txt | 7 +- 25 files changed, 108 insertions(+), 303 deletions(-) delete mode 100644 seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/AbstractExecutionContext.java delete mode 100644 seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/ExecutionFactory.java rename seatunnel-core/seatunnel-flink-starter/src/main/bin/{start-seatunnel-flink-new-connector.sh => start-seatunnel-flink-connector-v2.sh} (100%) create mode 100644 seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/FlinkEnvironmentFactory.java rename seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/{FlinkJobType.java => StarterConstant.java} (76%) rename seatunnel-core/seatunnel-spark-starter/src/main/bin/{start-seatunnel-spark-new-connector.sh => start-seatunnel-spark-connector-v2.sh} (95%) rename seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/{SparkEnvironment.java => SparkEnvironmentFactory.java} (53%) create mode 100644 seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/StarterConstant.java diff --git a/pom.xml b/pom.xml index 6f3c5414d1b..4233ebcbfd2 100644 --- a/pom.xml +++ b/pom.xml @@ -142,6 +142,7 @@ 2.7 2.12.6 1.18.0 + 1.20 8.0.16 42.3.3 8.1.2.141 @@ -290,6 +291,13 @@ + + + org.apache.commons + commons-compress + ${commons-compress.version} + + org.apache.logging.log4j log4j-core diff --git a/seatunnel-apis/seatunnel-api-base/pom.xml b/seatunnel-apis/seatunnel-api-base/pom.xml index 9fe328718ae..da26ee48172 100644 --- a/seatunnel-apis/seatunnel-api-base/pom.xml +++ b/seatunnel-apis/seatunnel-api-base/pom.xml @@ -35,11 +35,6 @@ seatunnel-common ${project.version} - - - org.apache.seatunnel - seatunnel-config-shade - diff --git a/seatunnel-core/seatunnel-core-starter/pom.xml b/seatunnel-core/seatunnel-core-starter/pom.xml index 775bb9a137f..53424d9ba57 100644 --- a/seatunnel-core/seatunnel-core-starter/pom.xml +++ b/seatunnel-core/seatunnel-core-starter/pom.xml @@ -40,19 +40,19 @@ org.apache.seatunnel - seatunnel-api-flink + seatunnel-api ${project.version} org.apache.seatunnel - seatunnel-api-spark + seatunnel-plugin-discovery ${project.version} + - org.apache.seatunnel - seatunnel-plugin-discovery - ${project.version} + org.apache.commons + commons-compress diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/AbstractExecutionContext.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/AbstractExecutionContext.java deleted file mode 100644 index a834965c099..00000000000 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/AbstractExecutionContext.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.seatunnel.core.starter.config; - -import org.apache.seatunnel.apis.base.api.BaseSink; -import org.apache.seatunnel.apis.base.api.BaseSource; -import org.apache.seatunnel.apis.base.api.BaseTransform; -import org.apache.seatunnel.apis.base.env.RuntimeEnv; -import org.apache.seatunnel.common.constants.JobMode; -import org.apache.seatunnel.common.constants.PluginType; -import org.apache.seatunnel.plugin.discovery.PluginIdentifier; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import java.net.URL; -import java.util.Arrays; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** - * The ExecutionContext contains all configuration needed to run the job. - * - * @param environment type. - */ -public abstract class AbstractExecutionContext { - - private final Config config; - private final EngineType engine; - - private final ENVIRONMENT environment; - private final JobMode jobMode; - - public AbstractExecutionContext(Config config, EngineType engine) { - this.config = config; - this.engine = engine; - this.environment = new EnvironmentFactory(config, engine).getEnvironment(); - this.jobMode = environment.getJobMode(); - } - - public Config getRootConfig() { - return config; - } - - public EngineType getEngine() { - return engine; - } - - public ENVIRONMENT getEnvironment() { - return environment; - } - - public JobMode getJobMode() { - return jobMode; - } - - public abstract List> getSources(); - - public abstract List> getTransforms(); - - public abstract List> getSinks(); - - public abstract List getPluginJars(); - - @SuppressWarnings("checkstyle:Indentation") - protected List getPluginIdentifiers(PluginType... pluginTypes) { - return Arrays.stream(pluginTypes).flatMap((Function>) pluginType -> { - List configList = config.getConfigList(pluginType.getType()); - return configList.stream() - .map(pluginConfig -> PluginIdentifier - .of(engine.getEngine(), - pluginType.getType(), - pluginConfig.getString("plugin_name"))); - }).collect(Collectors.toList()); - } -} diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/EnvironmentFactory.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/EnvironmentFactory.java index 20e77a1dbb1..f38a8add53c 100644 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/EnvironmentFactory.java +++ b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/EnvironmentFactory.java @@ -19,8 +19,6 @@ import org.apache.seatunnel.apis.base.env.RuntimeEnv; import org.apache.seatunnel.common.constants.JobMode; -import org.apache.seatunnel.flink.FlinkEnvironment; -import org.apache.seatunnel.spark.SparkEnvironment; import org.apache.seatunnel.shade.com.typesafe.config.Config; @@ -31,39 +29,27 @@ * * @param environment type */ -public class EnvironmentFactory { - +public abstract class EnvironmentFactory { private static final String PLUGIN_NAME_KEY = "plugin_name"; private final Config config; - private final EngineType engine; - public EnvironmentFactory(Config config, EngineType engine) { + public EnvironmentFactory(Config config) { this.config = config; - this.engine = engine; } // todo:put this method into submodule to avoid dependency on the engine public synchronized ENVIRONMENT getEnvironment() { Config envConfig = config.getConfig("env"); - boolean enableHive = checkIsContainHive(); - ENVIRONMENT env; - switch (engine) { - case SPARK: - env = (ENVIRONMENT) new SparkEnvironment().setEnableHive(enableHive); - break; - case FLINK: - env = (ENVIRONMENT) new FlinkEnvironment(); - break; - default: - throw new IllegalArgumentException("Engine: " + engine + " is not supported"); - } + ENVIRONMENT env = newEnvironment(); env.setConfig(envConfig) .setJobMode(getJobMode(envConfig)).prepare(); return env; } - private boolean checkIsContainHive() { + protected abstract ENVIRONMENT newEnvironment(); + + protected boolean checkIsContainHive() { List sourceConfigList = config.getConfigList(PluginType.SOURCE.getType()); for (Config c : sourceConfigList) { if (c.getString(PLUGIN_NAME_KEY).toLowerCase().contains("hive")) { diff --git a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/ExecutionFactory.java b/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/ExecutionFactory.java deleted file mode 100644 index d0d99cd948b..00000000000 --- a/seatunnel-core/seatunnel-core-starter/src/main/java/org/apache/seatunnel/core/starter/config/ExecutionFactory.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.seatunnel.core.starter.config; - -import org.apache.seatunnel.apis.base.api.BaseSink; -import org.apache.seatunnel.apis.base.api.BaseSource; -import org.apache.seatunnel.apis.base.api.BaseTransform; -import org.apache.seatunnel.apis.base.env.Execution; -import org.apache.seatunnel.apis.base.env.RuntimeEnv; -import org.apache.seatunnel.flink.FlinkEnvironment; -import org.apache.seatunnel.flink.batch.FlinkBatchExecution; -import org.apache.seatunnel.flink.stream.FlinkStreamExecution; -import org.apache.seatunnel.spark.SparkEnvironment; -import org.apache.seatunnel.spark.batch.SparkBatchExecution; -import org.apache.seatunnel.spark.stream.SparkStreamingExecution; -import org.apache.seatunnel.spark.structuredstream.StructuredStreamingExecution; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Used to create {@link Execution}. - * - * @param environment type - */ -public class ExecutionFactory { - - private static final Logger LOGGER = LoggerFactory.getLogger(ExecutionFactory.class); - - public AbstractExecutionContext executionContext; - - public ExecutionFactory(AbstractExecutionContext executionContext) { - this.executionContext = executionContext; - } - - public Execution, BaseTransform, BaseSink, ENVIRONMENT> createExecution() { - Execution execution = null; - switch (executionContext.getEngine()) { - case SPARK: - SparkEnvironment sparkEnvironment = (SparkEnvironment) executionContext.getEnvironment(); - switch (executionContext.getJobMode()) { - case STREAMING: - execution = new SparkStreamingExecution(sparkEnvironment); - break; - case STRUCTURED_STREAMING: - execution = new StructuredStreamingExecution(sparkEnvironment); - break; - default: - execution = new SparkBatchExecution(sparkEnvironment); - } - break; - case FLINK: - FlinkEnvironment flinkEnvironment = (FlinkEnvironment) executionContext.getEnvironment(); - switch (executionContext.getJobMode()) { - case STREAMING: - execution = new FlinkStreamExecution(flinkEnvironment); - break; - default: - execution = new FlinkBatchExecution(flinkEnvironment); - } - break; - default: - throw new IllegalArgumentException("No suitable engine"); - } - LOGGER.info("current execution is [{}]", execution.getClass().getName()); - return (Execution, BaseTransform, BaseSink, ENVIRONMENT>) execution; - } - -} diff --git a/seatunnel-core/seatunnel-flink-starter/pom.xml b/seatunnel-core/seatunnel-flink-starter/pom.xml index f542fbfe61b..a34426b126f 100644 --- a/seatunnel-core/seatunnel-flink-starter/pom.xml +++ b/seatunnel-core/seatunnel-flink-starter/pom.xml @@ -42,12 +42,6 @@ ${project.version} - - org.apache.seatunnel - seatunnel-api - ${project.version} - - org.apache.seatunnel seatunnel-translation-flink diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh b/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-connector-v2.sh similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh rename to seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-connector-v2.sh diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java index df10fc645ac..5a793986611 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java @@ -20,7 +20,7 @@ import org.apache.seatunnel.common.config.Common; import org.apache.seatunnel.core.starter.Starter; import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; -import org.apache.seatunnel.core.starter.flink.config.FlinkJobType; +import org.apache.seatunnel.core.starter.flink.config.StarterConstant; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; import java.util.ArrayList; @@ -46,7 +46,7 @@ public class FlinkStarter implements Starter { private final String appJar; FlinkStarter(String[] args) { - this.flinkCommandArgs = CommandLineUtils.parse(args, new FlinkCommandArgs(), FlinkJobType.JAR.getType(), true); + this.flinkCommandArgs = CommandLineUtils.parse(args, new FlinkCommandArgs(), StarterConstant.SHELL_NAME, true); // set the deployment mode, used to get the job jar path. Common.setDeployMode(flinkCommandArgs.getDeployMode()); Common.setStarter(true); diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeatunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeatunnelFlink.java index cf01b979d40..e221f471056 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeatunnelFlink.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeatunnelFlink.java @@ -22,13 +22,13 @@ import org.apache.seatunnel.core.starter.exception.CommandException; import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import org.apache.seatunnel.core.starter.flink.command.FlinkCommandBuilder; -import org.apache.seatunnel.core.starter.flink.config.FlinkJobType; +import org.apache.seatunnel.core.starter.flink.config.StarterConstant; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; public class SeatunnelFlink { public static void main(String[] args) throws CommandException { - FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parse(args, new FlinkCommandArgs(), FlinkJobType.JAR.getType(), true); + FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parse(args, new FlinkCommandArgs(), StarterConstant.SHELL_NAME, true); Command flinkCommand = new FlinkCommandBuilder() .buildCommand(flinkCommandArgs); Seatunnel.run(flinkCommand); diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/FlinkEnvironmentFactory.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/FlinkEnvironmentFactory.java new file mode 100644 index 00000000000..a3d3c2eec9a --- /dev/null +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/FlinkEnvironmentFactory.java @@ -0,0 +1,35 @@ +/* + * 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.seatunnel.core.starter.flink.config; + +import org.apache.seatunnel.core.starter.config.EnvironmentFactory; +import org.apache.seatunnel.flink.FlinkEnvironment; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +public class FlinkEnvironmentFactory extends EnvironmentFactory { + + public FlinkEnvironmentFactory(Config config) { + super(config); + } + + @Override + protected FlinkEnvironment newEnvironment() { + return new FlinkEnvironment(); + } +} diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/FlinkJobType.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/StarterConstant.java similarity index 76% rename from seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/FlinkJobType.java rename to seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/StarterConstant.java index f18f8346969..2ad1da85cbb 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/FlinkJobType.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/config/StarterConstant.java @@ -17,18 +17,8 @@ package org.apache.seatunnel.core.starter.flink.config; -public enum FlinkJobType { - JAR("start-seatunnel-flink.sh"), - SQL("start-seatunnel-sql.sh"), - ; +public class StarterConstant { - private final String type; + public static final String SHELL_NAME = "start-seatunnel-flink-connector-v2.sh"; - FlinkJobType(String type) { - this.type = type; - } - - public String getType() { - return this.type; - } } diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java index e6f5719fa0e..a4c52638a50 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java @@ -18,11 +18,11 @@ package org.apache.seatunnel.core.starter.flink.execution; import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.common.config.Common; -import org.apache.seatunnel.core.starter.config.EngineType; -import org.apache.seatunnel.core.starter.config.EnvironmentFactory; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; import org.apache.seatunnel.core.starter.execution.TaskExecution; +import org.apache.seatunnel.core.starter.flink.config.FlinkEnvironmentFactory; import org.apache.seatunnel.flink.FlinkEnvironment; import org.apache.seatunnel.shade.com.typesafe.config.Config; @@ -45,21 +45,18 @@ public class FlinkExecution implements TaskExecution { private static final Logger LOGGER = LoggerFactory.getLogger(FlinkExecution.class); - - private final Config config; private final FlinkEnvironment flinkEnvironment; private final PluginExecuteProcessor sourcePluginExecuteProcessor; private final PluginExecuteProcessor transformPluginExecuteProcessor; private final PluginExecuteProcessor sinkPluginExecuteProcessor; public FlinkExecution(Config config) { - this.config = config; - this.flinkEnvironment = new EnvironmentFactory(config, EngineType.FLINK).getEnvironment(); + this.flinkEnvironment = new FlinkEnvironmentFactory(config).getEnvironment(); JobContext jobContext = new JobContext(); jobContext.setJobMode(flinkEnvironment.getJobMode()); - this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList("source")); - this.transformPluginExecuteProcessor = new TransformExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList("transform")); - this.sinkPluginExecuteProcessor = new SinkExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList("sink")); + this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList(Constants.SOURCE)); + this.transformPluginExecuteProcessor = new TransformExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList(Constants.TRANSFORM)); + this.sinkPluginExecuteProcessor = new SinkExecuteProcessor(flinkEnvironment, jobContext, config.getConfigList(Constants.SINK)); registerPlugin(); } diff --git a/seatunnel-core/seatunnel-spark-starter/pom.xml b/seatunnel-core/seatunnel-spark-starter/pom.xml index e894a6a157e..91dfbf9392c 100644 --- a/seatunnel-core/seatunnel-spark-starter/pom.xml +++ b/seatunnel-core/seatunnel-spark-starter/pom.xml @@ -35,12 +35,6 @@ - - org.apache.seatunnel - seatunnel-api - ${project.version} - - org.apache.seatunnel seatunnel-core-starter diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-new-connector.sh b/seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-connector-v2.sh similarity index 95% rename from seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-new-connector.sh rename to seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-connector-v2.sh index a54dc55ee33..67626da6671 100755 --- a/seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-new-connector.sh +++ b/seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-connector-v2.sh @@ -50,7 +50,7 @@ else args=$@ fi -CMD=$(java -cp ${APP_JAR} ${APP_MAIN} ${args} | tail -n 1) && EXIT_CODE=$? || EXIT_CODE=$? +CMD=$(java -cp ${APP_JAR} ${APP_MAIN} ${args}) && EXIT_CODE=$? || EXIT_CODE=$? if [ ${EXIT_CODE} -eq 234 ]; then # print usage echo "${CMD}" diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeatunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeatunnelSpark.java index 338921ad12a..bc7c9a94daa 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeatunnelSpark.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeatunnelSpark.java @@ -22,12 +22,13 @@ import org.apache.seatunnel.core.starter.exception.CommandException; import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; import org.apache.seatunnel.core.starter.spark.command.SparkCommandBuilder; +import org.apache.seatunnel.core.starter.spark.config.StarterConstant; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; public class SeatunnelSpark { public static void main(String[] args) throws CommandException { - SparkCommandArgs sparkArgs = CommandLineUtils.parse(args, new SparkCommandArgs(), "start-seatunnel-spark.sh", true); + SparkCommandArgs sparkArgs = CommandLineUtils.parse(args, new SparkCommandArgs(), StarterConstant.SHELL_NAME, true); Command sparkCommand = new SparkCommandBuilder().buildCommand(sparkArgs); Seatunnel.run(sparkCommand); diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java index 3aaf02c6714..c455db37abe 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.core.starter.config.ConfigBuilder; import org.apache.seatunnel.core.starter.config.PluginType; import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; +import org.apache.seatunnel.core.starter.spark.config.StarterConstant; import org.apache.seatunnel.core.starter.utils.CommandLineUtils; import org.apache.seatunnel.core.starter.utils.CompressionUtils; import org.apache.seatunnel.plugin.discovery.PluginIdentifier; @@ -109,7 +110,7 @@ public static void main(String[] args) throws IOException { * {@link ClientModeSparkStarter} depending on deploy mode. */ static SparkStarter getInstance(String[] args) { - SparkCommandArgs commandArgs = CommandLineUtils.parse(args, new SparkCommandArgs(), "start-seatunnel-spark.sh", true); + SparkCommandArgs commandArgs = CommandLineUtils.parse(args, new SparkCommandArgs(), StarterConstant.SHELL_NAME, true); DeployMode deployMode = commandArgs.getDeployMode(); switch (deployMode) { case CLUSTER: diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkApiConfigChecker.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkApiConfigChecker.java index b08a4a752c2..3f340edafd5 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkApiConfigChecker.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkApiConfigChecker.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.core.starter.config.ConfigChecker; import org.apache.seatunnel.core.starter.exception.ConfigCheckException; +import org.apache.seatunnel.spark.SparkEnvironment; import org.apache.seatunnel.shade.com.typesafe.config.Config; diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkEnvironment.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkEnvironmentFactory.java similarity index 53% rename from seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkEnvironment.java rename to seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkEnvironmentFactory.java index f7d5b83d4a4..11764e44e19 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkEnvironment.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/SparkEnvironmentFactory.java @@ -18,49 +18,18 @@ package org.apache.seatunnel.core.starter.spark.config; -import org.apache.seatunnel.apis.base.env.RuntimeEnv; -import org.apache.seatunnel.common.config.CheckResult; -import org.apache.seatunnel.common.constants.JobMode; +import org.apache.seatunnel.core.starter.config.EnvironmentFactory; +import org.apache.seatunnel.spark.SparkEnvironment; import org.apache.seatunnel.shade.com.typesafe.config.Config; -import java.net.URL; -import java.util.List; - -public class SparkEnvironment implements RuntimeEnv { - - @Override - public SparkEnvironment setConfig(Config config) { - return null; - } - - @Override - public Config getConfig() { - return null; +public class SparkEnvironmentFactory extends EnvironmentFactory { + public SparkEnvironmentFactory(Config config) { + super(config); } @Override - public CheckResult checkConfig() { - return null; - } - - @Override - public SparkEnvironment prepare() { - return null; - } - - @Override - public SparkEnvironment setJobMode(JobMode mode) { - return null; - } - - @Override - public JobMode getJobMode() { - return null; - } - - @Override - public void registerPlugin(List pluginPaths) { - + protected SparkEnvironment newEnvironment() { + return new SparkEnvironment().setEnableHive(checkIsContainHive()); } } diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/StarterConstant.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/StarterConstant.java new file mode 100644 index 00000000000..e9598459a8d --- /dev/null +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/config/StarterConstant.java @@ -0,0 +1,24 @@ +/* + * 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.seatunnel.core.starter.spark.config; + +public class StarterConstant { + + public static final String SHELL_NAME = "start-seatunnel-spark-connector-v2.sh"; + +} diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java index c1d0c9c22cc..62890277c3e 100644 --- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java +++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java @@ -19,9 +19,8 @@ import org.apache.seatunnel.api.common.JobContext; import org.apache.seatunnel.common.Constants; -import org.apache.seatunnel.core.starter.config.EngineType; -import org.apache.seatunnel.core.starter.config.EnvironmentFactory; import org.apache.seatunnel.core.starter.exception.TaskExecuteException; +import org.apache.seatunnel.core.starter.spark.config.SparkEnvironmentFactory; import org.apache.seatunnel.spark.SparkEnvironment; import org.apache.seatunnel.shade.com.typesafe.config.Config; @@ -37,16 +36,13 @@ public class SparkExecution { private static final Logger LOGGER = LoggerFactory.getLogger(SparkExecution.class); - - private final Config config; private final SparkEnvironment sparkEnvironment; private final PluginExecuteProcessor sourcePluginExecuteProcessor; private final PluginExecuteProcessor transformPluginExecuteProcessor; private final PluginExecuteProcessor sinkPluginExecuteProcessor; public SparkExecution(Config config) { - this.config = config; - this.sparkEnvironment = (SparkEnvironment) new EnvironmentFactory<>(config, EngineType.SPARK).getEnvironment(); + this.sparkEnvironment = new SparkEnvironmentFactory(config).getEnvironment(); JobContext jobContext = new JobContext(); jobContext.setJobMode(sparkEnvironment.getJobMode()); this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(sparkEnvironment, jobContext, config.getConfigList(Constants.SOURCE)); diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE index ecb3c9ed021..9e124ebffe2 100644 --- a/seatunnel-dist/release-docs/LICENSE +++ b/seatunnel-dist/release-docs/LICENSE @@ -328,9 +328,7 @@ The text of each license is the standard Apache 2.0 license. (Apache License, Version 2.0) Apache Commons CLI (commons-cli:commons-cli:1.4 - http://commons.apache.org/proper/commons-cli/) (Apache License, Version 2.0) Apache Commons Codec (commons-codec:commons-codec:1.13 - https://commons.apache.org/proper/commons-codec/) (Apache License, Version 2.0) Apache Commons Collections (org.apache.commons:commons-collections4:4.4 - https://commons.apache.org/proper/commons-collections/) - (Apache License, Version 2.0) Apache Commons Compress (org.apache.commons:commons-compress:1.18 - https://commons.apache.org/proper/commons-compress/) (Apache License, Version 2.0) Apache Commons Compress (org.apache.commons:commons-compress:1.20 - https://commons.apache.org/proper/commons-compress/) - (Apache License, Version 2.0) Apache Commons Compress (org.apache.commons:commons-compress:1.21 - https://commons.apache.org/proper/commons-compress/) (Apache License, Version 2.0) Apache Commons Configuration (org.apache.commons:commons-configuration2:2.1.1 - http://commons.apache.org/proper/commons-configuration/) (Apache License, Version 2.0) Apache Commons Crypto (org.apache.commons:commons-crypto:1.0.0 - http://commons.apache.org/proper/commons-crypto/) (Apache License, Version 2.0) Apache Commons Email (org.apache.commons:commons-email:1.5 - http://commons.apache.org/proper/commons-email/) @@ -602,7 +600,6 @@ The text of each license is the standard Apache 2.0 license. (The Apache Software License, Version 2.0) Apache Avro IPC (org.apache.avro:avro-ipc:1.8.2 - http://avro.apache.org) (The Apache Software License, Version 2.0) Apache Avro Mapred API (org.apache.avro:avro-mapred:1.8.2 - http://avro.apache.org/avro-mapred) (The Apache Software License, Version 2.0) Apache Commons CSV (org.apache.commons:commons-csv:1.0 - http://commons.apache.org/proper/commons-csv/) - (The Apache Software License, Version 2.0) Apache Commons Compress (org.apache.commons:commons-compress:1.8.1 - http://commons.apache.org/proper/commons-compress/) (The Apache Software License, Version 2.0) Apache Commons DBCP (org.apache.commons:commons-dbcp2:2.9.0 - http://commons.apache.org/proper/commons-dbcp/) (The Apache Software License, Version 2.0) Apache Commons Lang (org.apache.commons:commons-lang3:3.3.2 - http://commons.apache.org/proper/commons-lang/) (The Apache Software License, Version 2.0) Apache Commons Logging (commons-logging:commons-logging:1.2 - http://commons.apache.org/proper/commons-logging/) @@ -649,7 +646,6 @@ The text of each license is the standard Apache 2.0 license. (The Apache Software License, Version 2.0) Bean Validation API (javax.validation:validation-api:2.0.1.Final - http://beanvalidation.org) (The Apache Software License, Version 2.0) Commons BeanUtils Core (commons-beanutils:commons-beanutils-core:1.8.0 - http://commons.apache.org/beanutils/) (The Apache Software License, Version 2.0) Commons CLI (commons-cli:commons-cli:1.2 - http://commons.apache.org/cli/) - (The Apache Software License, Version 2.0) Commons Compress (org.apache.commons:commons-compress:1.4.1 - http://commons.apache.org/compress/) (The Apache Software License, Version 2.0) Commons Configuration (commons-configuration:commons-configuration:1.6 - http://commons.apache.org/${pom.artifactId.substring(8)}/) (The Apache Software License, Version 2.0) Commons Configuration (commons-configuration:commons-configuration:1.7 - http://commons.apache.org/configuration/) (The Apache Software License, Version 2.0) Commons DBCP (commons-dbcp:commons-dbcp:1.4 - http://commons.apache.org/dbcp/) @@ -1127,7 +1123,6 @@ The following components are provided under the Public Domain License. See proje The text of each license is also included at licenses/LICENSE-[project].txt. (Public Domain) AOP alliance (aopalliance:aopalliance:1.0 - http://aopalliance.sourceforge.net) - (Public Domain) XZ for Java (org.tukaani:xz:1.0 - http://tukaani.org/xz/java.html) (Public Domain) XZ for Java (org.tukaani:xz:1.5 - http://tukaani.org/xz/java.html) (Public Domain) XZ for Java (org.tukaani:xz:1.8 - https://tukaani.org/xz/java.html) (Public Domain, per Creative Commons CC0) HdrHistogram (org.hdrhistogram:HdrHistogram:2.1.9 - http://hdrhistogram.github.io/HdrHistogram/) diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java index f6f02cce752..9380854b5ed 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-flink-e2e-base/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java @@ -38,7 +38,7 @@ protected String getStartModuleName() { @Override protected String getStartShellName() { - return "start-seatunnel-flink-new-connector.sh"; + return "start-seatunnel-flink-connector-v2.sh"; } @Override diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java index a26271655c9..68b8ff9bb10 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-spark-e2e-base/src/test/java/org/apache/seatunnel/e2e/spark/SparkContainer.java @@ -32,7 +32,7 @@ protected String getStartModuleName() { @Override protected String getStartShellName() { - return "start-seatunnel-spark-new-connector.sh"; + return "start-seatunnel-spark-connector-v2.sh"; } @Override diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index d80df589f21..4e28fd58fab 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -59,11 +59,7 @@ commons-codec-1.13.jar commons-collections-3.2.2.jar commons-collections4-4.4.jar commons-compiler-3.1.7.jar -commons-compress-1.18.jar -commons-compress-1.19.jar -commons-compress-1.21.jar -commons-compress-1.4.1.jar -commons-compress-1.8.1.jar +commons-compress-1.20.jar commons-configuration-1.7.jar commons-configuration2-2.1.1.jar commons-crypto-1.0.0.jar @@ -642,7 +638,6 @@ xercesImpl-2.9.1.jar xml-apis-1.3.04.jar xmlbeans-3.1.0.jar xmlenc-0.52.jar -xz-1.0.jar xz-1.5.jar zkclient-0.3.jar zookeeper-3.3.1.jar From 971af707e83376dfb7c1dd3e39e0b88c226fa33f Mon Sep 17 00:00:00 2001 From: hk__lrzy Date: Wed, 14 Sep 2022 19:02:59 +0800 Subject: [PATCH 34/44] [SeaTunnel][Improve]Add sink name in flink starter. (#2728) --- .../core/starter/flink/execution/SinkExecuteProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java index 6a06516d597..a7c48832b66 100644 --- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java @@ -76,7 +76,7 @@ public List> execute(List> upstreamDataStreams) SeaTunnelSink seaTunnelSink = plugins.get(i); DataStream stream = fromSourceTable(sinkConfig).orElse(input); seaTunnelSink.setTypeInfo((SeaTunnelRowType) TypeConverterUtils.convert(stream.getType())); - stream.sinkTo(new FlinkSink<>(seaTunnelSink)); + stream.sinkTo(new FlinkSink<>(seaTunnelSink)).name(seaTunnelSink.getPluginName()); } // the sink is the last stream return null; From 3b5927584db5302bc512801d9f82f4451d5891ca Mon Sep 17 00:00:00 2001 From: Laglangyue <35491928+laglangyue@users.noreply.github.com> Date: Wed, 14 Sep 2022 23:03:12 +0800 Subject: [PATCH 35/44] [hotfix][translation] fix data,time,timestamp type convert for spark (#2637) * [hotfix] data,time,timestamp convert for spark * fxi the instant convert * revert to nullable field value * Update exception info * Update exception info Co-authored-by: laglangyue Co-authored-by: Zongwen Li --- .../main/resources/examples/spark.batch.conf | 3 +- .../serialization/InternalRowConverter.java | 21 ++++---- .../common/utils/InstantConverterUtils.java | 51 +++++++++++++++++++ .../common/utils/TypeConverterUtils.java | 4 +- 4 files changed, 65 insertions(+), 14 deletions(-) create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/utils/InstantConverterUtils.java diff --git a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/resources/examples/spark.batch.conf b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/resources/examples/spark.batch.conf index 25ae6b5679c..00b2338c293 100644 --- a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/resources/examples/spark.batch.conf +++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/resources/examples/spark.batch.conf @@ -49,7 +49,6 @@ source { c_null = "null" c_bytes = bytes c_date = date - c_time = time c_timestamp = timestamp } } @@ -72,7 +71,7 @@ transform { # you can also use other transform plugins, such as sql sql { - sql = "select c_array,c_string,c_boolean,c_tinyint,c_smallint,c_int,c_bigint,c_float,c_double,c_null,c_bytes from fake" + sql = "select c_array,c_string,c_boolean,c_tinyint,c_smallint,c_int,c_bigint,c_float,c_double,c_null,c_bytes,c_date,c_timestamp from fake" result_table_name = "sql" } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/serialization/InternalRowConverter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/serialization/InternalRowConverter.java index e95989ec85d..31673c59591 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/serialization/InternalRowConverter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/serialization/InternalRowConverter.java @@ -24,6 +24,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.translation.serialization.RowConverter; +import org.apache.seatunnel.translation.spark.common.utils.InstantConverterUtils; import org.apache.seatunnel.translation.spark.common.utils.TypeConverterUtils; import org.apache.spark.sql.catalyst.InternalRow; @@ -43,11 +44,9 @@ import java.io.IOException; import java.math.BigDecimal; -import java.sql.Date; import java.sql.Timestamp; import java.time.LocalDate; import java.time.LocalDateTime; -import java.time.LocalTime; import java.util.HashMap; import java.util.Map; import java.util.function.BiFunction; @@ -74,13 +73,12 @@ private static Object convert(Object field, SeaTunnelDataType dataType) { SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; return convert(seaTunnelRow, rowType); case DATE: - return Date.valueOf((LocalDate) field); + return (int) ((LocalDate) field).toEpochDay(); case TIME: - //TODO: how reconvert? - LocalTime localTime = (LocalTime) field; - return Timestamp.valueOf(LocalDateTime.of(LocalDate.ofEpochDay(0), localTime)); + // TODO: Support TIME Type + throw new RuntimeException("time type is not supported now, but will be supported in the future."); case TIMESTAMP: - return Timestamp.valueOf((LocalDateTime) field); + return InstantConverterUtils.toEpochMicro(Timestamp.valueOf((LocalDateTime) field).toInstant()); case MAP: return convertMap((Map) field, (MapType) dataType, InternalRowConverter::convert); case STRING: @@ -141,8 +139,10 @@ private static MutableValue createMutableValue(SeaTunnelDataType dataType) { case SMALLINT: return new MutableShort(); case INT: + case DATE: return new MutableInt(); case BIGINT: + case TIMESTAMP: return new MutableLong(); case FLOAT: return new MutableFloat(); @@ -167,11 +167,12 @@ private static Object reconvert(Object field, SeaTunnelDataType dataType) { case ROW: return reconvert((InternalRow) field, (SeaTunnelRowType) dataType); case DATE: - return ((Date) field).toLocalDate(); + return LocalDate.ofEpochDay((int) field); case TIME: - return ((Timestamp) field).toLocalDateTime().toLocalTime(); + // TODO: Support TIME Type + throw new RuntimeException("time type is not supported now, but will be supported in the future."); case TIMESTAMP: - return ((Timestamp) field).toLocalDateTime(); + return Timestamp.from(InstantConverterUtils.ofEpochMicro((long) field)).toLocalDateTime(); case MAP: return convertMap((Map) field, (MapType) dataType, InternalRowConverter::reconvert); case STRING: diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/utils/InstantConverterUtils.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/utils/InstantConverterUtils.java new file mode 100644 index 00000000000..37a90ee23ad --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/utils/InstantConverterUtils.java @@ -0,0 +1,51 @@ +/* + * 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.seatunnel.translation.spark.common.utils; + +import java.time.Instant; + +public class InstantConverterUtils { + + private static final int MICRO_OF_SECOND = 1000_000; + private static final int MICRO_OF_NANOS = 1000; + + /** + * @see Instant#toEpochMilli() + */ + public static Long toEpochMicro(Instant instant) { + long seconds = instant.getEpochSecond(); + int nanos = instant.getNano(); + if (seconds < 0 && nanos > 0) { + long micro = Math.multiplyExact(seconds + 1, MICRO_OF_SECOND); + long adjustment = nanos / MICRO_OF_NANOS - MICRO_OF_SECOND; + return Math.addExact(micro, adjustment); + } else { + long millis = Math.multiplyExact(seconds, MICRO_OF_SECOND); + return Math.addExact(millis, nanos / MICRO_OF_NANOS); + } + } + + /** + * @see Instant#ofEpochMilli(long) + */ + public static Instant ofEpochMicro(long epochMicro) { + long secs = Math.floorDiv(epochMicro, MICRO_OF_SECOND); + int mos = (int) Math.floorMod(epochMicro, MICRO_OF_SECOND); + return Instant.ofEpochSecond(secs, Math.multiplyExact(mos, MICRO_OF_NANOS)); + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/utils/TypeConverterUtils.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/utils/TypeConverterUtils.java index 36260be058b..493fd9a9cfa 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/utils/TypeConverterUtils.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/common/utils/TypeConverterUtils.java @@ -85,8 +85,8 @@ public static DataType convert(SeaTunnelDataType dataType) { return DataTypes.BinaryType; case DATE: return DataTypes.DateType; - case TIME: - // TODO: how reconvert? + // case TIME: + // TODO: not support now, how reconvert? case TIMESTAMP: return DataTypes.TimestampType; case ARRAY: From a357dca36503c9ef49bdfb15c7a3bfeaa50ca0b8 Mon Sep 17 00:00:00 2001 From: TyrantLucifer Date: Thu, 15 Sep 2022 13:49:02 +0800 Subject: [PATCH 36/44] [Improve][Connector-V2] Refactor hive source & sink connector (#2708) --- docs/en/connector-v2/sink/Hive.md | 34 ++-- docs/en/connector-v2/source/Hive.md | 47 +++++ .../connector-hive/pom.xml | 131 +++---------- .../seatunnel/hive/config/Config.java | 21 -- .../seatunnel/hive/config/Constant.java | 23 --- .../seatunnel/hive/config/HiveConfig.java | 47 +++++ .../seatunnel/hive/config/SourceConfig.java | 29 --- .../hive/exception/HivePluginException.java | 29 --- .../hive/sink/BaseHiveCommitInfo.java | 36 ---- .../hive/sink/HiveAggregatedCommitInfo.java | 36 ---- .../seatunnel/hive/sink/HiveCommitInfo.java | 36 ---- .../seatunnel/hive/sink/HiveSink.java | 166 ++++++++-------- .../sink/HiveSinkAggregatedCommitter.java | 139 ------------- .../seatunnel/hive/sink/HiveSinkConfig.java | 155 --------------- .../seatunnel/hive/sink/HiveSinkState.java | 30 --- .../seatunnel/hive/sink/HiveSinkWriter.java | 160 --------------- .../seatunnel/hive/source/HadoopConf.java | 34 ---- .../seatunnel/hive/source/HiveSource.java | 100 ++++------ .../hive/source/HiveSourceReader.java | 98 ---------- .../hive/source/HiveSourceSplit.java | 36 ---- .../source/HiveSourceSplitEnumerator.java | 126 ------------ .../hive/source/HiveSourceState.java | 35 ---- .../reader/format/AbstractReadStrategy.java | 84 -------- .../file/reader/format/OrcReadStrategy.java | 168 ---------------- .../reader/format/ParquetReadStrategy.java | 185 ------------------ .../file/reader/format/ReadStrategy.java | 42 ---- .../reader/format/ReadStrategyFactory.java | 38 ---- .../file/reader/format/TextReadStrategy.java | 56 ------ .../source/file/reader/type/FileTypeEnum.java | 48 ----- .../hive/utils/HiveMetaStoreProxy.java | 45 ++++- .../hive/sink/HiveSinkConfigTest.java | 51 ----- 31 files changed, 293 insertions(+), 1972 deletions(-) create mode 100644 docs/en/connector-v2/source/Hive.md delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Config.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java create mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/HiveConfig.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/BaseHiveCommitInfo.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceReader.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplitEnumerator.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/OrcReadStrategy.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ParquetReadStrategy.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategy.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategyFactory.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/type/FileTypeEnum.java delete mode 100644 seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfigTest.java diff --git a/docs/en/connector-v2/sink/Hive.md b/docs/en/connector-v2/sink/Hive.md index 56b49ad7ba9..59d8dc5c33c 100644 --- a/docs/en/connector-v2/sink/Hive.md +++ b/docs/en/connector-v2/sink/Hive.md @@ -22,22 +22,22 @@ By default, we use 2PC commit to ensure `exactly-once` ## Options -| name | type | required | default value | -| --------------------------------- | ------ | -------- | ------------------------------------------------------------- | -| hive_table_name | string | yes | - | -| hive_metastore_uris | string | yes | - | -| partition_by | array | no | - | -| sink_columns | array | no | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean| no | true | -| save_mode | string | no | "append" | +| name | type | required | default value | +|-----------------------| ------ | -------- | ------------------------------------------------------------- | +| table_name | string | yes | - | +| metastore_uri | string | yes | - | +| partition_by | array | no | - | +| sink_columns | array | no | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean| no | true | +| save_mode | string | no | "append" | -### hive_table_name [string] +### table_name [string] Target Hive table name eg: db1.table1 -### hive_metastore_uris [string] +### metastore_uri [string] -Hive metastore uris +Hive metastore uri ### partition_by [array] @@ -64,13 +64,9 @@ Streaming Job not support `overwrite`. ```bash -Hive { - hive_table_name="db1.table1" - hive_metastore_uris="thrift://localhost:9083" - partition_by=["age"] - sink_columns=["name","age"] - is_enable_transaction=true - save_mode="append" -} + Hive { + table_name = "default.seatunnel_orc" + metastore_uri = "thrift://namenode001:9083" + } ``` diff --git a/docs/en/connector-v2/source/Hive.md b/docs/en/connector-v2/source/Hive.md new file mode 100644 index 00000000000..dcc4030f573 --- /dev/null +++ b/docs/en/connector-v2/source/Hive.md @@ -0,0 +1,47 @@ +# Hive + +> Hive source connector + +## Description + +Read data from Hive. + +In order to use this connector, You must ensure your spark/flink cluster already integrated hive. The tested hive version is 2.3.9. + +## Key features + +- [x] [exactly-once](../../concept/connector-v2-features.md) + +By default, we use 2PC commit to ensure `exactly-once` + +- [ ] [schema projection](../../concept/connector-v2-features.md) +- [x] file format + - [x] text + - [x] parquet + - [x] orc + +## Options + +| name | type | required | default value | +|-----------------------| ------ | -------- | ------------------------------------------------------------- | +| table_name | string | yes | - | +| metastore_uri | string | yes | - | + +### table_name [string] + +Target Hive table name eg: db1.table1 + +### metastore_uri [string] + +Hive metastore uri + +## Example + +```bash + + Hive { + table_name = "default.seatunnel_orc" + metastore_uri = "thrift://namenode001:9083" + } + +``` diff --git a/seatunnel-connectors-v2/connector-hive/pom.xml b/seatunnel-connectors-v2/connector-hive/pom.xml index a00016488e2..39c6f642991 100644 --- a/seatunnel-connectors-v2/connector-hive/pom.xml +++ b/seatunnel-connectors-v2/connector-hive/pom.xml @@ -30,146 +30,61 @@ connector-hive - 2.3.9 - 2.7.5-7.0 - 1.5.6 - 1.10.0 - 4.4 - 3.4 - 3.1.6 + 2.3.9 - org.apache.hive - hive-exec - ${hive.exec.version} - provided + org.apache.seatunnel + connector-file-hadoop + ${project.version} - org.pentaho - pentaho-aggdesigner-algorithm - - - javax.servlet - servlet-api - - - org.apache.logging.log4j - log4j-1.2-api - - - org.apache.logging.log4j - log4j-web - - - com.fasterxml.jackson.core - * - - - org.apapche.hadoop - * - - - com.github.joshelser - dropwizard-metrics-hadoop-metrics2-reporter - - - org.apache.logging.log4j - * - - - org.apache.zookeeper - zookeeper - - - org.apache.hadoop - hadoop-yarn-server-resourcemanager - - - org.apache.hadoop - hadoop-hdfs + org.apache.flink + flink-shaded-hadoop-2 - org.apache.flink flink-shaded-hadoop-2 - ${flink.hadoop.version} provided - - - org.apache.seatunnel - seatunnel-core-base - ${project.version} - test - - - org.apache.commons - commons-lang3 - ${commons.lang3.version} - - - org.apache.orc - orc-core - ${orc.version} + org.apache.hive + hive-metastore + ${hive.metastore.version} - javax.servlet - servlet-api + log4j + log4j - org.apache.hadoop - hadoop-common + log4j-1.2-api + org.apache.logging.log4j - org.apache.hadoop - hadoop-hdfs + log4j-slf4j-impl + org.apache.logging.log4j + log4j-web org.apache.logging.log4j - * - com.fasterxml.jackson.core - * + slf4j-log4j12 + org.slf4j - org.apapche.hadoop - * + parquet-hadoop-bundle + org.apache.parquet - org.apache.curator - * + jdk.tools + jdk.tools - - - org.apache.parquet - parquet-avro - ${parquet-avro.version} - - - - org.apache.commons - commons-collections4 - ${commons.collecton4.version} - - - - org.apache.seatunnel - connector-file-hadoop - ${project.version} - - - - org.codehaus.janino - janino - ${janino.version} - + diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Config.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Config.java deleted file mode 100644 index 2177ffbdea1..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Config.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.config; - -public class Config { -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java deleted file mode 100644 index dd0b2ab5465..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.config; - -public class Constant { - public static final String HIVE_RESULT_TABLE_NAME = "hive_table_name"; - public static final String HIVE_METASTORE_URIS = "hive_metastore_uris"; -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/HiveConfig.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/HiveConfig.java new file mode 100644 index 00000000000..16ab34b5ec8 --- /dev/null +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/HiveConfig.java @@ -0,0 +1,47 @@ +/* + * 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.seatunnel.connectors.seatunnel.hive.config; + +import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hive.metastore.api.Table; + +public class HiveConfig { + public static final String TABLE_NAME = "table_name"; + public static final String METASTORE_URI = "metastore_uri"; + public static final String TEXT_INPUT_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextInputFormat"; + public static final String TEXT_OUTPUT_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"; + public static final String PARQUET_INPUT_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; + public static final String PARQUET_OUTPUT_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; + public static final String ORC_INPUT_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; + public static final String ORC_OUTPUT_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; + + public static Pair getTableInfo(Config config) { + String table = config.getString(TABLE_NAME); + String[] splits = table.split("\\."); + if (splits.length != 2) { + throw new RuntimeException("Please config " + TABLE_NAME + " as db.table format"); + } + HiveMetaStoreProxy hiveMetaStoreProxy = HiveMetaStoreProxy.getInstance(config); + Table tableInformation = hiveMetaStoreProxy.getTable(splits[0], splits[1]); + return Pair.of(splits, tableInformation); + } +} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java deleted file mode 100644 index 7054403164b..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.config; - -import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; - -public class SourceConfig { - - public static final String FILE_TYPE = "file.type"; - - public static final String DEFAULT_FS = FS_DEFAULT_NAME_KEY; - - public static final String FILE_PATH = "file.path"; -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java deleted file mode 100644 index 8e1c22548e3..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.exception; - -public class HivePluginException extends Exception{ - - public HivePluginException(String message) { - super(message); - } - - public HivePluginException(String message, Throwable cause) { - super(message, cause); - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/BaseHiveCommitInfo.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/BaseHiveCommitInfo.java deleted file mode 100644 index 7422ecf4cf3..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/BaseHiveCommitInfo.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.sink; - -import lombok.AllArgsConstructor; -import lombok.Data; -import lombok.NoArgsConstructor; -import org.apache.hadoop.hive.metastore.api.Table; - -import java.io.Serializable; - -@Data -@AllArgsConstructor -@NoArgsConstructor -public class BaseHiveCommitInfo implements Serializable { - - private String hiveMetastoreUris; - - private Table table; - -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java deleted file mode 100644 index 1311c925df2..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.sink; - -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo; - -import lombok.AllArgsConstructor; -import lombok.Data; -import org.apache.hadoop.hive.metastore.api.Table; - -@Data -@AllArgsConstructor -public class HiveAggregatedCommitInfo extends BaseHiveCommitInfo { - - private FileAggregatedCommitInfo fileAggregatedCommitInfo; - - public HiveAggregatedCommitInfo(FileAggregatedCommitInfo fileAggregatedCommitInfo, String hiveMetastoreUris, Table table) { - super(hiveMetastoreUris, table); - this.fileAggregatedCommitInfo = fileAggregatedCommitInfo; - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java deleted file mode 100644 index 509352f4641..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.sink; - -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo; - -import lombok.AllArgsConstructor; -import lombok.Data; -import org.apache.hadoop.hive.metastore.api.Table; - -@Data -@AllArgsConstructor -public class HiveCommitInfo extends BaseHiveCommitInfo { - - private FileCommitInfo fileCommitInfo; - - public HiveCommitInfo(FileCommitInfo fileCommitInfo, String hiveMetastoreUris, Table table) { - super(hiveMetastoreUris, table); - this.fileCommitInfo = fileCommitInfo; - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java index 36bd50ca327..9adae11580c 100644 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java @@ -17,40 +17,50 @@ package org.apache.seatunnel.connectors.seatunnel.hive.sink; -import org.apache.seatunnel.api.common.JobContext; +import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FIELD_DELIMITER; +import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FILE_FORMAT; +import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FILE_NAME_EXPRESSION; +import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.IS_PARTITION_FIELD_WRITE_IN_FILE; +import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.PATH; +import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.ROW_DELIMITER; +import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.SAVE_MODE; +import static org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig.ORC_OUTPUT_FORMAT_CLASSNAME; +import static org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig.PARQUET_OUTPUT_FORMAT_CLASSNAME; +import static org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig.TEXT_OUTPUT_FORMAT_CLASSNAME; + import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.serialization.DefaultSerializer; -import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.sink.SeaTunnelSink; -import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; -import org.apache.seatunnel.api.sink.SinkWriter; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.constants.JobMode; +import org.apache.seatunnel.common.config.CheckConfigUtil; +import org.apache.seatunnel.common.config.CheckResult; +import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; +import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.sink.BaseFileSink; import org.apache.seatunnel.connectors.seatunnel.file.sink.config.SaveMode; +import org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig; +import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy; import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; import com.google.auto.service.AutoService; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Table; -import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; import java.util.List; -import java.util.Optional; +import java.util.Map; +import java.util.stream.Collectors; -/** - * Hive Sink implementation by using SeaTunnel sink API. - * This class contains the method to create {@link HiveSinkWriter} and {@link HiveSinkAggregatedCommitter}. - */ @AutoService(SeaTunnelSink.class) -public class HiveSink implements SeaTunnelSink { - - private Config config; - private String jobId; - private Long checkpointId; - private SeaTunnelRowType seaTunnelRowTypeInfo; - private JobContext jobContext; - private HiveSinkConfig hiveSinkConfig; +public class HiveSink extends BaseFileSink { + private String dbName; + private String tableName; + private Table tableInformation; @Override public String getPluginName() { @@ -58,73 +68,67 @@ public String getPluginName() { } @Override - public void setTypeInfo(SeaTunnelRowType seaTunnelRowTypeInfo) { - this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo; - this.hiveSinkConfig = new HiveSinkConfig(config, seaTunnelRowTypeInfo); - } + public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) { + super.setTypeInfo(seaTunnelRowType); + HiveMetaStoreProxy hiveMetaStoreProxy = HiveMetaStoreProxy.getInstance(pluginConfig); + // --------------------Check textFileSinkConfig with the hive table info------------------- + List fields = hiveMetaStoreProxy.getTableFields(dbName, tableName); + List partitionKeys = tableInformation.getPartitionKeys(); + + // Remove partitionKeys from table fields + List fieldNotContainPartitionKey = fields.stream().filter(filed -> !partitionKeys.contains(filed)).collect(Collectors.toList()); + + // check fields size must same as sinkColumnList size + if (fieldNotContainPartitionKey.size() != textFileSinkConfig.getSinkColumnList().size()) { + throw new RuntimeException("sink columns size must same as hive table field size"); + } - @Override - public SeaTunnelDataType getConsumedType() { - return this.seaTunnelRowTypeInfo; + // check hivePartitionFieldList size must same as partitionFieldList size + if (partitionKeys.size() != textFileSinkConfig.getPartitionFieldList().size()) { + throw new RuntimeException("partition by columns size must same as hive table partition columns size"); + } + hiveMetaStoreProxy.close(); } @Override public void prepare(Config pluginConfig) throws PrepareFailException { - this.config = pluginConfig; - this.checkpointId = 1L; - } - - @Override - public SinkWriter createWriter(SinkWriter.Context context) throws IOException { - if (!jobContext.getJobMode().equals(JobMode.BATCH) && hiveSinkConfig.getTextFileSinkConfig().getSaveMode().equals(SaveMode.OVERWRITE)) { - throw new RuntimeException("only batch job can overwrite hive table"); + CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, HiveConfig.METASTORE_URI, HiveConfig.TABLE_NAME); + if (!result.isSuccess()) { + throw new PrepareFailException(getPluginName(), PluginType.SINK, result.getMsg()); } - - if (!this.getSinkConfig().getTextFileSinkConfig().isEnableTransaction()) { - throw new RuntimeException("Hive Sink Connector only support transaction now"); + Pair tableInfo = HiveConfig.getTableInfo(pluginConfig); + dbName = tableInfo.getLeft()[0]; + tableName = tableInfo.getLeft()[1]; + tableInformation = tableInfo.getRight(); + String outputFormat = tableInformation.getSd().getOutputFormat(); + if (TEXT_OUTPUT_FORMAT_CLASSNAME.equals(outputFormat)) { + Map parameters = tableInformation.getSd().getSerdeInfo().getParameters(); + pluginConfig = pluginConfig.withValue(FILE_FORMAT, ConfigValueFactory.fromAnyRef(FileFormat.TEXT.toString())) + .withValue(FIELD_DELIMITER, ConfigValueFactory.fromAnyRef(parameters.get("field.delim"))) + .withValue(ROW_DELIMITER, ConfigValueFactory.fromAnyRef(parameters.get("line.delim"))); + } else if (PARQUET_OUTPUT_FORMAT_CLASSNAME.equals(outputFormat)) { + pluginConfig = pluginConfig.withValue(FILE_FORMAT, ConfigValueFactory.fromAnyRef(FileFormat.PARQUET.toString())); + } else if (ORC_OUTPUT_FORMAT_CLASSNAME.equals(outputFormat)) { + pluginConfig = pluginConfig.withValue(FILE_FORMAT, ConfigValueFactory.fromAnyRef(FileFormat.ORC.toString())); + } else { + throw new RuntimeException("Only support [text parquet orc] file now"); } - return new HiveSinkWriter(seaTunnelRowTypeInfo, - config, - context, - getSinkConfig(), - jobId); - } + pluginConfig = pluginConfig.withValue(IS_PARTITION_FIELD_WRITE_IN_FILE, ConfigValueFactory.fromAnyRef(false)) + .withValue(FILE_NAME_EXPRESSION, ConfigValueFactory.fromAnyRef("${transactionId}")) + .withValue(PATH, ConfigValueFactory.fromAnyRef(tableInformation.getSd().getLocation())); - @Override - public SinkWriter restoreWriter(SinkWriter.Context context, List states) throws IOException { - return new HiveSinkWriter(seaTunnelRowTypeInfo, config, context, hiveSinkConfig, jobId, states); - } - - @Override - public void setJobContext(JobContext jobContext) { - this.jobContext = jobContext; - this.jobId = jobContext.getJobId(); - } - - @Override - public Optional> createAggregatedCommitter() throws IOException { - return Optional.of(new HiveSinkAggregatedCommitter()); - } - - @Override - public Optional> getWriterStateSerializer() { - return Optional.of(new DefaultSerializer<>()); - } - - @Override - public Optional> getAggregatedCommitInfoSerializer() { - return Optional.of(new DefaultSerializer<>()); - } - - @Override - public Optional> getCommitInfoSerializer() { - return Optional.of(new DefaultSerializer<>()); - } - - private HiveSinkConfig getSinkConfig() { - if (this.hiveSinkConfig == null && (this.seaTunnelRowTypeInfo != null && this.config != null)) { - this.hiveSinkConfig = new HiveSinkConfig(config, seaTunnelRowTypeInfo); + if (!pluginConfig.hasPath(SAVE_MODE) || StringUtils.isBlank(pluginConfig.getString(SAVE_MODE))) { + pluginConfig = pluginConfig.withValue(SAVE_MODE, ConfigValueFactory.fromAnyRef(SaveMode.APPEND.toString())); + } + String hdfsLocation = tableInformation.getSd().getLocation(); + try { + URI uri = new URI(hdfsLocation); + String path = uri.getPath(); + pluginConfig = pluginConfig.withValue(PATH, ConfigValueFactory.fromAnyRef(path)); + hadoopConf = new HadoopConf(hdfsLocation.replace(path, "")); + } catch (URISyntaxException e) { + throw new RuntimeException("Get hdfs cluster address failed, please check.", e); } - return this.hiveSinkConfig; + super.prepare(pluginConfig); } } diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java deleted file mode 100644 index 6532bad0732..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.sink; - -import org.apache.seatunnel.api.sink.SinkAggregatedCommitter; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.util.HdfsUtils; -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy; - -import org.apache.commons.collections4.CollectionUtils; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class HiveSinkAggregatedCommitter implements SinkAggregatedCommitter { - private static final Logger LOGGER = LoggerFactory.getLogger(HiveSinkAggregatedCommitter.class); - - @Override - public List commit(List aggregatedCommitInfoList) throws IOException { - LOGGER.info("=============================agg commit================================="); - if (CollectionUtils.isEmpty(aggregatedCommitInfoList)) { - return null; - } - List errorAggregatedCommitInfoList = new ArrayList<>(); - HiveMetaStoreProxy hiveMetaStoreProxy = new HiveMetaStoreProxy(aggregatedCommitInfoList.get(0).getHiveMetastoreUris()); - HiveMetaStoreClient hiveMetaStoreClient = hiveMetaStoreProxy.getHiveMetaStoreClient(); - try { - aggregatedCommitInfoList.forEach(aggregateCommitInfo -> { - try { - for (Map.Entry> entry : aggregateCommitInfo.getFileAggregatedCommitInfo().getTransactionMap().entrySet()) { - // rollback the file - for (Map.Entry mvFileEntry : entry.getValue().entrySet()) { - HdfsUtils.renameFile(mvFileEntry.getKey(), mvFileEntry.getValue(), true); - } - // delete the transaction dir - HdfsUtils.deleteFile(entry.getKey()); - } - // add hive partition - aggregateCommitInfo.getFileAggregatedCommitInfo().getPartitionDirAndValsMap().entrySet().forEach(entry -> { - Partition part = new Partition(); - part.setDbName(aggregateCommitInfo.getTable().getDbName()); - part.setTableName(aggregateCommitInfo.getTable().getTableName()); - part.setValues(entry.getValue()); - part.setParameters(new HashMap<>()); - part.setSd(aggregateCommitInfo.getTable().getSd().deepCopy()); - part.getSd().setSerdeInfo(aggregateCommitInfo.getTable().getSd().getSerdeInfo()); - part.getSd().setLocation(aggregateCommitInfo.getTable().getSd().getLocation() + "/" + entry.getKey()); - try { - hiveMetaStoreClient.add_partition(part); - } catch (TException e) { - throw new RuntimeException(e); - } - }); - } catch (Exception e) { - LOGGER.error("commit aggregateCommitInfo error ", e); - errorAggregatedCommitInfoList.add(aggregateCommitInfo); - } - }); - } finally { - hiveMetaStoreClient.close(); - } - - return errorAggregatedCommitInfoList; - } - - @Override - public HiveAggregatedCommitInfo combine(List commitInfos) { - if (CollectionUtils.isEmpty(commitInfos)) { - return null; - } - Map> aggregateCommitInfo = new HashMap<>(); - Map> partitionDirAndValsMap = new HashMap<>(); - commitInfos.forEach(commitInfo -> { - Map needMoveFileMap = aggregateCommitInfo.computeIfAbsent(commitInfo.getFileCommitInfo().getTransactionDir(), k -> new HashMap<>()); - needMoveFileMap.putAll(commitInfo.getFileCommitInfo().getNeedMoveFiles()); - Set>> entries = commitInfo.getFileCommitInfo().getPartitionDirAndValsMap().entrySet(); - if (!CollectionUtils.isEmpty(entries)) { - partitionDirAndValsMap.putAll(commitInfo.getFileCommitInfo().getPartitionDirAndValsMap()); - } - }); - return new HiveAggregatedCommitInfo( - new FileAggregatedCommitInfo(aggregateCommitInfo, partitionDirAndValsMap), - commitInfos.get(0).getHiveMetastoreUris(), - commitInfos.get(0).getTable()); - } - - @Override - public void abort(List aggregatedCommitInfoList) throws Exception { - if (aggregatedCommitInfoList == null || aggregatedCommitInfoList.size() == 0) { - return; - } - aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> { - try { - for (Map.Entry> entry : aggregateCommitInfo.getFileAggregatedCommitInfo().getTransactionMap().entrySet()) { - // rollback the file - for (Map.Entry mvFileEntry : entry.getValue().entrySet()) { - if (HdfsUtils.fileExist(mvFileEntry.getValue()) && !HdfsUtils.fileExist(mvFileEntry.getKey())) { - HdfsUtils.renameFile(mvFileEntry.getValue(), mvFileEntry.getKey(), true); - } - } - // delete the transaction dir - HdfsUtils.deleteFile(entry.getKey()); - - // The partitions that have been added will be preserved and will not be deleted - } - } catch (IOException e) { - LOGGER.error("abort aggregateCommitInfo error ", e); - } - }); - } - - @Override - public void close() throws IOException { - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java deleted file mode 100644 index b05b8d2e4fd..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java +++ /dev/null @@ -1,155 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.sink; - -import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FIELD_DELIMITER; -import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FILE_FORMAT; -import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FILE_NAME_EXPRESSION; -import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.IS_PARTITION_FIELD_WRITE_IN_FILE; -import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.PATH; -import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.ROW_DELIMITER; -import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.SAVE_MODE; -import static org.apache.seatunnel.connectors.seatunnel.hive.config.Constant.HIVE_METASTORE_URIS; -import static org.apache.seatunnel.connectors.seatunnel.hive.config.Constant.HIVE_RESULT_TABLE_NAME; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.config.Constant; -import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; -import org.apache.seatunnel.connectors.seatunnel.file.sink.config.SaveMode; -import org.apache.seatunnel.connectors.seatunnel.file.sink.config.TextFileSinkConfig; -import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; - -import lombok.Data; -import lombok.NonNull; -import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.thrift.TException; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -@Data -public class HiveSinkConfig implements Serializable { - private static final String TEXT_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"; - private static final String PARQUET_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; - private static final String ORC_FORMAT_CLASSNAME = "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; - private String hiveTableName; - private List hivePartitionFieldList; - private String hiveMetaUris; - - private String dbName; - - private String tableName; - - private Table table; - - private TextFileSinkConfig textFileSinkConfig; - - public HiveSinkConfig(@NonNull Config config, @NonNull SeaTunnelRowType seaTunnelRowTypeInfo) { - checkArgument(!CollectionUtils.isEmpty(Arrays.asList(seaTunnelRowTypeInfo.getFieldNames()))); - - if (config.hasPath(HIVE_RESULT_TABLE_NAME) && !StringUtils.isBlank(config.getString(HIVE_RESULT_TABLE_NAME))) { - this.hiveTableName = config.getString(HIVE_RESULT_TABLE_NAME); - } - checkNotNull(hiveTableName); - - if (config.hasPath(HIVE_METASTORE_URIS) && !StringUtils.isBlank(config.getString(HIVE_METASTORE_URIS))) { - this.hiveMetaUris = config.getString(HIVE_METASTORE_URIS); - } - checkNotNull(hiveMetaUris); - - String[] dbAndTableName = hiveTableName.split("\\."); - if (dbAndTableName == null || dbAndTableName.length != 2) { - throw new RuntimeException("Please config " + HIVE_RESULT_TABLE_NAME + " as db.table format"); - } - this.dbName = dbAndTableName[0]; - this.tableName = dbAndTableName[1]; - HiveMetaStoreProxy hiveMetaStoreProxy = new HiveMetaStoreProxy(hiveMetaUris); - HiveMetaStoreClient hiveMetaStoreClient = hiveMetaStoreProxy.getHiveMetaStoreClient(); - - try { - table = hiveMetaStoreClient.getTable(dbName, tableName); - String outputFormat = table.getSd().getOutputFormat(); - Map parameters = table.getSd().getSerdeInfo().getParameters(); - if (TEXT_FORMAT_CLASSNAME.equals(outputFormat)) { - config = config.withValue(FILE_FORMAT, ConfigValueFactory.fromAnyRef(FileFormat.TEXT.toString())) - .withValue(FIELD_DELIMITER, ConfigValueFactory.fromAnyRef(parameters.get("field.delim"))) - .withValue(ROW_DELIMITER, ConfigValueFactory.fromAnyRef(parameters.get("line.delim"))); - } else if (PARQUET_FORMAT_CLASSNAME.equals(outputFormat)) { - config = config.withValue(FILE_FORMAT, ConfigValueFactory.fromAnyRef(FileFormat.PARQUET.toString())); - } else if (ORC_FORMAT_CLASSNAME.equals(outputFormat)) { - config = config.withValue(FILE_FORMAT, ConfigValueFactory.fromAnyRef(FileFormat.ORC.toString())); - } else { - throw new RuntimeException("Only support [text parquet orc] file now"); - } - - config = config.withValue(IS_PARTITION_FIELD_WRITE_IN_FILE, ConfigValueFactory.fromAnyRef(false)) - .withValue(FILE_NAME_EXPRESSION, ConfigValueFactory.fromAnyRef("${transactionId}")) - .withValue(PATH, ConfigValueFactory.fromAnyRef(table.getSd().getLocation())); - - if (!config.hasPath(SAVE_MODE) || StringUtils.isBlank(config.getString(Constant.SAVE_MODE))) { - config = config.withValue(SAVE_MODE, ConfigValueFactory.fromAnyRef(SaveMode.APPEND.toString())); - } - - this.textFileSinkConfig = new TextFileSinkConfig(config, seaTunnelRowTypeInfo); - - // --------------------Check textFileSinkConfig with the hive table info------------------- - List fields = hiveMetaStoreClient.getFields(dbAndTableName[0], dbAndTableName[1]); - List partitionKeys = table.getPartitionKeys(); - - // Remove partitionKeys from table fields - List fieldNotContainPartitionKey = fields.stream().filter(filed -> !partitionKeys.contains(filed)).collect(Collectors.toList()); - - // check fields size must same as sinkColumnList size - if (fieldNotContainPartitionKey.size() != textFileSinkConfig.getSinkColumnList().size()) { - throw new RuntimeException("sink columns size must same as hive table field size"); - } - - // check hivePartitionFieldList size must same as partitionFieldList size - if (partitionKeys.size() != textFileSinkConfig.getPartitionFieldList().size()) { - throw new RuntimeException("partition by columns size must same as hive table partition columns size"); - } - - // --------------------Check textFileSinkConfig with the hive table info end---------------- - } catch (TException e) { - throw new RuntimeException(e); - } finally { - hiveMetaStoreClient.close(); - } - - // hive only support append or overwrite - if (!this.textFileSinkConfig.getSaveMode().equals(SaveMode.APPEND) && !this.textFileSinkConfig.getSaveMode().equals(SaveMode.OVERWRITE)) { - throw new RuntimeException("hive only support append or overwrite save mode"); - } - } - - public TextFileSinkConfig getTextFileSinkConfig() { - return textFileSinkConfig; - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java deleted file mode 100644 index a104151c36a..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.sink; - -import lombok.AllArgsConstructor; -import lombok.Data; - -import java.io.Serializable; - -@Data -@AllArgsConstructor -public class HiveSinkState implements Serializable { - private String transactionId; - private Long checkpointId; -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java deleted file mode 100644 index 69283a55a6a..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java +++ /dev/null @@ -1,160 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.sink; - -import org.apache.seatunnel.api.sink.SinkWriter; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.HdfsFileSinkPlugin; -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkState; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkTransactionFileNameGenerator; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import lombok.NonNull; -import org.apache.commons.collections4.CollectionUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; - -public class HiveSinkWriter implements SinkWriter { - private static final Logger LOGGER = LoggerFactory.getLogger(HiveSinkWriter.class); - - private SeaTunnelRowType seaTunnelRowTypeInfo; - private Config pluginConfig; - private Context context; - private String jobId; - - private TransactionStateFileWriter fileWriter; - - private HiveSinkConfig hiveSinkConfig; - - public HiveSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo, - @NonNull Config pluginConfig, - @NonNull SinkWriter.Context context, - @NonNull HiveSinkConfig hiveSinkConfig, - @NonNull String jobId) { - this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo; - this.pluginConfig = pluginConfig; - this.context = context; - this.jobId = jobId; - this.hiveSinkConfig = hiveSinkConfig; - this.fileWriter = createFileWriter(); - - fileWriter.beginTransaction(1L); - } - - public HiveSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo, - @NonNull Config pluginConfig, - @NonNull SinkWriter.Context context, - @NonNull HiveSinkConfig hiveSinkConfig, - @NonNull String jobId, - @NonNull List hiveSinkStates) { - this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo; - this.pluginConfig = pluginConfig; - this.context = context; - this.jobId = jobId; - this.hiveSinkConfig = hiveSinkConfig; - this.fileWriter = createFileWriter(); - - // Rollback dirty transaction - if (hiveSinkStates.size() > 0) { - List transactionAfter = fileWriter.getTransactionAfter(hiveSinkStates.get(0).getTransactionId()); - fileWriter.abortTransactions(transactionAfter); - } - fileWriter.beginTransaction(hiveSinkStates.get(0).getCheckpointId() + 1); - } - - @Override - public void write(SeaTunnelRow element) throws IOException { - fileWriter.write(element); - } - - @Override - public Optional prepareCommit() throws IOException { - Optional fileCommitInfoOptional = fileWriter.prepareCommit(); - if (fileCommitInfoOptional.isPresent()) { - FileCommitInfo fileCommitInfo = fileCommitInfoOptional.get(); - return Optional.of(new HiveCommitInfo(fileCommitInfo, hiveSinkConfig.getHiveMetaUris(), this.hiveSinkConfig.getTable())); - } - return Optional.empty(); - } - - @Override - public void close() throws IOException { - fileWriter.finishAndCloseWriteFile(); - } - - @Override - public List snapshotState(long checkpointId) throws IOException { - List fileSinkStates = fileWriter.snapshotState(checkpointId); - if (!CollectionUtils.isEmpty(fileSinkStates)) { - return fileSinkStates.stream().map(state -> - new HiveSinkState(state.getTransactionId(), state.getCheckpointId())) - .collect(Collectors.toList()); - } - return Collections.emptyList(); - } - - @Override - public void abortPrepare() { - fileWriter.abortTransaction(); - } - - private TransactionStateFileWriter createFileWriter() { - SinkFileSystemPlugin sinkFileSystemPlugin = new HdfsFileSinkPlugin(); - Optional transactionStateFileWriterOpt = sinkFileSystemPlugin.getTransactionStateFileWriter(this.seaTunnelRowTypeInfo, - getFilenameGenerator(), - getPartitionDirNameGenerator(), - this.hiveSinkConfig.getTextFileSinkConfig().getSinkColumnsIndexInRow(), - this.hiveSinkConfig.getTextFileSinkConfig().getTmpPath(), - this.hiveSinkConfig.getTextFileSinkConfig().getPath(), - this.jobId, - this.context.getIndexOfSubtask(), - this.hiveSinkConfig.getTextFileSinkConfig().getFieldDelimiter(), - this.hiveSinkConfig.getTextFileSinkConfig().getRowDelimiter(), - sinkFileSystemPlugin.getFileSystem().get()); - if (!transactionStateFileWriterOpt.isPresent()) { - throw new RuntimeException("A TransactionStateFileWriter is need"); - } - return transactionStateFileWriterOpt.get(); - } - - private FileSinkTransactionFileNameGenerator getFilenameGenerator() { - return new FileSinkTransactionFileNameGenerator( - this.hiveSinkConfig.getTextFileSinkConfig().getFileFormat(), - this.hiveSinkConfig.getTextFileSinkConfig().getFileNameExpression(), - this.hiveSinkConfig.getTextFileSinkConfig().getFileNameTimeFormat()); - } - - private FileSinkPartitionDirNameGenerator getPartitionDirNameGenerator() { - return new FileSinkPartitionDirNameGenerator( - this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldList(), - this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldsIndexInRow(), - this.hiveSinkConfig.getTextFileSinkConfig().getPartitionDirExpression()); - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java deleted file mode 100644 index 31dddf7b49a..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source; - -import lombok.Data; - -import java.io.Serializable; - -@Data -public class HadoopConf implements Serializable { - - private String hdfsNameKey; - - private String fsHdfsImpl = "org.apache.hadoop.hdfs.DistributedFileSystem"; - - public HadoopConf(String hdfsNameKey) { - this.hdfsNameKey = hdfsNameKey; - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java index 1f1a8c3fbb1..92173892e53 100644 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java @@ -17,42 +17,34 @@ package org.apache.seatunnel.connectors.seatunnel.hive.source; -import static org.apache.seatunnel.connectors.seatunnel.hive.config.SourceConfig.FILE_PATH; -import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; +import static org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig.ORC_INPUT_FORMAT_CLASSNAME; +import static org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig.PARQUET_INPUT_FORMAT_CLASSNAME; +import static org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig.TEXT_INPUT_FORMAT_CLASSNAME; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.config.CheckConfigUtil; import org.apache.seatunnel.common.config.CheckResult; import org.apache.seatunnel.common.constants.PluginType; -import org.apache.seatunnel.connectors.seatunnel.hive.config.SourceConfig; -import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException; -import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.ReadStrategy; -import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.ReadStrategyFactory; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; +import org.apache.seatunnel.connectors.seatunnel.file.hdfs.source.HdfsFileSource; +import org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig; import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; import com.google.auto.service.AutoService; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.hive.metastore.api.Table; -import java.io.IOException; -import java.util.List; +import java.net.URI; +import java.net.URISyntaxException; @AutoService(SeaTunnelSource.class) -public class HiveSource implements SeaTunnelSource { - - private SeaTunnelRowType typeInfo; - - private ReadStrategy readStrategy; - - private HadoopConf hadoopConf; - - private List filesPath; +public class HiveSource extends HdfsFileSource { + private Table tableInformation; @Override public String getPluginName() { @@ -60,51 +52,33 @@ public String getPluginName() { } @Override - public void prepare(Config pluginConfig) { - CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, FILE_PATH, FS_DEFAULT_NAME_KEY); + public void prepare(Config pluginConfig) throws PrepareFailException { + CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, HiveConfig.METASTORE_URI, HiveConfig.TABLE_NAME); if (!result.isSuccess()) { throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg()); } - // use factory to generate readStrategy - readStrategy = ReadStrategyFactory.of(pluginConfig.getString(SourceConfig.FILE_TYPE)); - String path = pluginConfig.getString(FILE_PATH); - hadoopConf = new HadoopConf(pluginConfig.getString(FS_DEFAULT_NAME_KEY)); - try { - filesPath = readStrategy.getFileNamesByPath(hadoopConf, path); - } catch (IOException e) { - throw new PrepareFailException(getPluginName(), PluginType.SOURCE, "Check file path fail."); + Pair tableInfo = HiveConfig.getTableInfo(pluginConfig); + tableInformation = tableInfo.getRight(); + String inputFormat = tableInformation.getSd().getInputFormat(); + if (TEXT_INPUT_FORMAT_CLASSNAME.equals(inputFormat)) { + pluginConfig = pluginConfig.withValue(BaseSourceConfig.FILE_TYPE, ConfigValueFactory.fromAnyRef(FileFormat.TEXT.toString())); + } else if (PARQUET_INPUT_FORMAT_CLASSNAME.equals(inputFormat)) { + pluginConfig = pluginConfig.withValue(BaseSourceConfig.FILE_TYPE, ConfigValueFactory.fromAnyRef(FileFormat.PARQUET.toString())); + } else if (ORC_INPUT_FORMAT_CLASSNAME.equals(inputFormat)) { + pluginConfig = pluginConfig.withValue(BaseSourceConfig.FILE_TYPE, ConfigValueFactory.fromAnyRef(FileFormat.ORC.toString())); + } else { + throw new RuntimeException("Only support [text parquet orc] file now"); } + String hdfsLocation = tableInformation.getSd().getLocation(); try { - // should read from config or read from hive metadata( wait catlog done) - this.typeInfo = readStrategy.getSeaTunnelRowTypeInfo(hadoopConf, filesPath.get(0)); - } catch (HivePluginException e) { - throw new PrepareFailException(getPluginName(), PluginType.SOURCE, "Read hive file type error.", e); + URI uri = new URI(hdfsLocation); + String path = uri.getPath(); + String defaultFs = hdfsLocation.replace(path, ""); + pluginConfig = pluginConfig.withValue(BaseSourceConfig.FILE_PATH, ConfigValueFactory.fromAnyRef(path)) + .withValue(FS_DEFAULT_NAME_KEY, ConfigValueFactory.fromAnyRef(defaultFs)); + } catch (URISyntaxException e) { + throw new RuntimeException("Get hdfs cluster address failed, please check.", e); } + super.prepare(pluginConfig); } - - @Override - public SeaTunnelDataType getProducedType() { - return this.typeInfo; - } - - @Override - public SourceReader createReader(SourceReader.Context readerContext) throws Exception { - return new HiveSourceReader(this.readStrategy, this.hadoopConf, readerContext); - } - - @Override - public Boundedness getBoundedness() { - return Boundedness.BOUNDED; - } - - @Override - public SourceSplitEnumerator createEnumerator(SourceSplitEnumerator.Context enumeratorContext) throws Exception { - return new HiveSourceSplitEnumerator(enumeratorContext, filesPath); - } - - @Override - public SourceSplitEnumerator restoreEnumerator(SourceSplitEnumerator.Context enumeratorContext, HiveSourceState checkpointState) throws Exception { - return new HiveSourceSplitEnumerator(enumeratorContext, filesPath, checkpointState); - } - } diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceReader.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceReader.java deleted file mode 100644 index 570f48e35da..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceReader.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source; - -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.ReadStrategy; - -import lombok.extern.slf4j.Slf4j; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -@Slf4j -public class HiveSourceReader implements SourceReader { - - private static final long THREAD_WAIT_TIME = 500L; - - private ReadStrategy readStrategy; - - private HadoopConf hadoopConf; - - private Set sourceSplits; - - private final SourceReader.Context context; - - public HiveSourceReader(ReadStrategy readStrategy, HadoopConf hadoopConf, SourceReader.Context context) { - this.readStrategy = readStrategy; - this.hadoopConf = hadoopConf; - this.context = context; - this.sourceSplits = new HashSet<>(); - } - - @Override - public void open() { - readStrategy.init(hadoopConf); - } - - @Override - public void close() { - - } - - @Override - public void pollNext(Collector output) throws Exception { - if (sourceSplits.isEmpty()) { - Thread.sleep(THREAD_WAIT_TIME); - return; - } - sourceSplits.forEach(source -> { - try { - readStrategy.read(source.splitId(), output); - } catch (Exception e) { - throw new RuntimeException("Hive source read error", e); - } - - }); - context.signalNoMoreElement(); - } - - @Override - public List snapshotState(long checkpointId) { - return new ArrayList<>(sourceSplits); - } - - @Override - public void addSplits(List splits) { - sourceSplits.addAll(splits); - } - - @Override - public void handleNoMoreSplits() { - - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java deleted file mode 100644 index 44e062e54c3..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source; - -import org.apache.seatunnel.api.source.SourceSplit; - -public class HiveSourceSplit implements SourceSplit { - - private static final long serialVersionUID = -1L; - - private String splitId; - - public HiveSourceSplit(String splitId) { - this.splitId = splitId; - } - - @Override - public String splitId() { - return this.splitId; - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplitEnumerator.java deleted file mode 100644 index 301b1506fe6..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplitEnumerator.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source; - -import org.apache.seatunnel.api.source.SourceSplitEnumerator; -import org.apache.seatunnel.common.config.Common; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class HiveSourceSplitEnumerator implements SourceSplitEnumerator { - - private final Context context; - private Set pendingSplit; - private Set assignedSplit; - private List filePaths; - - public HiveSourceSplitEnumerator(SourceSplitEnumerator.Context context, List filePaths) { - this.context = context; - this.filePaths = filePaths; - } - - public HiveSourceSplitEnumerator(SourceSplitEnumerator.Context context, List filePaths, - HiveSourceState sourceState) { - this(context, filePaths); - this.assignedSplit = sourceState.getAssignedSplit(); - } - - @Override - public void open() { - this.assignedSplit = new HashSet<>(); - this.pendingSplit = new HashSet<>(); - } - - @Override - public void run() { - pendingSplit = getHiveFileSplit(); - assignSplit(context.registeredReaders()); - } - - private Set getHiveFileSplit() { - Set hiveSourceSplits = new HashSet<>(); - filePaths.forEach(k -> hiveSourceSplits.add(new HiveSourceSplit(k))); - return hiveSourceSplits; - - } - - @Override - public void close() throws IOException { - - } - - @Override - public void addSplitsBack(List splits, int subtaskId) { - if (!splits.isEmpty()) { - pendingSplit.addAll(splits); - assignSplit(Collections.singletonList(subtaskId)); - } - } - - private void assignSplit(Collection taskIDList) { - Map> readySplit = new HashMap<>(Common.COLLECTION_SIZE); - for (int taskID : taskIDList) { - readySplit.computeIfAbsent(taskID, id -> new ArrayList<>()); - } - - pendingSplit.forEach(s -> readySplit.get(getSplitOwner(s.splitId(), taskIDList.size())) - .add(s)); - readySplit.forEach(context::assignSplit); - assignedSplit.addAll(pendingSplit); - pendingSplit.clear(); - } - - private static int getSplitOwner(String tp, int numReaders) { - return tp.hashCode() % numReaders; - } - - @Override - public int currentUnassignedSplitSize() { - return pendingSplit.size(); - } - - @Override - public void registerReader(int subtaskId) { - if (!pendingSplit.isEmpty()) { - assignSplit(Collections.singletonList(subtaskId)); - } - } - - @Override - public HiveSourceState snapshotState(long checkpointId) { - return new HiveSourceState(assignedSplit); - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - - } - - @Override - public void handleSplitRequest(int subtaskId) { - - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java deleted file mode 100644 index f982a71cb22..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source; - -import java.io.Serializable; -import java.util.Set; - -public class HiveSourceState implements Serializable { - - - private Set assignedSplit; - - public HiveSourceState(Set assignedSplit) { - this.assignedSplit = assignedSplit; - } - - public Set getAssignedSplit() { - return assignedSplit; - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java deleted file mode 100644 index 2df0a21f73d..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source.file.reader.format; - -import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY; - -import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException; -import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -public abstract class AbstractReadStrategy implements ReadStrategy { - - HadoopConf hadoopConf; - - @Override - public void init(HadoopConf conf) { - this.hadoopConf = conf; - } - - @Override - public Configuration getConfiguration(HadoopConf hadoopConf) { - Configuration configuration = new Configuration(); - configuration.set(FS_DEFAULT_NAME_KEY, hadoopConf.getHdfsNameKey()); - configuration.set("fs.hdfs.impl", hadoopConf.getFsHdfsImpl()); - return configuration; - } - - Configuration getConfiguration() throws HivePluginException { - if (null == hadoopConf) { - throw new HivePluginException("Not init read config"); - } - return getConfiguration(hadoopConf); - } - - boolean checkFileType(String path) { - return true; - } - - @Override - public List getFileNamesByPath(HadoopConf hadoopConf, String path) throws IOException { - Configuration configuration = getConfiguration(hadoopConf); - List fileNames = new ArrayList<>(); - FileSystem hdfs = FileSystem.get(configuration); - Path listFiles = new Path(path); - FileStatus[] stats = hdfs.listStatus(listFiles); - for (FileStatus fileStatus : stats) { - if (fileStatus.isDirectory()) { - fileNames.addAll(getFileNamesByPath(hadoopConf, fileStatus.getPath().toString())); - continue; - } - if (fileStatus.isFile()) { - // filter '_SUCCESS' file - if (!fileStatus.getPath().getName().equals("_SUCCESS")) { - fileNames.add(fileStatus.getPath().toString()); - } - } - } - return fileNames; - } - -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/OrcReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/OrcReadStrategy.java deleted file mode 100644 index 96ff1e983cf..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/OrcReadStrategy.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source.file.reader.format; - -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException; -import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf; - -import lombok.extern.slf4j.Slf4j; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.io.orc.OrcFile; -import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcSerde; -import org.apache.hadoop.hive.ql.io.orc.OrcStruct; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; -import org.apache.hadoop.mapred.Reporter; -import org.apache.orc.Reader; -import org.apache.orc.TypeDescription; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Properties; - -@Slf4j -public class OrcReadStrategy extends AbstractReadStrategy { - - private SeaTunnelRowType seaTunnelRowTypeInfo; - private static final long MIN_SIZE = 16 * 1024; - - @Override - public void read(String path, Collector output) throws Exception { - if (Boolean.FALSE.equals(checkFileType(path))) { - throw new Exception("please check file type"); - } - JobConf conf = new JobConf(); - Path filePath = new Path(path); - Properties p = new Properties(); - OrcSerde serde = new OrcSerde(); - String columns = String.join(",", seaTunnelRowTypeInfo.getFieldNames()); - p.setProperty("columns", columns); - //support types - serde.initialize(conf, p); - StructObjectInspector inspector = (StructObjectInspector) serde.getObjectInspector(); - InputFormat in = new OrcInputFormat(); - FileInputFormat.setInputPaths(conf, filePath); - InputSplit[] splits = in.getSplits(conf, 1); - - conf.set("hive.io.file.readcolumn.ids", "1"); - RecordReader reader = in.getRecordReader(splits[0], conf, Reporter.NULL); - NullWritable key = reader.createKey(); - OrcStruct value = reader.createValue(); - List fields = inspector.getAllStructFieldRefs(); - while (reader.next(key, value)) { - Object[] datas = new Object[fields.size()]; - for (int i = 0; i < fields.size(); i++) { - Object data = inspector.getStructFieldData(value, fields.get(i)); - if (null != data) { - datas[i] = String.valueOf(data); - } else { - datas[i] = null; - } - } - output.collect(new SeaTunnelRow(datas)); - } - reader.close(); - } - - @Override - public SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path) throws HivePluginException { - - if (null != seaTunnelRowTypeInfo) { - return seaTunnelRowTypeInfo; - } - Configuration configuration = getConfiguration(hadoopConf); - Path dstDir = new Path(path); - Reader reader; - try { - reader = OrcFile.createReader(FileSystem.get(configuration), dstDir); - } catch (IOException e) { - throw new HivePluginException("Create OrcReader Fail", e); - } - - TypeDescription schema = reader.getSchema(); - String[] fields = new String[schema.getFieldNames().size()]; - SeaTunnelDataType[] types = new SeaTunnelDataType[schema.getFieldNames().size()]; - - for (int i = 0; i < schema.getFieldNames().size(); i++) { - fields[i] = schema.getFieldNames().get(i); - types[i] = BasicType.STRING_TYPE; - } - seaTunnelRowTypeInfo = new SeaTunnelRowType(fields, types); - return seaTunnelRowTypeInfo; - } - - @SuppressWarnings("checkstyle:MagicNumber") - @Override - boolean checkFileType(String path) { - try { - boolean checkResult; - Configuration configuration = getConfiguration(); - FileSystem fileSystem = FileSystem.get(configuration); - Path filePath = new Path(path); - FSDataInputStream in = fileSystem.open(filePath); - // try to get Postscript in orc file - long size = fileSystem.getFileStatus(filePath).getLen(); - int readSize = (int) Math.min(size, MIN_SIZE); - in.seek(size - readSize); - ByteBuffer buffer = ByteBuffer.allocate(readSize); - in.readFully(buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); - int psLen = buffer.get(readSize - 1) & 0xff; - int len = OrcFile.MAGIC.length(); - if (psLen < len + 1) { - in.close(); - return false; - } - int offset = buffer.arrayOffset() + buffer.position() + buffer.limit() - 1 - len; - byte[] array = buffer.array(); - if (Text.decode(array, offset, len).equals(OrcFile.MAGIC)) { - checkResult = true; - } else { - // If it isn't there, this may be the 0.11.0 version of ORC. - // Read the first 3 bytes of the file to check for the header - in.seek(0); - byte[] header = new byte[len]; - in.readFully(header, 0, len); - // if it isn't there, this isn't an ORC file - checkResult = Text.decode(header, 0, len).equals(OrcFile.MAGIC); - } - in.close(); - return checkResult; - } catch (HivePluginException | IOException e) { - String errorMsg = String.format("Check orc file [%s] error", path); - throw new RuntimeException(errorMsg, e); - } - } -} - diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ParquetReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ParquetReadStrategy.java deleted file mode 100644 index 35d70e3ea23..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ParquetReadStrategy.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source.file.reader.format; - -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException; -import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.avro.AvroParquetReader; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.ParquetReader; -import org.apache.parquet.hadoop.metadata.FileMetaData; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.hadoop.util.HadoopInputFile; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.MessageType; - -import java.io.IOException; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; - -@Slf4j -public class ParquetReadStrategy extends AbstractReadStrategy { - - private SeaTunnelRowType seaTunnelRowType; - - private static final byte[] PARQUET_MAGIC = new byte[]{(byte) 'P', (byte) 'A', (byte) 'R', (byte) '1'}; - - @Override - public void read(String path, Collector output) throws Exception { - if (Boolean.FALSE.equals(checkFileType(path))) { - throw new Exception("please check file type"); - } - Path filePath = new Path(path); - HadoopInputFile hadoopInputFile = HadoopInputFile.fromPath(filePath, getConfiguration()); - int fieldsCount = seaTunnelRowType.getTotalFields(); - GenericRecord record; - try (ParquetReader reader = AvroParquetReader.builder(hadoopInputFile).build()) { - while ((record = reader.read()) != null) { - Object[] fields = new Object[fieldsCount]; - for (int i = 0; i < fieldsCount; i++) { - Object data = record.get(i); - try { - if (data instanceof GenericData.Fixed) { - // judge the data in upstream is or not decimal type - data = fixed2String((GenericData.Fixed) data); - } else if (data instanceof ArrayList) { - // judge the data in upstream is or not array type - data = array2String((ArrayList) data); - } - } catch (Exception e) { - data = record.get(i); - } finally { - fields[i] = data.toString(); - } - } - output.collect(new SeaTunnelRow(fields)); - } - } - } - - @Override - public SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path) throws HivePluginException { - if (seaTunnelRowType != null) { - return seaTunnelRowType; - } - Configuration configuration = getConfiguration(hadoopConf); - Path filePath = new Path(path); - ParquetMetadata metadata; - try { - metadata = ParquetFileReader.readFooter(configuration, filePath); - } catch (IOException e) { - throw new HivePluginException("Create parquet reader failed", e); - } - FileMetaData fileMetaData = metadata.getFileMetaData(); - MessageType schema = fileMetaData.getSchema(); - int fieldCount = schema.getFieldCount(); - String[] fields = new String[fieldCount]; - SeaTunnelDataType[] types = new SeaTunnelDataType[fieldCount]; - for (int i = 0; i < fieldCount; i++) { - fields[i] = schema.getFieldName(i); - // Temporarily each field is treated as a string type - // I think we can use the schema information to build seatunnel column type - types[i] = BasicType.STRING_TYPE; - } - seaTunnelRowType = new SeaTunnelRowType(fields, types); - return seaTunnelRowType; - } - - private String fixed2String(GenericData.Fixed fixed) { - Schema schema = fixed.getSchema(); - byte[] bytes = fixed.bytes(); - int precision = Integer.parseInt(schema.getObjectProps().get("precision").toString()); - int scale = Integer.parseInt(schema.getObjectProps().get("scale").toString()); - BigDecimal bigDecimal = bytes2Decimal(bytes, precision, scale); - return bigDecimal.toString(); - } - - @SuppressWarnings("checkstyle:MagicNumber") - private BigDecimal bytes2Decimal(byte[] bytesArray, int precision, int scale) { - Binary value = Binary.fromConstantByteArray(bytesArray); - if (precision <= 18) { - ByteBuffer buffer = value.toByteBuffer(); - byte[] bytes = buffer.array(); - int start = buffer.arrayOffset() + buffer.position(); - int end = buffer.arrayOffset() + buffer.limit(); - long unscaled = 0L; - int i = start; - while (i < end) { - unscaled = unscaled << 8 | bytes[i] & 0xff; - i++; - } - int bits = 8 * (end - start); - long unscaledNew = (unscaled << (64 - bits)) >> (64 - bits); - if (unscaledNew <= -Math.pow(10, 18) || unscaledNew >= Math.pow(10, 18)) { - return new BigDecimal(unscaledNew); - } else { - return BigDecimal.valueOf(unscaledNew / Math.pow(10, scale)); - } - } else { - return new BigDecimal(new BigInteger(value.getBytes()), scale); - } - } - - @Override - boolean checkFileType(String path) { - boolean checkResult; - byte[] magic = new byte[PARQUET_MAGIC.length]; - try { - Configuration configuration = getConfiguration(); - FileSystem fileSystem = FileSystem.get(configuration); - Path filePath = new Path(path); - FSDataInputStream in = fileSystem.open(filePath); - // try to get header information in a parquet file - in.seek(0); - in.readFully(magic); - checkResult = Arrays.equals(magic, PARQUET_MAGIC); - in.close(); - return checkResult; - } catch (HivePluginException | IOException e) { - String errorMsg = String.format("Check parquet file [%s] error", path); - throw new RuntimeException(errorMsg, e); - } - } - - private String array2String(ArrayList data) throws JsonProcessingException { - ObjectMapper objectMapper = new ObjectMapper(); - List values = data.stream().map(record -> record.get(0).toString()).collect(Collectors.toList()); - return objectMapper.writeValueAsString(values); - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategy.java deleted file mode 100644 index 5e73019147a..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategy.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source.file.reader.format; - -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException; -import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf; - -import org.apache.hadoop.conf.Configuration; - -import java.io.IOException; -import java.io.Serializable; -import java.util.List; - -public interface ReadStrategy extends Serializable { - void init(HadoopConf conf); - - Configuration getConfiguration(HadoopConf conf); - - void read(String path, Collector output) throws Exception; - - SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path) throws HivePluginException; - - List getFileNamesByPath(HadoopConf hadoopConf, String path) throws IOException; -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategyFactory.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategyFactory.java deleted file mode 100644 index 56e88aa4465..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategyFactory.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source.file.reader.format; - -import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.type.FileTypeEnum; - -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public class ReadStrategyFactory { - - private ReadStrategyFactory() {} - - public static ReadStrategy of(String fileType) { - try { - FileTypeEnum fileTypeEnum = FileTypeEnum.valueOf(fileType.toUpperCase()); - return fileTypeEnum.getReadStrategy(); - } catch (IllegalArgumentException e) { - log.warn("Hive plugin not support this file type [{}], it will be treated as a plain text file", fileType); - return new TextReadStrategy(); - } - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java deleted file mode 100644 index 6b014d73799..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source.file.reader.format; - -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException; -import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; - -public class TextReadStrategy extends AbstractReadStrategy { - - private static final String TEXT_FIELD_NAME = "lines"; - - @Override - public void read(String path, Collector output) throws IOException, HivePluginException { - Configuration conf = getConfiguration(); - FileSystem fs = FileSystem.get(conf); - Path filePath = new Path(path); - try (BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(filePath), StandardCharsets.UTF_8))) { - reader.lines().forEach(line -> output.collect(new SeaTunnelRow(new String[]{line}))); - } - } - - @Override - public SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path) { - return new SeaTunnelRowType(new String[]{TEXT_FIELD_NAME}, - new SeaTunnelDataType[]{BasicType.STRING_TYPE}); - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/type/FileTypeEnum.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/type/FileTypeEnum.java deleted file mode 100644 index 3cf986b1805..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/type/FileTypeEnum.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.source.file.reader.type; - -import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.OrcReadStrategy; -import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.ParquetReadStrategy; -import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.ReadStrategy; -import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.TextReadStrategy; - -public enum FileTypeEnum { - ORC { - @Override - public ReadStrategy getReadStrategy() { - return new OrcReadStrategy(); - } - }, - PARQUET { - @Override - public ReadStrategy getReadStrategy() { - return new ParquetReadStrategy(); - } - }, - TEXT { - @Override - public ReadStrategy getReadStrategy() { - return new TextReadStrategy(); - } - }; - - public ReadStrategy getReadStrategy() { - return null; - } -} diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java index 30c9a2eba5b..d813a03a376 100644 --- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java +++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java @@ -17,18 +17,26 @@ package org.apache.seatunnel.connectors.seatunnel.hive.utils; +import org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + import lombok.NonNull; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.thrift.TException; -public class HiveMetaStoreProxy { +import java.util.List; +import java.util.Objects; - private HiveMetaStoreClient hiveMetaStoreClient; +public class HiveMetaStoreProxy { + private final HiveMetaStoreClient hiveMetaStoreClient; + private static volatile HiveMetaStoreProxy INSTANCE = null; - public HiveMetaStoreProxy(@NonNull String uris) { + private HiveMetaStoreProxy(@NonNull String uris) { HiveConf hiveConf = new HiveConf(); hiveConf.set("hive.metastore.uris", uris); try { @@ -38,15 +46,40 @@ public HiveMetaStoreProxy(@NonNull String uris) { } } + public static HiveMetaStoreProxy getInstance(Config config) { + if (INSTANCE == null) { + synchronized (HiveMetaStoreProxy.class) { + if (INSTANCE == null) { + String metastoreUri = config.getString(HiveConfig.METASTORE_URI); + INSTANCE = new HiveMetaStoreProxy(metastoreUri); + } + } + } + return INSTANCE; + } + public Table getTable(@NonNull String dbName, @NonNull String tableName) { try { return hiveMetaStoreClient.getTable(dbName, tableName); } catch (TException e) { - throw new RuntimeException(e); + String errorMsg = String.format("Get table [%s.%s] information failed", dbName, tableName); + throw new RuntimeException(errorMsg, e); } } - public HiveMetaStoreClient getHiveMetaStoreClient() { - return hiveMetaStoreClient; + public List getTableFields(@NonNull String dbName, @NonNull String tableName) { + try { + return hiveMetaStoreClient.getFields(dbName, tableName); + } catch (TException e) { + String errorMsg = String.format("Get table [%s.%s] fields information failed", dbName, tableName); + throw new RuntimeException(errorMsg, e); + } + } + + public synchronized void close() { + if (Objects.nonNull(hiveMetaStoreClient)) { + hiveMetaStoreClient.close(); + HiveMetaStoreProxy.INSTANCE = null; + } } } diff --git a/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfigTest.java b/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfigTest.java deleted file mode 100644 index e822a49c67f..00000000000 --- a/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfigTest.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.hive.sink; - -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigResolveOptions; - -import java.io.File; -import java.util.List; - -public class HiveSinkConfigTest { - - /** - * test hive sink config. - *

TODO: Uncouple from the hive environment - */ - public void testCreateHiveSinkConfig() { - String[] fieldNames = new String[]{"name", "age"}; - SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[]{BasicType.STRING_TYPE, BasicType.INT_TYPE}; - SeaTunnelRowType seaTunnelRowTypeInfo = new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); - String configFile = "fakesource_to_hive.conf"; - String configFilePath = System.getProperty("user.dir") + "/src/test/resources/" + configFile; - Config config = ConfigFactory - .parseFile(new File(configFilePath)) - .resolve(ConfigResolveOptions.defaults().setAllowUnresolved(true)) - .resolveWith(ConfigFactory.systemProperties(), - ConfigResolveOptions.defaults().setAllowUnresolved(true)); - List sink = config.getConfigList("sink"); - HiveSinkConfig hiveSinkConfig = new HiveSinkConfig(sink.get(0), seaTunnelRowTypeInfo); - } -} From 6129c025678ec25ee155d89084f52d324b9f364a Mon Sep 17 00:00:00 2001 From: TyrantLucifer Date: Thu, 15 Sep 2022 19:47:01 +0800 Subject: [PATCH 37/44] [Improve][Connector-V2] Refactor hdfs file sink connector code structure (#2701) * [Improve][Connector-V2] Refactor hdfs file sink codes --- docs/en/connector-v2/sink/HdfsFile.md | 44 ++-- .../file/hdfs/sink/HdfsFileSink.java | 30 ++- .../file/hdfs/sink/HdfsFileSinkPlugin.java | 69 ----- .../hdfs/sink/filesystem/HdfsFileSystem.java | 40 --- .../filesystem/HdfsFileSystemCommitter.java | 54 ---- .../file/hdfs/sink/util/HdfsUtils.java | 138 ---------- .../HdfsJsonTransactionStateFileWriter.java | 122 --------- .../HdfsOrcTransactionStateFileWriter.java | 244 ------------------ ...HdfsParquetTransactionStateFileWriter.java | 169 ------------ .../HdfsTransactionStateFileWriteFactory.java | 115 --------- .../HdfsTxtTransactionStateFileWriter.java | 125 --------- .../sink/FileSinkAggregatedCommitterTest.java | 147 ----------- ...TestHdfsTxtTransactionStateFileWriter.java | 109 -------- .../e2e/flink/v2/file/FakeSourceToFileIT.java | 27 -- .../file/fakesource_to_hdfs_json.conf | 70 ----- .../file/fakesource_to_hdfs_parquet.conf | 71 ----- .../file/fakesource_to_hdfs_text.conf | 71 ----- .../e2e/spark/v2/file/FakeSourceToFileIT.java | 28 -- .../file/fakesource_to_hdfs_json.conf | 69 ----- .../file/fakesource_to_hdfs_parquet.conf | 70 ----- .../file/fakesource_to_hdfs_text.conf | 70 ----- 21 files changed, 51 insertions(+), 1831 deletions(-) delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSinkPlugin.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/filesystem/HdfsFileSystem.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/filesystem/HdfsFileSystemCommitter.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/util/HdfsUtils.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsJsonTransactionStateFileWriter.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsOrcTransactionStateFileWriter.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsParquetTransactionStateFileWriter.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsTransactionStateFileWriteFactory.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsTxtTransactionStateFileWriter.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/FileSinkAggregatedCommitterTest.java delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/TestHdfsTxtTransactionStateFileWriter.java delete mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_json.conf delete mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_parquet.conf delete mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_text.conf delete mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_json.conf delete mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_parquet.conf delete mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_text.conf diff --git a/docs/en/connector-v2/sink/HdfsFile.md b/docs/en/connector-v2/sink/HdfsFile.md index e2e3b7561ff..9281567606d 100644 --- a/docs/en/connector-v2/sink/HdfsFile.md +++ b/docs/en/connector-v2/sink/HdfsFile.md @@ -24,24 +24,29 @@ By default, we use 2PC commit to ensure `exactly-once` In order to use this connector, You must ensure your spark/flink cluster already integrated hadoop. The tested hadoop version is 2.x. -| name | type | required | default value | -| --------------------------------- | ------ | -------- |---------------------------------------------------------| -| path | string | yes | - | -| file_name_expression | string | no | "${transactionId}" | -| file_format | string | no | "text" | -| filename_time_format | string | no | "yyyy.MM.dd" | -| field_delimiter | string | no | '\001' | -| row_delimiter | string | no | "\n" | -| partition_by | array | no | - | -| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | -| is_partition_field_write_in_file | boolean| no | false | -| sink_columns | array | no | When this parameter is empty, all fields are sink columns | -| is_enable_transaction | boolean| no | true | -| save_mode | string | no | "error" | +| name | type | required | default value | +|----------------------------------| ------ | -------- |---------------------------------------------------------| +| fs.defaultFS | string | yes | - | +| path | string | yes | - | +| file_name_expression | string | no | "${transactionId}" | +| file_format | string | no | "text" | +| filename_time_format | string | no | "yyyy.MM.dd" | +| field_delimiter | string | no | '\001' | +| row_delimiter | string | no | "\n" | +| partition_by | array | no | - | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | +| is_partition_field_write_in_file | boolean| no | false | +| sink_columns | array | no | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean| no | true | +| save_mode | string | no | "error" | + +### fs.defaultFS [string] + +The hadoop cluster address that start with `hdfs://`, for example: `hdfs://hadoopcluster` ### path [string] -The target dir path is required. The `hdfs file` starts with `hdfs://`. +The target dir path is required. ### file_name_expression [string] @@ -125,7 +130,8 @@ For text file format ```bash HdfsFile { - path="hdfs://mycluster/tmp/hive/warehouse/test2" + fs.defaultFS="hdfs://hadoopcluster" + path="/tmp/hive/warehouse/test2" field_delimiter="\t" row_delimiter="\n" partition_by=["age"] @@ -145,7 +151,8 @@ For parquet file format ```bash HdfsFile { - path="hdfs://mycluster/tmp/hive/warehouse/test2" + fs.defaultFS="hdfs://hadoopcluster" + path="/tmp/hive/warehouse/test2" partition_by=["age"] partition_dir_expression="${k0}=${v0}" is_partition_field_write_in_file=true @@ -163,7 +170,8 @@ For orc file format ```bash HdfsFile { - path="hdfs://mycluster/tmp/hive/warehouse/test2" + fs.defaultFS="hdfs://hadoopcluster" + path="/tmp/hive/warehouse/test2" partition_by=["age"] partition_dir_expression="${k0}=${v0}" is_partition_field_write_in_file=true diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSink.java index 01f23e1fef3..a484a6345a1 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSink.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSink.java @@ -17,16 +17,36 @@ package org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; + +import org.apache.seatunnel.api.common.PrepareFailException; import org.apache.seatunnel.api.sink.SeaTunnelSink; -import org.apache.seatunnel.connectors.seatunnel.file.sink.AbstractFileSink; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin; +import org.apache.seatunnel.common.config.CheckConfigUtil; +import org.apache.seatunnel.common.config.CheckResult; +import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.sink.BaseFileSink; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; import com.google.auto.service.AutoService; @AutoService(SeaTunnelSink.class) -public class HdfsFileSink extends AbstractFileSink { +public class HdfsFileSink extends BaseFileSink { + + @Override + public String getPluginName() { + return FileSystemType.HDFS.getFileSystemPluginName(); + } + @Override - public SinkFileSystemPlugin getSinkFileSystemPlugin() { - return new HdfsFileSinkPlugin(); + public void prepare(Config pluginConfig) throws PrepareFailException { + CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, FS_DEFAULT_NAME_KEY); + if (!result.isSuccess()) { + throw new PrepareFailException(getPluginName(), PluginType.SINK, result.getMsg()); + } + super.prepare(pluginConfig); + hadoopConf = new HadoopConf(pluginConfig.getString(FS_DEFAULT_NAME_KEY)); } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSinkPlugin.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSinkPlugin.java deleted file mode 100644 index 33ffb1f9110..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/HdfsFileSinkPlugin.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink; - -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.filesystem.HdfsFileSystem; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.filesystem.HdfsFileSystemCommitter; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.writer.HdfsTransactionStateFileWriteFactory; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator; - -import lombok.NonNull; - -import java.util.List; -import java.util.Optional; - -public class HdfsFileSinkPlugin implements SinkFileSystemPlugin { - @Override - public String getPluginName() { - return FileSystemType.HDFS.getFileSystemPluginName(); - } - - @Override - public Optional getTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo, - @NonNull TransactionFileNameGenerator transactionFileNameGenerator, - @NonNull PartitionDirNameGenerator partitionDirNameGenerator, - @NonNull List sinkColumnsIndexInRow, - @NonNull String tmpPath, - @NonNull String targetPath, - @NonNull String jobId, - int subTaskIndex, - @NonNull String fieldDelimiter, - @NonNull String rowDelimiter, - @NonNull FileSystem fileSystem) { - // using factory to generate transaction state file writer - TransactionStateFileWriter writer = HdfsTransactionStateFileWriteFactory.of(seaTunnelRowTypeInfo, transactionFileNameGenerator, partitionDirNameGenerator, sinkColumnsIndexInRow, tmpPath, targetPath, jobId, subTaskIndex, fieldDelimiter, rowDelimiter, fileSystem); - return Optional.of(writer); - } - - @Override - public Optional getFileSystemCommitter() { - return Optional.of(new HdfsFileSystemCommitter()); - } - - @Override - public Optional getFileSystem() { - return Optional.of(new HdfsFileSystem()); - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/filesystem/HdfsFileSystem.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/filesystem/HdfsFileSystem.java deleted file mode 100644 index a70e2a3d680..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/filesystem/HdfsFileSystem.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink.filesystem; - -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.util.HdfsUtils; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem; - -import org.apache.hadoop.fs.Path; - -import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; - -public class HdfsFileSystem implements FileSystem { - @Override - public void deleteFile(String path) throws IOException { - HdfsUtils.deleteFile(path); - } - - @Override - public List dirList(String dirPath) throws IOException { - List paths = HdfsUtils.dirList(dirPath); - return paths.stream().map(dir -> dir.getName()).collect(Collectors.toList()); - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/filesystem/HdfsFileSystemCommitter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/filesystem/HdfsFileSystemCommitter.java deleted file mode 100644 index ae80856e6f7..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/filesystem/HdfsFileSystemCommitter.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink.filesystem; - -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.util.HdfsUtils; -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter; - -import lombok.NonNull; - -import java.io.IOException; -import java.util.Map; - -public class HdfsFileSystemCommitter implements FileSystemCommitter { - @Override - public void commitTransaction(@NonNull FileAggregatedCommitInfo aggregateCommitInfo) throws IOException { - for (Map.Entry> entry : aggregateCommitInfo.getTransactionMap().entrySet()) { - for (Map.Entry mvFileEntry : entry.getValue().entrySet()) { - HdfsUtils.renameFile(mvFileEntry.getKey(), mvFileEntry.getValue(), true); - } - // delete the transaction dir - HdfsUtils.deleteFile(entry.getKey()); - } - } - - @Override - public void abortTransaction(@NonNull FileAggregatedCommitInfo aggregateCommitInfo) throws IOException { - for (Map.Entry> entry : aggregateCommitInfo.getTransactionMap().entrySet()) { - // rollback the file - for (Map.Entry mvFileEntry : entry.getValue().entrySet()) { - if (HdfsUtils.fileExist(mvFileEntry.getValue()) && !HdfsUtils.fileExist(mvFileEntry.getKey())) { - HdfsUtils.renameFile(mvFileEntry.getValue(), mvFileEntry.getKey(), true); - } - } - // delete the transaction dir - HdfsUtils.deleteFile(entry.getKey()); - } - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/util/HdfsUtils.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/util/HdfsUtils.java deleted file mode 100644 index 7b5b972edd9..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/util/HdfsUtils.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink.util; - -import lombok.NonNull; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.net.URI; -import java.util.ArrayList; -import java.util.List; - -public class HdfsUtils { - private static final Logger LOGGER = LoggerFactory.getLogger(HdfsUtils.class); - - public static final int WRITE_BUFFER_SIZE = 2048; - - public static final Configuration CONF = new Configuration(); - - // make the configuration object static, so orc and parquet reader can get it - static { - LOGGER.info(System.getenv("HADOOP_CONF_DIR")); - CONF.addResource(new Path(System.getenv("HADOOP_CONF_DIR") + "/core-site.xml")); - CONF.addResource(new Path(System.getenv("HADOOP_CONF_DIR") + "/hdfs-site.xml")); - CONF.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem"); - } - - public static FileSystem getHdfsFs(@NonNull String path) - throws IOException { - return FileSystem.get(URI.create(path), CONF); - } - - public static FSDataOutputStream getOutputStream(@NonNull String outFilePath) throws IOException { - FileSystem hdfsFs = getHdfsFs(outFilePath); - Path path = new Path(outFilePath); - FSDataOutputStream fsDataOutputStream = hdfsFs.create(path, true, WRITE_BUFFER_SIZE); - return fsDataOutputStream; - } - - public static void createFile(@NonNull String filePath) throws IOException { - FileSystem hdfsFs = getHdfsFs(filePath); - Path path = new Path(filePath); - if (!hdfsFs.createNewFile(path)) { - throw new IOException("create file " + filePath + " error"); - } - } - - public static void deleteFile(@NonNull String file) throws IOException { - FileSystem hdfsFs = getHdfsFs(file); - if (!hdfsFs.delete(new Path(file), true)) { - throw new IOException("delete file " + file + " error"); - } - } - - /** - * rename file - * - * @param oldName old file name - * @param newName target file name - * @param rmWhenExist if this is true, we will delete the target file when it already exists - * @throws IOException throw IOException - */ - public static void renameFile(@NonNull String oldName, @NonNull String newName, boolean rmWhenExist) throws IOException { - FileSystem hdfsFs = getHdfsFs(newName); - LOGGER.info("begin rename file oldName :[" + oldName + "] to newName :[" + newName + "]"); - - Path oldPath = new Path(oldName); - Path newPath = new Path(newName); - if (rmWhenExist) { - if (fileExist(newName) && fileExist(oldName)) { - hdfsFs.delete(newPath, true); - } - } - if (!fileExist(newName.substring(0, newName.lastIndexOf("/")))) { - createDir(newName.substring(0, newName.lastIndexOf("/"))); - } - - if (hdfsFs.rename(oldPath, newPath)) { - LOGGER.info("rename file :[" + oldPath + "] to [" + newPath + "] finish"); - } else { - throw new IOException("rename file :[" + oldPath + "] to [" + newPath + "] error"); - } - } - - public static void createDir(@NonNull String filePath) throws IOException { - FileSystem hdfsFs = getHdfsFs(filePath); - Path dfs = new Path(filePath); - if (!hdfsFs.mkdirs(dfs)) { - throw new IOException("create dir " + filePath + " error"); - } - } - - public static boolean fileExist(@NonNull String filePath) throws IOException { - FileSystem hdfsFs = getHdfsFs(filePath); - Path fileName = new Path(filePath); - return hdfsFs.exists(fileName); - } - - /** - * get the dir in filePath - */ - public static List dirList(@NonNull String filePath) throws FileNotFoundException, IOException { - FileSystem hdfsFs = getHdfsFs(filePath); - List pathList = new ArrayList(); - Path fileName = new Path(filePath); - FileStatus[] status = hdfsFs.listStatus(fileName); - if (status != null && status.length > 0) { - for (FileStatus fileStatus : status) { - if (fileStatus.isDirectory()) { - pathList.add(fileStatus.getPath()); - } - } - } - return pathList; - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsJsonTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsJsonTransactionStateFileWriter.java deleted file mode 100644 index c11d57ce6a9..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsJsonTransactionStateFileWriter.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink.writer; - -import org.apache.seatunnel.api.serialization.SerializationSchema; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.util.HdfsUtils; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.AbstractTransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator; -import org.apache.seatunnel.format.json.JsonSerializationSchema; - -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; -import org.apache.hadoop.fs.FSDataOutputStream; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -@Slf4j -public class HdfsJsonTransactionStateFileWriter extends AbstractTransactionStateFileWriter { - - private static final long serialVersionUID = -5432828969702531646L; - - private final byte[] rowDelimiter; - private final SerializationSchema serializationSchema; - private Map beingWrittenOutputStream; - - public HdfsJsonTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo, - @NonNull TransactionFileNameGenerator transactionFileNameGenerator, - @NonNull PartitionDirNameGenerator partitionDirNameGenerator, - @NonNull List sinkColumnsIndexInRow, - @NonNull String tmpPath, - @NonNull String targetPath, - @NonNull String jobId, - int subTaskIndex, - @NonNull String rowDelimiter, - @NonNull FileSystem fileSystem) { - super(seaTunnelRowTypeInfo, transactionFileNameGenerator, partitionDirNameGenerator, sinkColumnsIndexInRow, tmpPath, targetPath, jobId, subTaskIndex, fileSystem); - - this.rowDelimiter = rowDelimiter.getBytes(); - this.serializationSchema = new JsonSerializationSchema(seaTunnelRowTypeInfo); - beingWrittenOutputStream = new HashMap<>(); - } - - @Override - public void beginTransaction(String transactionId) { - this.beingWrittenOutputStream = new HashMap<>(); - } - - @Override - public void abortTransaction(String transactionId) { - this.beingWrittenOutputStream = new HashMap<>(); - } - - @Override - public void write(@NonNull SeaTunnelRow seaTunnelRow) { - String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow); - FSDataOutputStream fsDataOutputStream = getOrCreateOutputStream(filePath); - try { - byte[] rowBytes = serializationSchema.serialize(seaTunnelRow); - fsDataOutputStream.write(rowBytes); - fsDataOutputStream.write(rowDelimiter); - } catch (IOException e) { - log.error("write data to file {} error", filePath); - throw new RuntimeException(e); - } - } - - @Override - public void finishAndCloseWriteFile() { - beingWrittenOutputStream.entrySet().forEach(entry -> { - try { - entry.getValue().flush(); - } catch (IOException e) { - log.error("error when flush file {}", entry.getKey()); - throw new RuntimeException(e); - } finally { - try { - entry.getValue().close(); - } catch (IOException e) { - log.error("error when close output stream {}", entry.getKey()); - } - } - - needMoveFiles.put(entry.getKey(), getTargetLocation(entry.getKey())); - }); - } - - private FSDataOutputStream getOrCreateOutputStream(@NonNull String filePath) { - FSDataOutputStream fsDataOutputStream = beingWrittenOutputStream.get(filePath); - if (fsDataOutputStream == null) { - try { - fsDataOutputStream = HdfsUtils.getOutputStream(filePath); - beingWrittenOutputStream.put(filePath, fsDataOutputStream); - } catch (IOException e) { - log.error("can not get output file stream"); - throw new RuntimeException(e); - } - } - return fsDataOutputStream; - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsOrcTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsOrcTransactionStateFileWriter.java deleted file mode 100644 index 23648359cf6..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsOrcTransactionStateFileWriter.java +++ /dev/null @@ -1,244 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink.writer; - -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.util.HdfsUtils; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.AbstractTransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator; - -import lombok.NonNull; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import org.apache.orc.CompressionKind; -import org.apache.orc.OrcFile; -import org.apache.orc.TypeDescription; -import org.apache.orc.Writer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.math.BigInteger; -import java.nio.charset.StandardCharsets; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class HdfsOrcTransactionStateFileWriter extends AbstractTransactionStateFileWriter { - private static final Logger LOGGER = LoggerFactory.getLogger(HdfsOrcTransactionStateFileWriter.class); - private Map beingWrittenWriter; - - public HdfsOrcTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo, @NonNull TransactionFileNameGenerator transactionFileNameGenerator, - @NonNull PartitionDirNameGenerator partitionDirNameGenerator, - @NonNull List sinkColumnsIndexInRow, - @NonNull String tmpPath, - @NonNull String targetPath, - @NonNull String jobId, - int subTaskIndex, - @NonNull FileSystem fileSystem) { - super(seaTunnelRowTypeInfo, transactionFileNameGenerator, partitionDirNameGenerator, sinkColumnsIndexInRow, tmpPath, targetPath, jobId, subTaskIndex, fileSystem); - this.beingWrittenWriter = new HashMap<>(); - } - - @Override - public void write(@NonNull SeaTunnelRow seaTunnelRow) { - String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow); - Writer writer = getOrCreateWriter(filePath); - TypeDescription schema = buildSchemaWithRowType(); - VectorizedRowBatch rowBatch = schema.createRowBatch(); - int i = 0; - int row = rowBatch.size++; - for (Integer index : sinkColumnsIndexInRow) { - Object value = seaTunnelRow.getField(index); - ColumnVector vector = rowBatch.cols[i]; - setColumn(value, vector, row); - i++; - } - try { - writer.addRowBatch(rowBatch); - rowBatch.reset(); - } catch (IOException e) { - String errorMsg = String.format("Write data to orc file [%s] error", filePath); - throw new RuntimeException(errorMsg, e); - } - } - - @Override - public void finishAndCloseWriteFile() { - this.beingWrittenWriter.forEach((k, v) -> { - try { - v.close(); - } catch (IOException e) { - String errorMsg = String.format("Close file [%s] orc writer failed, error msg: [%s]", k, e.getMessage()); - throw new RuntimeException(errorMsg, e); - } catch (NullPointerException e) { - // Because orc writer not support be closed multi times, so if the second time close orc writer it will throw NullPointerException - // In a whole process of file sink, it will experience four stages: - // 1. beginTransaction 2. prepareCommit 3. commit 4. close - // In the first stage, it will not close any writers, start with the second stage, writer will be closed. - // In the last stage, it will not close any writers - // So orc writer will be closed one extra time after is closed. - LOGGER.info("Close file [{}] orc writer", k); - } - needMoveFiles.put(k, getTargetLocation(k)); - }); - } - - @Override - public void beginTransaction(String transactionId) { - this.beingWrittenWriter = new HashMap<>(); - } - - @Override - public void abortTransaction(String transactionId) { - this.beingWrittenWriter = new HashMap<>(); - } - - private Writer getOrCreateWriter(@NonNull String filePath) { - Writer writer = this.beingWrittenWriter.get(filePath); - if (writer == null) { - TypeDescription schema = buildSchemaWithRowType(); - Path path = new Path(filePath); - try { - OrcFile.WriterOptions options = OrcFile.writerOptions(HdfsUtils.CONF) - .setSchema(schema) - // temporarily used snappy - .compress(CompressionKind.SNAPPY) - // use orc version 0.12 - .version(OrcFile.Version.V_0_12) - .overwrite(true); - Writer newWriter = OrcFile.createWriter(path, options); - this.beingWrittenWriter.put(filePath, newWriter); - return newWriter; - } catch (IOException e) { - String errorMsg = String.format("Get orc writer for file [%s] error", filePath); - throw new RuntimeException(errorMsg, e); - } - } - return writer; - } - - private TypeDescription buildFieldWithRowType(SeaTunnelDataType type) { - if (BasicType.BOOLEAN_TYPE.equals(type)) { - return TypeDescription.createBoolean(); - } - if (BasicType.SHORT_TYPE.equals(type)) { - return TypeDescription.createShort(); - } - if (BasicType.INT_TYPE.equals(type)) { - return TypeDescription.createInt(); - } - if (BasicType.LONG_TYPE.equals(type)) { - return TypeDescription.createLong(); - } - if (BasicType.FLOAT_TYPE.equals(type)) { - return TypeDescription.createFloat(); - } - if (BasicType.DOUBLE_TYPE.equals(type)) { - return TypeDescription.createDouble(); - } - if (BasicType.BYTE_TYPE.equals(type)) { - return TypeDescription.createByte(); - } - return TypeDescription.createString(); - } - - private TypeDescription buildSchemaWithRowType() { - TypeDescription schema = TypeDescription.createStruct(); - for (Integer i : sinkColumnsIndexInRow) { - TypeDescription fieldType = buildFieldWithRowType(seaTunnelRowTypeInfo.getFieldType(i)); - schema.addField(seaTunnelRowTypeInfo.getFieldName(i), fieldType); - } - return schema; - } - - private void setColumn(Object value, ColumnVector vector, int row) { - if (value == null) { - vector.isNull[row] = true; - vector.noNulls = false; - } else { - switch (vector.type) { - case LONG: - LongColumnVector longVector = (LongColumnVector) vector; - setLongColumnVector(value, longVector, row); - break; - case DOUBLE: - DoubleColumnVector doubleColumnVector = (DoubleColumnVector) vector; - setDoubleVector(value, doubleColumnVector, row); - break; - case BYTES: - BytesColumnVector bytesColumnVector = (BytesColumnVector) vector; - setByteColumnVector(value, bytesColumnVector, row); - break; - default: - throw new RuntimeException("Unexpected ColumnVector subtype"); - } - } - } - - private void setLongColumnVector(Object value, LongColumnVector longVector, int row) { - if (value instanceof Boolean) { - Boolean bool = (Boolean) value; - longVector.vector[row] = (bool.equals(Boolean.TRUE)) ? Long.valueOf(1) : Long.valueOf(0); - } else if (value instanceof Integer) { - longVector.vector[row] = (Integer) value; - } else if (value instanceof Long) { - longVector.vector[row] = (Long) value; - } else if (value instanceof BigInteger) { - BigInteger bigInt = (BigInteger) value; - longVector.vector[row] = bigInt.longValue(); - } else { - throw new RuntimeException("Long or Integer type expected for field"); - } - } - - private void setByteColumnVector(Object value, BytesColumnVector bytesColVector, int rowNum) { - if (value instanceof byte[] || value instanceof String) { - byte[] byteVec; - if (value instanceof String) { - String strVal = (String) value; - byteVec = strVal.getBytes(StandardCharsets.UTF_8); - } else { - byteVec = (byte[]) value; - } - bytesColVector.setRef(rowNum, byteVec, 0, byteVec.length); - } else { - throw new RuntimeException("byte[] or String type expected for field "); - } - } - - private void setDoubleVector(Object value, DoubleColumnVector doubleVector, int rowNum) { - if (value instanceof Double) { - doubleVector.vector[rowNum] = (Double) value; - } else if (value instanceof Float) { - Float floatValue = (Float) value; - doubleVector.vector[rowNum] = floatValue.doubleValue(); - } else { - throw new RuntimeException("Double or Float type expected for field "); - } - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsParquetTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsParquetTransactionStateFileWriter.java deleted file mode 100644 index 8b904fda9a2..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsParquetTransactionStateFileWriter.java +++ /dev/null @@ -1,169 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink.writer; - -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.util.HdfsUtils; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.AbstractTransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator; - -import lombok.NonNull; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.avro.AvroParquetWriter; -import org.apache.parquet.column.ParquetProperties; -import org.apache.parquet.hadoop.ParquetFileWriter; -import org.apache.parquet.hadoop.ParquetWriter; -import org.apache.parquet.hadoop.metadata.CompressionCodecName; -import org.apache.parquet.hadoop.util.HadoopOutputFile; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class HdfsParquetTransactionStateFileWriter extends AbstractTransactionStateFileWriter { - private static final Logger LOGGER = LoggerFactory.getLogger(HdfsParquetTransactionStateFileWriter.class); - private Map> beingWrittenWriter; - - public HdfsParquetTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo, - @NonNull TransactionFileNameGenerator transactionFileNameGenerator, - @NonNull PartitionDirNameGenerator partitionDirNameGenerator, - @NonNull List sinkColumnsIndexInRow, @NonNull String tmpPath, - @NonNull String targetPath, - @NonNull String jobId, - int subTaskIndex, - @NonNull FileSystem fileSystem) { - super(seaTunnelRowTypeInfo, transactionFileNameGenerator, partitionDirNameGenerator, sinkColumnsIndexInRow, tmpPath, targetPath, jobId, subTaskIndex, fileSystem); - beingWrittenWriter = new HashMap<>(); - } - - @Override - public void write(@NonNull SeaTunnelRow seaTunnelRow) { - String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow); - ParquetWriter writer = getOrCreateWriter(filePath); - Schema schema = buildSchemaWithRowType(); - GenericRecordBuilder recordBuilder = new GenericRecordBuilder(schema); - sinkColumnsIndexInRow.forEach(index -> recordBuilder.set(seaTunnelRowTypeInfo.getFieldName(index), seaTunnelRow.getField(index))); - GenericData.Record record = recordBuilder.build(); - try { - writer.write(record); - } catch (IOException e) { - String errorMsg = String.format("Write data to file [%s] error", filePath); - throw new RuntimeException(errorMsg, e); - } - } - - @Override - public void finishAndCloseWriteFile() { - this.beingWrittenWriter.forEach((k, v) -> { - try { - v.close(); - } catch (IOException e) { - String errorMsg = String.format("Close file [%s] parquet writer failed, error msg: [%s]", k, e.getMessage()); - throw new RuntimeException(errorMsg, e); - } - needMoveFiles.put(k, getTargetLocation(k)); - }); - } - - @Override - public void beginTransaction(String transactionId) { - this.beingWrittenWriter = new HashMap<>(); - } - - @Override - public void abortTransaction(String transactionId) { - this.beingWrittenWriter = new HashMap<>(); - } - - private ParquetWriter getOrCreateWriter(@NonNull String filePath) { - ParquetWriter writer = this.beingWrittenWriter.get(filePath); - if (writer == null) { - Schema schema = buildSchemaWithRowType(); - Path path = new Path(filePath); - try { - HadoopOutputFile outputFile = HadoopOutputFile.fromPath(path, HdfsUtils.CONF); - ParquetWriter newWriter = AvroParquetWriter.builder(outputFile) - .withWriteMode(ParquetFileWriter.Mode.OVERWRITE) - // use parquet v1 to improve compatibility - .withWriterVersion(ParquetProperties.WriterVersion.PARQUET_1_0) - // Temporarily use snappy compress - // I think we can use the compress option in config to control this - .withCompressionCodec(CompressionCodecName.SNAPPY) - .withSchema(schema) - .build(); - this.beingWrittenWriter.put(filePath, newWriter); - return newWriter; - } catch (IOException e) { - String errorMsg = String.format("Get parquet writer for file [%s] error", filePath); - throw new RuntimeException(errorMsg, e); - } - } - return writer; - } - - private Schema buildSchemaWithRowType() { - ArrayList fields = new ArrayList<>(); - SeaTunnelDataType[] fieldTypes = seaTunnelRowTypeInfo.getFieldTypes(); - String[] fieldNames = seaTunnelRowTypeInfo.getFieldNames(); - sinkColumnsIndexInRow.forEach(index -> { - if (BasicType.BOOLEAN_TYPE.equals(fieldTypes[index])) { - Schema.Field field = new Schema.Field(fieldNames[index], Schema.create(Schema.Type.BOOLEAN), null, null); - fields.add(field); - } else if (BasicType.SHORT_TYPE.equals(fieldTypes[index]) || BasicType.INT_TYPE.equals(fieldTypes[index])) { - Schema.Field field = new Schema.Field(fieldNames[index], Schema.create(Schema.Type.INT), null, null); - fields.add(field); - } else if (BasicType.LONG_TYPE.equals(fieldTypes[index])) { - Schema.Field field = new Schema.Field(fieldNames[index], Schema.create(Schema.Type.LONG), null, null); - fields.add(field); - } else if (BasicType.FLOAT_TYPE.equals(fieldTypes[index])) { - Schema.Field field = new Schema.Field(fieldNames[index], Schema.create(Schema.Type.FLOAT), null, null); - fields.add(field); - } else if (BasicType.DOUBLE_TYPE.equals(fieldTypes[index])) { - Schema.Field field = new Schema.Field(fieldNames[index], Schema.create(Schema.Type.DOUBLE), null, null); - fields.add(field); - } else if (BasicType.STRING_TYPE.equals(fieldTypes[index])) { - Schema.Field field = new Schema.Field(fieldNames[index], Schema.create(Schema.Type.STRING), null, null); - fields.add(field); - } else if (BasicType.BYTE_TYPE.equals(fieldTypes[index])) { - Schema.Field field = new Schema.Field(fieldNames[index], Schema.create(Schema.Type.BYTES), null, null); - fields.add(field); - } else if (BasicType.VOID_TYPE.equals(fieldTypes[index])) { - Schema.Field field = new Schema.Field(fieldNames[index], Schema.create(Schema.Type.NULL), null, null); - fields.add(field); - } - }); - return Schema.createRecord("SeatunnelRecord", - "The record generated by seatunnel file connector", - "org.apache.parquet.avro", - false, - fields); - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsTransactionStateFileWriteFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsTransactionStateFileWriteFactory.java deleted file mode 100644 index 814491b4f55..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsTransactionStateFileWriteFactory.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink.writer; - -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkTransactionFileNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator; - -import lombok.NonNull; - -import java.util.List; - -public class HdfsTransactionStateFileWriteFactory { - - private HdfsTransactionStateFileWriteFactory() {} - - public static TransactionStateFileWriter of(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo, - @NonNull TransactionFileNameGenerator transactionFileNameGenerator, - @NonNull PartitionDirNameGenerator partitionDirNameGenerator, - @NonNull List sinkColumnsIndexInRow, - @NonNull String tmpPath, - @NonNull String targetPath, - @NonNull String jobId, - int subTaskIndex, - @NonNull String fieldDelimiter, - @NonNull String rowDelimiter, - @NonNull FileSystem fileSystem) { - FileSinkTransactionFileNameGenerator fileSinkTransactionFileNameGenerator = (FileSinkTransactionFileNameGenerator) transactionFileNameGenerator; - FileFormat fileFormat = fileSinkTransactionFileNameGenerator.getFileFormat(); - if (fileFormat.equals(FileFormat.CSV)) { - // #2133 wait this issue closed, there will be replaced using csv writer - return new HdfsTxtTransactionStateFileWriter( - seaTunnelRowTypeInfo, - transactionFileNameGenerator, - partitionDirNameGenerator, - sinkColumnsIndexInRow, - tmpPath, - targetPath, - jobId, - subTaskIndex, - fieldDelimiter, - rowDelimiter, - fileSystem); - } - if (fileFormat.equals(FileFormat.PARQUET)) { - return new HdfsParquetTransactionStateFileWriter( - seaTunnelRowTypeInfo, - transactionFileNameGenerator, - partitionDirNameGenerator, - sinkColumnsIndexInRow, - tmpPath, - targetPath, - jobId, - subTaskIndex, - fileSystem); - } - if (fileFormat.equals(FileFormat.ORC)) { - return new HdfsOrcTransactionStateFileWriter( - seaTunnelRowTypeInfo, - transactionFileNameGenerator, - partitionDirNameGenerator, - sinkColumnsIndexInRow, - tmpPath, - targetPath, - jobId, - subTaskIndex, - fileSystem); - } - if (fileFormat.equals(FileFormat.JSON)) { - return new HdfsJsonTransactionStateFileWriter( - seaTunnelRowTypeInfo, - transactionFileNameGenerator, - partitionDirNameGenerator, - sinkColumnsIndexInRow, - tmpPath, - targetPath, - jobId, - subTaskIndex, - rowDelimiter, - fileSystem); - } - // if file type not supported by file connector, default txt writer will be generated - return new HdfsTxtTransactionStateFileWriter( - seaTunnelRowTypeInfo, - transactionFileNameGenerator, - partitionDirNameGenerator, - sinkColumnsIndexInRow, - tmpPath, - targetPath, - jobId, - subTaskIndex, - fieldDelimiter, - rowDelimiter, - fileSystem); - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsTxtTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsTxtTransactionStateFileWriter.java deleted file mode 100644 index 2ee792a10eb..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/writer/HdfsTxtTransactionStateFileWriter.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink.writer; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.util.HdfsUtils; -import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.AbstractTransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator; - -import lombok.NonNull; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class HdfsTxtTransactionStateFileWriter extends AbstractTransactionStateFileWriter { - private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTxtTransactionStateFileWriter.class); - private Map beingWrittenOutputStream; - - private String fieldDelimiter; - private String rowDelimiter; - - public HdfsTxtTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo, - @NonNull TransactionFileNameGenerator transactionFileNameGenerator, - @NonNull PartitionDirNameGenerator partitionDirNameGenerator, - @NonNull List sinkColumnsIndexInRow, - @NonNull String tmpPath, - @NonNull String targetPath, - @NonNull String jobId, - int subTaskIndex, - @NonNull String fieldDelimiter, - @NonNull String rowDelimiter, - @NonNull FileSystem fileSystem) { - super(seaTunnelRowTypeInfo, transactionFileNameGenerator, partitionDirNameGenerator, sinkColumnsIndexInRow, tmpPath, targetPath, jobId, subTaskIndex, fileSystem); - - this.fieldDelimiter = fieldDelimiter; - this.rowDelimiter = rowDelimiter; - beingWrittenOutputStream = new HashMap<>(); - } - - @Override - public void beginTransaction(String transactionId) { - this.beingWrittenOutputStream = new HashMap<>(); - } - - @Override - public void abortTransaction(String transactionId) { - this.beingWrittenOutputStream = new HashMap<>(); - } - - @Override - public void write(@NonNull SeaTunnelRow seaTunnelRow) { - String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow); - FSDataOutputStream fsDataOutputStream = getOrCreateOutputStream(filePath); - String line = transformRowToLine(seaTunnelRow); - try { - fsDataOutputStream.write(line.getBytes()); - fsDataOutputStream.write(rowDelimiter.getBytes()); - } catch (IOException e) { - LOGGER.error("write data to file {} error", filePath); - throw new RuntimeException(e); - } - } - - @Override - public void finishAndCloseWriteFile() { - beingWrittenOutputStream.entrySet().forEach(entry -> { - try { - entry.getValue().flush(); - } catch (IOException e) { - LOGGER.error("error when flush file {}", entry.getKey()); - throw new RuntimeException(e); - } finally { - try { - entry.getValue().close(); - } catch (IOException e) { - LOGGER.error("error when close output stream {}", entry.getKey()); - } - } - - needMoveFiles.put(entry.getKey(), getTargetLocation(entry.getKey())); - }); - } - - private FSDataOutputStream getOrCreateOutputStream(@NonNull String filePath) { - FSDataOutputStream fsDataOutputStream = beingWrittenOutputStream.get(filePath); - if (fsDataOutputStream == null) { - try { - fsDataOutputStream = HdfsUtils.getOutputStream(filePath); - beingWrittenOutputStream.put(filePath, fsDataOutputStream); - } catch (IOException e) { - LOGGER.error("can not get output file stream"); - throw new RuntimeException(e); - } - } - return fsDataOutputStream; - } - - private String transformRowToLine(@NonNull SeaTunnelRow seaTunnelRow) { - return this.sinkColumnsIndexInRow.stream().map(index -> seaTunnelRow.getFields()[index] == null ? "" : seaTunnelRow.getFields()[index].toString()).collect(Collectors.joining(fieldDelimiter)); - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/FileSinkAggregatedCommitterTest.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/FileSinkAggregatedCommitterTest.java deleted file mode 100644 index e76ee1d43d2..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/FileSinkAggregatedCommitterTest.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink; - -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.filesystem.HdfsFileSystemCommitter; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.util.HdfsUtils; -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkAggregatedCommitter; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.EnabledOnOs; -import org.junit.jupiter.api.condition.OS; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.stream.Collectors; - -@EnabledOnOs(value = {OS.MAC, OS.LINUX}) -public class FileSinkAggregatedCommitterTest { - @SuppressWarnings("checkstyle:UnnecessaryParentheses") - @Test - public void testCommit() throws Exception { - FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter()); - Map> transactionFiles = new HashMap<>(); - Random random = new Random(); - Long jobId = random.nextLong(); - String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId); - String targetDir = String.format("/tmp/hive/warehouse/%s", jobId); - Map needMoveFiles = new HashMap<>(); - needMoveFiles.put(transactionDir + "/c3=4/c4=rrr/test1.txt", targetDir + "/c3=4/c4=rrr/test1.txt"); - needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt"); - HdfsUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt"); - HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt"); - - transactionFiles.put(transactionDir, needMoveFiles); - - Map> partitionDirAndVals = new HashMap<>(); - partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList())); - partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList())); - - FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals); - List fileAggregatedCommitInfoList = new ArrayList<>(); - fileAggregatedCommitInfoList.add(fileAggregatedCommitInfo); - fileSinkAggregatedCommitter.commit(fileAggregatedCommitInfoList); - - Assertions.assertTrue(HdfsUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt")); - Assertions.assertTrue(HdfsUtils.fileExist(targetDir + "/c3=4/c4=rrr/test1.txt")); - Assertions.assertTrue(!HdfsUtils.fileExist(transactionDir)); - } - - @SuppressWarnings("checkstyle:UnnecessaryParentheses") - @Test - public void testCombine() throws Exception { - FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter()); - Map> transactionFiles = new HashMap<>(); - Random random = new Random(); - Long jobId = random.nextLong(); - String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId); - String targetDir = String.format("/tmp/hive/warehouse/%s", jobId); - Map needMoveFiles = new HashMap<>(); - needMoveFiles.put(transactionDir + "/c3=3/c4=rrr/test1.txt", targetDir + "/c3=3/c4=rrr/test1.txt"); - needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt"); - Map> partitionDirAndVals = new HashMap<>(); - partitionDirAndVals.put("/c3=3/c4=rrr", Arrays.stream((new String[]{"3", "rrr"})).collect(Collectors.toList())); - partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList())); - FileCommitInfo fileCommitInfo = new FileCommitInfo(needMoveFiles, partitionDirAndVals, transactionDir); - HdfsUtils.createFile(transactionDir + "/c3=3/c4=rrr/test1.txt"); - HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt"); - - Map needMoveFiles1 = new HashMap<>(); - needMoveFiles1.put(transactionDir + "/c3=4/c4=rrr/test2.txt", targetDir + "/c3=4/c4=rrr/test2.txt"); - needMoveFiles1.put(transactionDir + "/c3=4/c4=bbb/test2.txt", targetDir + "/c3=4/c4=bbb/test2.txt"); - Map> partitionDirAndVals1 = new HashMap<>(); - partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList())); - partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList())); - FileCommitInfo fileCommitInfo1 = new FileCommitInfo(needMoveFiles1, partitionDirAndVals1, transactionDir); - List fileCommitInfoList = new ArrayList<>(); - fileCommitInfoList.add(fileCommitInfo); - fileCommitInfoList.add(fileCommitInfo1); - - FileAggregatedCommitInfo combine = fileSinkAggregatedCommitter.combine(fileCommitInfoList); - Assertions.assertEquals(1, combine.getTransactionMap().size()); - Assertions.assertEquals(4, combine.getTransactionMap().get(transactionDir).size()); - Assertions.assertEquals(targetDir + "/c3=3/c4=rrr/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=3/c4=rrr/test1.txt")); - Assertions.assertEquals(targetDir + "/c3=4/c4=bbb/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test1.txt")); - Assertions.assertEquals(targetDir + "/c3=4/c4=rrr/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=rrr/test2.txt")); - Assertions.assertEquals(targetDir + "/c3=4/c4=bbb/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test2.txt")); - Assertions.assertEquals(3, combine.getPartitionDirAndValsMap().keySet().size()); - } - - @SuppressWarnings("checkstyle:UnnecessaryParentheses") - @Test - public void testAbort() throws Exception { - FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter()); - Map> transactionFiles = new HashMap<>(); - Random random = new Random(); - Long jobId = random.nextLong(); - String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId); - String targetDir = String.format("/tmp/hive/warehouse/%s", jobId); - Map needMoveFiles = new HashMap<>(); - needMoveFiles.put(transactionDir + "/c3=4/c4=rrr/test1.txt", targetDir + "/c3=4/c4=rrr/test1.txt"); - needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt"); - Map> partitionDirAndVals = new HashMap<>(); - partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList())); - partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList())); - HdfsUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt"); - HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt"); - - transactionFiles.put(transactionDir, needMoveFiles); - FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals); - List fileAggregatedCommitInfoList = new ArrayList<>(); - fileAggregatedCommitInfoList.add(fileAggregatedCommitInfo); - fileSinkAggregatedCommitter.commit(fileAggregatedCommitInfoList); - - Assertions.assertTrue(HdfsUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt")); - Assertions.assertTrue(HdfsUtils.fileExist(targetDir + "/c3=4/c4=rrr/test1.txt")); - Assertions.assertTrue(!HdfsUtils.fileExist(transactionDir)); - - fileSinkAggregatedCommitter.abort(fileAggregatedCommitInfoList); - Assertions.assertTrue(!HdfsUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt")); - Assertions.assertTrue(!HdfsUtils.fileExist(targetDir + "/c3=4/c4=rrr/test1.txt")); - - // transactionDir will being delete when abort - Assertions.assertTrue(!HdfsUtils.fileExist(transactionDir)); - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/TestHdfsTxtTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/TestHdfsTxtTransactionStateFileWriter.java deleted file mode 100644 index 88ca2b9f4c2..00000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/sink/TestHdfsTxtTransactionStateFileWriter.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.seatunnel.connectors.seatunnel.file.hdfs.sink; - -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.filesystem.HdfsFileSystem; -import org.apache.seatunnel.connectors.seatunnel.file.hdfs.sink.writer.HdfsTxtTransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo; -import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator; -import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkTransactionFileNameGenerator; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.condition.EnabledOnOs; -import org.junit.jupiter.api.condition.OS; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -@EnabledOnOs(value = {OS.MAC, OS.LINUX}) -public class TestHdfsTxtTransactionStateFileWriter { - - @SuppressWarnings("checkstyle:MagicNumber") - @Test - public void testHdfsTextTransactionStateFileWriter() throws Exception { - String[] fieldNames = new String[]{"c1", "c2", "c3", "c4"}; - SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[]{BasicType.BOOLEAN_TYPE, BasicType.INT_TYPE, BasicType.STRING_TYPE, BasicType.INT_TYPE}; - SeaTunnelRowType seaTunnelRowTypeInfo = new SeaTunnelRowType(fieldNames, seaTunnelDataTypes); - - List sinkColumnIndexInRow = new ArrayList<>(); - sinkColumnIndexInRow.add(0); - sinkColumnIndexInRow.add(1); - - List hivePartitionFieldList = new ArrayList<>(); - hivePartitionFieldList.add("c3"); - hivePartitionFieldList.add("c4"); - - List partitionFieldIndexInRow = new ArrayList<>(); - partitionFieldIndexInRow.add(2); - partitionFieldIndexInRow.add(3); - - String jobId = System.currentTimeMillis() + ""; - String targetPath = "/tmp/hive/warehouse/seatunnel.db/test1"; - String tmpPath = "/tmp/seatunnel"; - - TransactionStateFileWriter fileWriter = new HdfsTxtTransactionStateFileWriter(seaTunnelRowTypeInfo, - new FileSinkTransactionFileNameGenerator(FileFormat.TEXT, null, "yyyy.MM.dd"), - new FileSinkPartitionDirNameGenerator(hivePartitionFieldList, partitionFieldIndexInRow, "${k0}=${v0}/${k1}=${v1}"), - sinkColumnIndexInRow, - tmpPath, - targetPath, - jobId, - 0, - String.valueOf('\001'), - "\n", - new HdfsFileSystem()); - - String transactionId = fileWriter.beginTransaction(1L); - - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(new Object[]{true, 1, "str1", "str2"}); - fileWriter.write(seaTunnelRow); - - SeaTunnelRow seaTunnelRow1 = new SeaTunnelRow(new Object[]{true, 1, "str1", "str3"}); - fileWriter.write(seaTunnelRow1); - - Optional fileCommitInfoOptional = fileWriter.prepareCommit(); - //check file exists and file content - Assertions.assertTrue(fileCommitInfoOptional.isPresent()); - FileCommitInfo fileCommitInfo = fileCommitInfoOptional.get(); - String transactionDir = tmpPath + "/seatunnel/" + jobId + "/" + transactionId; - Assertions.assertEquals(transactionDir, fileCommitInfo.getTransactionDir()); - Assertions.assertEquals(2, fileCommitInfo.getNeedMoveFiles().size()); - Map needMoveFiles = fileCommitInfo.getNeedMoveFiles(); - Assertions.assertEquals(targetPath + "/c3=str1/c4=str2/" + transactionId + ".txt", needMoveFiles.get(transactionDir + "/c3=str1/c4=str2/" + transactionId + ".txt")); - Assertions.assertEquals(targetPath + "/c3=str1/c4=str3/" + transactionId + ".txt", needMoveFiles.get(transactionDir + "/c3=str1/c4=str3/" + transactionId + ".txt")); - - Map> partitionDirAndValsMap = fileCommitInfo.getPartitionDirAndValsMap(); - Assertions.assertEquals(2, partitionDirAndValsMap.size()); - Assertions.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str2")); - Assertions.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str3")); - Assertions.assertTrue(partitionDirAndValsMap.get("c3=str1/c4=str2").size() == 2); - Assertions.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str2").get(0)); - Assertions.assertEquals("str2", partitionDirAndValsMap.get("c3=str1/c4=str2").get(1)); - Assertions.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str3").get(0)); - Assertions.assertEquals("str3", partitionDirAndValsMap.get("c3=str1/c4=str3").get(1)); - } -} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/file/FakeSourceToFileIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/file/FakeSourceToFileIT.java index 50d7c34d212..d6bfdefc550 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/file/FakeSourceToFileIT.java +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/file/FakeSourceToFileIT.java @@ -53,31 +53,4 @@ public void testFakeSourceToLocalFileJson() throws IOException, InterruptedExcep Container.ExecResult execResult = executeSeaTunnelFlinkJob("/file/fakesource_to_local_json.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } - - /** - * fake source -> hdfs text file sink - */ - @Test - public void testFakeSourceToHdfsFileText() throws IOException, InterruptedException { - Container.ExecResult execResult = executeSeaTunnelFlinkJob("/file/fakesource_to_hdfs_text.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } - - /** - * fake source -> hdfs parquet file sink - */ - @Test - public void testFakeSourceToHdfsFileParquet() throws IOException, InterruptedException { - Container.ExecResult execResult = executeSeaTunnelFlinkJob("/file/fakesource_to_hdfs_parquet.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } - - /** - * fake source -> hdfs json file sink - */ - @Test - public void testFakeSourceToHdfsFileJson() throws IOException, InterruptedException { - Container.ExecResult execResult = executeSeaTunnelFlinkJob("/file/fakesource_to_hdfs_json.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } } diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_json.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_json.conf deleted file mode 100644 index 269b85d0840..00000000000 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_json.conf +++ /dev/null @@ -1,70 +0,0 @@ -# -# 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. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set flink configuration here - execution.parallelism = 1 - job.mode = "BATCH" - #execution.checkpoint.interval = 10000 - #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" -} - -source { - FakeSource { - result_table_name = "fake" - schema = { - fields { - name = "string" - age = "int" - } - } - } - - # If you would like to get more information about how to configure seatunnel and see full list of source plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource -} - -transform { - sql { - sql = "select name,age from fake" - } - - # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, - # please go to https://seatunnel.apache.org/docs/transform/sql -} - -sink { - HdfsFile { - path="/tmp/hive/warehouse/test2" - row_delimiter="\n" - partition_by=["age"] - partition_dir_expression="${k0}=${v0}" - is_partition_field_write_in_file=true - file_name_expression="${transactionId}_${now}" - file_format="json" - sink_columns=["name","age"] - filename_time_format="yyyy.MM.dd" - is_enable_transaction=true - save_mode="error" - } - - # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/sink/HdfsFile -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_parquet.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_parquet.conf deleted file mode 100644 index 5e1ea5c01d1..00000000000 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_parquet.conf +++ /dev/null @@ -1,71 +0,0 @@ -# -# 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. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set flink configuration here - execution.parallelism = 1 - job.mode = "BATCH" - #execution.checkpoint.interval = 10000 - #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" -} - -source { - FakeSource { - result_table_name = "fake" - schema = { - fields { - name = "string" - age = "int" - } - } - } - - # If you would like to get more information about how to configure seatunnel and see full list of source plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource -} - -transform { - sql { - sql = "select name,age from fake" - } - - # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, - # please go to https://seatunnel.apache.org/docs/transform/sql -} - -sink { - HdfsFile { - path="/tmp/hive/warehouse/test2" - field_delimiter="\t" - row_delimiter="\n" - partition_by=["age"] - partition_dir_expression="${k0}=${v0}" - is_partition_field_write_in_file=true - file_name_expression="${transactionId}_${now}" - file_format="parquet" - sink_columns=["name","age"] - filename_time_format="yyyy.MM.dd" - is_enable_transaction=true - save_mode="error" - } - - # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/sink/HdfsFile -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_text.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_text.conf deleted file mode 100644 index d4a8a745cc2..00000000000 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-file-flink-e2e/src/test/resources/file/fakesource_to_hdfs_text.conf +++ /dev/null @@ -1,71 +0,0 @@ -# -# 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. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set flink configuration here - execution.parallelism = 1 - job.mode = "BATCH" - #execution.checkpoint.interval = 10000 - #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" -} - -source { - FakeSource { - result_table_name = "fake" - schema = { - fields { - name = "string" - age = "int" - } - } - } - - # If you would like to get more information about how to configure seatunnel and see full list of source plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource -} - -transform { - sql { - sql = "select name,age from fake" - } - - # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, - # please go to https://seatunnel.apache.org/docs/transform/sql -} - -sink { - HdfsFile { - path="/tmp/hive/warehouse/test2" - field_delimiter="\t" - row_delimiter="\n" - partition_by=["age"] - partition_dir_expression="${k0}=${v0}" - is_partition_field_write_in_file=true - file_name_expression="${transactionId}_${now}" - file_format="text" - sink_columns=["name","age"] - filename_time_format="yyyy.MM.dd" - is_enable_transaction=true - save_mode="error" - } - - # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/sink/HdfsFile -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java index fccd8db06a2..b2535ebf281 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java @@ -57,32 +57,4 @@ public void testFakeSourceToLocalFileJson() throws IOException, InterruptedExcep Container.ExecResult execResult = executeSeaTunnelSparkJob("/file/fakesource_to_local_json.conf"); Assertions.assertEquals(0, execResult.getExitCode()); } - - - /** - * fake source -> hdfs text file sink - */ - @Test - public void testFakeSourceToHdfsFileText() throws IOException, InterruptedException { - Container.ExecResult execResult = executeSeaTunnelSparkJob("/file/fakesource_to_hdfs_text.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } - - /** - * fake source -> hdfs parquet file sink - */ - @Test - public void testFakeSourceToHdfsFileParquet() throws IOException, InterruptedException { - Container.ExecResult execResult = executeSeaTunnelSparkJob("/file/fakesource_to_hdfs_parquet.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } - - /** - * fake source -> hdfs json file sink - */ - @Test - public void testFakeSourceToHdfsFileJson() throws IOException, InterruptedException { - Container.ExecResult execResult = executeSeaTunnelSparkJob("/file/fakesource_to_hdfs_json.conf"); - Assertions.assertEquals(0, execResult.getExitCode()); - } } diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_json.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_json.conf deleted file mode 100644 index 40454bce061..00000000000 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_json.conf +++ /dev/null @@ -1,69 +0,0 @@ -# -# 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. -# - -env { - # You can set spark configuration here - spark.app.name = "SeaTunnel" - spark.executor.instances = 2 - spark.executor.cores = 1 - spark.executor.memory = "1g" - spark.master = local - job.mode = "BATCH" -} - -source { - FakeSource { - result_table_name = "fake" - schema = { - fields { - name = "string" - age = "int" - } - } - } - - # If you would like to get more information about how to configure seatunnel and see full list of source plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource -} - -transform { - sql { - sql = "select name,age from fake" - } - - # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, - # please go to https://seatunnel.apache.org/docs/category/transform -} - -sink { - HdfsFile { - path="/tmp/hive/warehouse/test2" - row_delimiter="\n" - partition_by=["age"] - partition_dir_expression="${k0}=${v0}" - is_partition_field_write_in_file=true - file_name_expression="${transactionId}_${now}" - file_format="json" - sink_columns=["name","age"] - filename_time_format="yyyy.MM.dd" - is_enable_transaction=true - save_mode="error" - } - - # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/sink/HdfsFile -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_parquet.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_parquet.conf deleted file mode 100644 index 550990eeace..00000000000 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_parquet.conf +++ /dev/null @@ -1,70 +0,0 @@ -# -# 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. -# - -env { - # You can set spark configuration here - spark.app.name = "SeaTunnel" - spark.executor.instances = 2 - spark.executor.cores = 1 - spark.executor.memory = "1g" - spark.master = local - job.mode = "BATCH" -} - -source { - FakeSource { - result_table_name = "fake" - schema = { - fields { - name = "string" - age = "int" - } - } - } - - # If you would like to get more information about how to configure seatunnel and see full list of source plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource -} - -transform { - sql { - sql = "select name,age from fake" - } - - # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, - # please go to https://seatunnel.apache.org/docs/category/transform -} - -sink { - HdfsFile { - path="/tmp/hive/warehouse/test2" - field_delimiter="\t" - row_delimiter="\n" - partition_by=["age"] - partition_dir_expression="${k0}=${v0}" - is_partition_field_write_in_file=true - file_name_expression="${transactionId}_${now}" - file_format="parquet" - sink_columns=["name","age"] - filename_time_format="yyyy.MM.dd" - is_enable_transaction=true - save_mode="error" - } - - # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/sink/HdfsFile -} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_text.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_text.conf deleted file mode 100644 index 2bf6afba61f..00000000000 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-file-spark-e2e/src/test/resources/file/fakesource_to_hdfs_text.conf +++ /dev/null @@ -1,70 +0,0 @@ -# -# 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. -# - -env { - # You can set spark configuration here - spark.app.name = "SeaTunnel" - spark.executor.instances = 2 - spark.executor.cores = 1 - spark.executor.memory = "1g" - spark.master = local - job.mode = "BATCH" -} - -source { - FakeSource { - result_table_name = "fake" - schema = { - fields { - name = "string" - age = "int" - } - } - } - - # If you would like to get more information about how to configure seatunnel and see full list of source plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/source/FakeSource -} - -transform { - sql { - sql = "select name,age from fake" - } - - # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, - # please go to https://seatunnel.apache.org/docs/category/transform -} - -sink { - HdfsFile { - path="/tmp/hive/warehouse/test2" - field_delimiter="\t" - row_delimiter="\n" - partition_by=["age"] - partition_dir_expression="${k0}=${v0}" - is_partition_field_write_in_file=true - file_name_expression="${transactionId}_${now}" - file_format="text" - sink_columns=["name","age"] - filename_time_format="yyyy.MM.dd" - is_enable_transaction=true - save_mode="error" - } - - # If you would like to get more information about how to configure seatunnel and see full list of sink plugins, - # please go to https://seatunnel.apache.org/docs/connector-v2/sink/HdfsFile -} \ No newline at end of file From 90ce3851db2266c89a57bece479c4ed79cd69988 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Fri, 16 Sep 2022 10:58:02 +0800 Subject: [PATCH 38/44] [Improve][Connector-V2] Fix kafka connector (#2745) * Fix SeaTunnelRow convert to json format error * Rename `bootstrap.server` to `bootstrap.servers` in kafka source config * Remove `kafka.` prefix in sink producer configuration key --- .../connector-kafka/pom.xml | 5 +++++ .../seatunnel/kafka/config/Config.java | 2 +- .../DefaultSeaTunnelRowSerializer.java | 22 +++++-------------- .../seatunnel/kafka/sink/KafkaSinkWriter.java | 17 +++++++------- .../kafka/source/ConsumerMetadata.java | 10 ++++----- .../seatunnel/kafka/source/KafkaSource.java | 6 ++--- .../kafka/source/KafkaSourceReader.java | 2 +- .../source/KafkaSourceSplitEnumerator.java | 6 ++++- 8 files changed, 34 insertions(+), 36 deletions(-) diff --git a/seatunnel-connectors-v2/connector-kafka/pom.xml b/seatunnel-connectors-v2/connector-kafka/pom.xml index 75e56237b24..4159e8783aa 100644 --- a/seatunnel-connectors-v2/connector-kafka/pom.xml +++ b/seatunnel-connectors-v2/connector-kafka/pom.xml @@ -41,6 +41,11 @@ kafka-clients ${kafka.client.version} + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java index ee1492044e6..d48d12cf646 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java @@ -31,7 +31,7 @@ public class Config { /** * The server address of kafka cluster. */ - public static final String BOOTSTRAP_SERVER = "bootstrap.server"; + public static final String BOOTSTRAP_SERVERS = "bootstrap.servers"; public static final String KAFKA_CONFIG_PREFIX = "kafka."; diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java index ad9a24ef27a..e80a7d306b0 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java @@ -19,32 +19,22 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.config.Common; -import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.format.json.JsonSerializationSchema; import org.apache.kafka.clients.producer.ProducerRecord; -import java.util.HashMap; -import java.util.Map; - -public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer { +public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer { private final String topic; - private final SeaTunnelRowType seaTunnelRowType; + private final JsonSerializationSchema jsonSerializationSchema; public DefaultSeaTunnelRowSerializer(String topic, SeaTunnelRowType seaTunnelRowType) { this.topic = topic; - this.seaTunnelRowType = seaTunnelRowType; + this.jsonSerializationSchema = new JsonSerializationSchema(seaTunnelRowType); } @Override - public ProducerRecord serializeRow(SeaTunnelRow row) { - Map map = new HashMap<>(Common.COLLECTION_SIZE); - String[] fieldNames = seaTunnelRowType.getFieldNames(); - Object[] fields = row.getFields(); - for (int i = 0; i < fieldNames.length; i++) { - map.put(fieldNames[i], fields[i]); - } - return new ProducerRecord<>(topic, null, JsonUtils.toJsonString(map)); + public ProducerRecord serializeRow(SeaTunnelRow row) { + return new ProducerRecord<>(topic, null, jsonSerializationSchema.serialize(row)); } } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java index b3a20d0b152..3f71c308525 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java @@ -33,7 +33,7 @@ import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import java.util.List; import java.util.Optional; @@ -51,19 +51,18 @@ public class KafkaSinkWriter implements SinkWriter kafkaProducerSender; + private final KafkaProduceSender kafkaProducerSender; + private final SeaTunnelRowSerializer seaTunnelRowSerializer; private static final int PREFIX_RANGE = 10000; // check config @Override public void write(SeaTunnelRow element) { - ProducerRecord producerRecord = seaTunnelRowSerializer.serializeRow(element); + ProducerRecord producerRecord = seaTunnelRowSerializer.serializeRow(element); kafkaProducerSender.send(producerRecord); } - private final SeaTunnelRowSerializer seaTunnelRowSerializer; - public KafkaSinkWriter( SinkWriter.Context context, SeaTunnelRowType seaTunnelRowType, @@ -124,19 +123,19 @@ public void close() { private Properties getKafkaProperties(Config pluginConfig) { Config kafkaConfig = TypesafeConfigUtils.extractSubConfig(pluginConfig, - org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.KAFKA_CONFIG_PREFIX, true); + org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.KAFKA_CONFIG_PREFIX, false); Properties kafkaProperties = new Properties(); kafkaConfig.entrySet().forEach(entry -> { kafkaProperties.put(entry.getKey(), entry.getValue().unwrapped()); }); kafkaProperties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, pluginConfig.getString(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); - kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); return kafkaProperties; } // todo: parse the target field from config - private SeaTunnelRowSerializer getSerializer(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { + private SeaTunnelRowSerializer getSerializer(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { return new DefaultSeaTunnelRowSerializer(pluginConfig.getString("topics"), seaTunnelRowType); } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java index 6f6b921d89d..ee942aaefc7 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/ConsumerMetadata.java @@ -27,7 +27,7 @@ public class ConsumerMetadata implements Serializable { private String topic; private boolean isPattern = false; - private String bootstrapServer; + private String bootstrapServers; private Properties properties; private String consumerGroup; private boolean commitOnCheckpoint = false; @@ -56,12 +56,12 @@ public void setPattern(boolean pattern) { isPattern = pattern; } - public String getBootstrapServer() { - return bootstrapServer; + public String getBootstrapServers() { + return bootstrapServers; } - public void setBootstrapServer(String bootstrapServer) { - this.bootstrapServer = bootstrapServer; + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; } public Properties getProperties() { diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java index 62314b87a63..a6b6199f5f0 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.kafka.source; -import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.BOOTSTRAP_SERVER; +import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.BOOTSTRAP_SERVERS; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.COMMIT_ON_CHECKPOINT; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.CONSUMER_GROUP; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PATTERN; @@ -67,7 +67,7 @@ public String getPluginName() { @Override public void prepare(Config config) throws PrepareFailException { - CheckResult result = CheckConfigUtil.checkAllExists(config, TOPIC, BOOTSTRAP_SERVER); + CheckResult result = CheckConfigUtil.checkAllExists(config, TOPIC, BOOTSTRAP_SERVERS); if (!result.isSuccess()) { throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg()); } @@ -75,7 +75,7 @@ public void prepare(Config config) throws PrepareFailException { if (config.hasPath(PATTERN)) { this.metadata.setPattern(config.getBoolean(PATTERN)); } - this.metadata.setBootstrapServer(config.getString(BOOTSTRAP_SERVER)); + this.metadata.setBootstrapServers(config.getString(BOOTSTRAP_SERVERS)); this.metadata.setProperties(new Properties()); if (config.hasPath(CONSUMER_GROUP)) { diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java index d4272e4a3fd..5a53aae3b2e 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceReader.java @@ -73,7 +73,7 @@ public class KafkaSourceReader implements SourceReader { private static final String CLIENT_ID_PREFIX = "seatunnel"; @@ -119,7 +121,7 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { private AdminClient initAdminClient(Properties properties) { Properties props = new Properties(properties); - props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.metadata.getBootstrapServer()); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.metadata.getBootstrapServers()); props.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, CLIENT_ID_PREFIX + "-enumerator-admin-client-" + this.hashCode()); return AdminClient.create(props); } @@ -133,6 +135,8 @@ private Set getTopicInfo() throws ExecutionException, Interrup } else { topics = Arrays.asList(this.metadata.getTopic().split(",")); } + log.info("Discovered topics: {}", topics); + Collection partitions = adminClient.describeTopics(topics).all().get().values().stream().flatMap(t -> t.partitions().stream() .map(p -> new TopicPartition(t.name(), p.partition()))).collect(Collectors.toSet()); From e4691f7d38950aa0a9f2dec6bfcb306665865d56 Mon Sep 17 00:00:00 2001 From: songjianet <1778651752@qq.com> Date: Fri, 16 Sep 2022 13:57:53 +0800 Subject: [PATCH 39/44] [Feat][UI] Add themes in the project. (#2753) --- seatunnel-ui/src/views/login/index.tsx | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/views/login/index.tsx index 9be980c0b0e..87cac2fbe5b 100644 --- a/seatunnel-ui/src/views/login/index.tsx +++ b/seatunnel-ui/src/views/login/index.tsx @@ -37,9 +37,9 @@ const Login = defineComponent({ }, render() { return ( - -

-

{this.t('login.login_to_sea_tunnel')}

+ +
+

{this.t('login.login_to_sea_tunnel')}

Date: Fri, 16 Sep 2022 14:57:51 +0800 Subject: [PATCH 40/44] [Feat][UI] Update user service in the project. (#2756) --- seatunnel-ui/src/service/user/index.ts | 58 ++++++++++++++++---------- seatunnel-ui/src/service/user/types.ts | 13 +++--- 2 files changed, 41 insertions(+), 30 deletions(-) diff --git a/seatunnel-ui/src/service/user/index.ts b/seatunnel-ui/src/service/user/index.ts index f28844408f4..801059b1626 100644 --- a/seatunnel-ui/src/service/user/index.ts +++ b/seatunnel-ui/src/service/user/index.ts @@ -16,52 +16,64 @@ */ import { axios } from '@/service/service' -import type { UserList, UserAdd, UserUpdate } from '@/service/user/types' +import type { UserList, UserLogin, UserDetail } from '@/service/user/types' -export function userDisable(id: number): any { +export function userList(params: UserList): any { return axios({ - url: '/user/disable', - method: 'put', - data: { id } + url: '/user', + method: 'get', + params }) } -export function userEnable(id: number): any { +export function userAdd(data: UserDetail): any { return axios({ - url: '/user/enable', - method: 'put', - data: { id } + url: '/user', + method: 'post', + data }) } -export function userList(userListReq: UserList): any { +export function userLogin(data: UserLogin): any { return axios({ - url: '/user/list', + url: '/user/login', method: 'post', - data: { userListReq } + data }) } -export function userDelete(id: number): any { +export function userLogout(): any { return axios({ - url: '/user/user', - method: 'delete', - data: { id } + url: '/user/logout', + method: 'patch' }) } -export function userAdd(addReq: UserAdd): any { +export function userDelete(userId: number): any { return axios({ - url: '/user/user', - method: 'post', - data: { addReq } + url: `/user/${userId}`, + method: 'delete' }) } -export function userUpdate(updateReq: UserUpdate): any { +export function userUpdate(userId: number, data: UserDetail): any { return axios({ - url: '/user/user', + url: `/user/${userId}`, method: 'put', - data: { updateReq } + data }) } + +export function userDisable(userId: number): any { + return axios({ + url: `/user/${userId}/disable`, + method: 'put' + }) +} + +export function userEnable(userId: number): any { + return axios({ + url: `/user/${userId}/enable`, + method: 'patch' + }) +} \ No newline at end of file diff --git a/seatunnel-ui/src/service/user/types.ts b/seatunnel-ui/src/service/user/types.ts index 8d28ca55466..427ff86d5b0 100644 --- a/seatunnel-ui/src/service/user/types.ts +++ b/seatunnel-ui/src/service/user/types.ts @@ -16,20 +16,19 @@ */ interface UserList { - name: string + name?: string pageNo: number pageSize: number } -interface UserAdd { +interface UserLogin { password: string - status: string - type: string username: string } -interface UserUpdate extends UserAdd { - id: number +interface UserDetail extends UserLogin { + status: string + type: string } -export { UserList, UserAdd, UserUpdate } +export { UserList, UserLogin, UserDetail } \ No newline at end of file From ffc6088a79a2d346fd8940673b6e0c06b1534c46 Mon Sep 17 00:00:00 2001 From: hailin0 Date: Fri, 16 Sep 2022 17:21:09 +0800 Subject: [PATCH 41/44] [Feature][Connector-V2] Add iceberg source connector (#2615) * [Feature][Connector-V2] Add iceberg source connector Co-authored-by: wanghailin --- LICENSE | 6 +- NOTICE | 19 +- docs/en/connector-v2/source/Iceberg.md | 157 +++++++++++++ plugin-mapping.properties | 2 +- .../console/sink/ConsoleSinkWriter.java | 3 + .../connector-iceberg/pom.xml | 193 ++++++++++++++++ .../iceberg/IcebergCatalogFactory.java | 82 +++++++ .../seatunnel/iceberg/IcebergTableLoader.java | 77 +++++++ .../iceberg/config/CommonConfig.java | 79 +++++++ .../iceberg/config/SourceConfig.java | 81 +++++++ .../iceberg/data/DefaultDeserializer.java | 139 ++++++++++++ .../seatunnel/iceberg/data/Deserializer.java | 27 +++ .../iceberg/data/IcebergRecordProjection.java | 97 ++++++++ .../iceberg/data/IcebergTypeMapper.java | 111 ++++++++++ .../iceberg/source/IcebergSource.java | 178 +++++++++++++++ .../enumerator/AbstractSplitEnumerator.java | 130 +++++++++++ .../IcebergBatchSplitEnumerator.java | 71 ++++++ .../enumerator/IcebergEnumerationResult.java | 38 ++++ .../enumerator/IcebergEnumeratorPosition.java | 39 ++++ .../IcebergSplitEnumeratorState.java | 41 ++++ .../IcebergStreamSplitEnumerator.java | 85 +++++++ .../enumerator/scan/IcebergScanContext.java | 85 +++++++ .../scan/IcebergScanSplitPlanner.java | 207 ++++++++++++++++++ .../scan/IcebergStreamScanStrategy.java | 41 ++++ .../reader/IcebergFileScanTaskReader.java | 143 ++++++++++++ .../IcebergFileScanTaskSplitReader.java | 86 ++++++++ .../source/reader/IcebergSourceReader.java | 145 ++++++++++++ .../split/IcebergFileScanTaskSplit.java | 68 ++++++ .../iceberg/TestIcebergMetastore.java | 63 ++++++ seatunnel-connectors-v2/pom.xml | 1 + seatunnel-dist/pom.xml | 6 + .../connector-iceberg-flink-e2e/pom.xml | 45 ++++ .../e2e/flink/v2/iceberg/IcebergSourceIT.java | 165 ++++++++++++++ .../resources/iceberg/iceberg_source.conf | 82 +++++++ .../src/test/resources/log4j.properties | 22 ++ .../seatunnel-flink-connector-v2-e2e/pom.xml | 1 + .../connector-iceberg-spark-e2e/pom.xml | 45 ++++ .../e2e/flink/v2/iceberg/IcebergSourceIT.java | 164 ++++++++++++++ .../resources/iceberg/iceberg_source.conf | 85 +++++++ .../src/test/resources/log4j.properties | 22 ++ 40 files changed, 3128 insertions(+), 3 deletions(-) create mode 100644 docs/en/connector-v2/source/Iceberg.md create mode 100644 seatunnel-connectors-v2/connector-iceberg/pom.xml create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergTableLoader.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/CommonConfig.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/DefaultDeserializer.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/Deserializer.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergRecordProjection.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/AbstractSplitEnumerator.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergBatchSplitEnumerator.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumerationResult.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumeratorPosition.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergSplitEnumeratorState.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergStreamSplitEnumerator.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanContext.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanSplitPlanner.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergStreamScanStrategy.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskReader.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskSplitReader.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergSourceReader.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/split/IcebergFileScanTaskSplit.java create mode 100644 seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/TestIcebergMetastore.java create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iceberg/IcebergSourceIT.java create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/resources/iceberg/iceberg_source.conf create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/resources/log4j.properties create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iceberg/IcebergSourceIT.java create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/resources/iceberg/iceberg_source.conf create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/resources/log4j.properties diff --git a/LICENSE b/LICENSE index 3239c7bec9e..b0e8126be8f 100644 --- a/LICENSE +++ b/LICENSE @@ -222,4 +222,8 @@ seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connec seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialectFactory.java from https://github.com/apache/flink seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialectLoader.java from https://github.com/apache/flink seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/BiConsumerWithException.java from https://github.com/apache/flink -seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/JdbcStatementBuilder.java from https://github.com/apache/flink \ No newline at end of file +seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/executor/JdbcStatementBuilder.java from https://github.com/apache/flink +seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanSplitPlanner.java from https://github.com/apache/iceberg +seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergStreamScanStrategy.java from https://github.com/apache/iceberg +seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumerationResult.java from https://github.com/apache/iceberg +seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumeratorPosition.java from https://github.com/apache/iceberg \ No newline at end of file diff --git a/NOTICE b/NOTICE index 49b6fb66d18..4682b14abb6 100644 --- a/NOTICE +++ b/NOTICE @@ -18,4 +18,21 @@ The Apache Software Foundation (http://www.apache.org/). Flink : Connectors : JDBC -Copyright 2014-2022 The Apache Software Foundation \ No newline at end of file +Copyright 2014-2022 The Apache Software Foundation + + + +// ------------------------------------------------------------------ +// NOTICE file corresponding to the section 4d of The Apache License, +// Version 2.0, in this case for Apache Iceberg +// ------------------------------------------------------------------ + +Apache Iceberg +Copyright 2017-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +Iceberg : Flink +Copyright 2017-2022 The Apache Software Foundation \ No newline at end of file diff --git a/docs/en/connector-v2/source/Iceberg.md b/docs/en/connector-v2/source/Iceberg.md new file mode 100644 index 00000000000..85458b0eae1 --- /dev/null +++ b/docs/en/connector-v2/source/Iceberg.md @@ -0,0 +1,157 @@ +# Apache Iceberg + +> Apache Iceberg source connector + +## Description + +Source connector for Apache Iceberg. It can support batch and stream mode. + +## Key features + +- [x] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [x] [schema projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) + +- [x] data format + - [x] parquet + - [x] orc + - [x] avro +- [x] iceberg catalog + - [x] hadoop(2.7.5) + - [x] hive(2.3.9) + +## Options + +| name | type | required | default value | +|-----------------------------------|----------|----------|-------------------------| +| catalog_name | string | yes | - | +| catalog_type | string | yes | - | +| uri | string | false | - | +| warehouse | string | yes | - | +| namespace | string | yes | - | +| table | string | yes | - | +| case_sensitive | boolean | false | false | +| start_snapshot_timestamp | long | false | - | +| start_snapshot_id | long | false | - | +| end_snapshot_id | long | false | - | +| use_snapshot_id | long | false | - | +| use_snapshot_timestamp | long | false | - | +| stream_scan_strategy | enum | false | FROM_LATEST_SNAPSHOT | + +### catalog_name [string] + +User-specified catalog name. + +### catalog_type [string] + +The optional values are: +- hive: The hive metastore catalog. +- hadoop: The hadoop catalog. + +### uri [string] + +The Hive metastore’s thrift URI. + +### warehouse [string] + +The location to store metadata files and data files. + +### namespace [string] + +The iceberg database name in the backend catalog. + +### table [string] + +The iceberg table name in the backend catalog. + +### case_sensitive [boolean] + +If data columns where selected via fields(Collection), controls whether the match to the schema will be done with case sensitivity. + +### fields [array] + +Use projection to select data columns and columns order. + +### start_snapshot_id [long] + +Instructs this scan to look for changes starting from a particular snapshot (exclusive). + +### start_snapshot_timestamp [long] + +Instructs this scan to look for changes starting from the most recent snapshot for the table as of the timestamp. timestamp – the timestamp in millis since the Unix epoch + +### end_snapshot_id [long] + +Instructs this scan to look for changes up to a particular snapshot (inclusive). + +### use_snapshot_id [long] + +Instructs this scan to look for use the given snapshot ID. + +### use_snapshot_timestamp [long] + +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 + +### stream_scan_strategy [enum] + +Starting strategy for stream mode execution, Default to use `FROM_LATEST_SNAPSHOT` if don’t specify any value. +The optional values are: +- TABLE_SCAN_THEN_INCREMENTAL: Do a regular table scan then switch to the incremental mode. +- FROM_LATEST_SNAPSHOT: Start incremental mode from the latest snapshot inclusive. +- FROM_EARLIEST_SNAPSHOT: Start incremental mode from the earliest snapshot inclusive. +- FROM_SNAPSHOT_ID: Start incremental mode from a snapshot with a specific id inclusive. +- FROM_SNAPSHOT_TIMESTAMP: Start incremental mode from a snapshot with a specific timestamp inclusive. + +## Example + +simple + +```hocon +source { + Iceberg { + catalog_name = "seatunnel" + catalog_type = "hadoop" + warehouse = "hdfs://your_cluster//tmp/seatunnel/iceberg/" + namespace = "your_iceberg_database" + table = "your_iceberg_table" + } +} +``` +Or + +```hocon +source { + Iceberg { + catalog_name = "seatunnel" + catalog_type = "hive" + uri = "thrift://localhost:9083" + warehouse = "hdfs://your_cluster//tmp/seatunnel/iceberg/" + namespace = "your_iceberg_database" + table = "your_iceberg_table" + } +} +``` + +schema projection + +```hocon +source { + Iceberg { + catalog_name = "seatunnel" + catalog_type = "hadoop" + warehouse = "hdfs://your_cluster/tmp/seatunnel/iceberg/" + namespace = "your_iceberg_database" + table = "your_iceberg_table" + + fields { + f2 = "boolean" + f1 = "bigint" + f3 = "int" + f4 = "bigint" + } + } +} +``` \ No newline at end of file diff --git a/plugin-mapping.properties b/plugin-mapping.properties index cde36d782a4..6236e9485d9 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -128,4 +128,4 @@ seatunnel.sink.DataHub = connector-datahub seatunnel.sink.Sentry = connector-sentry seatunnel.source.MongoDB = connector-mongodb seatunnel.sink.MongoDB = connector-mongodb - +seatunnel.source.Iceberg = connector-iceberg diff --git a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java index 3a5b334f8f6..2a276d4c798 100644 --- a/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java +++ b/seatunnel-connectors-v2/connector-console/src/main/java/org/apache/seatunnel/connectors/seatunnel/console/sink/ConsoleSinkWriter.java @@ -59,6 +59,9 @@ public void close() { } private String fieldToString(SeaTunnelDataType type, Object value) { + if (value == null) { + return null; + } switch (type.getSqlType()) { case ARRAY: case BYTES: diff --git a/seatunnel-connectors-v2/connector-iceberg/pom.xml b/seatunnel-connectors-v2/connector-iceberg/pom.xml new file mode 100644 index 00000000000..f474989d550 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/pom.xml @@ -0,0 +1,193 @@ + + + + + seatunnel-connectors-v2 + org.apache.seatunnel + ${revision} + + 4.0.0 + + connector-iceberg + + + 0.14.0 + 1.12.3 + 1.10.2 + 2.3.9 + + + + + org.apache.seatunnel + connector-common + ${project.version} + + + + org.apache.iceberg + iceberg-core + ${iceberg.version} + + + org.apache.iceberg + iceberg-api + ${iceberg.version} + + + org.apache.iceberg + iceberg-data + ${iceberg.version} + + + org.apache.iceberg + iceberg-orc + ${iceberg.version} + + + org.apache.orc + orc-core + nohive + 1.7.5 + + + org.apache.iceberg + iceberg-parquet + ${iceberg.version} + + + org.apache.parquet + parquet-avro + ${parquet-avro.version} + + + org.apache.avro + avro + ${avro.version} + + + org.apache.iceberg + iceberg-hive-metastore + ${iceberg.version} + + + org.apache.hive + hive-exec + ${hive.version} + core + + + org.apache.logging.log4j + * + + + org.pentaho + * + + + org.apache.parquet + * + + + org.apache.orc + * + + + org.apache.avro + * + + + + + org.apache.flink + flink-shaded-hadoop-2 + + + org.apache.avro + * + + + + + + org.apache.iceberg + iceberg-hive-metastore + ${iceberg.version} + tests + test + + + org.apache.hive + hive-exec + ${hive.version} + test + + + org.apache.logging.log4j + * + + + org.pentaho + * + + + + + org.apache.derby + derby + 10.14.2.0 + test + + + org.datanucleus + datanucleus-rdbms + 4.1.17 + test + + + org.datanucleus + datanucleus-api-jdo + 4.1.4 + test + + + commons-dbcp + commons-dbcp + 1.4 + test + + + com.jolbox + bonecp + 0.8.0.RELEASE + test + + + junit + junit + 4.13.2 + test + + + + + \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogFactory.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogFactory.java new file mode 100644 index 00000000000..e1f4b3de440 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergCatalogFactory.java @@ -0,0 +1,82 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; + +import lombok.NonNull; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hadoop.SerializableConfiguration; +import org.apache.iceberg.hive.HiveCatalog; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +public class IcebergCatalogFactory implements Serializable { + + private static final long serialVersionUID = -6003040601422350869L; + + private final String catalogName; + private final String catalogType; + private final String warehouse; + private final String uri; + + public IcebergCatalogFactory(@NonNull String catalogName, + @NonNull String catalogType, + @NonNull String warehouse, + String uri) { + this.catalogName = catalogName; + this.catalogType = catalogType; + this.warehouse = warehouse; + this.uri = uri; + } + + public Catalog create() { + Configuration conf = new Configuration(); + SerializableConfiguration serializableConf = new SerializableConfiguration(conf); + Map properties = new HashMap<>(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouse); + + switch (catalogType) { + case CommonConfig.CATALOG_TYPE_HADOOP: + return hadoop(catalogName, serializableConf, properties); + case CommonConfig.CATALOG_TYPE_HIVE: + properties.put(CatalogProperties.URI, uri); + return hive(catalogName, serializableConf, properties); + default: + throw new UnsupportedOperationException("Unsupported catalogType: " + catalogType); + } + } + + private static Catalog hadoop(String catalogName, + SerializableConfiguration conf, + Map properties) { + return CatalogUtil.loadCatalog(HadoopCatalog.class.getName(), catalogName, properties, conf.get()); + } + + private static Catalog hive(String catalogName, + SerializableConfiguration conf, + Map properties) { + return CatalogUtil.loadCatalog(HiveCatalog.class.getName(), catalogName, properties, conf.get()); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergTableLoader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergTableLoader.java new file mode 100644 index 00000000000..2dc448e696a --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/IcebergTableLoader.java @@ -0,0 +1,77 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg; + +import static com.google.common.base.Preconditions.checkArgument; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; + +import lombok.NonNull; +import org.apache.iceberg.CachingCatalog; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; + +public class IcebergTableLoader implements Closeable, Serializable { + + private static final long serialVersionUID = 9061073826700804273L; + + private final IcebergCatalogFactory icebergCatalogFactory; + private final String tableIdentifierStr; + + private Catalog catalog; + + public IcebergTableLoader(@NonNull IcebergCatalogFactory icebergCatalogFactory, + @NonNull TableIdentifier tableIdentifier) { + this.icebergCatalogFactory = icebergCatalogFactory; + this.tableIdentifierStr = tableIdentifier.toString(); + } + + public void open() { + catalog = CachingCatalog.wrap(icebergCatalogFactory.create()); + } + + public Table loadTable() { + TableIdentifier tableIdentifier = TableIdentifier.parse(tableIdentifierStr); + checkArgument(catalog.tableExists(tableIdentifier), + "Illegal source table: " + tableIdentifier); + return catalog.loadTable(tableIdentifier); + } + + @Override + public void close() throws IOException { + if (catalog != null && catalog instanceof Closeable) { + ((Closeable) catalog).close(); + } + } + + public static IcebergTableLoader create(SourceConfig sourceConfig) { + IcebergCatalogFactory catalogFactory = new IcebergCatalogFactory( + sourceConfig.getCatalogName(), + sourceConfig.getCatalogType(), + sourceConfig.getWarehouse(), + sourceConfig.getUri()); + return new IcebergTableLoader(catalogFactory, + TableIdentifier.of(Namespace.of(sourceConfig.getNamespace()), sourceConfig.getTable())); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/CommonConfig.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/CommonConfig.java new file mode 100644 index 00000000000..abf54aaf337 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/CommonConfig.java @@ -0,0 +1,79 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.config; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import lombok.Getter; +import lombok.ToString; + +import java.io.Serializable; + +@Getter +@ToString +public class CommonConfig implements Serializable { + private static final long serialVersionUID = 239821141534421580L; + + private static final String KEY_CATALOG_NAME = "catalog_name"; + private static final String KEY_CATALOG_TYPE = "catalog_type"; + private static final String KEY_NAMESPACE = "namespace"; + private static final String KEY_TABLE = "table"; + private static final String KEY_URI = "uri"; + private static final String KEY_WAREHOUSE = "warehouse"; + private static final String KEY_CASE_SENSITIVE = "case_sensitive"; + + public static final String KEY_FIELDS = "fields"; + public static final String CATALOG_TYPE_HADOOP = "hadoop"; + public static final String CATALOG_TYPE_HIVE = "hive"; + + private String catalogName; + private String catalogType; + private String uri; + private String warehouse; + private String namespace; + private String table; + private boolean caseSensitive; + + public CommonConfig(Config pluginConfig) { + String catalogType = checkArgumentNotNull(pluginConfig.getString(KEY_CATALOG_TYPE)); + checkArgument(CATALOG_TYPE_HADOOP.equals(catalogType) + || CATALOG_TYPE_HIVE.equals(catalogType), + "Illegal catalogType: " + catalogType); + + this.catalogType = catalogType; + this.catalogName = checkArgumentNotNull(pluginConfig.getString(KEY_CATALOG_NAME)); + if (pluginConfig.hasPath(KEY_URI)) { + this.uri = checkArgumentNotNull(pluginConfig.getString(KEY_URI)); + } + this.warehouse = checkArgumentNotNull(pluginConfig.getString(KEY_WAREHOUSE)); + this.namespace = checkArgumentNotNull(pluginConfig.getString(KEY_NAMESPACE)); + this.table = checkArgumentNotNull(pluginConfig.getString(KEY_TABLE)); + + if (pluginConfig.hasPath(KEY_CASE_SENSITIVE)) { + this.caseSensitive = pluginConfig.getBoolean(KEY_CASE_SENSITIVE); + } + } + + protected T checkArgumentNotNull(T argument) { + checkNotNull(argument); + return argument; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java new file mode 100644 index 00000000000..4fe2b3ef7f4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/config/SourceConfig.java @@ -0,0 +1,81 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.config; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergStreamScanStrategy; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import lombok.Getter; +import lombok.ToString; +import org.apache.iceberg.expressions.Expression; + +@Getter +@ToString +public class SourceConfig extends CommonConfig { + private static final long serialVersionUID = -1965861967575264253L; + + private static final String KEY_START_SNAPSHOT_TIMESTAMP = "start_snapshot_timestamp"; + private static final String KEY_START_SNAPSHOT_ID = "start_snapshot_id"; + private static final String KEY_END_SNAPSHOT_ID = "end_snapshot_id"; + private static final String KEY_USE_SNAPSHOT_ID = "use_snapshot_id"; + private static final String KEY_USE_SNAPSHOT_TIMESTAMP = "use_snapshot_timestamp"; + private static final String KEY_STREAM_SCAN_STRATEGY = "stream_scan_strategy"; + + private Long startSnapshotTimestamp; + private Long startSnapshotId; + private Long endSnapshotId; + + private Long useSnapshotId; + private Long useSnapshotTimestamp; + + private IcebergStreamScanStrategy streamScanStrategy = IcebergStreamScanStrategy.FROM_LATEST_SNAPSHOT; + private Expression filter; + private Long splitSize; + private Integer splitLookback; + private Long splitOpenFileCost; + + public SourceConfig(Config pluginConfig) { + super(pluginConfig); + if (pluginConfig.hasPath(KEY_START_SNAPSHOT_TIMESTAMP)) { + this.startSnapshotTimestamp = pluginConfig.getLong(KEY_START_SNAPSHOT_TIMESTAMP); + } + if (pluginConfig.hasPath(KEY_START_SNAPSHOT_ID)) { + this.startSnapshotId = pluginConfig.getLong(KEY_START_SNAPSHOT_ID); + } + if (pluginConfig.hasPath(KEY_END_SNAPSHOT_ID)) { + this.endSnapshotId = pluginConfig.getLong(KEY_END_SNAPSHOT_ID); + } + if (pluginConfig.hasPath(KEY_USE_SNAPSHOT_ID)) { + this.useSnapshotId = pluginConfig.getLong(KEY_USE_SNAPSHOT_ID); + } + if (pluginConfig.hasPath(KEY_USE_SNAPSHOT_TIMESTAMP)) { + this.useSnapshotTimestamp = pluginConfig.getLong(KEY_USE_SNAPSHOT_TIMESTAMP); + } + if (pluginConfig.hasPath(KEY_STREAM_SCAN_STRATEGY)) { + this.streamScanStrategy = pluginConfig.getEnum( + IcebergStreamScanStrategy.class, KEY_STREAM_SCAN_STRATEGY); + } + } + + public static SourceConfig loadConfig(Config pluginConfig) { + return new SourceConfig(pluginConfig); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/DefaultDeserializer.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/DefaultDeserializer.java new file mode 100644 index 00000000000..7ae641d1481 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/DefaultDeserializer.java @@ -0,0 +1,139 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.data; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +import lombok.AllArgsConstructor; +import lombok.NonNull; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@AllArgsConstructor +public class DefaultDeserializer implements Deserializer { + + @NonNull + private final SeaTunnelRowType seaTunnelRowType; + @NonNull + private final Schema icebergSchema; + + @Override + public SeaTunnelRow deserialize(@NonNull Record record) { + SeaTunnelRow seaTunnelRow = new SeaTunnelRow(seaTunnelRowType.getTotalFields()); + for (int i = 0; i < seaTunnelRowType.getTotalFields(); i++) { + String seaTunnelFieldName = seaTunnelRowType.getFieldName(i); + SeaTunnelDataType seaTunnelFieldType = seaTunnelRowType.getFieldType(i); + Types.NestedField icebergField = icebergSchema.findField(seaTunnelFieldName); + Object icebergValue = record.getField(seaTunnelFieldName); + + seaTunnelRow.setField(i, convert(icebergField.type(), icebergValue, seaTunnelFieldType)); + } + return seaTunnelRow; + } + + private Object convert(@NonNull Type icebergType, + Object icebergValue, + @NonNull SeaTunnelDataType seaTunnelType) { + if (icebergValue == null) { + return null; + } + switch (icebergType.typeId()) { + case BOOLEAN: + return Boolean.class.cast(icebergValue); + case INTEGER: + return Integer.class.cast(icebergValue); + case LONG: + return Long.class.cast(icebergValue); + case FLOAT: + return Float.class.cast(icebergValue); + case DOUBLE: + return Double.class.cast(icebergValue); + case DATE: + return LocalDate.class.cast(icebergValue); + case TIME: + return LocalTime.class.cast(icebergValue); + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) icebergType; + if (timestampType.shouldAdjustToUTC()) { + return OffsetDateTime.class.cast(icebergValue).toLocalDateTime(); + } + return LocalDateTime.class.cast(icebergValue); + case STRING: + return String.class.cast(icebergValue); + case FIXED: + return byte[].class.cast(icebergValue); + case BINARY: + return ByteBuffer.class.cast(icebergValue).array(); + case DECIMAL: + return BigDecimal.class.cast(icebergValue); + case STRUCT: + Record icebergStruct = Record.class.cast(icebergValue); + Types.StructType icebergStructType = (Types.StructType) icebergType; + SeaTunnelRowType seaTunnelRowType = (SeaTunnelRowType) seaTunnelType; + SeaTunnelRow seatunnelRow = new SeaTunnelRow(seaTunnelRowType.getTotalFields()); + for (int i = 0; i < seaTunnelRowType.getTotalFields(); i++) { + String seatunnelFieldName = seaTunnelRowType.getFieldName(i); + Object seatunnelFieldValue = convert(icebergStructType.fieldType(seatunnelFieldName), + icebergStruct.getField(seatunnelFieldName), + seaTunnelRowType.getFieldType(i)); + seatunnelRow.setField(i, seatunnelFieldValue); + } + return seatunnelRow; + case LIST: + List icebergList = List.class.cast(icebergValue); + Types.ListType icebergListType = (Types.ListType) icebergType; + List seatunnelList = new ArrayList(icebergList.size()); + ArrayType seatunnelListType = (ArrayType) seaTunnelType; + for (int i = 0; i < icebergList.size(); i++) { + seatunnelList.add(convert(icebergListType.elementType(), + icebergList.get(i), seatunnelListType.getElementType())); + } + return seatunnelList.toArray(); + case MAP: + Map icebergMap = Map.class.cast(icebergValue); + Types.MapType icebergMapType = (Types.MapType) icebergType; + Map seatunnelMap = new HashMap(); + MapType seatunnelMapType = (MapType) seaTunnelType; + for (Map.Entry entry : icebergMap.entrySet()) { + seatunnelMap.put( + convert(icebergMapType.keyType(), entry.getKey(), seatunnelMapType.getKeyType()), + convert(icebergMapType.valueType(), entry.getValue(), seatunnelMapType.getValueType())); + } + return seatunnelMap; + default: + throw new UnsupportedOperationException("Unsupported iceberg type: " + icebergType); + } + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/Deserializer.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/Deserializer.java new file mode 100644 index 00000000000..1f88713d5fc --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/Deserializer.java @@ -0,0 +1,27 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.data; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import org.apache.iceberg.data.Record; + +public interface Deserializer { + + SeaTunnelRow deserialize(Record record); +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergRecordProjection.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergRecordProjection.java new file mode 100644 index 00000000000..48be1b5b05d --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergRecordProjection.java @@ -0,0 +1,97 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.data; + +import lombok.NonNull; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; + +import java.util.HashMap; +import java.util.Map; + +public class IcebergRecordProjection implements Record { + + private final Record record; + private final Types.StructType structType; + private final Types.StructType projectStructType; + private final Map posMapping; + + public IcebergRecordProjection(@NonNull Record record, + @NonNull Types.StructType structType, + @NonNull Types.StructType projectStructType) { + Map posMapping = new HashMap<>(); + for (int projectPos = 0, len = projectStructType.fields().size(); + projectPos < len; projectPos++) { + Types.NestedField projectField = projectStructType.fields().get(projectPos); + + Types.NestedField field = structType.field(projectField.fieldId()); + int fieldPos = structType.fields().indexOf(field); + posMapping.put(projectPos, fieldPos); + } + + this.record = record; + this.structType = structType; + this.projectStructType = projectStructType; + this.posMapping = posMapping; + } + + @Override + public Types.StructType struct() { + return projectStructType; + } + + @Override + public Object getField(String name) { + return record.getField(name); + } + + @Override + public void setField(String name, Object value) { + record.setField(name, value); + } + + @Override + public Object get(int pos) { + return record.get(posMapping.get(pos)); + } + + @Override + public Record copy() { + return new IcebergRecordProjection(record.copy(), structType, projectStructType); + } + + @Override + public Record copy(Map overwriteValues) { + return new IcebergRecordProjection(record.copy(overwriteValues), structType, projectStructType); + } + + @Override + public int size() { + return projectStructType.fields().size(); + } + + @Override + public T get(int pos, Class javaClass) { + return record.get(posMapping.get(pos), javaClass); + } + + @Override + public void set(int pos, T value) { + record.set(posMapping.get(pos), value); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java new file mode 100644 index 00000000000..485ecaf88d3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/data/IcebergTypeMapper.java @@ -0,0 +1,111 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.data; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +import lombok.NonNull; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +import java.util.ArrayList; +import java.util.List; + +public class IcebergTypeMapper { + + public static SeaTunnelDataType mapping(@NonNull Type icebergType) { + switch (icebergType.typeId()) { + case BOOLEAN: + return BasicType.BOOLEAN_TYPE; + case INTEGER: + return BasicType.INT_TYPE; + case LONG: + return BasicType.LONG_TYPE; + case FLOAT: + return BasicType.FLOAT_TYPE; + case DOUBLE: + return BasicType.DOUBLE_TYPE; + case DATE: + return LocalTimeType.LOCAL_DATE_TYPE; + case TIME: + return LocalTimeType.LOCAL_TIME_TYPE; + case TIMESTAMP: + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + case STRING: + return BasicType.STRING_TYPE; + case FIXED: + case BINARY: + return PrimitiveByteArrayType.INSTANCE; + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) icebergType; + return new DecimalType(decimalType.precision(), decimalType.scale()); + case STRUCT: + return mappingStructType((Types.StructType) icebergType); + case LIST: + return mappingListType((Types.ListType) icebergType); + case MAP: + return mappingMapType((Types.MapType) icebergType); + default: + throw new UnsupportedOperationException( + "Unsupported iceberg data type: " + icebergType.typeId()); + } + } + + private static SeaTunnelRowType mappingStructType(Types.StructType structType) { + List fields = structType.fields(); + List fieldNames = new ArrayList<>(fields.size()); + List> fieldTypes = new ArrayList<>(fields.size()); + for (Types.NestedField field : fields) { + fieldNames.add(field.name()); + fieldTypes.add(mapping(field.type())); + } + return new SeaTunnelRowType(fieldNames.toArray(new String[0]), + fieldTypes.toArray(new SeaTunnelDataType[0])); + } + + private static ArrayType mappingListType(Types.ListType listType) { + switch (listType.elementType().typeId()) { + case BOOLEAN: + return ArrayType.BOOLEAN_ARRAY_TYPE; + case INTEGER: + return ArrayType.INT_ARRAY_TYPE; + case LONG: + return ArrayType.LONG_ARRAY_TYPE; + case FLOAT: + return ArrayType.FLOAT_ARRAY_TYPE; + case DOUBLE: + return ArrayType.DOUBLE_ARRAY_TYPE; + case STRING: + return ArrayType.STRING_ARRAY_TYPE; + default: + throw new UnsupportedOperationException( + "Unsupported iceberg list element type: " + listType.elementType().typeId()); + } + } + + private static MapType mappingMapType(Types.MapType mapType) { + return new MapType(mapping(mapType.keyType()), mapping(mapType.valueType())); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java new file mode 100644 index 00000000000..3ec0963684d --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/IcebergSource.java @@ -0,0 +1,178 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source; + +import static com.google.common.base.Preconditions.checkArgument; + +import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.common.PrepareFailException; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.config.CheckConfigUtil; +import org.apache.seatunnel.common.config.CheckResult; +import org.apache.seatunnel.common.constants.JobMode; +import org.apache.seatunnel.connectors.seatunnel.common.schema.SeaTunnelSchema; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.CommonConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.data.IcebergTypeMapper; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.IcebergBatchSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.IcebergSplitEnumeratorState; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.IcebergStreamSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanContext; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.reader.IcebergSourceReader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import com.google.auto.service.AutoService; +import lombok.SneakyThrows; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; + +import java.util.ArrayList; +import java.util.List; + +@AutoService(SeaTunnelSource.class) +public class IcebergSource implements SeaTunnelSource { + + private static final long serialVersionUID = 4343414808223919870L; + + private SourceConfig sourceConfig; + private Schema tableSchema; + private Schema projectedSchema; + private SeaTunnelRowType seaTunnelRowType; + private JobContext jobContext; + + @Override + public String getPluginName() { + return "Iceberg"; + } + + @Override + public void prepare(Config pluginConfig) throws PrepareFailException { + this.sourceConfig = SourceConfig.loadConfig(pluginConfig); + this.tableSchema = loadIcebergSchema(sourceConfig); + this.seaTunnelRowType = loadSeaTunnelRowType(tableSchema, pluginConfig); + this.projectedSchema = tableSchema.select(seaTunnelRowType.getFieldNames()); + } + + @SneakyThrows + private Schema loadIcebergSchema(SourceConfig sourceConfig) { + IcebergTableLoader icebergTableLoader = IcebergTableLoader.create(sourceConfig); + try { + icebergTableLoader.open(); + return icebergTableLoader.loadTable().schema(); + } finally { + icebergTableLoader.close(); + } + } + + private SeaTunnelRowType loadSeaTunnelRowType(Schema tableSchema, + Config pluginConfig) { + List columnNames = new ArrayList<>(tableSchema.columns().size()); + List> columnDataTypes = new ArrayList<>(tableSchema.columns().size()); + for (Types.NestedField column : tableSchema.columns()) { + columnNames.add(column.name()); + columnDataTypes.add(IcebergTypeMapper.mapping(column.type())); + } + SeaTunnelRowType originalRowType = new SeaTunnelRowType( + columnNames.toArray(new String[0]), + columnDataTypes.toArray(new SeaTunnelDataType[0])); + + CheckResult checkResult = CheckConfigUtil.checkAllExists(pluginConfig, CommonConfig.KEY_FIELDS); + if (checkResult.isSuccess()) { + SeaTunnelSchema configSchema = SeaTunnelSchema.buildWithConfig(pluginConfig); + SeaTunnelRowType projectedRowType = configSchema.getSeaTunnelRowType(); + for (int i = 0; i < projectedRowType.getFieldNames().length; i++) { + String fieldName = projectedRowType.getFieldName(i); + SeaTunnelDataType projectedFieldType = projectedRowType.getFieldType(i); + int originalFieldIndex = originalRowType.indexOf(fieldName); + SeaTunnelDataType originalFieldType = originalRowType.getFieldType(originalFieldIndex); + checkArgument( + projectedFieldType.equals(originalFieldType), + String.format("Illegal field: %s, original: %s <-> projected: %s", + fieldName, originalFieldType, projectedFieldType)); + } + return projectedRowType; + } + return originalRowType; + } + + @Override + public Boundedness getBoundedness() { + return JobMode.BATCH.equals(jobContext.getJobMode()) ? + Boundedness.BOUNDED : Boundedness.UNBOUNDED; + } + + @Override + public void setJobContext(JobContext jobContext) { + this.jobContext = jobContext; + } + + @Override + public SeaTunnelDataType getProducedType() { + return seaTunnelRowType; + } + + @Override + public SourceReader createReader( + SourceReader.Context readerContext) { + return new IcebergSourceReader(readerContext, + seaTunnelRowType, + tableSchema, + projectedSchema, + sourceConfig); + } + + @Override + public SourceSplitEnumerator createEnumerator( + SourceSplitEnumerator.Context enumeratorContext) { + if (Boundedness.BOUNDED.equals(getBoundedness())) { + return new IcebergBatchSplitEnumerator(enumeratorContext, + IcebergScanContext.scanContext(sourceConfig, projectedSchema), + sourceConfig, + null); + } + return new IcebergStreamSplitEnumerator(enumeratorContext, + IcebergScanContext.streamScanContext(sourceConfig, projectedSchema), + sourceConfig, + null); + } + + @Override + public SourceSplitEnumerator restoreEnumerator( + SourceSplitEnumerator.Context enumeratorContext, + IcebergSplitEnumeratorState checkpointState) { + if (Boundedness.BOUNDED.equals(getBoundedness())) { + return new IcebergBatchSplitEnumerator(enumeratorContext, + IcebergScanContext.scanContext(sourceConfig, projectedSchema), + sourceConfig, + checkpointState); + } + return new IcebergStreamSplitEnumerator(enumeratorContext, + IcebergScanContext.streamScanContext(sourceConfig, projectedSchema), + sourceConfig, + checkpointState); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/AbstractSplitEnumerator.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/AbstractSplitEnumerator.java new file mode 100644 index 00000000000..e84abdc8406 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/AbstractSplitEnumerator.java @@ -0,0 +1,130 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.iceberg.Table; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +@Slf4j +public abstract class AbstractSplitEnumerator implements SourceSplitEnumerator { + + protected final SourceSplitEnumerator.Context context; + protected final SourceConfig sourceConfig; + protected final Map> pendingSplits; + + protected IcebergTableLoader icebergTableLoader; + + public AbstractSplitEnumerator(@NonNull SourceSplitEnumerator.Context context, + @NonNull SourceConfig sourceConfig, + @NonNull Map> pendingSplits) { + this.context = context; + this.sourceConfig = sourceConfig; + this.pendingSplits = new HashMap<>(pendingSplits); + } + + @Override + public void open() { + icebergTableLoader = IcebergTableLoader.create(sourceConfig); + icebergTableLoader.open(); + } + + @Override + public void run() { + refreshPendingSplits(); + assignPendingSplits(context.registeredReaders()); + } + + @Override + public void close() throws IOException { + icebergTableLoader.close(); + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + addPendingSplits(splits); + if (context.registeredReaders().contains(subtaskId)) { + assignPendingSplits(Collections.singleton(subtaskId)); + } + } + + @Override + public int currentUnassignedSplitSize() { + return pendingSplits.size(); + } + + @Override + public void registerReader(int subtaskId) { + log.debug("Adding reader {} to IcebergSourceEnumerator.", + subtaskId); + assignPendingSplits(Collections.singleton(subtaskId)); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + } + + protected void refreshPendingSplits() { + List newSplits = loadNewSplits(icebergTableLoader.loadTable()); + addPendingSplits(newSplits); + } + + protected abstract List loadNewSplits(Table table); + + private void addPendingSplits(Collection newSplits) { + int numReaders = context.currentParallelism(); + for (IcebergFileScanTaskSplit newSplit : newSplits) { + int ownerReader = newSplit.splitId().hashCode() % numReaders; + pendingSplits + .computeIfAbsent(ownerReader, r -> new ArrayList<>()) + .add(newSplit); + log.info("Assigning {} to {} reader.", newSplit, ownerReader); + } + } + + protected void assignPendingSplits(Set pendingReaders) { + for (int pendingReader : pendingReaders) { + List pendingAssignmentForReader = pendingSplits.remove(pendingReader); + if (pendingAssignmentForReader != null && !pendingAssignmentForReader.isEmpty()) { + log.info("Assign splits {} to reader {}", + pendingAssignmentForReader, pendingReader); + try { + context.assignSplit(pendingReader, pendingAssignmentForReader); + } catch (Exception e) { + log.error("Failed to assign splits {} to reader {}", + pendingAssignmentForReader, pendingReader, e); + pendingSplits.put(pendingReader, pendingAssignmentForReader); + } + } + } + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergBatchSplitEnumerator.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergBatchSplitEnumerator.java new file mode 100644 index 00000000000..9c09e8febd4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergBatchSplitEnumerator.java @@ -0,0 +1,71 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanContext; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanSplitPlanner; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.iceberg.Table; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +@Slf4j +public class IcebergBatchSplitEnumerator extends AbstractSplitEnumerator { + + private final IcebergScanContext icebergScanContext; + + public IcebergBatchSplitEnumerator(@NonNull SourceSplitEnumerator.Context context, + @NonNull IcebergScanContext icebergScanContext, + @NonNull SourceConfig sourceConfig, + IcebergSplitEnumeratorState restoreState) { + super(context, sourceConfig, restoreState != null ? + restoreState.getPendingSplits() : Collections.EMPTY_MAP); + this.icebergScanContext = icebergScanContext; + } + + @Override + public void run() { + super.run(); + + Set readers = context.registeredReaders(); + log.debug("No more splits to assign." + + " Sending NoMoreSplitsEvent to reader {}.", readers); + readers.forEach(context::signalNoMoreSplits); + } + + @Override + public IcebergSplitEnumeratorState snapshotState(long checkpointId) { + return new IcebergSplitEnumeratorState(null, pendingSplits); + } + + @Override + public void handleSplitRequest(int subtaskId) { + } + + @Override + protected List loadNewSplits(Table table) { + return IcebergScanSplitPlanner.planSplits(table, icebergScanContext); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumerationResult.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumerationResult.java new file mode 100644 index 00000000000..9b4338cc082 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumerationResult.java @@ -0,0 +1,38 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NonNull; +import lombok.ToString; + +import java.util.List; + +@Getter +@ToString +@AllArgsConstructor +public class IcebergEnumerationResult { + @NonNull + private final List splits; + private final IcebergEnumeratorPosition fromPosition; + @NonNull + private final IcebergEnumeratorPosition toPosition; +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumeratorPosition.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumeratorPosition.java new file mode 100644 index 00000000000..3161545c6d2 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergEnumeratorPosition.java @@ -0,0 +1,39 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator; + +import lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; + +import java.io.Serializable; + +@Getter +@AllArgsConstructor +@EqualsAndHashCode +@ToString +public class IcebergEnumeratorPosition implements Serializable { + + private static final long serialVersionUID = 5703291468632501375L; + + public static final IcebergEnumeratorPosition EMPTY = new IcebergEnumeratorPosition(null, null); + + private final Long snapshotId; + private final Long snapshotTimestampMs; +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergSplitEnumeratorState.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergSplitEnumeratorState.java new file mode 100644 index 00000000000..4b170989898 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergSplitEnumeratorState.java @@ -0,0 +1,41 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +@Getter +@Setter +@AllArgsConstructor +@ToString +public class IcebergSplitEnumeratorState implements Serializable { + + private static final long serialVersionUID = -529307606400995298L; + + private final IcebergEnumeratorPosition lastEnumeratedPosition; + private final Map> pendingSplits; +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergStreamSplitEnumerator.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergStreamSplitEnumerator.java new file mode 100644 index 00000000000..9ced61e3a13 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/IcebergStreamSplitEnumerator.java @@ -0,0 +1,85 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanContext; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan.IcebergScanSplitPlanner; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.iceberg.Table; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; + +@Slf4j +public class IcebergStreamSplitEnumerator extends AbstractSplitEnumerator { + + private final IcebergScanContext icebergScanContext; + private final AtomicReference enumeratorPosition; + + public IcebergStreamSplitEnumerator(@NonNull SourceSplitEnumerator.Context context, + @NonNull IcebergScanContext icebergScanContext, + @NonNull SourceConfig sourceConfig, + IcebergSplitEnumeratorState restoreState) { + super(context, sourceConfig, restoreState != null ? + restoreState.getPendingSplits() : Collections.EMPTY_MAP); + this.icebergScanContext = icebergScanContext; + this.enumeratorPosition = new AtomicReference<>(); + if (restoreState != null) { + enumeratorPosition.set(restoreState.getLastEnumeratedPosition()); + } + } + + @Override + public IcebergSplitEnumeratorState snapshotState(long checkpointId) throws Exception { + return new IcebergSplitEnumeratorState(enumeratorPosition.get(), pendingSplits); + } + + @Override + public void handleSplitRequest(int subtaskId) { + synchronized (this) { + if (pendingSplits.isEmpty() || + pendingSplits.get(subtaskId) == null) { + refreshPendingSplits(); + } + assignPendingSplits(Collections.singleton(subtaskId)); + } + } + + @Override + protected List loadNewSplits(Table table) { + IcebergEnumerationResult result = IcebergScanSplitPlanner.planStreamSplits( + table, icebergScanContext, enumeratorPosition.get()); + if (!Objects.equals(result.getFromPosition(), enumeratorPosition.get())) { + log.info("Skip {} loaded splits because the scan starting position doesn't match " + + "the current enumerator position: enumerator position = {}, scan starting position = {}", + result.getSplits().size(), enumeratorPosition.get(), result.getFromPosition()); + return Collections.emptyList(); + } else { + enumeratorPosition.set(result.getToPosition()); + log.debug("Update enumerator position to {}", result.getToPosition()); + return result.getSplits(); + } + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanContext.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanContext.java new file mode 100644 index 00000000000..8e3904108b9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanContext.java @@ -0,0 +1,85 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; + +import lombok.Builder; +import lombok.Getter; +import lombok.ToString; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; + +@Getter +@Builder(toBuilder = true) +@ToString +public class IcebergScanContext { + + private final boolean streaming; + private final IcebergStreamScanStrategy streamScanStrategy; + + private final Long startSnapshotId; + private final Long startSnapshotTimestamp; + private final Long endSnapshotId; + + private final Long useSnapshotId; + private final Long useSnapshotTimestamp; + + private final boolean caseSensitive; + + private final Schema schema; + private final Expression filter; + private final Long splitSize; + private final Integer splitLookback; + private final Long splitOpenFileCost; + + public IcebergScanContext copyWithAppendsBetween(Long newStartSnapshotId, + long newEndSnapshotId) { + return this.toBuilder() + .useSnapshotId(null) + .useSnapshotTimestamp(null) + .startSnapshotId(newStartSnapshotId) + .endSnapshotId(newEndSnapshotId) + .build(); + } + + public static IcebergScanContext scanContext(SourceConfig sourceConfig, + Schema schema) { + return IcebergScanContext.builder() + .startSnapshotTimestamp(sourceConfig.getStartSnapshotTimestamp()) + .startSnapshotId(sourceConfig.getStartSnapshotId()) + .endSnapshotId(sourceConfig.getEndSnapshotId()) + .useSnapshotId(sourceConfig.getUseSnapshotId()) + .useSnapshotTimestamp(sourceConfig.getUseSnapshotTimestamp()) + .caseSensitive(sourceConfig.isCaseSensitive()) + .schema(schema) + .filter(sourceConfig.getFilter()) + .splitSize(sourceConfig.getSplitSize()) + .splitLookback(sourceConfig.getSplitLookback()) + .splitOpenFileCost(sourceConfig.getSplitOpenFileCost()) + .build(); + } + + public static IcebergScanContext streamScanContext(SourceConfig sourceConfig, + Schema schema) { + return scanContext(sourceConfig, schema).toBuilder() + .streaming(true) + .streamScanStrategy(sourceConfig.getStreamScanStrategy()) + .build(); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanSplitPlanner.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanSplitPlanner.java new file mode 100644 index 00000000000..f3d29f81451 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergScanSplitPlanner.java @@ -0,0 +1,207 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan; + +import static com.google.common.base.Preconditions.checkArgument; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.IcebergEnumerationResult; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.enumerator.IcebergEnumeratorPosition; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import lombok.extern.slf4j.Slf4j; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.IncrementalAppendScan; +import org.apache.iceberg.Scan; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.util.SnapshotUtil; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +@Slf4j +public class IcebergScanSplitPlanner { + + public static IcebergEnumerationResult planStreamSplits(Table table, + IcebergScanContext icebergScanContext, + IcebergEnumeratorPosition lastPosition) { + // Load increment files + table.refresh(); + + if (lastPosition == null) { + return initialStreamSplits(table, icebergScanContext); + } + return incrementalStreamSplits(table, icebergScanContext, lastPosition); + } + + private static IcebergEnumerationResult incrementalStreamSplits(Table table, + IcebergScanContext icebergScanContext, + IcebergEnumeratorPosition lastPosition) { + Snapshot currentSnapshot = table.currentSnapshot(); + if (currentSnapshot == null) { + checkArgument(lastPosition.getSnapshotId() == null, + "Invalid last enumerated position for an empty table: not null"); + log.info("Skip incremental scan because table is empty"); + return new IcebergEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); + } else if (lastPosition.getSnapshotId() != null + && currentSnapshot.snapshotId() == lastPosition.getSnapshotId()) { + log.debug("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); + return new IcebergEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); + } + + IcebergEnumeratorPosition newPosition = new IcebergEnumeratorPosition( + currentSnapshot.snapshotId(), currentSnapshot.timestampMillis()); + IcebergScanContext incrementalScan = icebergScanContext + .copyWithAppendsBetween(lastPosition.getSnapshotId(), currentSnapshot.snapshotId()); + List splits = planSplits(table, incrementalScan); + log.info("Discovered {} splits from incremental scan: " + + "from snapshot (exclusive) is {}, to snapshot (inclusive) is {}", + splits.size(), lastPosition, newPosition); + return new IcebergEnumerationResult(splits, lastPosition, newPosition); + } + + private static IcebergEnumerationResult initialStreamSplits(Table table, + IcebergScanContext icebergScanContext) { + Optional startSnapshotOptional = getStreamStartSnapshot(table, icebergScanContext); + if (!startSnapshotOptional.isPresent()) { + return new IcebergEnumerationResult(Collections.emptyList(), + null, IcebergEnumeratorPosition.EMPTY); + } + + Snapshot startSnapshot = startSnapshotOptional.get(); + List splits = Collections.emptyList(); + IcebergEnumeratorPosition toPosition = IcebergEnumeratorPosition.EMPTY; + if (IcebergStreamScanStrategy.TABLE_SCAN_THEN_INCREMENTAL.equals(icebergScanContext.getStreamScanStrategy())) { + splits = planSplits(table, icebergScanContext); + log.info("Discovered {} splits from initial batch table scan with snapshot Id {}", + splits.size(), startSnapshot.snapshotId()); + + toPosition = new IcebergEnumeratorPosition(startSnapshot.snapshotId(), + startSnapshot.timestampMillis()); + } else { + Long parentSnapshotId = startSnapshot.parentId(); + if (parentSnapshotId != null) { + Snapshot parentSnapshot = table.snapshot(parentSnapshotId); + Long parentSnapshotTimestampMs = parentSnapshot != null ? parentSnapshot.timestampMillis() : null; + toPosition = new IcebergEnumeratorPosition(parentSnapshotId, parentSnapshotTimestampMs); + } + log.info("Start incremental scan with start snapshot (inclusive): id = {}, timestamp = {}", + startSnapshot.snapshotId(), startSnapshot.timestampMillis()); + } + + return new IcebergEnumerationResult(splits, null, toPosition); + } + + private static Optional getStreamStartSnapshot(Table table, + IcebergScanContext icebergScanContext) { + switch (icebergScanContext.getStreamScanStrategy()) { + case TABLE_SCAN_THEN_INCREMENTAL: + case FROM_LATEST_SNAPSHOT: + return Optional.ofNullable(table.currentSnapshot()); + case FROM_EARLIEST_SNAPSHOT: + return Optional.ofNullable(SnapshotUtil.oldestAncestor(table)); + case FROM_SNAPSHOT_ID: + return Optional.of(table.snapshot(icebergScanContext.getStartSnapshotId())); + case FROM_SNAPSHOT_TIMESTAMP: + long snapshotIdAsOfTime = SnapshotUtil.snapshotIdAsOfTime(table, + icebergScanContext.getStartSnapshotTimestamp()); + Snapshot matchedSnapshot = table.snapshot(snapshotIdAsOfTime); + if (matchedSnapshot.timestampMillis() == icebergScanContext.getStartSnapshotTimestamp()) { + return Optional.of(matchedSnapshot); + } else { + return Optional.of(SnapshotUtil.snapshotAfter(table, snapshotIdAsOfTime)); + } + default: + throw new UnsupportedOperationException("Unsupported stream scan strategy: " + + icebergScanContext.getStreamScanStrategy()); + } + } + + public static List planSplits(Table table, + IcebergScanContext context) { + try (CloseableIterable tasksIterable = planTasks(table, context)) { + List splits = new ArrayList<>(); + for (CombinedScanTask combinedScanTask : tasksIterable) { + for (FileScanTask fileScanTask : combinedScanTask.files()) { + splits.add(new IcebergFileScanTaskSplit(fileScanTask)); + } + } + return splits; + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to scan iceberg splits from: " + table.name(), e); + } + } + + private static CloseableIterable planTasks(Table table, + IcebergScanContext context) { + if (context.isStreaming() + || context.getStartSnapshotId() != null + || context.getEndSnapshotId() != null) { + IncrementalAppendScan scan = table.newIncrementalAppendScan(); + scan = rebuildScanWithBaseConfig(scan, context); + if (context.getStartSnapshotId() != null) { + scan = scan.fromSnapshotExclusive(context.getStartSnapshotId()); + } + if (context.getEndSnapshotId() != null) { + scan = scan.toSnapshot(context.getEndSnapshotId()); + } + return scan.planTasks(); + } else { + TableScan scan = table.newScan(); + scan = rebuildScanWithBaseConfig(scan, context); + if (context.getUseSnapshotId() != null) { + scan = scan.useSnapshot(context.getUseSnapshotId()); + } + if (context.getUseSnapshotTimestamp() != null) { + scan = scan.asOfTime(context.getUseSnapshotTimestamp()); + } + return scan.planTasks(); + } + } + + private static > T rebuildScanWithBaseConfig( + T scan, IcebergScanContext context) { + T newScan = scan.caseSensitive(context.isCaseSensitive()) + .project(context.getSchema()); + if (context.getFilter() != null) { + newScan = newScan.filter(context.getFilter()); + } + if (context.getSplitSize() != null) { + newScan = newScan.option(TableProperties.SPLIT_SIZE, + context.getSplitSize().toString()); + } + if (context.getSplitLookback() != null) { + newScan = newScan.option(TableProperties.SPLIT_LOOKBACK, + context.getSplitLookback().toString()); + } + if (context.getSplitOpenFileCost() != null) { + newScan = newScan.option(TableProperties.SPLIT_OPEN_FILE_COST, + context.getSplitOpenFileCost().toString()); + } + return newScan; + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergStreamScanStrategy.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergStreamScanStrategy.java new file mode 100644 index 00000000000..8692b9993cb --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/enumerator/scan/IcebergStreamScanStrategy.java @@ -0,0 +1,41 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.enumerator.scan; + +public enum IcebergStreamScanStrategy { + /** + * Do a regular table scan then switch to the incremental mode. + */ + TABLE_SCAN_THEN_INCREMENTAL, + /** + * Start incremental mode from the latest snapshot inclusive. + */ + FROM_LATEST_SNAPSHOT, + /** + * Start incremental mode from the earliest snapshot inclusive. + */ + FROM_EARLIEST_SNAPSHOT, + /** + * Start incremental mode from a snapshot with a specific id inclusive. + */ + FROM_SNAPSHOT_ID, + /** + * Start incremental mode from a snapshot with a specific timestamp inclusive. + */ + FROM_SNAPSHOT_TIMESTAMP +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskReader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskReader.java new file mode 100644 index 00000000000..a50706847cf --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskReader.java @@ -0,0 +1,143 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.reader; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.data.IcebergRecordProjection; + +import com.google.common.collect.Sets; +import lombok.Builder; +import lombok.NonNull; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.data.GenericDeleteFilter; +import org.apache.iceberg.data.IdentityPartitionConverters; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PartitionUtil; + +import java.io.Closeable; +import java.util.Map; + +@Builder +public class IcebergFileScanTaskReader implements Closeable { + + private final FileIO fileIO; + private final Schema tableSchema; + private final Schema projectedSchema; + private final boolean caseSensitive; + private final boolean reuseContainers; + + public CloseableIterator open(@NonNull FileScanTask task) { + CloseableIterable iterable = icebergGenericRead(task); + return iterable.iterator(); + } + + private CloseableIterable icebergGenericRead(FileScanTask task) { + DeleteFilter deletes = new GenericDeleteFilter(fileIO, task, tableSchema, projectedSchema); + Schema readSchema = deletes.requiredSchema(); + + CloseableIterable records = openFile(task, readSchema); + records = deletes.filter(records); + records = applyResidual(records, readSchema, task.residual()); + + if (!projectedSchema.sameSchema(readSchema)) { + // filter metadata columns + records = CloseableIterable.transform(records, record -> + new IcebergRecordProjection(record, readSchema.asStruct(), projectedSchema.asStruct())); + } + return records; + } + + private CloseableIterable applyResidual(CloseableIterable records, + Schema recordSchema, + Expression residual) { + if (residual != null && residual != Expressions.alwaysTrue()) { + InternalRecordWrapper wrapper = new InternalRecordWrapper(recordSchema.asStruct()); + Evaluator filter = new Evaluator(recordSchema.asStruct(), residual, caseSensitive); + return CloseableIterable.filter(records, record -> filter.eval(wrapper.wrap(record))); + } + + return records; + } + + private CloseableIterable openFile(FileScanTask task, Schema fileProjection) { + if (task.isDataTask()) { + throw new UnsupportedOperationException("Cannot read data task."); + } + InputFile input = fileIO.newInputFile(task.file().path().toString()); + Map partition = PartitionUtil.constantsMap(task, IdentityPartitionConverters::convertConstant); + + switch (task.file().format()) { + case AVRO: + Avro.ReadBuilder avro = Avro.read(input) + .project(fileProjection) + .createReaderFunc( + avroSchema -> DataReader.create(fileProjection, avroSchema, partition)) + .split(task.start(), task.length()); + if (reuseContainers) { + avro.reuseContainers(); + } + return avro.build(); + case PARQUET: + Parquet.ReadBuilder parquet = Parquet.read(input) + .caseSensitive(caseSensitive) + .project(fileProjection) + .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(fileProjection, fileSchema, partition)) + .split(task.start(), task.length()) + .filter(task.residual()); + if (reuseContainers) { + parquet.reuseContainers(); + } + return parquet.build(); + case ORC: + Schema projectionWithoutConstantAndMetadataFields = TypeUtil.selectNot(fileProjection, + Sets.union(partition.keySet(), MetadataColumns.metadataFieldIds())); + ORC.ReadBuilder orc = ORC.read(input) + .caseSensitive(caseSensitive) + .project(projectionWithoutConstantAndMetadataFields) + .createReaderFunc(fileSchema -> GenericOrcReader.buildReader(fileProjection, fileSchema, partition)) + .split(task.start(), task.length()) + .filter(task.residual()); + return orc.build(); + default: + throw new UnsupportedOperationException(String.format("Cannot read %s file: %s", + task.file().format().name(), task.file().path())); + } + } + + @Override + public void close() { + fileIO.close(); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskSplitReader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskSplitReader.java new file mode 100644 index 00000000000..c170dbd90ef --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergFileScanTaskSplitReader.java @@ -0,0 +1,86 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.reader; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.iceberg.data.Deserializer; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import lombok.AllArgsConstructor; +import lombok.NonNull; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterator; + +import java.io.Closeable; +import java.io.IOException; + +@AllArgsConstructor +public class IcebergFileScanTaskSplitReader implements Closeable { + + private Deserializer deserializer; + private IcebergFileScanTaskReader icebergFileScanTaskReader; + + public CloseableIterator open(@NonNull IcebergFileScanTaskSplit split) { + CloseableIterator iterator = icebergFileScanTaskReader.open(split.getTask()); + + OffsetSeekIterator seekIterator = new OffsetSeekIterator(iterator); + seekIterator.seek(split.getRecordOffset()); + + return CloseableIterator.transform(seekIterator, record -> { + SeaTunnelRow seaTunnelRow = deserializer.deserialize(record); + split.setRecordOffset(split.getRecordOffset() + 1); + return seaTunnelRow; + }); + } + + @Override + public void close() { + icebergFileScanTaskReader.close(); + } + + @AllArgsConstructor + private static class OffsetSeekIterator implements CloseableIterator { + private final CloseableIterator iterator; + + public void seek(long startingRecordOffset) { + for (long i = 0; i < startingRecordOffset; ++i) { + if (hasNext()) { + next(); + } else { + throw new IllegalStateException(String.format( + "Invalid starting record offset %d", startingRecordOffset)); + } + } + } + + @Override + public void close() throws IOException { + iterator.close(); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public T next() { + return iterator.next(); + } + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergSourceReader.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergSourceReader.java new file mode 100644 index 00000000000..f432c5e2235 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/reader/IcebergSourceReader.java @@ -0,0 +1,145 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.reader; + +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergTableLoader; +import org.apache.seatunnel.connectors.seatunnel.iceberg.config.SourceConfig; +import org.apache.seatunnel.connectors.seatunnel.iceberg.data.DefaultDeserializer; +import org.apache.seatunnel.connectors.seatunnel.iceberg.data.Deserializer; +import org.apache.seatunnel.connectors.seatunnel.iceberg.source.split.IcebergFileScanTaskSplit; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + +@Slf4j +public class IcebergSourceReader implements SourceReader { + + private static final long POLL_WAIT_MS = 1000; + + private final SourceReader.Context context; + private final Queue pendingSplits; + private final Deserializer deserializer; + private final Schema tableSchema; + private final Schema projectedSchema; + private final SourceConfig sourceConfig; + + private IcebergTableLoader icebergTableLoader; + private IcebergFileScanTaskSplitReader icebergFileScanTaskSplitReader; + + private IcebergFileScanTaskSplit currentReadSplit; + private boolean noMoreSplitsAssignment; + + public IcebergSourceReader(@NonNull SourceReader.Context context, + @NonNull SeaTunnelRowType seaTunnelRowType, + @NonNull Schema tableSchema, + @NonNull Schema projectedSchema, + @NonNull SourceConfig sourceConfig) { + this.context = context; + this.pendingSplits = new LinkedList<>(); + this.deserializer = new DefaultDeserializer(seaTunnelRowType, projectedSchema); + this.tableSchema = tableSchema; + this.projectedSchema = projectedSchema; + this.sourceConfig = sourceConfig; + } + + @Override + public void open() { + icebergTableLoader = IcebergTableLoader.create(sourceConfig); + icebergTableLoader.open(); + + icebergFileScanTaskSplitReader = new IcebergFileScanTaskSplitReader(deserializer, + IcebergFileScanTaskReader.builder() + .fileIO(icebergTableLoader.loadTable().io()) + .tableSchema(tableSchema) + .projectedSchema(projectedSchema) + .caseSensitive(sourceConfig.isCaseSensitive()) + .reuseContainers(true) + .build()); + } + + @Override + public void close() throws IOException { + if (icebergFileScanTaskSplitReader != null) { + icebergFileScanTaskSplitReader.close(); + } + icebergTableLoader.close(); + } + + @Override + public void pollNext(Collector output) throws Exception { + for (IcebergFileScanTaskSplit pendingSplit = pendingSplits.poll(); + pendingSplit != null; pendingSplit = pendingSplits.poll()) { + currentReadSplit = pendingSplit; + try (CloseableIterator rowIterator = icebergFileScanTaskSplitReader.open(currentReadSplit)) { + while (rowIterator.hasNext()) { + output.collect(rowIterator.next()); + } + } + } + + if (noMoreSplitsAssignment && Boundedness.BOUNDED.equals(context.getBoundedness())) { + context.signalNoMoreElement(); + } else { + context.sendSplitRequest(); + if (pendingSplits.isEmpty()) { + Thread.sleep(POLL_WAIT_MS); + } + } + } + + @Override + public List snapshotState(long checkpointId) { + List readerState = new ArrayList<>(); + if (!pendingSplits.isEmpty()) { + readerState.addAll(pendingSplits); + } + if (currentReadSplit != null) { + readerState.add(currentReadSplit); + } + return readerState; + } + + @Override + public void addSplits(List splits) { + log.info("Add {} splits to reader", splits.size()); + pendingSplits.addAll(splits); + } + + @Override + public void handleNoMoreSplits() { + log.info("Reader received NoMoreSplits event."); + noMoreSplitsAssignment = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/split/IcebergFileScanTaskSplit.java b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/split/IcebergFileScanTaskSplit.java new file mode 100644 index 00000000000..19653dfef47 --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/main/java/org/apache/seatunnel/connectors/seatunnel/iceberg/source/split/IcebergFileScanTaskSplit.java @@ -0,0 +1,68 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg.source.split; + +import org.apache.seatunnel.api.source.SourceSplit; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NonNull; +import lombok.Setter; +import org.apache.iceberg.FileScanTask; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +@Getter +@AllArgsConstructor +public class IcebergFileScanTaskSplit implements SourceSplit { + + private static final long serialVersionUID = -9043797960947110643L; + + private final FileScanTask task; + @Setter + private volatile long recordOffset; + + public IcebergFileScanTaskSplit(@NonNull FileScanTask task) { + this(task, 0); + } + + @Override + public String splitId() { + return task.file().path().toString(); + } + + @Override + public String toString() { + return "IcebergFileScanTaskSplit{" + + "task=" + toString(task) + + ", recordOffset=" + recordOffset + + '}'; + } + + private String toString(FileScanTask task) { + Map taskInfo = new HashMap<>(); + taskInfo.put("file", task.file().path().toString()); + taskInfo.put("start", task.start()); + taskInfo.put("length", task.length()); + taskInfo.put("deletes", task.deletes().stream().map( + deleteFile -> deleteFile.path()).collect(Collectors.toList())); + return taskInfo.toString(); + } +} diff --git a/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/TestIcebergMetastore.java b/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/TestIcebergMetastore.java new file mode 100644 index 00000000000..a8429fc3d3b --- /dev/null +++ b/seatunnel-connectors-v2/connector-iceberg/src/test/java/org/apache/seatunnel/connectors/seatunnel/iceberg/TestIcebergMetastore.java @@ -0,0 +1,63 @@ +/* + * 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.seatunnel.connectors.seatunnel.iceberg; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import java.io.File; + +public class TestIcebergMetastore { + + private static TestHiveMetastore METASTORE = null; + private static String METASTORE_URI; + + @BeforeEach + public void start() { + METASTORE = new TestHiveMetastore(); + METASTORE.start(); + METASTORE_URI = METASTORE.hiveConf().get(HiveConf.ConfVars.METASTOREURIS.varname); + } + + @Disabled("Disabled because system environment does not support to run this test") + @Test + public void testUseHiveMetastore() { + String warehousePath = "/tmp/seatunnel/iceberg/hive/"; + new File(warehousePath).mkdirs(); + + HiveCatalog catalog = (HiveCatalog) new IcebergCatalogFactory("seatunnel", + "hive", + "file://" + warehousePath, + METASTORE_URI) + .create(); + catalog.createNamespace(Namespace.of("test_database")); + Assertions.assertTrue(catalog.namespaceExists(Namespace.of("test_database"))); + } + + @AfterEach + public void close() throws Exception { + METASTORE.stop(); + } +} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index 9f424ff960b..4b6c1c3f2d5 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -54,6 +54,7 @@ connector-datahub connector-sentry connector-mongodb + connector-iceberg diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 03c85791381..b9def6bc78d 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -260,6 +260,12 @@ ${project.version} provided + + org.apache.seatunnel + connector-iceberg + ${project.version} + provided + diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/pom.xml new file mode 100644 index 00000000000..fc016ed71e4 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/pom.xml @@ -0,0 +1,45 @@ + + + + + org.apache.seatunnel + seatunnel-flink-connector-v2-e2e + ${revision} + + 4.0.0 + + connector-iceberg-flink-e2e + + + + org.apache.seatunnel + connector-flink-e2e-base + ${project.version} + tests + test-jar + test + + + org.apache.seatunnel + connector-iceberg + ${project.version} + test + + + + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iceberg/IcebergSourceIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iceberg/IcebergSourceIT.java new file mode 100644 index 00000000000..c67f3a5a321 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iceberg/IcebergSourceIT.java @@ -0,0 +1,165 @@ +/* + * 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.seatunnel.e2e.flink.v2.iceberg; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergCatalogFactory; +import org.apache.seatunnel.e2e.flink.FlinkContainer; + +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.Container; +import org.testcontainers.utility.MountableFile; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +@Slf4j +public class IcebergSourceIT extends FlinkContainer { + + private static final TableIdentifier TABLE = TableIdentifier.of( + Namespace.of("database1"), "source"); + private static final Schema SCHEMA = new Schema( + Types.NestedField.optional(1, "f1", Types.LongType.get()), + Types.NestedField.optional(2, "f2", Types.BooleanType.get()), + Types.NestedField.optional(3, "f3", Types.IntegerType.get()), + Types.NestedField.optional(4, "f4", Types.LongType.get()), + Types.NestedField.optional(5, "f5", Types.FloatType.get()), + Types.NestedField.optional(6, "f6", Types.DoubleType.get()), + Types.NestedField.optional(7, "f7", Types.DateType.get()), + Types.NestedField.optional(8, "f8", Types.TimeType.get()), + Types.NestedField.optional(9, "f9", Types.TimestampType.withZone()), + Types.NestedField.optional(10, "f10", Types.TimestampType.withoutZone()), + Types.NestedField.optional(11, "f11", Types.StringType.get()), + Types.NestedField.optional(12, "f12", Types.FixedType.ofLength(10)), + Types.NestedField.optional(13, "f13", Types.BinaryType.get()), + Types.NestedField.optional(14, "f14", Types.DecimalType.of(19, 9)), + Types.NestedField.optional(15, "f15", Types.ListType.ofOptional( + 100, Types.IntegerType.get())), + Types.NestedField.optional(16, "f16", Types.MapType.ofOptional( + 200, 300, Types.StringType.get(), Types.IntegerType.get())), + Types.NestedField.optional(17, "f17", Types.StructType.of( + Types.NestedField.required(400, "f17_a", Types.StringType.get()))) + ); + + private static final String CATALOG_NAME = "seatunnel"; + private static final String CATALOG_TYPE = "hadoop"; + private static final String CATALOG_DIR = "/tmp/seatunnel/iceberg/flink/"; + private static final String WAREHOUSE = "file://" + CATALOG_DIR; + private static Catalog CATALOG; + + @BeforeEach + public void start() { + initializeIcebergTable(); + batchInsertData(); + MountableFile catalogPath = MountableFile.forHostPath(CATALOG_DIR); + jobManager.copyFileToContainer(catalogPath, CATALOG_DIR); + taskManager.copyFileToContainer(catalogPath, CATALOG_DIR); + } + + @Test + public void testIcebergSource() throws IOException, InterruptedException { + Container.ExecResult execResult = executeSeaTunnelFlinkJob("/iceberg/iceberg_source.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + private void initializeIcebergTable() { + CATALOG = new IcebergCatalogFactory(CATALOG_NAME, + CATALOG_TYPE, + WAREHOUSE, + null) + .create(); + if (!CATALOG.tableExists(TABLE)) { + CATALOG.createTable(TABLE, SCHEMA); + } + } + + private void batchInsertData() { + GenericRecord record = GenericRecord.create(SCHEMA); + record.setField("f1", Long.valueOf(0)); + record.setField("f2", true); + record.setField("f3", Integer.MAX_VALUE); + record.setField("f4", Long.MAX_VALUE); + record.setField("f5", Float.MAX_VALUE); + record.setField("f6", Double.MAX_VALUE); + record.setField("f7", LocalDate.now()); + record.setField("f8", LocalTime.now()); + record.setField("f9", OffsetDateTime.now()); + record.setField("f10", LocalDateTime.now()); + record.setField("f11", "test"); + record.setField("f12", "abcdefghij".getBytes()); + record.setField("f13", ByteBuffer.wrap("test".getBytes())); + record.setField("f14", new BigDecimal("1000000000.000000001")); + record.setField("f15", Arrays.asList(Integer.MAX_VALUE)); + record.setField("f16", Collections.singletonMap("key", Integer.MAX_VALUE)); + Record structRecord = GenericRecord.create(SCHEMA.findField("f17").type().asStructType()); + structRecord.setField("f17_a", "test"); + record.setField("f17", structRecord); + + Table table = CATALOG.loadTable(TABLE); + FileAppenderFactory appenderFactory = new GenericAppenderFactory(SCHEMA); + List records = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + records.add(record.copy("f1", Long.valueOf(i))); + if (i % 10 == 0) { + String externalFilePath = String.format(CATALOG_DIR + "external_file/datafile_%s.avro", i); + FileAppender fileAppender = appenderFactory.newAppender( + Files.localOutput(externalFilePath), FileFormat.fromFileName(externalFilePath)); + try (FileAppender fileAppenderCloseable = fileAppender) { + fileAppenderCloseable.addAll(records); + records.clear(); + } catch (IOException e) { + throw new RuntimeException(e); + } + DataFile datafile = DataFiles.builder(PartitionSpec.unpartitioned()) + .withInputFile(HadoopInputFile.fromLocation(externalFilePath, new Configuration())) + .withMetrics(fileAppender.metrics()) + .build(); + table.newAppend().appendFile(datafile).commit(); + } + } + } +} diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/resources/iceberg/iceberg_source.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/resources/iceberg/iceberg_source.conf new file mode 100644 index 00000000000..23a587910d9 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/resources/iceberg/iceberg_source.conf @@ -0,0 +1,82 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + execution.parallelism = 2 + job.mode = "BATCH" + execution.checkpoint.interval = 10 +} + +source { + Iceberg { + fields { + f2 = "boolean" + f1 = "bigint" + f3 = "int" + f4 = "bigint" + f5 = "float" + f6 = "double" + f7 = "date" + f8 = "time" + f9 = "timestamp" + f10 = "timestamp" + f11 = "string" + f12 = "bytes" + f13 = "bytes" + f14 = "decimal(19,9)" + f15 = "array" + f16 = "map" + } + catalog_name = "seatunnel" + catalog_type = "hadoop" + warehouse = "file:///tmp/seatunnel/iceberg/flink/" + namespace = "database1" + table = "source" + } +} + +transform { +} + +sink { + Console { + } + Assert { + rules = [ + { + field_name = f1 + field_type = long + field_value = [ + { + rule_type = NOT_NULL + }, + { + rule_type = MIN + rule_value = 0 + }, + { + rule_type = MAX + rule_value = 99 + } + ] + } + ] + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/resources/log4j.properties b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/resources/log4j.properties new file mode 100644 index 00000000000..db5d9e51220 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-iceberg-flink-e2e/src/test/resources/log4j.properties @@ -0,0 +1,22 @@ +# +# 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. +# +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml index f9f46ffa507..9f9f264ae6c 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml @@ -36,6 +36,7 @@ connector-assert-flink-e2e connector-fake-flink-e2e connector-mongodb-flink-e2e + connector-iceberg-flink-e2e diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/pom.xml new file mode 100644 index 00000000000..50607ada427 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/pom.xml @@ -0,0 +1,45 @@ + + + + + org.apache.seatunnel + seatunnel-spark-connector-v2-e2e + ${revision} + + 4.0.0 + + connector-iceberg-spark-e2e + + + + org.apache.seatunnel + connector-spark-e2e-base + ${project.version} + tests + test-jar + test + + + org.apache.seatunnel + connector-iceberg + ${project.version} + test + + + + \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iceberg/IcebergSourceIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iceberg/IcebergSourceIT.java new file mode 100644 index 00000000000..866cb4d0963 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/iceberg/IcebergSourceIT.java @@ -0,0 +1,164 @@ +/* + * 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.seatunnel.e2e.flink.v2.iceberg; + +import org.apache.seatunnel.connectors.seatunnel.iceberg.IcebergCatalogFactory; +import org.apache.seatunnel.e2e.spark.SparkContainer; + +import lombok.extern.slf4j.Slf4j; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.Container; +import org.testcontainers.utility.MountableFile; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +@Slf4j +public class IcebergSourceIT extends SparkContainer { + + private static final TableIdentifier TABLE = TableIdentifier.of( + Namespace.of("database1"), "source"); + private static final Schema SCHEMA = new Schema( + Types.NestedField.optional(1, "f1", Types.LongType.get()), + Types.NestedField.optional(2, "f2", Types.BooleanType.get()), + Types.NestedField.optional(3, "f3", Types.IntegerType.get()), + Types.NestedField.optional(4, "f4", Types.LongType.get()), + Types.NestedField.optional(5, "f5", Types.FloatType.get()), + Types.NestedField.optional(6, "f6", Types.DoubleType.get()), + Types.NestedField.optional(7, "f7", Types.DateType.get()), + Types.NestedField.optional(8, "f8", Types.TimeType.get()), + Types.NestedField.optional(9, "f9", Types.TimestampType.withZone()), + Types.NestedField.optional(10, "f10", Types.TimestampType.withoutZone()), + Types.NestedField.optional(11, "f11", Types.StringType.get()), + Types.NestedField.optional(12, "f12", Types.FixedType.ofLength(10)), + Types.NestedField.optional(13, "f13", Types.BinaryType.get()), + Types.NestedField.optional(14, "f14", Types.DecimalType.of(19, 9)), + Types.NestedField.optional(15, "f15", Types.ListType.ofOptional( + 100, Types.IntegerType.get())), + Types.NestedField.optional(16, "f16", Types.MapType.ofOptional( + 200, 300, Types.StringType.get(), Types.IntegerType.get())), + Types.NestedField.optional(17, "f17", Types.StructType.of( + Types.NestedField.required(400, "f17_a", Types.StringType.get()))) + ); + + private static final String CATALOG_NAME = "seatunnel"; + private static final String CATALOG_TYPE = "hadoop"; + private static final String CATALOG_DIR = "/tmp/seatunnel/iceberg/spark/"; + private static final String WAREHOUSE = "file://" + CATALOG_DIR; + private static Catalog CATALOG; + + @BeforeEach + public void start() { + initializeIcebergTable(); + batchInsertData(); + MountableFile catalogPath = MountableFile.forHostPath(CATALOG_DIR); + master.copyFileToContainer(catalogPath, CATALOG_DIR); + } + + @Test + public void testIcebergSource() throws IOException, InterruptedException { + Container.ExecResult execResult = executeSeaTunnelSparkJob("/iceberg/iceberg_source.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + } + + private void initializeIcebergTable() { + CATALOG = new IcebergCatalogFactory(CATALOG_NAME, + CATALOG_TYPE, + WAREHOUSE, + null) + .create(); + if (!CATALOG.tableExists(TABLE)) { + CATALOG.createTable(TABLE, SCHEMA); + } + } + + private void batchInsertData() { + GenericRecord record = GenericRecord.create(SCHEMA); + record.setField("f1", Long.valueOf(0)); + record.setField("f2", true); + record.setField("f3", Integer.MAX_VALUE); + record.setField("f4", Long.MAX_VALUE); + record.setField("f5", Float.MAX_VALUE); + record.setField("f6", Double.MAX_VALUE); + record.setField("f7", LocalDate.now()); + record.setField("f8", LocalTime.now()); + record.setField("f9", OffsetDateTime.now()); + record.setField("f10", LocalDateTime.now()); + record.setField("f11", "test"); + record.setField("f12", "abcdefghij".getBytes()); + record.setField("f13", ByteBuffer.wrap("test".getBytes())); + record.setField("f14", new BigDecimal("1000000000.000000001")); + record.setField("f15", Arrays.asList(Integer.MAX_VALUE)); + record.setField("f16", Collections.singletonMap("key", Integer.MAX_VALUE)); + Record structRecord = GenericRecord.create(SCHEMA.findField("f17").type().asStructType()); + structRecord.setField("f17_a", "test"); + record.setField("f17", structRecord); + + Table table = CATALOG.loadTable(TABLE); + FileAppenderFactory appenderFactory = new GenericAppenderFactory(SCHEMA); + List records = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + records.add(record.copy("f1", Long.valueOf(i))); + if (i % 10 == 0) { + String externalFilePath = String.format(CATALOG_DIR + "external_file/datafile_%s.avro", i); + FileAppender fileAppender = appenderFactory.newAppender( + Files.localOutput(externalFilePath), FileFormat.fromFileName(externalFilePath)); + try (FileAppender fileAppenderCloseable = fileAppender) { + fileAppenderCloseable.addAll(records); + records.clear(); + } catch (IOException e) { + throw new RuntimeException(e); + } + DataFile datafile = DataFiles.builder(PartitionSpec.unpartitioned()) + .withInputFile(HadoopInputFile.fromLocation(externalFilePath, new Configuration())) + .withMetrics(fileAppender.metrics()) + .build(); + table.newAppend().appendFile(datafile).commit(); + } + } + } +} diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/resources/iceberg/iceberg_source.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/resources/iceberg/iceberg_source.conf new file mode 100644 index 00000000000..afff183edcf --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/resources/iceberg/iceberg_source.conf @@ -0,0 +1,85 @@ +# +# 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. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + Iceberg { + fields { + f2 = "boolean" + f1 = "bigint" + f3 = "int" + f4 = "bigint" + f5 = "float" + f6 = "double" + f7 = "date" + f8 = "time" + f9 = "timestamp" + f10 = "timestamp" + f11 = "string" + f12 = "bytes" + f13 = "bytes" + f14 = "decimal(19,9)" + f15 = "array" + f16 = "map" + } + catalog_name = "seatunnel" + catalog_type = "hadoop" + warehouse = "file:///tmp/seatunnel/iceberg/spark/" + namespace = "database1" + table = "source" + } +} + +transform { +} + +sink { + Console { + } + Assert { + rules = [ + { + field_name = f1 + field_type = long + field_value = [ + { + rule_type = NOT_NULL + }, + { + rule_type = MIN + rule_value = 0 + }, + { + rule_type = MAX + rule_value = 99 + } + ] + } + ] + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/resources/log4j.properties b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/resources/log4j.properties new file mode 100644 index 00000000000..db5d9e51220 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-iceberg-spark-e2e/src/test/resources/log4j.properties @@ -0,0 +1,22 @@ +# +# 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. +# +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n From 94ed275ba334b762d37c0a2fdd039ce1d5cb2c7e Mon Sep 17 00:00:00 2001 From: songjianet <1778651752@qq.com> Date: Fri, 16 Sep 2022 19:01:25 +0800 Subject: [PATCH 42/44] [Feat][UI] Update task service in the project. (#2758) * [Feat][UI] Update task service in the project. * [Feat][UI] Update task service in the project. --- seatunnel-ui/src/service/task/index.ts | 43 +++++++++++++++++--------- seatunnel-ui/src/service/task/types.ts | 11 ++----- 2 files changed, 31 insertions(+), 23 deletions(-) diff --git a/seatunnel-ui/src/service/task/index.ts b/seatunnel-ui/src/service/task/index.ts index 29d4097971d..3b1c3e303be 100644 --- a/seatunnel-ui/src/service/task/index.ts +++ b/seatunnel-ui/src/service/task/index.ts @@ -20,37 +20,50 @@ import type { TaskList, TaskJobList, TaskRecycle, - TaskTmpExecute + TaskExecute } from '@/service/task/types' -export function taskList(req: TaskList): any { +export function taskInstanceList(params: TaskList): any { return axios({ - url: '/task/listInstance', + url: '/task/instance', method: 'get', - params: { req } + params }) } -export function taskJobList(req: TaskJobList): any { +export function taskJobList(params: TaskList): any { return axios({ - url: '/task/listJob', + url: '/task/job', method: 'get', - params: { req } + params }) } -export function taskRecycle(req: TaskRecycle): any { +export function taskExecute(scriptId: number, data: TaskExecute): any { return axios({ - url: '/task/recycle', - method: 'put', - data: { req } + url: `/task/${scriptId}/execute`, + method: 'post', + data }) } -export function taskTmpExecute(req: TaskTmpExecute): any { +export function taskRecycle(scriptId: number): any { return axios({ - url: '/task/tmpExecute', - method: 'get', - params: { req } + url: `/task/${scriptId}/recycle`, + method: 'patch' + }) +} + +export function taskInstanceDetail(taskInstanceId: number): any { + return axios({ + url: `/task/${taskInstanceId}`, + method: 'get' }) } + +export function taskInstanceKill(taskInstanceId: number): any { + return axios({ + url: `/task/${taskInstanceId}`, + method: 'patch' + }) +} \ No newline at end of file diff --git a/seatunnel-ui/src/service/task/types.ts b/seatunnel-ui/src/service/task/types.ts index 290d6154645..7258649afd3 100644 --- a/seatunnel-ui/src/service/task/types.ts +++ b/seatunnel-ui/src/service/task/types.ts @@ -16,7 +16,7 @@ */ interface TaskList { - name: string + name?: string pageNo: number pageSize: number } @@ -32,15 +32,10 @@ interface TaskRecycle { operatorId: number } -interface TaskTmpExecute { +interface TaskExecute { content: string - endTime: string executeType: number - operatorId: number - parallelismNum: number params: object - scriptId: number - startTime: string } -export { TaskList, TaskJobList, TaskRecycle, TaskTmpExecute } +export { TaskList, TaskJobList, TaskRecycle, TaskExecute } From 864780b530c11f7d298387c312caa09802fca91c Mon Sep 17 00:00:00 2001 From: iture123 Date: Fri, 16 Sep 2022 23:17:48 +0800 Subject: [PATCH 43/44] [Feature][Connector-V2]improve Elasticsearch source doc(#2553) --- docs/en/connector-v2/source/Elasticsearch.md | 38 +++++++++++--------- 1 file changed, 22 insertions(+), 16 deletions(-) diff --git a/docs/en/connector-v2/source/Elasticsearch.md b/docs/en/connector-v2/source/Elasticsearch.md index 77d8176e0a8..b5301148359 100644 --- a/docs/en/connector-v2/source/Elasticsearch.md +++ b/docs/en/connector-v2/source/Elasticsearch.md @@ -1,24 +1,29 @@ # Elasticsearch -## Description +> Elasticsearch source connector -Read data from `Elasticsearch`. +## Description -:::tip +Used to read data from Elasticsearch. -Engine Supported +support version >= 2.x and < 8.x. -* supported `ElasticSearch version is >= 2.x and < 8.x` +## Key features -::: +- [x] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [schema projection](../../concept/connector-v2-features.md) +- [ ] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) ## Options | name | type | required | default value | |-------------|--------| -------- |---------------| | hosts | array | yes | - | -| username | string | no | | -| password | string | no | | +| username | string | no | - | +| password | string | no | - | | index | string | yes | - | | source | array | yes | - | | scroll_time | string | no | 1m | @@ -27,29 +32,30 @@ Engine Supported ### hosts [array] -`Elasticsearch` cluster http address, the format is `host:port` , allowing multiple hosts to be specified. Such as `["host1:9200", "host2:9200"]`. +Elasticsearch cluster http address, the format is `host:port`, allowing multiple hosts to be specified. Such as `["host1:9200", "host2:9200"]`. ### username [string] -x-pack username +x-pack username. ### password [string] -x-pack password +x-pack password. ### index [string] -`Elasticsearch` index name, support * fuzzy matching +Elasticsearch index name, support * fuzzy matching. ### source [array] The fields of index. -You can get the document id by specifying the field `_id`.If sink _id to other index,you need specify an alias for _id due to the `Elasticsearch` limit. +You can get the document id by specifying the field `_id`.If sink _id to other index,you need specify an alias for _id due to the Elasticsearch limit. ### scroll_time [String] -Amount of time `Elasticsearch` will keep the search context alive for scroll requests. +Amount of time Elasticsearch will keep the search context alive for scroll requests. ### scroll_size [int] -Maximum number of hits to be returned with each `Elasticsearch` scroll request. +Maximum number of hits to be returned with each Elasticsearch scroll request. ## Examples -```bash +simple +```hocon Elasticsearch { hosts = ["localhost:9200"] index = "seatunnel-*" From 74dd305aabad981fae0daced279dbab6dad3c9a2 Mon Sep 17 00:00:00 2001 From: iture123 Date: Sun, 18 Sep 2022 14:41:13 +0800 Subject: [PATCH 44/44] [Feature][Connector-V2]improve elasticseach e2e module structure(#2553) --- .../connector-elasticsearch-flink-e2e/pom.xml | 59 +++++++++++++++++++ .../ElasticsearchSourceToConsoleIT.java | 0 .../FakeSourceToElasticsearchIT.java | 0 .../elasticsearch_to_console.conf | 0 .../fakesource_to_elasticsearch.conf | 0 .../src/test/resources/log4j.properties | 22 +++++++ .../seatunnel-flink-connector-v2-e2e/pom.xml | 3 +- .../connector-elasticsearch-spark-e2e/pom.xml | 48 +++++++++++++++ .../ElasticsearchSourceToConsoleIT.java | 2 +- .../FakeSourceToElasticsearchIT.java | 2 +- .../elasticsearch_to_console.conf | 0 .../fakesource_to_elasticsearch.conf | 0 .../src/test/resources/log4j.properties | 22 +++++++ .../seatunnel-spark-connector-v2-e2e/pom.xml | 3 +- 14 files changed, 157 insertions(+), 4 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/pom.xml rename seatunnel-e2e/seatunnel-flink-connector-v2-e2e/{ => connector-elasticsearch-flink-e2e}/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java (100%) rename seatunnel-e2e/seatunnel-flink-connector-v2-e2e/{ => connector-elasticsearch-flink-e2e}/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java (100%) rename seatunnel-e2e/seatunnel-flink-connector-v2-e2e/{ => connector-elasticsearch-flink-e2e}/src/test/resources/elasticsearch/elasticsearch_to_console.conf (100%) rename seatunnel-e2e/seatunnel-flink-connector-v2-e2e/{ => connector-elasticsearch-flink-e2e}/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf (100%) create mode 100644 seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/resources/log4j.properties create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/pom.xml rename seatunnel-e2e/seatunnel-spark-connector-v2-e2e/{ => connector-elasticsearch-spark-e2e}/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java (98%) rename seatunnel-e2e/seatunnel-spark-connector-v2-e2e/{ => connector-elasticsearch-spark-e2e}/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java (97%) rename seatunnel-e2e/seatunnel-spark-connector-v2-e2e/{ => connector-elasticsearch-spark-e2e}/src/test/resources/elasticsearch/elasticsearch_to_console.conf (100%) rename seatunnel-e2e/seatunnel-spark-connector-v2-e2e/{ => connector-elasticsearch-spark-e2e}/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf (100%) create mode 100644 seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/resources/log4j.properties diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/pom.xml new file mode 100644 index 00000000000..7f734e76040 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/pom.xml @@ -0,0 +1,59 @@ + + + + + org.apache.seatunnel + seatunnel-flink-connector-v2-e2e + ${revision} + + 4.0.0 + + connector-elasticsearch-flink-e2e + + + + org.apache.seatunnel + connector-flink-e2e-base + ${project.version} + tests + test-jar + test + + + + + org.apache.seatunnel + connector-elasticsearch + ${project.version} + test + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-console + ${project.version} + test + + + + diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java similarity index 100% rename from seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java rename to seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java similarity index 100% rename from seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java rename to seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf similarity index 100% rename from seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf rename to seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf similarity index 100% rename from seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf rename to seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/resources/log4j.properties b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/resources/log4j.properties new file mode 100644 index 00000000000..db5d9e51220 --- /dev/null +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/resources/log4j.properties @@ -0,0 +1,22 @@ +# +# 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. +# +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml index 8f914aad7fc..4f8ac1445dd 100644 --- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/pom.xml @@ -36,6 +36,7 @@ connector-assert-flink-e2e connector-fake-flink-e2e connector-mongodb-flink-e2e + connector-elasticsearch-flink-e2e @@ -69,4 +70,4 @@ test - \ No newline at end of file + diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/pom.xml new file mode 100644 index 00000000000..44f1739f153 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/pom.xml @@ -0,0 +1,48 @@ + + + + + org.apache.seatunnel + seatunnel-spark-connector-v2-e2e + ${revision} + + 4.0.0 + + connector-elasticsearch-spark-e2e + + + + org.apache.seatunnel + connector-spark-e2e-base + ${project.version} + tests + test-jar + test + + + + + org.apache.seatunnel + connector-elasticsearch + ${project.version} + test + + + + + diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java similarity index 98% rename from seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java rename to seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java index ae541039e4c..b7e64e7a590 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.e2e.spark.v2.elasticsearch; +package test.java.org.apache.seatunnel.e2e.spark.v2.elasticsearch; import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient; import org.apache.seatunnel.e2e.spark.SparkContainer; diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java similarity index 97% rename from seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java rename to seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java index 33aebc39441..5c1669d8ca3 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/FakeSourceToElasticsearchIT.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.e2e.spark.v2.elasticsearch; +package test.java.org.apache.seatunnel.e2e.spark.v2.elasticsearch; import org.apache.seatunnel.e2e.spark.SparkContainer; diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf similarity index 100% rename from seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf rename to seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/resources/elasticsearch/elasticsearch_to_console.conf diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf similarity index 100% rename from seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf rename to seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/resources/elasticsearch/fakesource_to_elasticsearch.conf diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/resources/log4j.properties b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/resources/log4j.properties new file mode 100644 index 00000000000..db5d9e51220 --- /dev/null +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/resources/log4j.properties @@ -0,0 +1,22 @@ +# +# 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. +# +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml index a05ec270d42..09a45521052 100644 --- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/pom.xml @@ -34,6 +34,7 @@ connector-iotdb-spark-e2e connector-jdbc-spark-e2e connector-redis-spark-e2e + connector-elasticsearch-spark-e2e @@ -69,4 +70,4 @@ - \ No newline at end of file +