From 7dfea7724c38df9aad24feca81512ab6a1968aa7 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Fri, 2 Dec 2022 20:07:16 +0800 Subject: [PATCH 01/36] [Feature][Connector-V2][Doris] Add Doris StreamLoad sink connector --- plugin-mapping.properties | 3 +- .../connector-doris/pom.xml | 63 ++++ .../doris/client/DorisFlushTuple.java | 33 ++ .../doris/client/DorisSinkManager.java | 158 +++++++++ .../DorisStreamLoadFailedException.java | 49 +++ .../doris/client/DorisStreamLoadVisitor.java | 227 +++++++++++++ .../connectors/doris/client/HttpHelper.java | 163 ++++++++++ .../connectors/doris/config/SinkConfig.java | 192 +++++++++++ .../doris/serialize/DorisBaseSerializer.java | 76 +++++ .../doris/serialize/DorisCsvSerializer.java | 47 +++ .../doris/serialize/DorisDelimiterParser.java | 73 +++++ .../doris/serialize/DorisISerializer.java | 27 ++ .../doris/serialize/DorisJsonSerializer.java | 46 +++ .../connectors/doris/sink/DorisSink.java | 76 +++++ .../doris/sink/DorisSinkFactory.java | 43 +++ .../doris/sink/DorisSinkWriter.java | 89 ++++++ .../connectors/doris/DorisFactoryTest.java | 30 ++ seatunnel-connectors-v2/pom.xml | 1 + seatunnel-dist/pom.xml | 5 + .../connector-doris-e2e/pom.xml | 48 +++ .../e2e/connector/doris/DorisIT.java | 302 ++++++++++++++++++ .../test/resources/doris-jdbc-to-doris.conf | 48 +++ .../seatunnel-connector-v2-e2e/pom.xml | 1 + 23 files changed, 1799 insertions(+), 1 deletion(-) create mode 100644 seatunnel-connectors-v2/connector-doris/pom.xml create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisFlushTuple.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadFailedException.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/HttpHelper.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisBaseSerializer.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisCsvSerializer.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisDelimiterParser.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisISerializer.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisJsonSerializer.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkFactory.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/DorisFactoryTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/doris-jdbc-to-doris.conf diff --git a/plugin-mapping.properties b/plugin-mapping.properties index e73f8c283d2..6d20fd8f37f 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -156,4 +156,5 @@ seatunnel.source.Jira = connector-http-jira seatunnel.source.Gitlab = connector-http-gitlab seatunnel.sink.RabbitMQ = connector-rabbitmq seatunnel.source.RabbitMQ = connector-rabbitmq -seatunnel.source.OpenMldb = connector-openmldb \ No newline at end of file +seatunnel.source.OpenMldb = connector-openmldb +seatunnel.sink.Doris = connector-doris \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-doris/pom.xml b/seatunnel-connectors-v2/connector-doris/pom.xml new file mode 100644 index 00000000000..84c42d4f8a9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/pom.xml @@ -0,0 +1,63 @@ + + + + + seatunnel-connectors-v2 + org.apache.seatunnel + ${revision} + + 4.0.0 + + connector-doris + + + 4.5.13 + 4.4.4 + + + + + org.apache.seatunnel + seatunnel-api + ${project.version} + + + org.apache.seatunnel + connector-common + ${project.version} + + + org.apache.httpcomponents + httpclient + ${httpclient.version} + + + org.apache.httpcomponents + httpcore + ${httpcore.version} + + + org.apache.seatunnel + seatunnel-format-json + ${project.version} + + + diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisFlushTuple.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisFlushTuple.java new file mode 100644 index 00000000000..cfae19a8621 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisFlushTuple.java @@ -0,0 +1,33 @@ +/* + * 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.doris.client; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; + +import java.util.List; + +@AllArgsConstructor +@Getter +@Setter +public class DorisFlushTuple { + private String label; + private Long bytes; + private List rows; +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java new file mode 100644 index 00000000000..9348cfa940b --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java @@ -0,0 +1,158 @@ +/* + * 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.doris.client; + +import org.apache.seatunnel.connectors.doris.config.SinkConfig; + +import com.google.common.base.Strings; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +@Slf4j +public class DorisSinkManager { + + private final SinkConfig sinkConfig; + private final List batchList; + + private DorisStreamLoadVisitor dorisStreamLoadVisitor; + private ScheduledExecutorService scheduler; + private ScheduledFuture scheduledFuture; + private volatile boolean initialize; + private volatile Exception flushException; + private int batchRowCount = 0; + private long batchBytesSize = 0; + + private Integer batchIntervalMs; + + public DorisSinkManager(SinkConfig sinkConfig, List fileNames) { + this.sinkConfig = sinkConfig; + this.batchList = new ArrayList<>(); + this.batchIntervalMs = sinkConfig.getBatchIntervalMs(); + dorisStreamLoadVisitor = new DorisStreamLoadVisitor(sinkConfig, fileNames); + } + + private void tryInit() throws IOException { + if (initialize) { + return; + } + initialize = true; + + if (batchIntervalMs != null) { + scheduler = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("Doris-sink-output-%s").build()); + scheduledFuture = scheduler.scheduleAtFixedRate( + () -> { + try { + flush(); + } catch (IOException e) { + flushException = e; + } + }, + batchIntervalMs, + batchIntervalMs, + TimeUnit.MILLISECONDS); + } + } + + public synchronized void write(String record) throws IOException { + tryInit(); + checkFlushException(); + byte[] bts = record.getBytes(StandardCharsets.UTF_8); + batchList.add(bts); + batchRowCount++; + batchBytesSize += bts.length; + if (batchRowCount >= sinkConfig.getBatchMaxSize() || batchBytesSize >= sinkConfig.getBatchMaxBytes()) { + flush(); + } + } + + public synchronized void close() throws IOException { + if (scheduledFuture != null) { + scheduledFuture.cancel(false); + scheduler.shutdown(); + } + + flush(); + } + + public synchronized void flush() throws IOException { + checkFlushException(); + if (batchList.isEmpty()) { + return; + } + String label = createBatchLabel(); + DorisFlushTuple tuple = new DorisFlushTuple(label, batchBytesSize, new ArrayList<>(batchList)); + for (int i = 0; i <= sinkConfig.getMaxRetries(); i++) { + try { + Boolean successFlag = dorisStreamLoadVisitor.doStreamLoad(tuple); + if (successFlag) { + break; + } + } catch (Exception e) { + log.warn("Writing records to Doris failed, retry times = {}", i, e); + if (i >= sinkConfig.getMaxRetries()) { + throw new IOException("Writing records to Doris failed.", e); + } + + if (e instanceof DorisStreamLoadFailedException && ((DorisStreamLoadFailedException) e).needReCreateLabel()) { + String newLabel = createBatchLabel(); + log.warn(String.format("Batch label changed from [%s] to [%s]", tuple.getLabel(), newLabel)); + tuple.setLabel(newLabel); + } + + try { + long backoff = Math.min(sinkConfig.getRetryBackoffMultiplierMs() * i, + sinkConfig.getMaxRetryBackoffMs()); + Thread.sleep(backoff); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new IOException( + "Unable to flush; interrupted while doing another attempt.", e); + } + } + } + batchList.clear(); + batchRowCount = 0; + batchBytesSize = 0; + } + + private void checkFlushException() { + if (flushException != null) { + throw new RuntimeException("Writing records to Doris failed.", flushException); + } + } + + public String createBatchLabel() { + StringBuilder sb = new StringBuilder(); + if (!Strings.isNullOrEmpty(sinkConfig.getLabelPrefix())) { + sb.append(sinkConfig.getLabelPrefix()); + } + return sb.append(UUID.randomUUID().toString()) + .toString(); + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadFailedException.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadFailedException.java new file mode 100644 index 00000000000..56be7358fc1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadFailedException.java @@ -0,0 +1,49 @@ +/* + * 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.doris.client; + +import java.io.IOException; +import java.util.Map; + +public class DorisStreamLoadFailedException extends IOException { + + static final long serialVersionUID = 1L; + + private final Map response; + private boolean reCreateLabel; + + public DorisStreamLoadFailedException(String message, Map response) { + super(message); + this.response = response; + } + + public DorisStreamLoadFailedException(String message, Map response, boolean reCreateLabel) { + super(message); + this.response = response; + this.reCreateLabel = reCreateLabel; + } + + public Map getFailedResponse() { + return response; + } + + public boolean needReCreateLabel() { + return reCreateLabel; + } + +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java new file mode 100644 index 00000000000..2955dec03ff --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java @@ -0,0 +1,227 @@ +/* + * 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.doris.client; + +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.connectors.doris.config.SinkConfig; +import org.apache.seatunnel.connectors.doris.serialize.DorisDelimiterParser; + +import org.apache.commons.codec.binary.Base64; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class DorisStreamLoadVisitor { + + private static final Logger LOG = LoggerFactory.getLogger(DorisStreamLoadVisitor.class); + + private final HttpHelper httpHelper = new HttpHelper(); + private static final int MAX_SLEEP_TIME = 5; + + private final SinkConfig sinkConfig; + private long pos; + private static final String RESULT_FAILED = "Fail"; + private static final String RESULT_SUCCESS = "Success"; + private static final String RESULT_LABEL_EXISTED = "Label Already Exists"; + private static final String LAEBL_STATE_VISIBLE = "VISIBLE"; + private static final String LAEBL_STATE_COMMITTED = "COMMITTED"; + private static final String RESULT_LABEL_PREPARE = "PREPARE"; + private static final String RESULT_LABEL_ABORTED = "ABORTED"; + private static final String RESULT_LABEL_UNKNOWN = "UNKNOWN"; + + private List fieldNames; + + public DorisStreamLoadVisitor(SinkConfig sinkConfig, List fieldNames) { + this.sinkConfig = sinkConfig; + this.fieldNames = fieldNames; + } + + public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { + String host = getAvailableHost(); + if (null == host) { + throw new IOException("None of the host in `load_url` could be connected."); + } + String loadUrl = new StringBuilder(host) + .append("/api/") + .append(sinkConfig.getDatabase()) + .append("/") + .append(sinkConfig.getTable()) + .append("/_stream_load") + .toString(); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Start to join batch data: rows[%d] bytes[%d] label[%s].", flushData.getRows().size(), flushData.getBytes(), flushData.getLabel())); + } + Map loadResult = httpHelper.doHttpPut(loadUrl, joinRows(flushData.getRows(), flushData.getBytes().intValue()), getStreamLoadHttpHeader(flushData.getLabel())); + final String keyStatus = "Status"; + if (null == loadResult || !loadResult.containsKey(keyStatus)) { + LOG.error("unknown result status. {}", loadResult); + throw new IOException("Unable to flush data to Doris: unknown result status. " + loadResult); + } + if (LOG.isDebugEnabled()) { + LOG.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); + } + if (RESULT_FAILED.equals(loadResult.get(keyStatus))) { + StringBuilder errorBuilder = new StringBuilder("Failed to flush data to Doris.\n"); + if (loadResult.containsKey("Message")) { + errorBuilder.append(loadResult.get("Message")); + errorBuilder.append('\n'); + } + if (loadResult.containsKey("ErrorURL")) { + LOG.error("StreamLoad response: {}", loadResult); + try { + errorBuilder.append(httpHelper.doHttpGet(loadResult.get("ErrorURL").toString())); + errorBuilder.append('\n'); + } catch (IOException e) { + LOG.warn("Get Error URL failed. {} ", loadResult.get("ErrorURL"), e); + } + } else { + errorBuilder.append(JsonUtils.toJsonString(loadResult)); + errorBuilder.append('\n'); + } + throw new IOException(errorBuilder.toString()); + } else if (RESULT_LABEL_EXISTED.equals(loadResult.get(keyStatus))) { + LOG.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); + // has to block-checking the state to get the final result + checkLabelState(host, flushData.getLabel()); + } + return RESULT_SUCCESS.equals(loadResult.get(keyStatus)); + } + + private String getAvailableHost() { + List hostList = sinkConfig.getNodeUrls(); + long tmp = pos + hostList.size(); + for (; pos < tmp; pos++) { + String host = new StringBuilder("http://").append(hostList.get((int) (pos % hostList.size()))).toString(); + if (httpHelper.tryHttpConnection(host)) { + return host; + } + } + return null; + } + + private byte[] joinRows(List rows, int totalBytes) { + if (SinkConfig.StreamLoadFormat.CSV.equals(sinkConfig.getLoadFormat())) { + Map props = sinkConfig.getStreamLoadProps(); + byte[] lineDelimiter = DorisDelimiterParser.parse((String) props.get("row_delimiter"), "\n").getBytes(StandardCharsets.UTF_8); + ByteBuffer bos = ByteBuffer.allocate(totalBytes + rows.size() * lineDelimiter.length); + for (byte[] row : rows) { + bos.put(row); + bos.put(lineDelimiter); + } + return bos.array(); + } + + if (SinkConfig.StreamLoadFormat.JSON.equals(sinkConfig.getLoadFormat())) { + ByteBuffer bos = ByteBuffer.allocate(totalBytes + (rows.isEmpty() ? 2 : rows.size() + 1)); + bos.put("[".getBytes(StandardCharsets.UTF_8)); + byte[] jsonDelimiter = ",".getBytes(StandardCharsets.UTF_8); + boolean isFirstElement = true; + for (byte[] row : rows) { + if (!isFirstElement) { + bos.put(jsonDelimiter); + } + bos.put(row); + isFirstElement = false; + } + bos.put("]".getBytes(StandardCharsets.UTF_8)); + return bos.array(); + } + throw new RuntimeException("Failed to join rows data, unsupported `format` from stream load properties:"); + } + + @SuppressWarnings("unchecked") + private void checkLabelState(String host, String label) throws IOException { + int idx = 0; + while (true) { + try { + TimeUnit.SECONDS.sleep(Math.min(++idx, MAX_SLEEP_TIME)); + } catch (InterruptedException ex) { + break; + } + try { + String queryLoadStateUrl = new StringBuilder(host).append("/api/").append(sinkConfig.getDatabase()).append("/get_load_state?label=").append(label).toString(); + Map result = httpHelper.doHttpGet(queryLoadStateUrl, getLoadStateHttpHeader(label)); + if (result == null) { + throw new IOException(String.format("Failed to flush data to Doris, Error " + + "could not get the final state of label[%s].\n", label), null); + } + String labelState = (String) result.get("state"); + if (null == labelState) { + throw new IOException(String.format("Failed to flush data to Doris, Error " + + "could not get the final state of label[%s]. response[%s]\n", label, JsonUtils.toJsonString(result)), null); + } + LOG.info(String.format("Checking label[%s] state[%s]\n", label, labelState)); + switch (labelState) { + case LAEBL_STATE_VISIBLE: + case LAEBL_STATE_COMMITTED: + return; + case RESULT_LABEL_PREPARE: + continue; + case RESULT_LABEL_ABORTED: + throw new DorisStreamLoadFailedException(String.format("Failed to flush data to Doris, Error " + + "label[%s] state[%s]\n", label, labelState), null, true); + case RESULT_LABEL_UNKNOWN: + default: + throw new DorisStreamLoadFailedException(String.format("Failed to flush data to Doris, Error " + + "label[%s] state[%s]\n", label, labelState), null); + } + } catch (IOException e) { + throw new IOException(e); + } + } + } + + private String getBasicAuthHeader(String username, String password) { + String auth = username + ":" + password; + byte[] encodedAuth = Base64.encodeBase64(auth.getBytes(StandardCharsets.UTF_8)); + return new StringBuilder("Basic ").append(new String(encodedAuth)).toString(); + } + + private Map getStreamLoadHttpHeader(String label) { + Map headerMap = new HashMap<>(); + if (null != fieldNames && !fieldNames.isEmpty() && SinkConfig.StreamLoadFormat.CSV.equals(sinkConfig.getLoadFormat())) { + headerMap.put("columns", String.join(",", fieldNames.stream().map(f -> String.format("`%s`", f)).collect(Collectors.toList()))); + } + if (null != sinkConfig.getStreamLoadProps()) { + for (Map.Entry entry : sinkConfig.getStreamLoadProps().entrySet()) { + headerMap.put(entry.getKey(), String.valueOf(entry.getValue())); + } + } + headerMap.put("strip_outer_array", "true"); + headerMap.put("Expect", "100-continue"); + headerMap.put("label", label); + headerMap.put("Content-Type", "application/x-www-form-urlencoded"); + headerMap.put("Authorization", getBasicAuthHeader(sinkConfig.getUsername(), sinkConfig.getPassword())); + return headerMap; + } + + private Map getLoadStateHttpHeader(String label) { + Map headerMap = new HashMap<>(); + headerMap.put("Authorization", getBasicAuthHeader(sinkConfig.getUsername(), sinkConfig.getPassword())); + headerMap.put("Connection", "close"); + return headerMap; + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/HttpHelper.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/HttpHelper.java new file mode 100644 index 00000000000..c8cc5eb5ab9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/HttpHelper.java @@ -0,0 +1,163 @@ +/* + * 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.doris.client; + +import org.apache.seatunnel.common.utils.JsonUtils; + +import lombok.extern.slf4j.Slf4j; +import org.apache.http.HttpEntity; +import org.apache.http.HttpStatus; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.DefaultRedirectStrategy; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.HashMap; +import java.util.Map; + +@Slf4j +public class HttpHelper { + private static final int DEFAULT_CONNECT_TIMEOUT = 1000000; + + public HttpEntity getHttpEntity(CloseableHttpResponse resp) { + int code = resp.getStatusLine().getStatusCode(); + if (HttpStatus.SC_OK != code) { + log.warn("Request failed with code:{}", code); + return null; + } + HttpEntity respEntity = resp.getEntity(); + if (null == respEntity) { + log.warn("Request failed with empty response."); + return null; + } + return respEntity; + } + + public String doHttpGet(String getUrl) throws IOException { + log.info("Executing GET from {}.", getUrl); + try (CloseableHttpClient httpclient = buildHttpClient()) { + HttpGet httpGet = new HttpGet(getUrl); + try (CloseableHttpResponse resp = httpclient.execute(httpGet)) { + HttpEntity respEntity = resp.getEntity(); + if (null == respEntity) { + log.warn("Request failed with empty response."); + return null; + } + return EntityUtils.toString(respEntity); + } + } + } + + public Map doHttpGet(String getUrl, Map header) throws IOException { + log.info("Executing GET from {}.", getUrl); + try (CloseableHttpClient httpclient = HttpClients.createDefault()) { + HttpGet httpGet = new HttpGet(getUrl); + if (null != header) { + for (Map.Entry entry : header.entrySet()) { + httpGet.setHeader(entry.getKey(), String.valueOf(entry.getValue())); + } + } + try (CloseableHttpResponse resp = httpclient.execute(httpGet)) { + HttpEntity respEntity = getHttpEntity(resp); + if (null == respEntity) { + log.warn("Request failed with empty response."); + return null; + } + return JsonUtils.parseObject(EntityUtils.toString(respEntity), Map.class); + } + } + } + + @SuppressWarnings("unchecked") + public Map doHttpPut(String url, byte[] data, Map header) throws IOException { + final HttpClientBuilder httpClientBuilder = HttpClients.custom() + .setRedirectStrategy(new DefaultRedirectStrategy() { + @Override + protected boolean isRedirectable(String method) { + return true; + } + }); + try (CloseableHttpClient httpclient = httpClientBuilder.build()) { + HttpPut httpPut = new HttpPut(url); + if (null != header) { + for (Map.Entry entry : header.entrySet()) { + httpPut.setHeader(entry.getKey(), String.valueOf(entry.getValue())); + } + } + httpPut.setEntity(new ByteArrayEntity(data)); + httpPut.setConfig(RequestConfig.custom().setRedirectsEnabled(true).build()); + try (CloseableHttpResponse resp = httpclient.execute(httpPut)) { + int code = resp.getStatusLine().getStatusCode(); + if (HttpStatus.SC_OK != code) { + String errorText; + try { + HttpEntity respEntity = resp.getEntity(); + errorText = EntityUtils.toString(respEntity); + } catch (Exception err) { + errorText = "find errorText failed: " + err.getMessage(); + } + log.warn("Request failed with code:{}, err:{}", code, errorText); + Map errorMap = new HashMap<>(); + errorMap.put("Status", "Fail"); + errorMap.put("Message", errorText); + return errorMap; + } + HttpEntity respEntity = resp.getEntity(); + if (null == respEntity) { + log.warn("Request failed with empty response."); + return null; + } + return JsonUtils.parseObject(EntityUtils.toString(respEntity), Map.class); + } + } + } + + private CloseableHttpClient buildHttpClient() { + final HttpClientBuilder httpClientBuilder = HttpClients.custom() + .setRedirectStrategy(new DefaultRedirectStrategy() { + @Override + protected boolean isRedirectable(String method) { + return true; + } + }); + return httpClientBuilder.build(); + } + + public boolean tryHttpConnection(String host) { + try { + URL url = new URL(host); + HttpURLConnection co = (HttpURLConnection) url.openConnection(); + co.setConnectTimeout(DEFAULT_CONNECT_TIMEOUT); + co.connect(); + co.disconnect(); + return true; + } catch (Exception e1) { + log.warn("Failed to connect to address:{}", host, e1); + return false; + } + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java new file mode 100644 index 00000000000..14e448447e1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java @@ -0,0 +1,192 @@ +/* + * 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.doris.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.common.config.TypesafeConfigUtils; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@Setter +@Getter +@ToString +public class SinkConfig { + + private static final int DEFAULT_BATCH_MAX_SIZE = 1024; + private static final long DEFAULT_BATCH_BYTES = 5 * 1024 * 1024; + + private static final String LOAD_FORMAT = "format"; + private static final StreamLoadFormat DEFAULT_LOAD_FORMAT = StreamLoadFormat.CSV; + private static final String COLUMN_SEPARATOR = "column_separator"; + + public static final Option> NODE_URLS = Options.key("nodeUrls") + .listType() + .noDefaultValue() + .withDescription("Doris cluster address, the format is [\"fe_ip:fe_http_port\", ...]"); + + public static final Option USERNAME = Options.key("username") + .stringType() + .noDefaultValue() + .withDescription("Doris user username"); + + public static final Option PASSWORD = Options.key("password") + .stringType() + .noDefaultValue() + .withDescription("Doris user password"); + + public static final Option LABEL_PREFIX = Options.key("labelPrefix") + .stringType() + .noDefaultValue() + .withDescription("The prefix of Doris stream load label"); + + public static final Option DATABASE = Options.key("database") + .stringType() + .noDefaultValue() + .withDescription("The name of Doris database"); + + public static final Option TABLE = Options.key("table") + .stringType() + .noDefaultValue() + .withDescription("The name of Doris table"); + + public static final Option DORIS_SINK_CONFIG_PREFIX = Options.key("sink.properties.") + .stringType() + .noDefaultValue() + .withDescription("The parameter of the stream load data_desc. " + + "The way to specify the parameter is to add the prefix `sink.properties.` to the original stream load parameter name "); + + public static final Option BATCH_MAX_SIZE = Options.key("batch_max_rows") + .intType() + .defaultValue(DEFAULT_BATCH_MAX_SIZE) + .withDescription("For batch writing, when the number of buffers reaches the number of batch_max_rows or the byte size of batch_max_bytes or the time reaches batch_interval_ms, the data will be flushed into the Doris"); + + public static final Option BATCH_MAX_BYTES = Options.key("batch_max_bytes") + .longType() + .defaultValue(DEFAULT_BATCH_BYTES) + .withDescription("For batch writing, when the number of buffers reaches the number of batch_max_rows or the byte size of batch_max_bytes or the time reaches batch_interval_ms, the data will be flushed into the Doris"); + + public static final Option BATCH_INTERVAL_MS = Options.key("batch_interval_ms") + .intType() + .noDefaultValue() + .withDescription("For batch writing, when the number of buffers reaches the number of batch_max_rows or the byte size of batch_max_bytes or the time reaches batch_interval_ms, the data will be flushed into the Doris"); + + public static final Option MAX_RETRIES = Options.key("max_retries") + .intType() + .noDefaultValue() + .withDescription("The number of retries to flush failed"); + + public static final Option RETRY_BACKOFF_MULTIPLIER_MS = Options.key("retry_backoff_multiplier_ms") + .intType() + .noDefaultValue() + .withDescription("Using as a multiplier for generating the next delay for backoff"); + + public static final Option MAX_RETRY_BACKOFF_MS = Options.key("max_retry_backoff_ms") + .intType() + .noDefaultValue() + .withDescription("The amount of time to wait before attempting to retry a request to Doris"); + + public enum StreamLoadFormat { + CSV, JSON; + public static StreamLoadFormat parse(String format) { + if (StreamLoadFormat.JSON.name().equals(format)) { + return JSON; + } + return CSV; + } + } + + private List nodeUrls; + private String username; + private String password; + private String database; + private String table; + private String labelPrefix; + private String columnSeparator; + private StreamLoadFormat loadFormat = DEFAULT_LOAD_FORMAT; + + private int batchMaxSize = DEFAULT_BATCH_MAX_SIZE; + private long batchMaxBytes = DEFAULT_BATCH_BYTES; + + private Integer batchIntervalMs; + private int maxRetries; + private int retryBackoffMultiplierMs; + private int maxRetryBackoffMs; + + private final Map streamLoadProps = new HashMap<>(); + + public static SinkConfig loadConfig(Config pluginConfig) { + SinkConfig sinkConfig = new SinkConfig(); + sinkConfig.setNodeUrls(pluginConfig.getStringList(NODE_URLS.key())); + sinkConfig.setDatabase(pluginConfig.getString(DATABASE.key())); + sinkConfig.setTable(pluginConfig.getString(TABLE.key())); + + if (pluginConfig.hasPath(USERNAME.key())) { + sinkConfig.setUsername(pluginConfig.getString(USERNAME.key())); + } + if (pluginConfig.hasPath(PASSWORD.key())) { + sinkConfig.setPassword(pluginConfig.getString(PASSWORD.key())); + } + if (pluginConfig.hasPath(LABEL_PREFIX.key())) { + sinkConfig.setLabelPrefix(pluginConfig.getString(LABEL_PREFIX.key())); + } + if (pluginConfig.hasPath(BATCH_MAX_SIZE.key())) { + sinkConfig.setBatchMaxSize(pluginConfig.getInt(BATCH_MAX_SIZE.key())); + } + if (pluginConfig.hasPath(BATCH_MAX_BYTES.key())) { + sinkConfig.setBatchMaxBytes(pluginConfig.getLong(BATCH_MAX_BYTES.key())); + } + if (pluginConfig.hasPath(BATCH_INTERVAL_MS.key())) { + sinkConfig.setBatchIntervalMs(pluginConfig.getInt(BATCH_INTERVAL_MS.key())); + } + if (pluginConfig.hasPath(MAX_RETRIES.key())) { + sinkConfig.setMaxRetries(pluginConfig.getInt(MAX_RETRIES.key())); + } + if (pluginConfig.hasPath(RETRY_BACKOFF_MULTIPLIER_MS.key())) { + sinkConfig.setRetryBackoffMultiplierMs(pluginConfig.getInt(RETRY_BACKOFF_MULTIPLIER_MS.key())); + } + if (pluginConfig.hasPath(MAX_RETRY_BACKOFF_MS.key())) { + sinkConfig.setMaxRetryBackoffMs(pluginConfig.getInt(MAX_RETRY_BACKOFF_MS.key())); + } + parseSinkStreamLoadProperties(pluginConfig, sinkConfig); + if (sinkConfig.streamLoadProps.containsKey(COLUMN_SEPARATOR)) { + sinkConfig.setColumnSeparator((String) sinkConfig.streamLoadProps.get(COLUMN_SEPARATOR)); + } + if (sinkConfig.streamLoadProps.containsKey(LOAD_FORMAT)) { + sinkConfig.setLoadFormat(StreamLoadFormat.parse((String) sinkConfig.streamLoadProps.get(LOAD_FORMAT))); + } + return sinkConfig; + } + + private static void parseSinkStreamLoadProperties(Config pluginConfig, SinkConfig sinkConfig) { + Config dorisConfig = TypesafeConfigUtils.extractSubConfig(pluginConfig, + DORIS_SINK_CONFIG_PREFIX.key(), false); + dorisConfig.entrySet().forEach(entry -> { + final String configKey = entry.getKey().toLowerCase(); + sinkConfig.streamLoadProps.put(configKey, entry.getValue().unwrapped()); + }); + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisBaseSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisBaseSerializer.java new file mode 100644 index 00000000000..95eec722cc7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisBaseSerializer.java @@ -0,0 +1,76 @@ +/* + * 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.doris.serialize; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.common.utils.DateTimeUtils; +import org.apache.seatunnel.common.utils.DateUtils; +import org.apache.seatunnel.common.utils.JsonUtils; +import org.apache.seatunnel.common.utils.TimeUtils; + +import lombok.Builder; + +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; + +public class DorisBaseSerializer { + @Builder.Default + private DateUtils.Formatter dateFormatter = DateUtils.Formatter.YYYY_MM_DD; + @Builder.Default + private DateTimeUtils.Formatter dateTimeFormatter = DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS; + @Builder.Default + private TimeUtils.Formatter timeFormatter = TimeUtils.Formatter.HH_MM_SS; + + protected String convert(SeaTunnelDataType dataType, Object val) { + if (val == null) { + return null; + } + switch (dataType.getSqlType()) { + case TINYINT: + case SMALLINT: + return String.valueOf(((Number) val).shortValue()); + case INT: + return String.valueOf(((Number) val).intValue()); + case BIGINT: + return String.valueOf(((Number) val).longValue()); + case FLOAT: + return String.valueOf(((Number) val).floatValue()); + case DOUBLE: + return String.valueOf(((Number) val).doubleValue()); + case DECIMAL: + case BOOLEAN: + return val.toString(); + case DATE: + return DateUtils.toString((LocalDate) val, dateFormatter); + case TIME: + return TimeUtils.toString((LocalTime) val, timeFormatter); + case TIMESTAMP: + return DateTimeUtils.toString((LocalDateTime) val, dateTimeFormatter); + case STRING: + return (String) val; + case ARRAY: + case MAP: + return JsonUtils.toJsonString(val); + case BYTES: + return new String((byte[]) val); + default: + throw new UnsupportedOperationException("Unsupported dataType: " + dataType); + } + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisCsvSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisCsvSerializer.java new file mode 100644 index 00000000000..f092b5f6e0b --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisCsvSerializer.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.doris.serialize; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +public class DorisCsvSerializer extends DorisBaseSerializer implements DorisISerializer { + private static final long serialVersionUID = 1L; + + private final String columnSeparator; + private final SeaTunnelRowType seaTunnelRowType; + + public DorisCsvSerializer(String sp, SeaTunnelRowType seaTunnelRowType) { + this.seaTunnelRowType = seaTunnelRowType; + this.columnSeparator = DorisDelimiterParser.parse(sp, "\t"); + } + + @Override + public String serialize(SeaTunnelRow row) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < row.getFields().length; i++) { + String value = convert(seaTunnelRowType.getFieldType(i), row.getField(i)); + sb.append(null == value ? "\\N" : value); + if (i < row.getFields().length - 1) { + sb.append(columnSeparator); + } + } + return sb.toString(); + } + +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisDelimiterParser.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisDelimiterParser.java new file mode 100644 index 00000000000..75bff1cce70 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisDelimiterParser.java @@ -0,0 +1,73 @@ +/* + * 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.doris.serialize; + +import com.google.common.base.Strings; + +import java.io.StringWriter; + +public class DorisDelimiterParser { + private static final int SHIFT = 4; + + private static final String HEX_STRING = "0123456789ABCDEF"; + + public static String parse(String sp, String dSp) throws RuntimeException { + if (Strings.isNullOrEmpty(sp)) { + return dSp; + } + if (!sp.toUpperCase().startsWith("\\X")) { + return sp; + } + String hexStr = sp.substring(2); + // check hex str + if (hexStr.isEmpty()) { + throw new RuntimeException("Failed to parse delimiter: `Hex str is empty`"); + } + if (hexStr.length() % 2 != 0) { + throw new RuntimeException("Failed to parse delimiter: `Hex str length error`"); + } + for (char hexChar : hexStr.toUpperCase().toCharArray()) { + if (HEX_STRING.indexOf(hexChar) == -1) { + throw new RuntimeException("Failed to parse delimiter: `Hex str format error`"); + } + } + // transform to separator + StringWriter writer = new StringWriter(); + for (byte b : hexStrToBytes(hexStr)) { + writer.append((char) b); + } + return writer.toString(); + } + + private static byte[] hexStrToBytes(String hexStr) { + String upperHexStr = hexStr.toUpperCase(); + int length = upperHexStr.length() / 2; + char[] hexChars = upperHexStr.toCharArray(); + byte[] bytes = new byte[length]; + for (int i = 0; i < length; i++) { + int pos = i * 2; + bytes[i] = (byte) (charToByte(hexChars[pos]) << SHIFT | charToByte(hexChars[pos + 1])); + } + return bytes; + } + + private static byte charToByte(char c) { + return (byte) HEX_STRING.indexOf(c); + } +} + diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisISerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisISerializer.java new file mode 100644 index 00000000000..37c483cd4e0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisISerializer.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.doris.serialize; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; + +import java.io.Serializable; + +public interface DorisISerializer extends Serializable { + + String serialize(SeaTunnelRow seaTunnelRow); +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisJsonSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisJsonSerializer.java new file mode 100644 index 00000000000..016ed0fadcb --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisJsonSerializer.java @@ -0,0 +1,46 @@ +/* + * 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.doris.serialize; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.utils.JsonUtils; + +import java.util.HashMap; +import java.util.Map; + +public class DorisJsonSerializer extends DorisBaseSerializer implements DorisISerializer { + + private static final long serialVersionUID = 1L; + private final SeaTunnelRowType seaTunnelRowType; + + public DorisJsonSerializer(SeaTunnelRowType seaTunnelRowType) { + this.seaTunnelRowType = seaTunnelRowType; + } + + @Override + public String serialize(SeaTunnelRow row) { + Map rowMap = new HashMap<>(row.getFields().length); + + for (int i = 0; i < row.getFields().length; i++) { + String value = convert(seaTunnelRowType.getFieldType(i), row.getField(i)); + rowMap.put(seaTunnelRowType.getFieldName(i), value); + } + return JsonUtils.toJsonString(rowMap); + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java new file mode 100644 index 00000000000..89ba9737b23 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java @@ -0,0 +1,76 @@ +/* + * 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.doris.sink; + +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.DATABASE; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.NODE_URLS; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.PASSWORD; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.TABLE; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.USERNAME; + +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.shade.com.typesafe.config.Config; + +import com.google.auto.service.AutoService; + +@AutoService(SeaTunnelSink.class) +public class DorisSink extends AbstractSimpleSink { + + private Config pluginConfig; + private SeaTunnelRowType seaTunnelRowType; + + @Override + public String getPluginName() { + return "Doris"; + } + + @Override + public void prepare(Config pluginConfig) throws PrepareFailException { + this.pluginConfig = pluginConfig; + CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, NODE_URLS.key(), DATABASE.key(), TABLE.key(), USERNAME.key(), PASSWORD.key()); + if (!result.isSuccess()) { + throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg()); + } + } + + @Override + public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) { + this.seaTunnelRowType = seaTunnelRowType; + } + + @Override + public SeaTunnelDataType getConsumedType() { + return this.seaTunnelRowType; + } + + @Override + public AbstractSinkWriter createWriter(SinkWriter.Context context) { + return new DorisSinkWriter(pluginConfig, seaTunnelRowType); + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkFactory.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkFactory.java new file mode 100644 index 00000000000..5fb721e7e20 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkFactory.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.doris.sink; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.connectors.doris.config.SinkConfig; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class DorisSinkFactory implements TableSinkFactory { + @Override + public String factoryIdentifier() { + return "Doris"; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required(SinkConfig.NODE_URLS, SinkConfig.USERNAME, SinkConfig.PASSWORD, SinkConfig.DATABASE, SinkConfig.TABLE) + .optional(SinkConfig.LABEL_PREFIX, SinkConfig.BATCH_MAX_SIZE, SinkConfig.BATCH_MAX_BYTES, + SinkConfig.BATCH_INTERVAL_MS, SinkConfig.MAX_RETRIES, SinkConfig.MAX_RETRY_BACKOFF_MS, + SinkConfig.RETRY_BACKOFF_MULTIPLIER_MS, SinkConfig.DORIS_SINK_CONFIG_PREFIX) + .build(); + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java new file mode 100644 index 00000000000..ddbe6cd24d3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java @@ -0,0 +1,89 @@ +/* + * 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.doris.sink; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.doris.client.DorisSinkManager; +import org.apache.seatunnel.connectors.doris.config.SinkConfig; +import org.apache.seatunnel.connectors.doris.serialize.DorisCsvSerializer; +import org.apache.seatunnel.connectors.doris.serialize.DorisISerializer; +import org.apache.seatunnel.connectors.doris.serialize.DorisJsonSerializer; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +@Slf4j +public class DorisSinkWriter extends AbstractSinkWriter { + + private final DorisISerializer serializer; + private final DorisSinkManager manager; + + public DorisSinkWriter(Config pluginConfig, + SeaTunnelRowType seaTunnelRowType) { + SinkConfig sinkConfig = SinkConfig.loadConfig(pluginConfig); + List fieldNames = Arrays.stream(seaTunnelRowType.getFieldNames()).collect(Collectors.toList()); + this.serializer = createSerializer(sinkConfig, seaTunnelRowType); + this.manager = new DorisSinkManager(sinkConfig, fieldNames); + } + + @Override + public void write(SeaTunnelRow element) throws IOException { + String record = serializer.serialize(element); + manager.write(record); + } + + @SneakyThrows + @Override + public Optional prepareCommit() { + // Flush to storage before snapshot state is performed + manager.flush(); + return super.prepareCommit(); + } + + @Override + public void close() throws IOException { + try { + if (manager != null) { + manager.close(); + } + } catch (IOException e) { + log.error("Close doris manager failed.", e); + throw new IOException("Close doris manager failed.", e); + } + } + + public static DorisISerializer createSerializer(SinkConfig sinkConfig, SeaTunnelRowType seaTunnelRowType) { + if (SinkConfig.StreamLoadFormat.CSV.equals(sinkConfig.getLoadFormat())) { + return new DorisCsvSerializer(sinkConfig.getColumnSeparator(), seaTunnelRowType); + } + if (SinkConfig.StreamLoadFormat.JSON.equals(sinkConfig.getLoadFormat())) { + return new DorisJsonSerializer(seaTunnelRowType); + } + throw new RuntimeException("Failed to create row serializer, unsupported `format` from stream load properties."); + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/DorisFactoryTest.java b/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/DorisFactoryTest.java new file mode 100644 index 00000000000..9f3d17050c3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/test/java/org/apache/seatunnel/connectors/doris/DorisFactoryTest.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.doris; + +import org.apache.seatunnel.connectors.doris.sink.DorisSinkFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class DorisFactoryTest { + @Test + void optionRule() { + Assertions.assertNotNull((new DorisSinkFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml index 7a7a7d3af32..4f0f4bf17db 100644 --- a/seatunnel-connectors-v2/pom.xml +++ b/seatunnel-connectors-v2/pom.xml @@ -65,6 +65,7 @@ connector-slack connector-rabbitmq connector-openmldb + connector-doris diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index bbbd55f299e..ae2ee428340 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -380,6 +380,11 @@ connector-openmldb ${project.version} + + org.apache.seatunnel + connector-doris + ${project.version} + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml new file mode 100644 index 00000000000..7528abb180a --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/pom.xml @@ -0,0 +1,48 @@ + + + + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + 4.0.0 + + connector-doris-e2e + + + + + org.apache.seatunnel + connector-doris + ${project.version} + test + + + org.apache.seatunnel + connector-jdbc + ${project.version} + test + + + org.apache.seatunnel + connector-jdbc-e2e + ${project.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java new file mode 100644 index 00000000000..d62a57f9b0d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -0,0 +1,302 @@ +/* + * 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.connector.doris; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.utils.ExceptionUtils; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.org.apache.commons.io.IOUtils; + +import java.io.IOException; +import java.io.InputStream; +import java.math.BigDecimal; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.sql.*; +import java.util.*; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.given; + +@Slf4j +public class DorisIT extends TestSuiteBase implements TestResource { + private static final String DOCKER_IMAGE = "taozex/doris:tagname"; + private static final String DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; + private static final String HOST = "doris_e2e"; + private static final int DOCKER_PORT = 9030; + private static final int PORT = 8961; + + private static final String URL = "jdbc:mysql://%s:" + PORT; + private static final String USERNAME = "root"; + private static final String PASSWORD = ""; + private static final String DATABASE = "test"; + private static final String SOURCE_TABLE = "e2e_table_source"; + private static final String SINK_TABLE = "e2e_table_sink"; + private static final String DRIVER_JAR = "https://repo1.maven.org/maven2/mysql/mysql-connector-java/8.0.16/mysql-connector-java-8.0.16.jar"; + private static final String COLUMN_STRING = "BIGINT_COL, LARGEINT_COL, SMALLINT_COL, TINYINT_COL, BOOLEAN_COL, DECIMAL_COL, DOUBLE_COL, FLOAT_COL, INT_COL, CHAR_COL, VARCHAR_11_COL, STRING_COL, DATETIME_COL, DATE_COL"; + + private static final String DDL_SOURCE = "create table " + DATABASE + "." + SOURCE_TABLE + " (\n" + + " BIGINT_COL BIGINT,\n" + + " LARGEINT_COL LARGEINT,\n" + + " SMALLINT_COL SMALLINT,\n" + + " TINYINT_COL TINYINT,\n" + + " BOOLEAN_COL BOOLEAN,\n" + + " DECIMAL_COL DECIMAL,\n" + + " DOUBLE_COL DOUBLE,\n" + + " FLOAT_COL FLOAT,\n" + + " INT_COL INT,\n" + + " CHAR_COL CHAR,\n" + + " VARCHAR_11_COL VARCHAR(11),\n" + + " STRING_COL STRING,\n" + + " DATETIME_COL DATETIME,\n" + + " DATE_COL DATE\n" + + ")ENGINE=OLAP\n" + + "DUPLICATE KEY(`BIGINT_COL`)\n" + + "DISTRIBUTED BY HASH(`BIGINT_COL`) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"replication_allocation\" = \"tag.location.default: 1\"" + + ")"; + + private static final String DDL_SINK = "create table " + DATABASE + "." + SINK_TABLE + " (\n" + + " BIGINT_COL BIGINT,\n" + + " LARGEINT_COL LARGEINT,\n" + + " SMALLINT_COL SMALLINT,\n" + + " TINYINT_COL TINYINT,\n" + + " BOOLEAN_COL BOOLEAN,\n" + + " DECIMAL_COL DECIMAL,\n" + + " DOUBLE_COL DOUBLE,\n" + + " FLOAT_COL FLOAT,\n" + + " INT_COL INT,\n" + + " CHAR_COL CHAR,\n" + + " VARCHAR_11_COL VARCHAR(11),\n" + + " STRING_COL STRING,\n" + + " DATETIME_COL DATETIME,\n" + + " DATE_COL DATE\n" + + ")ENGINE=OLAP\n" + + "DUPLICATE KEY(`BIGINT_COL`)\n" + + "DISTRIBUTED BY HASH(`BIGINT_COL`) BUCKETS 1\n" + + "PROPERTIES (\n" + + "\"replication_allocation\" = \"tag.location.default: 1\"" + + ")"; + + private static final String INIT_DATA_SQL = "insert into " + DATABASE + "." + SOURCE_TABLE + " (\n" + + " BIGINT_COL,\n" + + " LARGEINT_COL,\n" + + " SMALLINT_COL,\n" + + " TINYINT_COL,\n" + + " BOOLEAN_COL,\n" + + " DECIMAL_COL,\n" + + " DOUBLE_COL,\n" + + " FLOAT_COL,\n" + + " INT_COL,\n" + + " CHAR_COL,\n" + + " VARCHAR_11_COL,\n" + + " STRING_COL,\n" + + " DATETIME_COL,\n" + + " DATE_COL\n" + + ")values(\n" + + "\t?,?,?,?,?,?,?,?,?,?,?,?,?,?\n" + + ")"; + + private Connection jdbcConnection; + private GenericContainer dorisServer; + private static final List TEST_DATASET = generateTestDataSet(); + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = container -> { + Container.ExecResult extraCommands = container.execInContainer("bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && curl -O " + DRIVER_JAR); + Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + + @BeforeAll + @Override + public void startUp() throws Exception { + dorisServer = new GenericContainer<>(DOCKER_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(HOST) + .withLogConsumer(new Slf4jLogConsumer(log)); + dorisServer.setPortBindings(Lists.newArrayList( + String.format("%s:%s", PORT, DOCKER_PORT))); + Startables.deepStart(Stream.of(dorisServer)).join(); + log.info("Doris container started"); + + // wait for doris fully start + given().ignoreExceptions() + .await() + .atMost(360, TimeUnit.SECONDS) + .untilAsserted(this::initializeJdbcConnection); + initializeJdbcTable(); + batchInsertData(); + } + + private static List generateTestDataSet() { + + List rows = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + SeaTunnelRow row = new SeaTunnelRow( + new Object[]{ + Long.valueOf(i), + Long.valueOf(1123456), + Short.parseShort("1"), + Byte.parseByte("1"), + Boolean.FALSE, + BigDecimal.valueOf(2222243, 1), + Double.parseDouble("2222243.2222243"), + Float.parseFloat("222224"), + Integer.parseInt("1"), + "a", + "VARCHAR_COL", + "STRING_COL", + "2022-03-02 13:24:45", + "2022-03-02" + }); + rows.add(row); + } + return rows; + } + + @AfterAll + @Override + public void tearDown() throws Exception { + if (jdbcConnection != null) { + jdbcConnection.close(); + } + if (dorisServer != null) { + dorisServer.close(); + } + } + + @TestTemplate + public void testDorisSink(TestContainer container) throws IOException, InterruptedException { + Container.ExecResult execResult = container.executeJob("/doris-jdbc-to-doris.conf"); + Assertions.assertEquals(0, execResult.getExitCode()); + try { + assertHasData(SINK_TABLE); + + String sourceSql = String.format("select * from %s.%s", DATABASE, SOURCE_TABLE); + String sinkSql = String.format("select * from %s.%s", DATABASE, SINK_TABLE); + List columnList = Arrays.stream(COLUMN_STRING.split(",")).map(x -> x.trim()).collect(Collectors.toList()); + Statement sourceStatement = jdbcConnection.createStatement(); + Statement sinkStatement = jdbcConnection.createStatement(); + ResultSet sourceResultSet = sourceStatement.executeQuery(sourceSql); + ResultSet sinkResultSet = sinkStatement.executeQuery(sinkSql); + Assertions.assertEquals(sourceResultSet.getMetaData().getColumnCount(), sinkResultSet.getMetaData().getColumnCount()); + while (sourceResultSet.next()) { + if (sinkResultSet.next()) { + for (String column : columnList) { + Object source = sourceResultSet.getObject(column); + Object sink = sinkResultSet.getObject(column); + if (!Objects.deepEquals(source, sink)) { + InputStream sourceAsciiStream = sourceResultSet.getBinaryStream(column); + InputStream sinkAsciiStream = sinkResultSet.getBinaryStream(column); + String sourceValue = IOUtils.toString(sourceAsciiStream, StandardCharsets.UTF_8); + String sinkValue = IOUtils.toString(sinkAsciiStream, StandardCharsets.UTF_8); + Assertions.assertEquals(sourceValue, sinkValue); + } + } + } + } + //Check the row numbers is equal + sourceResultSet.last(); + sinkResultSet.last(); + Assertions.assertEquals(sourceResultSet.getRow(), sinkResultSet.getRow()); + clearSinkTable(); + } catch (Exception e) { + throw new RuntimeException("Get doris connection error", e); + } + } + + private void initializeJdbcConnection() throws SQLException, ClassNotFoundException, MalformedURLException, InstantiationException, IllegalAccessException { + URLClassLoader urlClassLoader = new URLClassLoader(new URL[]{new URL(DRIVER_JAR)}, DorisIT.class.getClassLoader()); + Thread.currentThread().setContextClassLoader(urlClassLoader); + Driver driver = (Driver) urlClassLoader.loadClass(DRIVER_CLASS).newInstance(); + Properties props = new Properties(); + props.put("user", USERNAME); + props.put("password", PASSWORD); + jdbcConnection = driver.connect(String.format(URL, dorisServer.getHost()), props); + } + + private void initializeJdbcTable() { + try (Statement statement = jdbcConnection.createStatement()) { + // create databases + statement.execute("create database test"); + // create source table + statement.execute(DDL_SOURCE); + // create sink table + statement.execute(DDL_SINK); + } catch (SQLException e) { + throw new RuntimeException("Initializing table failed!", e); + } + } + + private void batchInsertData() { + List rows = TEST_DATASET; + try { + jdbcConnection.setAutoCommit(false); + try (PreparedStatement preparedStatement = jdbcConnection.prepareStatement(INIT_DATA_SQL)) { + for (int i = 0; i < rows.size(); i++) { + for (int index = 0; index < rows.get(i).getFields().length; index++) { + preparedStatement.setObject(index + 1, rows.get(i).getFields()[index]); + } + preparedStatement.addBatch(); + } + preparedStatement.executeBatch(); + } + jdbcConnection.commit(); + } catch (Exception exception) { + log.error(ExceptionUtils.getMessage(exception)); + throw new RuntimeException("Get connection error", exception); + } + } + + private void assertHasData(String table) { + try (Statement statement = jdbcConnection.createStatement()) { + String sql = String.format("select * from %s.%s limit 1", DATABASE, table); + ResultSet source = statement.executeQuery(sql); + Assertions.assertTrue(source.next()); + } catch (Exception e) { + throw new RuntimeException("Test doris server image error", e); + } + } + + private void clearSinkTable() { + try (Statement statement = jdbcConnection.createStatement()) { + statement.execute(String.format("TRUNCATE TABLE %s.%s", DATABASE, SINK_TABLE)); + } catch (SQLException e) { + throw new RuntimeException("Test doris server image error", e); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/doris-jdbc-to-doris.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/doris-jdbc-to-doris.conf new file mode 100644 index 00000000000..ca6fb043718 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/resources/doris-jdbc-to-doris.conf @@ -0,0 +1,48 @@ +# +# 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 { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + driver = com.mysql.cj.jdbc.Driver + url = "jdbc:mysql://doris_e2e:9030" + user = root + password = "" + query = "select BIGINT_COL, LARGEINT_COL, SMALLINT_COL, TINYINT_COL, BOOLEAN_COL, DECIMAL_COL, DOUBLE_COL, FLOAT_COL, INT_COL, CHAR_COL, VARCHAR_11_COL, STRING_COL, DATETIME_COL, DATE_COL from `test`.`e2e_table_source`" + } +} + +transform { +} + +sink { + Doris { + nodeUrls = ["doris_e2e:8030"] + username = root + password = "" + database = "test" + table = "e2e_table_sink" + batch_max_rows = 100 + sink.properties.format = "JSON" + sink.properties.strip_outer_array = true + max_retries = 3 + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index d31cc75341a..6876327bbb4 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -37,6 +37,7 @@ connector-http-e2e connector-rabbitmq-e2e connector-kafka-e2e + connector-doris-e2e seatunnel-connector-v2-e2e From d8fe461cdb61079390bb685c9f6d172fe5e3c99a Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Fri, 2 Dec 2022 20:49:42 +0800 Subject: [PATCH 02/36] [Feature][Connector-V2][Doris] Update code style --- .../e2e/connector/doris/DorisIT.java | 50 +++++++++++-------- 1 file changed, 30 insertions(+), 20 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index d62a57f9b0d..ba53656194b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -17,8 +17,8 @@ package org.apache.seatunnel.e2e.connector.doris; -import com.google.common.collect.Lists; -import lombok.extern.slf4j.Slf4j; +import static org.awaitility.Awaitility.given; + import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.utils.ExceptionUtils; import org.apache.seatunnel.e2e.common.TestResource; @@ -26,6 +26,9 @@ import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; @@ -43,14 +46,21 @@ import java.net.URL; import java.net.URLClassLoader; import java.nio.charset.StandardCharsets; -import java.sql.*; -import java.util.*; +import java.sql.Connection; +import java.sql.Driver; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.awaitility.Awaitility.given; - @Slf4j public class DorisIT extends TestSuiteBase implements TestResource { private static final String DOCKER_IMAGE = "taozex/doris:tagname"; @@ -168,20 +178,20 @@ private static List generateTestDataSet() { for (int i = 0; i < 100; i++) { SeaTunnelRow row = new SeaTunnelRow( new Object[]{ - Long.valueOf(i), - Long.valueOf(1123456), - Short.parseShort("1"), - Byte.parseByte("1"), - Boolean.FALSE, - BigDecimal.valueOf(2222243, 1), - Double.parseDouble("2222243.2222243"), - Float.parseFloat("222224"), - Integer.parseInt("1"), - "a", - "VARCHAR_COL", - "STRING_COL", - "2022-03-02 13:24:45", - "2022-03-02" + Long.valueOf(i), + Long.valueOf(1123456), + Short.parseShort("1"), + Byte.parseByte("1"), + Boolean.FALSE, + BigDecimal.valueOf(2222243, 1), + Double.parseDouble("2222243.2222243"), + Float.parseFloat("222224"), + Integer.parseInt("1"), + "a", + "VARCHAR_COL", + "STRING_COL", + "2022-03-02 13:24:45", + "2022-03-02" }); rows.add(row); } From 4050d5349743af1be8e1152069ad0b2a2dad271f Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Fri, 2 Dec 2022 22:40:21 +0800 Subject: [PATCH 03/36] [Feature][Connector-V2][Doris] Use seatunnel format to serilize --- .../connector-doris/pom.xml | 5 ++ .../doris/serialize/DorisBaseSerializer.java | 76 ------------------- .../doris/serialize/DorisCsvSerializer.java | 47 ------------ .../doris/serialize/DorisDelimiterParser.java | 73 ------------------ .../doris/serialize/DorisISerializer.java | 27 ------- .../doris/serialize/DorisJsonSerializer.java | 46 ----------- .../doris/sink/DorisSinkWriter.java | 21 ++--- 7 files changed, 17 insertions(+), 278 deletions(-) delete mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisBaseSerializer.java delete mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisCsvSerializer.java delete mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisDelimiterParser.java delete mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisISerializer.java delete mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisJsonSerializer.java diff --git a/seatunnel-connectors-v2/connector-doris/pom.xml b/seatunnel-connectors-v2/connector-doris/pom.xml index 84c42d4f8a9..5d9e309a11c 100644 --- a/seatunnel-connectors-v2/connector-doris/pom.xml +++ b/seatunnel-connectors-v2/connector-doris/pom.xml @@ -59,5 +59,10 @@ seatunnel-format-json ${project.version} + + org.apache.seatunnel + seatunnel-format-text + ${project.version} + diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisBaseSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisBaseSerializer.java deleted file mode 100644 index 95eec722cc7..00000000000 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisBaseSerializer.java +++ /dev/null @@ -1,76 +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.doris.serialize; - -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.common.utils.DateTimeUtils; -import org.apache.seatunnel.common.utils.DateUtils; -import org.apache.seatunnel.common.utils.JsonUtils; -import org.apache.seatunnel.common.utils.TimeUtils; - -import lombok.Builder; - -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; - -public class DorisBaseSerializer { - @Builder.Default - private DateUtils.Formatter dateFormatter = DateUtils.Formatter.YYYY_MM_DD; - @Builder.Default - private DateTimeUtils.Formatter dateTimeFormatter = DateTimeUtils.Formatter.YYYY_MM_DD_HH_MM_SS; - @Builder.Default - private TimeUtils.Formatter timeFormatter = TimeUtils.Formatter.HH_MM_SS; - - protected String convert(SeaTunnelDataType dataType, Object val) { - if (val == null) { - return null; - } - switch (dataType.getSqlType()) { - case TINYINT: - case SMALLINT: - return String.valueOf(((Number) val).shortValue()); - case INT: - return String.valueOf(((Number) val).intValue()); - case BIGINT: - return String.valueOf(((Number) val).longValue()); - case FLOAT: - return String.valueOf(((Number) val).floatValue()); - case DOUBLE: - return String.valueOf(((Number) val).doubleValue()); - case DECIMAL: - case BOOLEAN: - return val.toString(); - case DATE: - return DateUtils.toString((LocalDate) val, dateFormatter); - case TIME: - return TimeUtils.toString((LocalTime) val, timeFormatter); - case TIMESTAMP: - return DateTimeUtils.toString((LocalDateTime) val, dateTimeFormatter); - case STRING: - return (String) val; - case ARRAY: - case MAP: - return JsonUtils.toJsonString(val); - case BYTES: - return new String((byte[]) val); - default: - throw new UnsupportedOperationException("Unsupported dataType: " + dataType); - } - } -} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisCsvSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisCsvSerializer.java deleted file mode 100644 index f092b5f6e0b..00000000000 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisCsvSerializer.java +++ /dev/null @@ -1,47 +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.doris.serialize; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; - -public class DorisCsvSerializer extends DorisBaseSerializer implements DorisISerializer { - private static final long serialVersionUID = 1L; - - private final String columnSeparator; - private final SeaTunnelRowType seaTunnelRowType; - - public DorisCsvSerializer(String sp, SeaTunnelRowType seaTunnelRowType) { - this.seaTunnelRowType = seaTunnelRowType; - this.columnSeparator = DorisDelimiterParser.parse(sp, "\t"); - } - - @Override - public String serialize(SeaTunnelRow row) { - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < row.getFields().length; i++) { - String value = convert(seaTunnelRowType.getFieldType(i), row.getField(i)); - sb.append(null == value ? "\\N" : value); - if (i < row.getFields().length - 1) { - sb.append(columnSeparator); - } - } - return sb.toString(); - } - -} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisDelimiterParser.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisDelimiterParser.java deleted file mode 100644 index 75bff1cce70..00000000000 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisDelimiterParser.java +++ /dev/null @@ -1,73 +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.doris.serialize; - -import com.google.common.base.Strings; - -import java.io.StringWriter; - -public class DorisDelimiterParser { - private static final int SHIFT = 4; - - private static final String HEX_STRING = "0123456789ABCDEF"; - - public static String parse(String sp, String dSp) throws RuntimeException { - if (Strings.isNullOrEmpty(sp)) { - return dSp; - } - if (!sp.toUpperCase().startsWith("\\X")) { - return sp; - } - String hexStr = sp.substring(2); - // check hex str - if (hexStr.isEmpty()) { - throw new RuntimeException("Failed to parse delimiter: `Hex str is empty`"); - } - if (hexStr.length() % 2 != 0) { - throw new RuntimeException("Failed to parse delimiter: `Hex str length error`"); - } - for (char hexChar : hexStr.toUpperCase().toCharArray()) { - if (HEX_STRING.indexOf(hexChar) == -1) { - throw new RuntimeException("Failed to parse delimiter: `Hex str format error`"); - } - } - // transform to separator - StringWriter writer = new StringWriter(); - for (byte b : hexStrToBytes(hexStr)) { - writer.append((char) b); - } - return writer.toString(); - } - - private static byte[] hexStrToBytes(String hexStr) { - String upperHexStr = hexStr.toUpperCase(); - int length = upperHexStr.length() / 2; - char[] hexChars = upperHexStr.toCharArray(); - byte[] bytes = new byte[length]; - for (int i = 0; i < length; i++) { - int pos = i * 2; - bytes[i] = (byte) (charToByte(hexChars[pos]) << SHIFT | charToByte(hexChars[pos + 1])); - } - return bytes; - } - - private static byte charToByte(char c) { - return (byte) HEX_STRING.indexOf(c); - } -} - diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisISerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisISerializer.java deleted file mode 100644 index 37c483cd4e0..00000000000 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisISerializer.java +++ /dev/null @@ -1,27 +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.doris.serialize; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -import java.io.Serializable; - -public interface DorisISerializer extends Serializable { - - String serialize(SeaTunnelRow seaTunnelRow); -} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisJsonSerializer.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisJsonSerializer.java deleted file mode 100644 index 016ed0fadcb..00000000000 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/serialize/DorisJsonSerializer.java +++ /dev/null @@ -1,46 +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.doris.serialize; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.common.utils.JsonUtils; - -import java.util.HashMap; -import java.util.Map; - -public class DorisJsonSerializer extends DorisBaseSerializer implements DorisISerializer { - - private static final long serialVersionUID = 1L; - private final SeaTunnelRowType seaTunnelRowType; - - public DorisJsonSerializer(SeaTunnelRowType seaTunnelRowType) { - this.seaTunnelRowType = seaTunnelRowType; - } - - @Override - public String serialize(SeaTunnelRow row) { - Map rowMap = new HashMap<>(row.getFields().length); - - for (int i = 0; i < row.getFields().length; i++) { - String value = convert(seaTunnelRowType.getFieldType(i), row.getField(i)); - rowMap.put(seaTunnelRowType.getFieldName(i), value); - } - return JsonUtils.toJsonString(rowMap); - } -} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java index ddbe6cd24d3..4662d8c99fe 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java @@ -17,14 +17,14 @@ package org.apache.seatunnel.connectors.doris.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.doris.client.DorisSinkManager; import org.apache.seatunnel.connectors.doris.config.SinkConfig; -import org.apache.seatunnel.connectors.doris.serialize.DorisCsvSerializer; -import org.apache.seatunnel.connectors.doris.serialize.DorisISerializer; -import org.apache.seatunnel.connectors.doris.serialize.DorisJsonSerializer; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.format.json.JsonSerializationSchema; +import org.apache.seatunnel.format.text.TextSerializationSchema; import org.apache.seatunnel.shade.com.typesafe.config.Config; @@ -40,20 +40,20 @@ @Slf4j public class DorisSinkWriter extends AbstractSinkWriter { - private final DorisISerializer serializer; + private final SerializationSchema serializationSchema; private final DorisSinkManager manager; public DorisSinkWriter(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { SinkConfig sinkConfig = SinkConfig.loadConfig(pluginConfig); List fieldNames = Arrays.stream(seaTunnelRowType.getFieldNames()).collect(Collectors.toList()); - this.serializer = createSerializer(sinkConfig, seaTunnelRowType); + this.serializationSchema = createSerializer(sinkConfig, seaTunnelRowType); this.manager = new DorisSinkManager(sinkConfig, fieldNames); } @Override public void write(SeaTunnelRow element) throws IOException { - String record = serializer.serialize(element); + String record = new String(serializationSchema.serialize(element)); manager.write(record); } @@ -77,12 +77,15 @@ public void close() throws IOException { } } - public static DorisISerializer createSerializer(SinkConfig sinkConfig, SeaTunnelRowType seaTunnelRowType) { + public static SerializationSchema createSerializer(SinkConfig sinkConfig, SeaTunnelRowType seaTunnelRowType) { if (SinkConfig.StreamLoadFormat.CSV.equals(sinkConfig.getLoadFormat())) { - return new DorisCsvSerializer(sinkConfig.getColumnSeparator(), seaTunnelRowType); + return TextSerializationSchema.builder() + .seaTunnelRowType(seaTunnelRowType) + .delimiter(sinkConfig.getColumnSeparator()) + .build(); } if (SinkConfig.StreamLoadFormat.JSON.equals(sinkConfig.getLoadFormat())) { - return new DorisJsonSerializer(seaTunnelRowType); + return new JsonSerializationSchema(seaTunnelRowType); } throw new RuntimeException("Failed to create row serializer, unsupported `format` from stream load properties."); } From 9a08512062222c4ac2dafcafa3a6d7556c4732e8 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Sat, 3 Dec 2022 01:15:17 +0800 Subject: [PATCH 04/36] [Feature][Connector-V2][Doris] Add delimiterParse util --- docs/en/connector-v2/sink/Doris.md | 130 ++++++++++++++++++ .../doris/client/DorisStreamLoadVisitor.java | 29 ++-- .../doris/sink/DorisSinkWriter.java | 4 +- .../doris/util/DelimiterParserUtil.java | 72 ++++++++++ 4 files changed, 218 insertions(+), 17 deletions(-) create mode 100644 docs/en/connector-v2/sink/Doris.md create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DelimiterParserUtil.java diff --git a/docs/en/connector-v2/sink/Doris.md b/docs/en/connector-v2/sink/Doris.md new file mode 100644 index 00000000000..568a047fee5 --- /dev/null +++ b/docs/en/connector-v2/sink/Doris.md @@ -0,0 +1,130 @@ +# Doris + +> Doris sink connector + +## Description +Used to send data to Doris. Both support streaming and batch mode. +The internal implementation of Doris sink connector is cached and imported by stream load in batches. +## Key features + +- [ ] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [schema projection](../../concept/connector-v2-features.md) + +## Options + +| name | type | required | default value | +|-----------------------------|------------------------------|----------|-----------------| +| node_urls | list | yes | - | +| username | string | yes | - | +| password | string | yes | - | +| database | string | yes | - | +| table | string | yes | - | +| labelPrefix | string | no | - | +| batch_max_rows | long | no | 1024 | +| batch_max_bytes | int | no | 5 * 1024 * 1024 | +| batch_interval_ms | int | no | - | +| max_retries | int | no | - | +| retry_backoff_multiplier_ms | int | no | - | +| max_retry_backoff_ms | int | no | - | +| sink.properties.* | doris stream load config | no | - | + +### node_urls [list] + +`Doris` cluster address, the format is `["fe_ip:fe_http_port", ...]` + +### username [string] + +`Doris` user username + +### password [string] + +`Doris` user password + +### database [string] + +The name of `Doris` database + +### table [string] + +The name of `Doris` table + +### labelPrefix [string] + +The prefix of `Doris` stream load label + +### batch_max_rows [long] + +For batch writing, when the number of buffers reaches the number of `batch_max_rows` or the byte size of `batch_max_bytes` or the time reaches `batch_interval_ms`, the data will be flushed into the Doris + +### batch_max_bytes [int] + +For batch writing, when the number of buffers reaches the number of `batch_max_rows` or the byte size of `batch_max_bytes` or the time reaches `batch_interval_ms`, the data will be flushed into the Doris + +### batch_interval_ms [int] + +For batch writing, when the number of buffers reaches the number of `batch_max_rows` or the byte size of `batch_max_bytes` or the time reaches `batch_interval_ms`, the data will be flushed into the Doris + +### max_retries [int] + +The number of retries to flush failed + +### retry_backoff_multiplier_ms [int] + +Using as a multiplier for generating the next delay for backoff + +### max_retry_backoff_ms [int] + +The amount of time to wait before attempting to retry a request to `Doris` + +### sink.properties.* [doris stream load config] + +The parameter of the stream load `data_desc` +The way to specify the parameter is to add the prefix `sink.properties.` to the original stream load parameter name +For example, the way to specify `strip_outer_array` is: `sink.properties.strip_outer_array` + +#### Supported import data formats + +The supported formats include CSV and JSON. Default value: CSV + +## Example + +Use JSON format to import data + +``` +sink { + Doris { + nodeUrls = ["e2e_dorisdb:8030"] + username = root + password = "" + database = "test" + table = "e2e_table_sink" + batch_max_rows = 100 + sink.properties.format = "JSON" + sink.properties.strip_outer_array = true + } +} + +``` + +Use CSV format to import data + +``` +sink { + Doris { + nodeUrls = ["e2e_dorisdb:8030"] + username = root + password = "" + database = "test" + table = "e2e_table_sink" + batch_max_rows = 100 + sink.properties.format = "CSV" + sink.properties.column_separator = "," + } +} +``` + +## Changelog + +### next version + +- Add Doris Sink Connector \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java index 2955dec03ff..9a360d36918 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java @@ -19,11 +19,10 @@ import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.connectors.doris.config.SinkConfig; -import org.apache.seatunnel.connectors.doris.serialize.DorisDelimiterParser; +import org.apache.seatunnel.connectors.doris.util.DelimiterParserUtil; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.codec.binary.Base64; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.ByteBuffer; @@ -34,10 +33,8 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +@Slf4j public class DorisStreamLoadVisitor { - - private static final Logger LOG = LoggerFactory.getLogger(DorisStreamLoadVisitor.class); - private final HttpHelper httpHelper = new HttpHelper(); private static final int MAX_SLEEP_TIME = 5; @@ -71,17 +68,17 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { .append(sinkConfig.getTable()) .append("/_stream_load") .toString(); - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("Start to join batch data: rows[%d] bytes[%d] label[%s].", flushData.getRows().size(), flushData.getBytes(), flushData.getLabel())); + if (log.isDebugEnabled()) { + log.debug(String.format("Start to join batch data: rows[%d] bytes[%d] label[%s].", flushData.getRows().size(), flushData.getBytes(), flushData.getLabel())); } Map loadResult = httpHelper.doHttpPut(loadUrl, joinRows(flushData.getRows(), flushData.getBytes().intValue()), getStreamLoadHttpHeader(flushData.getLabel())); final String keyStatus = "Status"; if (null == loadResult || !loadResult.containsKey(keyStatus)) { - LOG.error("unknown result status. {}", loadResult); + log.error("unknown result status. {}", loadResult); throw new IOException("Unable to flush data to Doris: unknown result status. " + loadResult); } - if (LOG.isDebugEnabled()) { - LOG.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); + if (log.isDebugEnabled()) { + log.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); } if (RESULT_FAILED.equals(loadResult.get(keyStatus))) { StringBuilder errorBuilder = new StringBuilder("Failed to flush data to Doris.\n"); @@ -90,12 +87,12 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { errorBuilder.append('\n'); } if (loadResult.containsKey("ErrorURL")) { - LOG.error("StreamLoad response: {}", loadResult); + log.error("StreamLoad response: {}", loadResult); try { errorBuilder.append(httpHelper.doHttpGet(loadResult.get("ErrorURL").toString())); errorBuilder.append('\n'); } catch (IOException e) { - LOG.warn("Get Error URL failed. {} ", loadResult.get("ErrorURL"), e); + log.warn("Get Error URL failed. {} ", loadResult.get("ErrorURL"), e); } } else { errorBuilder.append(JsonUtils.toJsonString(loadResult)); @@ -103,7 +100,7 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { } throw new IOException(errorBuilder.toString()); } else if (RESULT_LABEL_EXISTED.equals(loadResult.get(keyStatus))) { - LOG.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); + log.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); // has to block-checking the state to get the final result checkLabelState(host, flushData.getLabel()); } @@ -125,7 +122,7 @@ private String getAvailableHost() { private byte[] joinRows(List rows, int totalBytes) { if (SinkConfig.StreamLoadFormat.CSV.equals(sinkConfig.getLoadFormat())) { Map props = sinkConfig.getStreamLoadProps(); - byte[] lineDelimiter = DorisDelimiterParser.parse((String) props.get("row_delimiter"), "\n").getBytes(StandardCharsets.UTF_8); + byte[] lineDelimiter = DelimiterParserUtil.parse((String) props.get("row_delimiter"), "\n").getBytes(StandardCharsets.UTF_8); ByteBuffer bos = ByteBuffer.allocate(totalBytes + rows.size() * lineDelimiter.length); for (byte[] row : rows) { bos.put(row); @@ -173,7 +170,7 @@ private void checkLabelState(String host, String label) throws IOException { throw new IOException(String.format("Failed to flush data to Doris, Error " + "could not get the final state of label[%s]. response[%s]\n", label, JsonUtils.toJsonString(result)), null); } - LOG.info(String.format("Checking label[%s] state[%s]\n", label, labelState)); + log.info(String.format("Checking label[%s] state[%s]\n", label, labelState)); switch (labelState) { case LAEBL_STATE_VISIBLE: case LAEBL_STATE_COMMITTED: diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java index 4662d8c99fe..884576d4a06 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java @@ -22,6 +22,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.doris.client.DorisSinkManager; import org.apache.seatunnel.connectors.doris.config.SinkConfig; +import org.apache.seatunnel.connectors.doris.util.DelimiterParserUtil; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; import org.apache.seatunnel.format.json.JsonSerializationSchema; import org.apache.seatunnel.format.text.TextSerializationSchema; @@ -79,9 +80,10 @@ public void close() throws IOException { public static SerializationSchema createSerializer(SinkConfig sinkConfig, SeaTunnelRowType seaTunnelRowType) { if (SinkConfig.StreamLoadFormat.CSV.equals(sinkConfig.getLoadFormat())) { + String columnSeparator = DelimiterParserUtil.parse(sinkConfig.getColumnSeparator(), "\t"); return TextSerializationSchema.builder() .seaTunnelRowType(seaTunnelRowType) - .delimiter(sinkConfig.getColumnSeparator()) + .delimiter(columnSeparator) .build(); } if (SinkConfig.StreamLoadFormat.JSON.equals(sinkConfig.getLoadFormat())) { diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DelimiterParserUtil.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DelimiterParserUtil.java new file mode 100644 index 00000000000..125127a4699 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DelimiterParserUtil.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.doris.util; + +import com.google.common.base.Strings; + +import java.io.StringWriter; + +public class DelimiterParserUtil { + private static final int SHIFT = 4; + + private static final String HEX_STRING = "0123456789ABCDEF"; + + public static String parse(String sp, String dSp) throws RuntimeException { + if (Strings.isNullOrEmpty(sp)) { + return dSp; + } + if (!sp.toUpperCase().startsWith("\\X")) { + return sp; + } + String hexStr = sp.substring(2); + // check hex str + if (hexStr.isEmpty()) { + throw new RuntimeException("Failed to parse delimiter: `Hex str is empty`"); + } + if (hexStr.length() % 2 != 0) { + throw new RuntimeException("Failed to parse delimiter: `Hex str length error`"); + } + for (char hexChar : hexStr.toUpperCase().toCharArray()) { + if (HEX_STRING.indexOf(hexChar) == -1) { + throw new RuntimeException("Failed to parse delimiter: `Hex str format error`"); + } + } + // transform to separator + StringWriter writer = new StringWriter(); + for (byte b : hexStrToBytes(hexStr)) { + writer.append((char) b); + } + return writer.toString(); + } + + private static byte[] hexStrToBytes(String hexStr) { + String upperHexStr = hexStr.toUpperCase(); + int length = upperHexStr.length() / 2; + char[] hexChars = upperHexStr.toCharArray(); + byte[] bytes = new byte[length]; + for (int i = 0; i < length; i++) { + int pos = i * 2; + bytes[i] = (byte) (charToByte(hexChars[pos]) << SHIFT | charToByte(hexChars[pos + 1])); + } + return bytes; + } + + private static byte charToByte(char c) { + return (byte) HEX_STRING.indexOf(c); + } +} From 454f1b5115d024c302d1939b203eb04d38fba8aa Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Sat, 3 Dec 2022 01:50:01 +0800 Subject: [PATCH 05/36] [Feature][Connector-V2][Doris] Unify exception --- .../Error-Quick-Reference-Manual.md | 6 +++ .../doris/client/DorisSinkManager.java | 9 ++-- .../doris/client/DorisStreamLoadVisitor.java | 22 +++++---- .../exception/DorisConnectorErrorCode.java | 43 +++++++++++++++++ .../exception/DorisConnectorException.java | 46 +++++++++++++++++++ .../doris/util/DelimiterParserUtil.java | 9 ++-- 6 files changed, 119 insertions(+), 16 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java create mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorException.java diff --git a/docs/en/connector-v2/Error-Quick-Reference-Manual.md b/docs/en/connector-v2/Error-Quick-Reference-Manual.md index 4ed61a0df35..fccb84c144d 100644 --- a/docs/en/connector-v2/Error-Quick-Reference-Manual.md +++ b/docs/en/connector-v2/Error-Quick-Reference-Manual.md @@ -159,6 +159,12 @@ problems encountered by users. | HUDI-01 | Create ParquetMetadata failed | When the user encounters this error code, it indicates that ParquetMetadata creation failed. Please check | | HUDI-02 | Kerberos Authorized failed | When the user encounters this error code, it indicates that Kerberos authorization failed. Please check | +## Doris Connector Error Codes + +| code | description | solution | +|----------|-------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------| +| Doris-01 | Writing records to Doris failed. | When users encounter this error code, it means that writing records to Doris failed, please check data from files whether is correct | + ## Clickhouse Connector Error Codes | code | description | solution | diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java index 9348cfa940b..31bda854223 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java @@ -17,7 +17,10 @@ package org.apache.seatunnel.connectors.doris.client; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.connectors.doris.config.SinkConfig; +import org.apache.seatunnel.connectors.doris.exception.DorisConnectorErrorCode; +import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException; import com.google.common.base.Strings; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -116,10 +119,10 @@ public synchronized void flush() throws IOException { } catch (Exception e) { log.warn("Writing records to Doris failed, retry times = {}", i, e); if (i >= sinkConfig.getMaxRetries()) { - throw new IOException("Writing records to Doris failed.", e); + throw new DorisConnectorException(DorisConnectorErrorCode.WRITE_RECORDS_FAILED, "The number of retries was exceeded,writing records to Doris failed.", e); } - if (e instanceof DorisStreamLoadFailedException && ((DorisStreamLoadFailedException) e).needReCreateLabel()) { + if (e instanceof DorisConnectorException && ((DorisConnectorException) e).needReCreateLabel()) { String newLabel = createBatchLabel(); log.warn(String.format("Batch label changed from [%s] to [%s]", tuple.getLabel(), newLabel)); tuple.setLabel(newLabel); @@ -143,7 +146,7 @@ public synchronized void flush() throws IOException { private void checkFlushException() { if (flushException != null) { - throw new RuntimeException("Writing records to Doris failed.", flushException); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, flushException); } } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java index 9a360d36918..64d803dd485 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java @@ -17,8 +17,10 @@ package org.apache.seatunnel.connectors.doris.client; +import org.apache.seatunnel.common.exception.CommonErrorCode; import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.connectors.doris.config.SinkConfig; +import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException; import org.apache.seatunnel.connectors.doris.util.DelimiterParserUtil; import lombok.extern.slf4j.Slf4j; @@ -59,7 +61,7 @@ public DorisStreamLoadVisitor(SinkConfig sinkConfig, List fieldNames) { public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { String host = getAvailableHost(); if (null == host) { - throw new IOException("None of the host in `load_url` could be connected."); + throw new DorisConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT, "None of the host in `load_url` could be connected."); } String loadUrl = new StringBuilder(host) .append("/api/") @@ -75,7 +77,7 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { final String keyStatus = "Status"; if (null == loadResult || !loadResult.containsKey(keyStatus)) { log.error("unknown result status. {}", loadResult); - throw new IOException("Unable to flush data to Doris: unknown result status. " + loadResult); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, "Unable to flush data to Doris: unknown result status. " + loadResult); } if (log.isDebugEnabled()) { log.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); @@ -98,7 +100,7 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { errorBuilder.append(JsonUtils.toJsonString(loadResult)); errorBuilder.append('\n'); } - throw new IOException(errorBuilder.toString()); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, errorBuilder.toString()); } else if (RESULT_LABEL_EXISTED.equals(loadResult.get(keyStatus))) { log.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); // has to block-checking the state to get the final result @@ -146,7 +148,7 @@ private byte[] joinRows(List rows, int totalBytes) { bos.put("]".getBytes(StandardCharsets.UTF_8)); return bos.array(); } - throw new RuntimeException("Failed to join rows data, unsupported `format` from stream load properties:"); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, "Failed to join rows data, unsupported `format` from stream load properties:"); } @SuppressWarnings("unchecked") @@ -162,12 +164,12 @@ private void checkLabelState(String host, String label) throws IOException { String queryLoadStateUrl = new StringBuilder(host).append("/api/").append(sinkConfig.getDatabase()).append("/get_load_state?label=").append(label).toString(); Map result = httpHelper.doHttpGet(queryLoadStateUrl, getLoadStateHttpHeader(label)); if (result == null) { - throw new IOException(String.format("Failed to flush data to Doris, Error " + + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, String.format("Failed to flush data to Doris, Error " + "could not get the final state of label[%s].\n", label), null); } String labelState = (String) result.get("state"); if (null == labelState) { - throw new IOException(String.format("Failed to flush data to Doris, Error " + + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, String.format("Failed to flush data to Doris, Error " + "could not get the final state of label[%s]. response[%s]\n", label, JsonUtils.toJsonString(result)), null); } log.info(String.format("Checking label[%s] state[%s]\n", label, labelState)); @@ -178,12 +180,12 @@ private void checkLabelState(String host, String label) throws IOException { case RESULT_LABEL_PREPARE: continue; case RESULT_LABEL_ABORTED: - throw new DorisStreamLoadFailedException(String.format("Failed to flush data to Doris, Error " + - "label[%s] state[%s]\n", label, labelState), null, true); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, String.format("Failed to flush data to Doris, Error " + + "label[%s] state[%s]\n", label, labelState), true); case RESULT_LABEL_UNKNOWN: default: - throw new DorisStreamLoadFailedException(String.format("Failed to flush data to Doris, Error " + - "label[%s] state[%s]\n", label, labelState), null); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, String.format("Failed to flush data to StarRocks, Error " + + "label[%s] state[%s]\n", label, labelState)); } } catch (IOException e) { throw new IOException(e); diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java new file mode 100644 index 00000000000..716f9e9c9b2 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.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.doris.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; + +public enum DorisConnectorErrorCode implements SeaTunnelErrorCode { + WRITE_RECORDS_FAILED("STARROCKS-01", "Writing records to StarRocks failed."); + + private final String code; + private final String description; + + DorisConnectorErrorCode(String code, String description) { + this.code = code; + this.description = description; + } + + @Override + public String getCode() { + return null; + } + + @Override + public String getDescription() { + return null; + } + +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorException.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorException.java new file mode 100644 index 00000000000..49dde0637a3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorException.java @@ -0,0 +1,46 @@ +/* + * 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.doris.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class DorisConnectorException extends SeaTunnelRuntimeException { + private boolean reCreateLabel; + + public DorisConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } + + public DorisConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, boolean reCreateLabel) { + super(seaTunnelErrorCode, errorMessage); + this.reCreateLabel = reCreateLabel; + } + + public DorisConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { + super(seaTunnelErrorCode, errorMessage, cause); + } + + public DorisConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, Throwable cause) { + super(seaTunnelErrorCode, cause); + } + + public boolean needReCreateLabel() { + return reCreateLabel; + } +} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DelimiterParserUtil.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DelimiterParserUtil.java index 125127a4699..7f6512e44bd 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DelimiterParserUtil.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/util/DelimiterParserUtil.java @@ -17,6 +17,9 @@ package org.apache.seatunnel.connectors.doris.util; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException; + import com.google.common.base.Strings; import java.io.StringWriter; @@ -36,14 +39,14 @@ public static String parse(String sp, String dSp) throws RuntimeException { String hexStr = sp.substring(2); // check hex str if (hexStr.isEmpty()) { - throw new RuntimeException("Failed to parse delimiter: `Hex str is empty`"); + throw new DorisConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT, "Failed to parse delimiter: `Hex str is empty`"); } if (hexStr.length() % 2 != 0) { - throw new RuntimeException("Failed to parse delimiter: `Hex str length error`"); + throw new DorisConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT, "Failed to parse delimiter: `Hex str is empty`"); } for (char hexChar : hexStr.toUpperCase().toCharArray()) { if (HEX_STRING.indexOf(hexChar) == -1) { - throw new RuntimeException("Failed to parse delimiter: `Hex str format error`"); + throw new DorisConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT, "Failed to parse delimiter: `Hex str is empty`"); } } // transform to separator From 0094168b2d1391958fc466e04ff7eab4ec2df9bb Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Sat, 3 Dec 2022 01:51:43 +0800 Subject: [PATCH 06/36] [Feature][Connector-V2][Doris] Update error code --- .../connectors/doris/client/DorisStreamLoadVisitor.java | 2 +- .../connectors/doris/exception/DorisConnectorErrorCode.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java index 64d803dd485..8596f4a9744 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java @@ -184,7 +184,7 @@ private void checkLabelState(String host, String label) throws IOException { "label[%s] state[%s]\n", label, labelState), true); case RESULT_LABEL_UNKNOWN: default: - throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, String.format("Failed to flush data to StarRocks, Error " + + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, String.format("Failed to flush data to Doris, Error " + "label[%s] state[%s]\n", label, labelState)); } } catch (IOException e) { diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java index 716f9e9c9b2..83e669e76f4 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java @@ -20,7 +20,7 @@ import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; public enum DorisConnectorErrorCode implements SeaTunnelErrorCode { - WRITE_RECORDS_FAILED("STARROCKS-01", "Writing records to StarRocks failed."); + WRITE_RECORDS_FAILED("DORIS-01", "Writing records to Doris failed."); private final String code; private final String description; From 45d876068555c021c876a09de8b5328308e25ee8 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Sat, 3 Dec 2022 01:55:09 +0800 Subject: [PATCH 07/36] [Feature][Connector-V2][Doris] Update code --- .../connectors/doris/exception/DorisConnectorErrorCode.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java index 83e669e76f4..1d92e93dcbd 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/exception/DorisConnectorErrorCode.java @@ -32,12 +32,12 @@ public enum DorisConnectorErrorCode implements SeaTunnelErrorCode { @Override public String getCode() { - return null; + return code; } @Override public String getDescription() { - return null; + return description; } } From 422519ffdaadac7062b45424e46fcb9e14ee0dd8 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Sun, 4 Dec 2022 13:16:38 +0800 Subject: [PATCH 08/36] [Feature][Connector-V2][Doris] Update e2e code --- .../org/apache/seatunnel/e2e/connector/doris/DorisIT.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index ba53656194b..685c79d394a 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -260,6 +260,12 @@ private void initializeJdbcConnection() throws SQLException, ClassNotFoundExcept } private void initializeJdbcTable() { + //just for test + try { + Thread.sleep(300000); + } catch (InterruptedException e) { + e.printStackTrace(); + } try (Statement statement = jdbcConnection.createStatement()) { // create databases statement.execute("create database test"); From ecf85d7df40131d6bd95331316521219dc3cb112 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Sun, 4 Dec 2022 22:58:53 +0800 Subject: [PATCH 09/36] [Feature][Connector-V2][Doris] Update code --- .../java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index 685c79d394a..47505c87251 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -184,7 +184,7 @@ private static List generateTestDataSet() { Byte.parseByte("1"), Boolean.FALSE, BigDecimal.valueOf(2222243, 1), - Double.parseDouble("2222243.2222243"), + Double.parseDouble("3.14"), Float.parseFloat("222224"), Integer.parseInt("1"), "a", From cccf497bd561f8e5f54d46894ffa6ce0ec9e4d7e Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 00:10:22 +0800 Subject: [PATCH 10/36] [Feature][Connector-V2][Doris] Use String format --- .../doris/client/DorisSinkManager.java | 4 ++-- .../doris/client/DorisStreamLoadVisitor.java | 16 ++++------------ .../connectors/doris/sink/DorisSink.java | 6 +++++- .../connectors/doris/sink/DorisSinkWriter.java | 9 ++++++--- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java index 31bda854223..869d7774a48 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java @@ -134,8 +134,8 @@ public synchronized void flush() throws IOException { Thread.sleep(backoff); } catch (InterruptedException ex) { Thread.currentThread().interrupt(); - throw new IOException( - "Unable to flush; interrupted while doing another attempt.", e); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, + "Unable to flush, interrupted while doing another attempt.", e); } } } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java index 8596f4a9744..55592bc6c9a 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java @@ -63,20 +63,13 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { if (null == host) { throw new DorisConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT, "None of the host in `load_url` could be connected."); } - String loadUrl = new StringBuilder(host) - .append("/api/") - .append(sinkConfig.getDatabase()) - .append("/") - .append(sinkConfig.getTable()) - .append("/_stream_load") - .toString(); + String loadUrl = String.format("%s/api/%s/%s/_stream_load", host, sinkConfig.getDatabase(), sinkConfig.getTable()); if (log.isDebugEnabled()) { log.debug(String.format("Start to join batch data: rows[%d] bytes[%d] label[%s].", flushData.getRows().size(), flushData.getBytes(), flushData.getLabel())); } Map loadResult = httpHelper.doHttpPut(loadUrl, joinRows(flushData.getRows(), flushData.getBytes().intValue()), getStreamLoadHttpHeader(flushData.getLabel())); final String keyStatus = "Status"; if (null == loadResult || !loadResult.containsKey(keyStatus)) { - log.error("unknown result status. {}", loadResult); throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, "Unable to flush data to Doris: unknown result status. " + loadResult); } if (log.isDebugEnabled()) { @@ -89,7 +82,6 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { errorBuilder.append('\n'); } if (loadResult.containsKey("ErrorURL")) { - log.error("StreamLoad response: {}", loadResult); try { errorBuilder.append(httpHelper.doHttpGet(loadResult.get("ErrorURL").toString())); errorBuilder.append('\n'); @@ -113,7 +105,7 @@ private String getAvailableHost() { List hostList = sinkConfig.getNodeUrls(); long tmp = pos + hostList.size(); for (; pos < tmp; pos++) { - String host = new StringBuilder("http://").append(hostList.get((int) (pos % hostList.size()))).toString(); + String host = String.format("http://%s", hostList.get((int) (pos % hostList.size()))); if (httpHelper.tryHttpConnection(host)) { return host; } @@ -161,7 +153,7 @@ private void checkLabelState(String host, String label) throws IOException { break; } try { - String queryLoadStateUrl = new StringBuilder(host).append("/api/").append(sinkConfig.getDatabase()).append("/get_load_state?label=").append(label).toString(); + String queryLoadStateUrl = String.format("%s/api/%s/get_load_state?label=%s", host, sinkConfig.getDatabase(), label); Map result = httpHelper.doHttpGet(queryLoadStateUrl, getLoadStateHttpHeader(label)); if (result == null) { throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, String.format("Failed to flush data to Doris, Error " + @@ -188,7 +180,7 @@ private void checkLabelState(String host, String label) throws IOException { "label[%s] state[%s]\n", label, labelState)); } } catch (IOException e) { - throw new IOException(e); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, e); } } } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java index 89ba9737b23..791e82f13de 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java @@ -24,6 +24,7 @@ import static org.apache.seatunnel.connectors.doris.config.SinkConfig.USERNAME; import org.apache.seatunnel.api.common.PrepareFailException; +import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.sink.SinkWriter; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -32,6 +33,7 @@ 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.doris.exception.DorisConnectorException; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; @@ -55,7 +57,9 @@ public void prepare(Config pluginConfig) throws PrepareFailException { this.pluginConfig = pluginConfig; CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, NODE_URLS.key(), DATABASE.key(), TABLE.key(), USERNAME.key(), PASSWORD.key()); if (!result.isSuccess()) { - throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg()); + throw new DorisConnectorException(SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + String.format("PluginName: %s, PluginType: %s, Message: %s", + getPluginName(), PluginType.SINK, result.getMsg())); } } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java index 884576d4a06..ff380fe048e 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java @@ -20,8 +20,10 @@ 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.common.exception.CommonErrorCode; import org.apache.seatunnel.connectors.doris.client.DorisSinkManager; import org.apache.seatunnel.connectors.doris.config.SinkConfig; +import org.apache.seatunnel.connectors.doris.exception.DorisConnectorException; import org.apache.seatunnel.connectors.doris.util.DelimiterParserUtil; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; import org.apache.seatunnel.format.json.JsonSerializationSchema; @@ -73,8 +75,8 @@ public void close() throws IOException { manager.close(); } } catch (IOException e) { - log.error("Close doris manager failed.", e); - throw new IOException("Close doris manager failed.", e); + throw new DorisConnectorException(CommonErrorCode.WRITER_OPERATION_FAILED, + "Close doris manager failed.", e); } } @@ -89,6 +91,7 @@ public static SerializationSchema createSerializer(SinkConfig sinkConfig, SeaTun if (SinkConfig.StreamLoadFormat.JSON.equals(sinkConfig.getLoadFormat())) { return new JsonSerializationSchema(seaTunnelRowType); } - throw new RuntimeException("Failed to create row serializer, unsupported `format` from stream load properties."); + throw new DorisConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT, + "Failed to create row serializer, unsupported `format` from stream load properties."); } } From 60a1150263cdce667720b601ab2907834bdbf59b Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 16:10:59 +0800 Subject: [PATCH 11/36] [Feature][Connector-V2][Doris] Update code --- .../doris/client/DorisFlushTuple.java | 6 +-- .../doris/client/DorisSinkManager.java | 4 +- .../DorisStreamLoadFailedException.java | 49 ------------------- .../doris/client/DorisStreamLoadVisitor.java | 6 +-- .../connectors/doris/config/SinkConfig.java | 8 +-- 5 files changed, 9 insertions(+), 64 deletions(-) delete mode 100644 seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadFailedException.java diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisFlushTuple.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisFlushTuple.java index cfae19a8621..7492ec259d4 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisFlushTuple.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisFlushTuple.java @@ -18,14 +18,12 @@ package org.apache.seatunnel.connectors.doris.client; import lombok.AllArgsConstructor; -import lombok.Getter; -import lombok.Setter; +import lombok.Data; import java.util.List; @AllArgsConstructor -@Getter -@Setter +@Data public class DorisFlushTuple { private String label; private Long bytes; diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java index 869d7774a48..600e9081dc3 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java @@ -42,7 +42,7 @@ public class DorisSinkManager { private final SinkConfig sinkConfig; private final List batchList; - private DorisStreamLoadVisitor dorisStreamLoadVisitor; + private final DorisStreamLoadVisitor dorisStreamLoadVisitor; private ScheduledExecutorService scheduler; private ScheduledFuture scheduledFuture; private volatile boolean initialize; @@ -50,7 +50,7 @@ public class DorisSinkManager { private int batchRowCount = 0; private long batchBytesSize = 0; - private Integer batchIntervalMs; + private final Integer batchIntervalMs; public DorisSinkManager(SinkConfig sinkConfig, List fileNames) { this.sinkConfig = sinkConfig; diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadFailedException.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadFailedException.java deleted file mode 100644 index 56be7358fc1..00000000000 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadFailedException.java +++ /dev/null @@ -1,49 +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.doris.client; - -import java.io.IOException; -import java.util.Map; - -public class DorisStreamLoadFailedException extends IOException { - - static final long serialVersionUID = 1L; - - private final Map response; - private boolean reCreateLabel; - - public DorisStreamLoadFailedException(String message, Map response) { - super(message); - this.response = response; - } - - public DorisStreamLoadFailedException(String message, Map response, boolean reCreateLabel) { - super(message); - this.response = response; - this.reCreateLabel = reCreateLabel; - } - - public Map getFailedResponse() { - return response; - } - - public boolean needReCreateLabel() { - return reCreateLabel; - } - -} diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java index 55592bc6c9a..8e14292f842 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java @@ -73,7 +73,7 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, "Unable to flush data to Doris: unknown result status. " + loadResult); } if (log.isDebugEnabled()) { - log.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); + log.debug(String.format("StreamLoad response:\n%s"), JsonUtils.toJsonString(loadResult)); } if (RESULT_FAILED.equals(loadResult.get(keyStatus))) { StringBuilder errorBuilder = new StringBuilder("Failed to flush data to Doris.\n"); @@ -94,7 +94,7 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { } throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, errorBuilder.toString()); } else if (RESULT_LABEL_EXISTED.equals(loadResult.get(keyStatus))) { - log.debug(new StringBuilder("StreamLoad response:\n").append(JsonUtils.toJsonString(loadResult)).toString()); + log.debug(String.format("StreamLoad response:\n%s"), JsonUtils.toJsonString(loadResult)); // has to block-checking the state to get the final result checkLabelState(host, flushData.getLabel()); } @@ -188,7 +188,7 @@ private void checkLabelState(String host, String label) throws IOException { private String getBasicAuthHeader(String username, String password) { String auth = username + ":" + password; byte[] encodedAuth = Base64.encodeBase64(auth.getBytes(StandardCharsets.UTF_8)); - return new StringBuilder("Basic ").append(new String(encodedAuth)).toString(); + return String.format("Basic %s", new String(encodedAuth)); } private Map getStreamLoadHttpHeader(String label) { diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java index 14e448447e1..cdeefe5783b 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java @@ -23,17 +23,13 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; -import lombok.Getter; -import lombok.Setter; -import lombok.ToString; +import lombok.Data; import java.util.HashMap; import java.util.List; import java.util.Map; -@Setter -@Getter -@ToString +@Data public class SinkConfig { private static final int DEFAULT_BATCH_MAX_SIZE = 1024; From ed55d36645cdb1b19de48b583931a3ffbedf7dda Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 16:40:43 +0800 Subject: [PATCH 12/36] [Feature][Connector-V2][Doris] update code --- seatunnel-engine/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-engine/README.md b/seatunnel-engine/README.md index 74827c5b721..1de13240519 100644 --- a/seatunnel-engine/README.md +++ b/seatunnel-engine/README.md @@ -32,4 +32,4 @@ In the future, SeaTunnel Engine will further optimize its functions to support f ### Quick Start -https://seatunnel.apache.org/docs/start-v2/local](https://seatunnel.apache.org/docs/start-v2/locally/deployment/ +https://seatunnel.apache.org/docs/start-v2/locally/deployment From c7f29cdde2d8f6965489fe94669b74b470a367fa Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 16:46:45 +0800 Subject: [PATCH 13/36] [Feature][Connector-V2][Doris] Update code --- docs/en/seatunnel-engine/about.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/seatunnel-engine/about.md b/docs/en/seatunnel-engine/about.md index 72da2a3de5e..8de9462b88d 100644 --- a/docs/en/seatunnel-engine/about.md +++ b/docs/en/seatunnel-engine/about.md @@ -36,4 +36,4 @@ In the future, SeaTunnel Engine will further optimize its functions to support f ### Quick Start -https://seatunnel.apache.org/docs/start-v2/local +https://seatunnel.apache.org/docs/start-v2/locally/deployment From f613bc33cbb7603073c5273bf1a091d4162b78b3 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 16:56:55 +0800 Subject: [PATCH 14/36] [Feature][Connector-V2][Doris] Update code --- docs/en/start-v2/locally/quick-start-flink.md | 2 +- docs/en/start-v2/locally/quick-start-seatunnel-engine.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/start-v2/locally/quick-start-flink.md b/docs/en/start-v2/locally/quick-start-flink.md index 6ad92f01f82..323aa886ba2 100644 --- a/docs/en/start-v2/locally/quick-start-flink.md +++ b/docs/en/start-v2/locally/quick-start-flink.md @@ -50,7 +50,7 @@ sink { ``` -More information about config please check [config concept](../concept/config) +More information about config please check [config concept](../../concept/config) ## Step 3: Run SeaTunnel Application diff --git a/docs/en/start-v2/locally/quick-start-seatunnel-engine.md b/docs/en/start-v2/locally/quick-start-seatunnel-engine.md index b7ffa14783a..8b8380e25cf 100644 --- a/docs/en/start-v2/locally/quick-start-seatunnel-engine.md +++ b/docs/en/start-v2/locally/quick-start-seatunnel-engine.md @@ -42,7 +42,7 @@ sink { ``` -More information about config please check [config concept](../concept/config) +More information about config please check [config concept](../../concept/config) ## Step 3: Run SeaTunnel Application From a5dcd93fb762ee5aaf9bca68aa21495c0e2ef5ab Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Mon, 5 Dec 2022 18:48:20 +0800 Subject: [PATCH 15/36] Update README.md --- seatunnel-engine/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-engine/README.md b/seatunnel-engine/README.md index 1de13240519..74827c5b721 100644 --- a/seatunnel-engine/README.md +++ b/seatunnel-engine/README.md @@ -32,4 +32,4 @@ In the future, SeaTunnel Engine will further optimize its functions to support f ### Quick Start -https://seatunnel.apache.org/docs/start-v2/locally/deployment +https://seatunnel.apache.org/docs/start-v2/local](https://seatunnel.apache.org/docs/start-v2/locally/deployment/ From ef3376b778e23c50f218926a1200f9408282df34 Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Mon, 5 Dec 2022 18:50:10 +0800 Subject: [PATCH 16/36] Update quick-start-flink.md --- docs/en/start-v2/locally/quick-start-flink.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/start-v2/locally/quick-start-flink.md b/docs/en/start-v2/locally/quick-start-flink.md index 323aa886ba2..6ad92f01f82 100644 --- a/docs/en/start-v2/locally/quick-start-flink.md +++ b/docs/en/start-v2/locally/quick-start-flink.md @@ -50,7 +50,7 @@ sink { ``` -More information about config please check [config concept](../../concept/config) +More information about config please check [config concept](../concept/config) ## Step 3: Run SeaTunnel Application From c3b82d0a48cebd175a3a1d7d70a7209a1198b01d Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Mon, 5 Dec 2022 18:50:31 +0800 Subject: [PATCH 17/36] Update quick-start-seatunnel-engine.md --- docs/en/start-v2/locally/quick-start-seatunnel-engine.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/start-v2/locally/quick-start-seatunnel-engine.md b/docs/en/start-v2/locally/quick-start-seatunnel-engine.md index 8b8380e25cf..b7ffa14783a 100644 --- a/docs/en/start-v2/locally/quick-start-seatunnel-engine.md +++ b/docs/en/start-v2/locally/quick-start-seatunnel-engine.md @@ -42,7 +42,7 @@ sink { ``` -More information about config please check [config concept](../../concept/config) +More information about config please check [config concept](../concept/config) ## Step 3: Run SeaTunnel Application From 76b12cf436edbb20de30178383234c0698d44a6e Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 19:06:02 +0800 Subject: [PATCH 18/36] [Feature][Connector-V2][Doris] Update code --- docs/en/start-v2/locally/quick-start-flink.md | 2 +- docs/en/start-v2/locally/quick-start-seatunnel-engine.md | 2 +- seatunnel-engine/README.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/start-v2/locally/quick-start-flink.md b/docs/en/start-v2/locally/quick-start-flink.md index 6ad92f01f82..323aa886ba2 100644 --- a/docs/en/start-v2/locally/quick-start-flink.md +++ b/docs/en/start-v2/locally/quick-start-flink.md @@ -50,7 +50,7 @@ sink { ``` -More information about config please check [config concept](../concept/config) +More information about config please check [config concept](../../concept/config) ## Step 3: Run SeaTunnel Application diff --git a/docs/en/start-v2/locally/quick-start-seatunnel-engine.md b/docs/en/start-v2/locally/quick-start-seatunnel-engine.md index b7ffa14783a..8b8380e25cf 100644 --- a/docs/en/start-v2/locally/quick-start-seatunnel-engine.md +++ b/docs/en/start-v2/locally/quick-start-seatunnel-engine.md @@ -42,7 +42,7 @@ sink { ``` -More information about config please check [config concept](../concept/config) +More information about config please check [config concept](../../concept/config) ## Step 3: Run SeaTunnel Application diff --git a/seatunnel-engine/README.md b/seatunnel-engine/README.md index 74827c5b721..1de13240519 100644 --- a/seatunnel-engine/README.md +++ b/seatunnel-engine/README.md @@ -32,4 +32,4 @@ In the future, SeaTunnel Engine will further optimize its functions to support f ### Quick Start -https://seatunnel.apache.org/docs/start-v2/local](https://seatunnel.apache.org/docs/start-v2/locally/deployment/ +https://seatunnel.apache.org/docs/start-v2/locally/deployment From 9e17c78bf056d51a68a205e120e279a5656b047b Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 19:11:17 +0800 Subject: [PATCH 19/36] [Feature][Connector-V2][Doris] Update code --- docs/en/about.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/about.md b/docs/en/about.md index 43468f327cf..e9a9e5ddc1e 100644 --- a/docs/en/about.md +++ b/docs/en/about.md @@ -35,7 +35,7 @@ SeaTunnel focuses on data integration and data synchronization, and is mainly de ## SeaTunnel work flowchart -![SeaTunnel work flowchart](../images/architecture_diagram.png) +![SeaTunnel work flowchart](./images/architecture_diagram.png) The runtime process of SeaTunnel is shown in the figure above. From 02268c4cfbda1885993352deb6a10fa4daad55ce Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 19:17:55 +0800 Subject: [PATCH 20/36] [Feature][Connector-V2][Doris] Update code --- docs/en/start-v2/locally/quick-start-flink.md | 2 +- docs/en/start-v2/locally/quick-start-seatunnel-engine.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/start-v2/locally/quick-start-flink.md b/docs/en/start-v2/locally/quick-start-flink.md index 323aa886ba2..f7f58a8d1b6 100644 --- a/docs/en/start-v2/locally/quick-start-flink.md +++ b/docs/en/start-v2/locally/quick-start-flink.md @@ -50,7 +50,7 @@ sink { ``` -More information about config please check [config concept](../../concept/config) +More information about config please check [config concept](../../concept/config.md) ## Step 3: Run SeaTunnel Application diff --git a/docs/en/start-v2/locally/quick-start-seatunnel-engine.md b/docs/en/start-v2/locally/quick-start-seatunnel-engine.md index 8b8380e25cf..d2ce05b373e 100644 --- a/docs/en/start-v2/locally/quick-start-seatunnel-engine.md +++ b/docs/en/start-v2/locally/quick-start-seatunnel-engine.md @@ -42,7 +42,7 @@ sink { ``` -More information about config please check [config concept](../../concept/config) +More information about config please check [config concept](../../concept/config.md) ## Step 3: Run SeaTunnel Application From 5275611a40d1ce6518443a030aaa06d61e4edb46 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 19:27:06 +0800 Subject: [PATCH 21/36] [Feature][Connector-V2][Doris] Update code --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ac88e8ce55a..8932319a2df 100644 --- a/README.md +++ b/README.md @@ -95,7 +95,7 @@ https://seatunnel.apache.org/docs/category/start-v2 https://seatunnel.apache.org/docs/category/start-v2 Detailed documentation on SeaTunnel -https://seatunnel.apache.org/docs/intro/about +https://seatunnel.apache.org/docs/about/ ## Application practice cases From e78d309de88ac350168f60496e7668c58a4e877d Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Mon, 5 Dec 2022 19:32:11 +0800 Subject: [PATCH 22/36] [Feature][Connector-V2][Doris] Update code --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8932319a2df..cf992f4f8db 100644 --- a/README.md +++ b/README.md @@ -42,7 +42,7 @@ SeaTunnel focuses on data integration and data synchronization, and is mainly de ## SeaTunnel work flowchart -![SeaTunnel work flowchart](/docs/en/images/architecture_diagram.png) +![SeaTunnel work flowchart](./docs/en/images/architecture_diagram.png) The runtime process of SeaTunnel is shown in the figure above. From b489b7eef0b76a793379e4d43926dbea85b0b3c5 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Tue, 6 Dec 2022 11:46:52 +0800 Subject: [PATCH 23/36] [Feature][Connector-V2][Doris] Update code --- .../apache/seatunnel/connectors/doris/config/SinkConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java index cdeefe5783b..5b46eccf548 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java @@ -132,7 +132,7 @@ public static StreamLoadFormat parse(String format) { private int retryBackoffMultiplierMs; private int maxRetryBackoffMs; - private final Map streamLoadProps = new HashMap<>(); + private final Map streamLoadProps = new HashMap<>(); public static SinkConfig loadConfig(Config pluginConfig) { SinkConfig sinkConfig = new SinkConfig(); @@ -182,7 +182,7 @@ private static void parseSinkStreamLoadProperties(Config pluginConfig, SinkConfi DORIS_SINK_CONFIG_PREFIX.key(), false); dorisConfig.entrySet().forEach(entry -> { final String configKey = entry.getKey().toLowerCase(); - sinkConfig.streamLoadProps.put(configKey, entry.getValue().unwrapped()); + sinkConfig.streamLoadProps.put(configKey, entry.getValue().unwrapped().toString()); }); } } From 2df826588f2be4e899f2c48e0e9f6b78f2328069 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Tue, 6 Dec 2022 12:30:23 +0800 Subject: [PATCH 24/36] [Feature][Connector-V2][Doris] Update code --- .../doris/client/DorisSinkManager.java | 33 ++++++++----------- .../doris/client/DorisStreamLoadVisitor.java | 4 +-- .../connectors/doris/config/SinkConfig.java | 4 +-- 3 files changed, 17 insertions(+), 24 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java index 600e9081dc3..7dc55e903ab 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisSinkManager.java @@ -65,21 +65,15 @@ private void tryInit() throws IOException { } initialize = true; - if (batchIntervalMs != null) { - scheduler = Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder().setNameFormat("Doris-sink-output-%s").build()); - scheduledFuture = scheduler.scheduleAtFixedRate( - () -> { - try { - flush(); - } catch (IOException e) { - flushException = e; - } - }, - batchIntervalMs, - batchIntervalMs, - TimeUnit.MILLISECONDS); - } + scheduler = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("Doris-sink-output-%s").build()); + scheduledFuture = scheduler.scheduleAtFixedRate(() -> { + try { + flush(); + } catch (IOException e) { + flushException = e; + } + }, batchIntervalMs, batchIntervalMs, TimeUnit.MILLISECONDS); } public synchronized void write(String record) throws IOException { @@ -109,7 +103,7 @@ public synchronized void flush() throws IOException { return; } String label = createBatchLabel(); - DorisFlushTuple tuple = new DorisFlushTuple(label, batchBytesSize, new ArrayList<>(batchList)); + DorisFlushTuple tuple = new DorisFlushTuple(label, batchBytesSize, batchList); for (int i = 0; i <= sinkConfig.getMaxRetries(); i++) { try { Boolean successFlag = dorisStreamLoadVisitor.doStreamLoad(tuple); @@ -151,11 +145,10 @@ private void checkFlushException() { } public String createBatchLabel() { - StringBuilder sb = new StringBuilder(); + String labelPrefix = ""; if (!Strings.isNullOrEmpty(sinkConfig.getLabelPrefix())) { - sb.append(sinkConfig.getLabelPrefix()); + labelPrefix = sinkConfig.getLabelPrefix(); } - return sb.append(UUID.randomUUID().toString()) - .toString(); + return String.format("%s%s", labelPrefix, UUID.randomUUID().toString()); } } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java index 8e14292f842..0ecb4abbf89 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java @@ -115,7 +115,7 @@ private String getAvailableHost() { private byte[] joinRows(List rows, int totalBytes) { if (SinkConfig.StreamLoadFormat.CSV.equals(sinkConfig.getLoadFormat())) { - Map props = sinkConfig.getStreamLoadProps(); + Map props = sinkConfig.getStreamLoadProps(); byte[] lineDelimiter = DelimiterParserUtil.parse((String) props.get("row_delimiter"), "\n").getBytes(StandardCharsets.UTF_8); ByteBuffer bos = ByteBuffer.allocate(totalBytes + rows.size() * lineDelimiter.length); for (byte[] row : rows) { @@ -197,7 +197,7 @@ private Map getStreamLoadHttpHeader(String label) { headerMap.put("columns", String.join(",", fieldNames.stream().map(f -> String.format("`%s`", f)).collect(Collectors.toList()))); } if (null != sinkConfig.getStreamLoadProps()) { - for (Map.Entry entry : sinkConfig.getStreamLoadProps().entrySet()) { + for (Map.Entry entry : sinkConfig.getStreamLoadProps().entrySet()) { headerMap.put(entry.getKey(), String.valueOf(entry.getValue())); } } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java index 5b46eccf548..8f6c3b10779 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java @@ -33,6 +33,7 @@ public class SinkConfig { private static final int DEFAULT_BATCH_MAX_SIZE = 1024; + private static final int DEFAULT_BATCH_INTERVAL_MS = 1000; private static final long DEFAULT_BATCH_BYTES = 5 * 1024 * 1024; private static final String LOAD_FORMAT = "format"; @@ -126,8 +127,7 @@ public static StreamLoadFormat parse(String format) { private int batchMaxSize = DEFAULT_BATCH_MAX_SIZE; private long batchMaxBytes = DEFAULT_BATCH_BYTES; - - private Integer batchIntervalMs; + private int batchIntervalMs = DEFAULT_BATCH_INTERVAL_MS; private int maxRetries; private int retryBackoffMultiplierMs; private int maxRetryBackoffMs; From b1986a06de413d7cc5b61df2dfd4ce18bdf19bd5 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Tue, 6 Dec 2022 12:33:59 +0800 Subject: [PATCH 25/36] [Feature][Connector-V2][Doris] Update doc --- docs/en/connector-v2/sink/Doris.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/connector-v2/sink/Doris.md b/docs/en/connector-v2/sink/Doris.md index 568a047fee5..6cac1bc3217 100644 --- a/docs/en/connector-v2/sink/Doris.md +++ b/docs/en/connector-v2/sink/Doris.md @@ -22,8 +22,8 @@ The internal implementation of Doris sink connector is cached and imported by st | labelPrefix | string | no | - | | batch_max_rows | long | no | 1024 | | batch_max_bytes | int | no | 5 * 1024 * 1024 | -| batch_interval_ms | int | no | - | -| max_retries | int | no | - | +| batch_interval_ms | int | no | 1000 | +| max_retries | int | no | 1 | | retry_backoff_multiplier_ms | int | no | - | | max_retry_backoff_ms | int | no | - | | sink.properties.* | doris stream load config | no | - | From c6a58bc86020e13d66249a8b4d9e9712771881dd Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Tue, 6 Dec 2022 21:07:53 +0800 Subject: [PATCH 26/36] Update DorisStreamLoadVisitor.java --- .../doris/client/DorisStreamLoadVisitor.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java index 0ecb4abbf89..d22de2faa2e 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/client/DorisStreamLoadVisitor.java @@ -76,23 +76,23 @@ public Boolean doStreamLoad(DorisFlushTuple flushData) throws IOException { log.debug(String.format("StreamLoad response:\n%s"), JsonUtils.toJsonString(loadResult)); } if (RESULT_FAILED.equals(loadResult.get(keyStatus))) { - StringBuilder errorBuilder = new StringBuilder("Failed to flush data to Doris.\n"); + String errorMsg = "Failed to flush data to Doris.\n"; + String message = ""; if (loadResult.containsKey("Message")) { - errorBuilder.append(loadResult.get("Message")); - errorBuilder.append('\n'); + message = loadResult.get("Message") + "\n"; } + String errorURL = ""; if (loadResult.containsKey("ErrorURL")) { try { - errorBuilder.append(httpHelper.doHttpGet(loadResult.get("ErrorURL").toString())); - errorBuilder.append('\n'); + errorURL = httpHelper.doHttpGet(loadResult.get("ErrorURL").toString()) + "\n"; } catch (IOException e) { log.warn("Get Error URL failed. {} ", loadResult.get("ErrorURL"), e); } } else { - errorBuilder.append(JsonUtils.toJsonString(loadResult)); - errorBuilder.append('\n'); + errorURL = JsonUtils.toJsonString(loadResult) + "\n"; } - throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, errorBuilder.toString()); + throw new DorisConnectorException(CommonErrorCode.FLUSH_DATA_FAILED, + String.format("%s%s%s", errorMsg, message, errorURL)); } else if (RESULT_LABEL_EXISTED.equals(loadResult.get(keyStatus))) { log.debug(String.format("StreamLoad response:\n%s"), JsonUtils.toJsonString(loadResult)); // has to block-checking the state to get the final result From 1c5c0a26d1a526baa0a562adc9a9543e47eab3a0 Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Tue, 6 Dec 2022 21:45:17 +0800 Subject: [PATCH 27/36] Update pom.xml --- seatunnel-dist/pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index ae2ee428340..4d20d3c344b 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -379,11 +379,13 @@ org.apache.seatunnel connector-openmldb ${project.version} + provided org.apache.seatunnel connector-doris ${project.version} + provided From c7715b437a3cf12a48670016db6f9b246616ad53 Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Tue, 6 Dec 2022 21:46:58 +0800 Subject: [PATCH 28/36] Update DorisIT.java --- .../java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index 47505c87251..9092266d214 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -38,6 +38,7 @@ import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; import org.testcontainers.shaded.org.apache.commons.io.IOUtils; +import org.testcontainers.utility.DockerLoggerFactory; import java.io.IOException; import java.io.InputStream; @@ -157,7 +158,7 @@ public void startUp() throws Exception { dorisServer = new GenericContainer<>(DOCKER_IMAGE) .withNetwork(NETWORK) .withNetworkAliases(HOST) - .withLogConsumer(new Slf4jLogConsumer(log)); + .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger(DOCKER_IMAGE))); dorisServer.setPortBindings(Lists.newArrayList( String.format("%s:%s", PORT, DOCKER_PORT))); Startables.deepStart(Stream.of(dorisServer)).join(); From d82604fafbb03cfd77d477be2a102d936daadea1 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Wed, 7 Dec 2022 17:54:12 +0800 Subject: [PATCH 29/36] [Feature][Connector-V2][Doris] Update use ReadonlyConfig --- .../connectors/doris/config/SinkConfig.java | 52 +++++++------------ .../connectors/doris/sink/DorisSink.java | 21 ++------ .../doris/sink/DorisSinkWriter.java | 6 ++- 3 files changed, 27 insertions(+), 52 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java index 8f6c3b10779..076ab69e2bf 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.common.config.TypesafeConfigUtils; import org.apache.seatunnel.shade.com.typesafe.config.Config; @@ -88,7 +89,7 @@ public class SinkConfig { public static final Option BATCH_INTERVAL_MS = Options.key("batch_interval_ms") .intType() - .noDefaultValue() + .defaultValue(DEFAULT_BATCH_INTERVAL_MS) .withDescription("For batch writing, when the number of buffers reaches the number of batch_max_rows or the byte size of batch_max_bytes or the time reaches batch_interval_ms, the data will be flushed into the Doris"); public static final Option MAX_RETRIES = Options.key("max_retries") @@ -134,45 +135,28 @@ public static StreamLoadFormat parse(String format) { private final Map streamLoadProps = new HashMap<>(); - public static SinkConfig loadConfig(Config pluginConfig) { + public static SinkConfig loadConfig(ReadonlyConfig config, Config pluginConfig) { SinkConfig sinkConfig = new SinkConfig(); - sinkConfig.setNodeUrls(pluginConfig.getStringList(NODE_URLS.key())); - sinkConfig.setDatabase(pluginConfig.getString(DATABASE.key())); - sinkConfig.setTable(pluginConfig.getString(TABLE.key())); + sinkConfig.setNodeUrls(config.get(SinkConfig.NODE_URLS)); + sinkConfig.setDatabase(config.get(SinkConfig.DATABASE)); + sinkConfig.setTable(config.get(SinkConfig.TABLE)); + + sinkConfig.setUsername(config.get(SinkConfig.USERNAME)); + sinkConfig.setPassword(config.get(SinkConfig.PASSWORD)); + sinkConfig.setBatchMaxSize(config.get(SinkConfig.BATCH_MAX_SIZE)); + sinkConfig.setLabelPrefix(config.get(SinkConfig.LABEL_PREFIX)); + sinkConfig.setBatchMaxBytes(config.get(SinkConfig.BATCH_MAX_BYTES)); + sinkConfig.setBatchIntervalMs(config.get(SinkConfig.BATCH_INTERVAL_MS)); + sinkConfig.setMaxRetries(config.get(SinkConfig.MAX_RETRIES)); + sinkConfig.setRetryBackoffMultiplierMs(config.get(SinkConfig.RETRY_BACKOFF_MULTIPLIER_MS)); + sinkConfig.setMaxRetryBackoffMs(config.get(SinkConfig.MAX_RETRY_BACKOFF_MS)); - if (pluginConfig.hasPath(USERNAME.key())) { - sinkConfig.setUsername(pluginConfig.getString(USERNAME.key())); - } - if (pluginConfig.hasPath(PASSWORD.key())) { - sinkConfig.setPassword(pluginConfig.getString(PASSWORD.key())); - } - if (pluginConfig.hasPath(LABEL_PREFIX.key())) { - sinkConfig.setLabelPrefix(pluginConfig.getString(LABEL_PREFIX.key())); - } - if (pluginConfig.hasPath(BATCH_MAX_SIZE.key())) { - sinkConfig.setBatchMaxSize(pluginConfig.getInt(BATCH_MAX_SIZE.key())); - } - if (pluginConfig.hasPath(BATCH_MAX_BYTES.key())) { - sinkConfig.setBatchMaxBytes(pluginConfig.getLong(BATCH_MAX_BYTES.key())); - } - if (pluginConfig.hasPath(BATCH_INTERVAL_MS.key())) { - sinkConfig.setBatchIntervalMs(pluginConfig.getInt(BATCH_INTERVAL_MS.key())); - } - if (pluginConfig.hasPath(MAX_RETRIES.key())) { - sinkConfig.setMaxRetries(pluginConfig.getInt(MAX_RETRIES.key())); - } - if (pluginConfig.hasPath(RETRY_BACKOFF_MULTIPLIER_MS.key())) { - sinkConfig.setRetryBackoffMultiplierMs(pluginConfig.getInt(RETRY_BACKOFF_MULTIPLIER_MS.key())); - } - if (pluginConfig.hasPath(MAX_RETRY_BACKOFF_MS.key())) { - sinkConfig.setMaxRetryBackoffMs(pluginConfig.getInt(MAX_RETRY_BACKOFF_MS.key())); - } parseSinkStreamLoadProperties(pluginConfig, sinkConfig); if (sinkConfig.streamLoadProps.containsKey(COLUMN_SEPARATOR)) { - sinkConfig.setColumnSeparator((String) sinkConfig.streamLoadProps.get(COLUMN_SEPARATOR)); + sinkConfig.setColumnSeparator(sinkConfig.streamLoadProps.get(COLUMN_SEPARATOR)); } if (sinkConfig.streamLoadProps.containsKey(LOAD_FORMAT)) { - sinkConfig.setLoadFormat(StreamLoadFormat.parse((String) sinkConfig.streamLoadProps.get(LOAD_FORMAT))); + sinkConfig.setLoadFormat(StreamLoadFormat.parse(sinkConfig.streamLoadProps.get(LOAD_FORMAT))); } return sinkConfig; } diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java index 791e82f13de..f51d43eb279 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java @@ -17,23 +17,14 @@ package org.apache.seatunnel.connectors.doris.sink; -import static org.apache.seatunnel.connectors.doris.config.SinkConfig.DATABASE; -import static org.apache.seatunnel.connectors.doris.config.SinkConfig.NODE_URLS; -import static org.apache.seatunnel.connectors.doris.config.SinkConfig.PASSWORD; -import static org.apache.seatunnel.connectors.doris.config.SinkConfig.TABLE; -import static org.apache.seatunnel.connectors.doris.config.SinkConfig.USERNAME; - import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.ConfigValidator; 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.doris.exception.DorisConnectorException; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; @@ -55,12 +46,8 @@ public String getPluginName() { @Override public void prepare(Config pluginConfig) throws PrepareFailException { this.pluginConfig = pluginConfig; - CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, NODE_URLS.key(), DATABASE.key(), TABLE.key(), USERNAME.key(), PASSWORD.key()); - if (!result.isSuccess()) { - throw new DorisConnectorException(SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - String.format("PluginName: %s, PluginType: %s, Message: %s", - getPluginName(), PluginType.SINK, result.getMsg())); - } + ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(pluginConfig); + ConfigValidator.of(readonlyConfig).validate(new DorisSinkFactory().optionRule()); } @Override diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java index ff380fe048e..f08307a5801 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.doris.sink; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.serialization.SerializationSchema; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -43,12 +44,15 @@ @Slf4j public class DorisSinkWriter extends AbstractSinkWriter { + private ReadonlyConfig readonlyConfig; private final SerializationSchema serializationSchema; private final DorisSinkManager manager; public DorisSinkWriter(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { - SinkConfig sinkConfig = SinkConfig.loadConfig(pluginConfig); + ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(pluginConfig); + SinkConfig sinkConfig = SinkConfig.loadConfig(readonlyConfig, pluginConfig); + List fieldNames = Arrays.stream(seaTunnelRowType.getFieldNames()).collect(Collectors.toList()); this.serializationSchema = createSerializer(sinkConfig, seaTunnelRowType); this.manager = new DorisSinkManager(sinkConfig, fieldNames); From 0a9b208a64be9bdf829441b0c3c1f51c79fd46ea Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Wed, 7 Dec 2022 18:52:34 +0800 Subject: [PATCH 30/36] [Feature][Connector-V2][Doris] Update code --- .../connectors/doris/config/SinkConfig.java | 37 ++++++++++++++----- 1 file changed, 27 insertions(+), 10 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java index 076ab69e2bf..3caf44390ff 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java @@ -141,16 +141,33 @@ public static SinkConfig loadConfig(ReadonlyConfig config, Config pluginConfig) sinkConfig.setDatabase(config.get(SinkConfig.DATABASE)); sinkConfig.setTable(config.get(SinkConfig.TABLE)); - sinkConfig.setUsername(config.get(SinkConfig.USERNAME)); - sinkConfig.setPassword(config.get(SinkConfig.PASSWORD)); - sinkConfig.setBatchMaxSize(config.get(SinkConfig.BATCH_MAX_SIZE)); - sinkConfig.setLabelPrefix(config.get(SinkConfig.LABEL_PREFIX)); - sinkConfig.setBatchMaxBytes(config.get(SinkConfig.BATCH_MAX_BYTES)); - sinkConfig.setBatchIntervalMs(config.get(SinkConfig.BATCH_INTERVAL_MS)); - sinkConfig.setMaxRetries(config.get(SinkConfig.MAX_RETRIES)); - sinkConfig.setRetryBackoffMultiplierMs(config.get(SinkConfig.RETRY_BACKOFF_MULTIPLIER_MS)); - sinkConfig.setMaxRetryBackoffMs(config.get(SinkConfig.MAX_RETRY_BACKOFF_MS)); - + if (pluginConfig.hasPath(USERNAME.key())) { + sinkConfig.setUsername(config.get(SinkConfig.USERNAME)); + } + if (pluginConfig.hasPath(PASSWORD.key())) { + sinkConfig.setPassword(config.get(SinkConfig.PASSWORD)); + } + if (pluginConfig.hasPath(LABEL_PREFIX.key())) { + sinkConfig.setLabelPrefix(config.get(SinkConfig.LABEL_PREFIX)); + } + if (pluginConfig.hasPath(BATCH_MAX_SIZE.key())) { + sinkConfig.setBatchMaxSize(config.get(SinkConfig.BATCH_MAX_SIZE)); + } + if (pluginConfig.hasPath(BATCH_MAX_BYTES.key())) { + sinkConfig.setBatchMaxBytes(config.get(SinkConfig.BATCH_MAX_BYTES)); + } + if (pluginConfig.hasPath(BATCH_INTERVAL_MS.key())) { + sinkConfig.setBatchIntervalMs(config.get(SinkConfig.BATCH_INTERVAL_MS)); + } + if (pluginConfig.hasPath(MAX_RETRIES.key())) { + sinkConfig.setMaxRetries(config.get(SinkConfig.MAX_RETRIES)); + } + if (pluginConfig.hasPath(RETRY_BACKOFF_MULTIPLIER_MS.key())) { + sinkConfig.setRetryBackoffMultiplierMs(config.get(SinkConfig.RETRY_BACKOFF_MULTIPLIER_MS)); + } + if (pluginConfig.hasPath(MAX_RETRY_BACKOFF_MS.key())) { + sinkConfig.setMaxRetryBackoffMs(config.get(SinkConfig.MAX_RETRY_BACKOFF_MS)); + } parseSinkStreamLoadProperties(pluginConfig, sinkConfig); if (sinkConfig.streamLoadProps.containsKey(COLUMN_SEPARATOR)) { sinkConfig.setColumnSeparator(sinkConfig.streamLoadProps.get(COLUMN_SEPARATOR)); From 50db8eff24fba83f95add0fad06d0619c6e2ea16 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Wed, 7 Dec 2022 22:03:31 +0800 Subject: [PATCH 31/36] [Feature][Connector-V2][Doris] Update code --- .../connectors/doris/sink/DorisSink.java | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java index f51d43eb279..791e82f13de 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSink.java @@ -17,14 +17,23 @@ package org.apache.seatunnel.connectors.doris.sink; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.DATABASE; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.NODE_URLS; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.PASSWORD; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.TABLE; +import static org.apache.seatunnel.connectors.doris.config.SinkConfig.USERNAME; + import org.apache.seatunnel.api.common.PrepareFailException; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; -import org.apache.seatunnel.api.configuration.util.ConfigValidator; +import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; 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.doris.exception.DorisConnectorException; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; @@ -46,8 +55,12 @@ public String getPluginName() { @Override public void prepare(Config pluginConfig) throws PrepareFailException { this.pluginConfig = pluginConfig; - ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(pluginConfig); - ConfigValidator.of(readonlyConfig).validate(new DorisSinkFactory().optionRule()); + CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, NODE_URLS.key(), DATABASE.key(), TABLE.key(), USERNAME.key(), PASSWORD.key()); + if (!result.isSuccess()) { + throw new DorisConnectorException(SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + String.format("PluginName: %s, PluginType: %s, Message: %s", + getPluginName(), PluginType.SINK, result.getMsg())); + } } @Override From 8d56dab27de9572fd6a398f1433c919a0c3a5d83 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Wed, 7 Dec 2022 22:45:46 +0800 Subject: [PATCH 32/36] [Feature][Connector-V2][Doris] fix ci --- .../connectors/doris/config/SinkConfig.java | 27 +++++++++---------- .../doris/sink/DorisSinkWriter.java | 4 +-- 2 files changed, 14 insertions(+), 17 deletions(-) diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java index 3caf44390ff..d1f88cbc678 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/config/SinkConfig.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; -import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.common.config.TypesafeConfigUtils; import org.apache.seatunnel.shade.com.typesafe.config.Config; @@ -135,38 +134,38 @@ public static StreamLoadFormat parse(String format) { private final Map streamLoadProps = new HashMap<>(); - public static SinkConfig loadConfig(ReadonlyConfig config, Config pluginConfig) { + public static SinkConfig loadConfig(Config pluginConfig) { SinkConfig sinkConfig = new SinkConfig(); - sinkConfig.setNodeUrls(config.get(SinkConfig.NODE_URLS)); - sinkConfig.setDatabase(config.get(SinkConfig.DATABASE)); - sinkConfig.setTable(config.get(SinkConfig.TABLE)); + sinkConfig.setNodeUrls(pluginConfig.getStringList(NODE_URLS.key())); + sinkConfig.setDatabase(pluginConfig.getString(DATABASE.key())); + sinkConfig.setTable(pluginConfig.getString(TABLE.key())); if (pluginConfig.hasPath(USERNAME.key())) { - sinkConfig.setUsername(config.get(SinkConfig.USERNAME)); + sinkConfig.setUsername(pluginConfig.getString(USERNAME.key())); } if (pluginConfig.hasPath(PASSWORD.key())) { - sinkConfig.setPassword(config.get(SinkConfig.PASSWORD)); + sinkConfig.setPassword(pluginConfig.getString(PASSWORD.key())); } if (pluginConfig.hasPath(LABEL_PREFIX.key())) { - sinkConfig.setLabelPrefix(config.get(SinkConfig.LABEL_PREFIX)); + sinkConfig.setLabelPrefix(pluginConfig.getString(LABEL_PREFIX.key())); } if (pluginConfig.hasPath(BATCH_MAX_SIZE.key())) { - sinkConfig.setBatchMaxSize(config.get(SinkConfig.BATCH_MAX_SIZE)); + sinkConfig.setBatchMaxSize(pluginConfig.getInt(BATCH_MAX_SIZE.key())); } if (pluginConfig.hasPath(BATCH_MAX_BYTES.key())) { - sinkConfig.setBatchMaxBytes(config.get(SinkConfig.BATCH_MAX_BYTES)); + sinkConfig.setBatchMaxBytes(pluginConfig.getLong(BATCH_MAX_BYTES.key())); } if (pluginConfig.hasPath(BATCH_INTERVAL_MS.key())) { - sinkConfig.setBatchIntervalMs(config.get(SinkConfig.BATCH_INTERVAL_MS)); + sinkConfig.setBatchIntervalMs(pluginConfig.getInt(BATCH_INTERVAL_MS.key())); } if (pluginConfig.hasPath(MAX_RETRIES.key())) { - sinkConfig.setMaxRetries(config.get(SinkConfig.MAX_RETRIES)); + sinkConfig.setMaxRetries(pluginConfig.getInt(MAX_RETRIES.key())); } if (pluginConfig.hasPath(RETRY_BACKOFF_MULTIPLIER_MS.key())) { - sinkConfig.setRetryBackoffMultiplierMs(config.get(SinkConfig.RETRY_BACKOFF_MULTIPLIER_MS)); + sinkConfig.setRetryBackoffMultiplierMs(pluginConfig.getInt(RETRY_BACKOFF_MULTIPLIER_MS.key())); } if (pluginConfig.hasPath(MAX_RETRY_BACKOFF_MS.key())) { - sinkConfig.setMaxRetryBackoffMs(config.get(SinkConfig.MAX_RETRY_BACKOFF_MS)); + sinkConfig.setMaxRetryBackoffMs(pluginConfig.getInt(MAX_RETRY_BACKOFF_MS.key())); } parseSinkStreamLoadProperties(pluginConfig, sinkConfig); if (sinkConfig.streamLoadProps.containsKey(COLUMN_SEPARATOR)) { diff --git a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java index f08307a5801..5f8e478b546 100644 --- a/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java +++ b/seatunnel-connectors-v2/connector-doris/src/main/java/org/apache/seatunnel/connectors/doris/sink/DorisSinkWriter.java @@ -50,9 +50,7 @@ public class DorisSinkWriter extends AbstractSinkWriter { public DorisSinkWriter(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { - ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(pluginConfig); - SinkConfig sinkConfig = SinkConfig.loadConfig(readonlyConfig, pluginConfig); - + SinkConfig sinkConfig = SinkConfig.loadConfig(pluginConfig); List fieldNames = Arrays.stream(seaTunnelRowType.getFieldNames()).collect(Collectors.toList()); this.serializationSchema = createSerializer(sinkConfig, seaTunnelRowType); this.manager = new DorisSinkManager(sinkConfig, fieldNames); From 1001d47ab0dcb466dbfddf8534e5316dd408f1fc Mon Sep 17 00:00:00 2001 From: TaoZex <45089228+TaoZex@users.noreply.github.com> Date: Thu, 8 Dec 2022 11:35:57 +0800 Subject: [PATCH 33/36] Update DorisIT.java --- .../java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index 9092266d214..37914647eaf 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -261,7 +261,7 @@ private void initializeJdbcConnection() throws SQLException, ClassNotFoundExcept } private void initializeJdbcTable() { - //just for test + // wait for BE start try { Thread.sleep(300000); } catch (InterruptedException e) { From 7dfa10c5ead69ab945d2dd5b5f55ad27c533bde5 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Thu, 8 Dec 2022 12:08:49 +0800 Subject: [PATCH 34/36] [Feature][Connector-V2][Doris] Update e2e test --- .../seatunnel/e2e/connector/doris/DorisIT.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index 37914647eaf..dc9db0bb7fb 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -167,7 +167,7 @@ public void startUp() throws Exception { // wait for doris fully start given().ignoreExceptions() .await() - .atMost(360, TimeUnit.SECONDS) + .atMost(600, TimeUnit.SECONDS) .untilAsserted(this::initializeJdbcConnection); initializeJdbcTable(); batchInsertData(); @@ -258,18 +258,14 @@ private void initializeJdbcConnection() throws SQLException, ClassNotFoundExcept props.put("user", USERNAME); props.put("password", PASSWORD); jdbcConnection = driver.connect(String.format(URL, dorisServer.getHost()), props); + Statement statement = jdbcConnection.createStatement(); + statement.execute("CREATE DATABASE IF NOT EXISTS test"); } private void initializeJdbcTable() { - // wait for BE start - try { - Thread.sleep(300000); - } catch (InterruptedException e) { - e.printStackTrace(); - } try (Statement statement = jdbcConnection.createStatement()) { // create databases - statement.execute("create database test"); + statement.execute("CREATE DATABASE IF NOT EXISTS test"); // create source table statement.execute(DDL_SOURCE); // create sink table From 2be750620a5fb70631bef4390fa969f0b7fa7549 Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Thu, 8 Dec 2022 12:22:25 +0800 Subject: [PATCH 35/36] [Feature][Connector-V2][Doris] Update e2e test --- .../org/apache/seatunnel/e2e/connector/doris/DorisIT.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index dc9db0bb7fb..416226713e8 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -258,8 +258,9 @@ private void initializeJdbcConnection() throws SQLException, ClassNotFoundExcept props.put("user", USERNAME); props.put("password", PASSWORD); jdbcConnection = driver.connect(String.format(URL, dorisServer.getHost()), props); - Statement statement = jdbcConnection.createStatement(); - statement.execute("CREATE DATABASE IF NOT EXISTS test"); + try (Statement statement = jdbcConnection.createStatement()) { + statement.execute("CREATE DATABASE IF NOT EXISTS test"); + } } private void initializeJdbcTable() { From c5046e76ed3e984809b401a0976b78cbd60e1ffc Mon Sep 17 00:00:00 2001 From: TaoZex <2633363995@qq.com> Date: Thu, 8 Dec 2022 13:44:35 +0800 Subject: [PATCH 36/36] [Feature][Connector-V2][Doris] Update code --- .../seatunnel/e2e/connector/doris/DorisIT.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java index 416226713e8..0d3ec6c5ded 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-doris-e2e/src/test/java/org/apache/seatunnel/e2e/connector/doris/DorisIT.java @@ -79,7 +79,8 @@ public class DorisIT extends TestSuiteBase implements TestResource { private static final String DRIVER_JAR = "https://repo1.maven.org/maven2/mysql/mysql-connector-java/8.0.16/mysql-connector-java-8.0.16.jar"; private static final String COLUMN_STRING = "BIGINT_COL, LARGEINT_COL, SMALLINT_COL, TINYINT_COL, BOOLEAN_COL, DECIMAL_COL, DOUBLE_COL, FLOAT_COL, INT_COL, CHAR_COL, VARCHAR_11_COL, STRING_COL, DATETIME_COL, DATE_COL"; - private static final String DDL_SOURCE = "create table " + DATABASE + "." + SOURCE_TABLE + " (\n" + + private static final String CREATE_DATABASE = "CREATE DATABASE IF NOT EXISTS " + DATABASE; + private static final String DDL_SOURCE = "CREATE TABLE IF NOT EXISTS " + DATABASE + "." + SOURCE_TABLE + " (\n" + " BIGINT_COL BIGINT,\n" + " LARGEINT_COL LARGEINT,\n" + " SMALLINT_COL SMALLINT,\n" + @@ -101,7 +102,7 @@ public class DorisIT extends TestSuiteBase implements TestResource { "\"replication_allocation\" = \"tag.location.default: 1\"" + ")"; - private static final String DDL_SINK = "create table " + DATABASE + "." + SINK_TABLE + " (\n" + + private static final String DDL_SINK = "CREATE TABLE IF NOT EXISTS " + DATABASE + "." + SINK_TABLE + " (\n" + " BIGINT_COL BIGINT,\n" + " LARGEINT_COL LARGEINT,\n" + " SMALLINT_COL SMALLINT,\n" + @@ -123,7 +124,7 @@ public class DorisIT extends TestSuiteBase implements TestResource { "\"replication_allocation\" = \"tag.location.default: 1\"" + ")"; - private static final String INIT_DATA_SQL = "insert into " + DATABASE + "." + SOURCE_TABLE + " (\n" + + private static final String INIT_DATA_SQL = "INSERT INTO " + DATABASE + "." + SOURCE_TABLE + " (\n" + " BIGINT_COL,\n" + " LARGEINT_COL,\n" + " SMALLINT_COL,\n" + @@ -259,14 +260,15 @@ private void initializeJdbcConnection() throws SQLException, ClassNotFoundExcept props.put("password", PASSWORD); jdbcConnection = driver.connect(String.format(URL, dorisServer.getHost()), props); try (Statement statement = jdbcConnection.createStatement()) { - statement.execute("CREATE DATABASE IF NOT EXISTS test"); + statement.execute(CREATE_DATABASE); + statement.execute(DDL_SOURCE); } } private void initializeJdbcTable() { try (Statement statement = jdbcConnection.createStatement()) { // create databases - statement.execute("CREATE DATABASE IF NOT EXISTS test"); + statement.execute(CREATE_DATABASE); // create source table statement.execute(DDL_SOURCE); // create sink table