EMsnap commented on code in PR #6213: URL: https://github.com/apache/inlong/pull/6213#discussion_r1002936551
########## inlong-sort/sort-connectors/doris/src/main/java/org/apache/inlong/sort/doris/table/DorisDynamicSchemaOutputFormat.java: ########## @@ -0,0 +1,337 @@ +/* + * 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.inlong.sort.doris.table; + +import org.apache.commons.lang3.StringUtils; +import org.apache.doris.flink.cfg.DorisExecutionOptions; +import org.apache.doris.flink.cfg.DorisOptions; +import org.apache.doris.flink.cfg.DorisReadOptions; +import org.apache.doris.flink.exception.DorisException; +import org.apache.doris.flink.exception.StreamLoadException; +import org.apache.doris.flink.rest.RestService; +import org.apache.doris.shaded.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.inlong.sort.base.format.DynamicSchemaFormatFactory; +import org.apache.inlong.sort.base.format.JsonDynamicSchemaFormat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + * DorisDynamicSchemaOutputFormat, copy from {@link org.apache.doris.flink.table.DorisDynamicOutputFormat} + * It is used in the multiple sink scenario, in this scenario, we directly convert the data format by + * 'sink.multiple.format' in the data stream to doris json that is used to load + */ +public class DorisDynamicSchemaOutputFormat<T> extends RichOutputFormat<T> { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(DorisDynamicSchemaOutputFormat.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String COLUMNS_KEY = "columns"; + private static final String DORIS_DELETE_SIGN = "__DORIS_DELETE_SIGN__"; + private static final String UNIQUE_KEYS_TYPE = "UNIQUE_KEYS"; + @SuppressWarnings({"rawtypes"}) + private final Map<String, List> batchMap = new HashMap<>(); + private final DorisOptions options; + private final DorisReadOptions readOptions; + private final DorisExecutionOptions executionOptions; + private final String databasePattern; + private final String tablePattern; + private long batchBytes = 0L; + private DorisStreamLoad dorisStreamLoad; + private String dynamicSchemaFormat; + private transient volatile boolean closed = false; + private transient ScheduledExecutorService scheduler; + private transient ScheduledFuture<?> scheduledFuture; + private transient volatile Exception flushException; + + public DorisDynamicSchemaOutputFormat(DorisOptions option, + DorisReadOptions readOptions, + DorisExecutionOptions executionOptions, + String dynamicSchemaFormat, + String databasePattern, + String tablePattern) { + this.options = option; + this.readOptions = readOptions; + this.executionOptions = executionOptions; + this.dynamicSchemaFormat = dynamicSchemaFormat; + this.databasePattern = databasePattern; + this.tablePattern = tablePattern; + handleStreamloadProp(); + } + + /** + * A builder used to set parameters to the output format's configuration in a fluent way. + * + * @return builder + */ + public static DorisDynamicSchemaOutputFormat.Builder builder() { + return new DorisDynamicSchemaOutputFormat.Builder(); + } + + private void handleStreamloadProp() { + Properties streamLoadProp = executionOptions.getStreamLoadProp(); + //add column key when fieldNames is not empty +// if (!streamLoadProp.containsKey(COLUMNS_KEY) && fieldNames != null && fieldNames.length > 0) { +// String columns = String.join(",", Arrays +// .stream(fieldNames).map(item -> String.format("`%s`", item.trim().replace("`", ""))).collect( +// Collectors.toList())); +// if (enableBatchDelete()) { +// columns = String.format("%s,%s", columns, DORIS_DELETE_SIGN); +// } +// streamLoadProp.put(COLUMNS_KEY, columns); +// } + } + + private boolean enableBatchDelete() { + return executionOptions.getEnableDelete(); + } + + @Override + public void configure(Configuration configuration) { + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + dorisStreamLoad = new DorisStreamLoad( + getBackend(), + options.getUsername(), + options.getPassword(), + executionOptions.getStreamLoadProp()); + if (executionOptions.getBatchIntervalMs() != 0 && executionOptions.getBatchSize() != 1) { + this.scheduler = Executors.newScheduledThreadPool(1, + new ExecutorThreadFactory("doris-streamload-output-format")); + this.scheduledFuture = this.scheduler.scheduleWithFixedDelay(() -> { + synchronized (DorisDynamicSchemaOutputFormat.this) { + if (!closed) { + try { + flush(); + } catch (Exception e) { + flushException = e; + } + } + } + }, executionOptions.getBatchIntervalMs(), executionOptions.getBatchIntervalMs(), TimeUnit.MILLISECONDS); + } + } + + private void checkFlushException() { + if (flushException != null) { + throw new RuntimeException("Writing records to streamload failed.", flushException); + } + } + + @Override + public synchronized void writeRecord(T row) throws IOException { + checkFlushException(); + addBatch(row); + if ((executionOptions.getBatchSize() > 0 && batchMap.size() >= executionOptions.getBatchSize()) + || batchBytes >= executionOptions.getMaxBatchBytes()) { + flush(); + } + } + + @SuppressWarnings({"unchecked"}) + private void addBatch(T row) throws IOException { + if (row instanceof RowData) { + RowData rowData = (RowData) row; + if (null == dynamicSchemaFormat) { + LOG.error("dynamicSchemaFormat is null"); + throw new RuntimeException("dynamicSchemaFormat can not be null"); + } + JsonDynamicSchemaFormat jsonDynamicSchemaFormat = (JsonDynamicSchemaFormat) + DynamicSchemaFormatFactory.getFormat(dynamicSchemaFormat); + JsonNode rootNode = jsonDynamicSchemaFormat.deserialize(rowData.getBinary(0)); + String tableIdentifier = StringUtils.join( + jsonDynamicSchemaFormat.parse(rowData.getBinary(0), databasePattern), + ".", + jsonDynamicSchemaFormat.parse(rowData.getBinary(0), tablePattern)); + Map<String, String> physicalData = jsonDynamicSchemaFormat.physicalDataToMap(rootNode); + batchBytes += physicalData.toString().getBytes(StandardCharsets.UTF_8).length; + // add doris delete sign + if (enableBatchDelete()) { + physicalData.put(DORIS_DELETE_SIGN, parseDeleteSign(rowData.getRowKind())); + } + batchMap.computeIfAbsent(tableIdentifier, k -> new ArrayList<>()).add(physicalData); + } else { + throw new RuntimeException("The type of element should be 'RowData' only."); + } + } + + private String parseDeleteSign(RowKind rowKind) { + if (RowKind.INSERT.equals(rowKind) || RowKind.UPDATE_AFTER.equals(rowKind)) { + return "0"; Review Comment: magic num, please add some doc -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@inlong.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org