Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[pipeline-connector][doris] add doris pipeline connector. #2810

Merged
merged 1 commit into from
Dec 5, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.time.ZoneId;

/**
* An internal data structure representing data of {@link LocalZonedTimestampType}.
Expand Down Expand Up @@ -74,7 +75,9 @@ public Instant toInstant() {
milliOfSecond += 1000;
}
long nanoAdjustment = milliOfSecond * 1_000_000 + epochNanoOfMillisecond;
return Instant.ofEpochSecond(epochSecond, nanoAdjustment);
return Instant.ofEpochSecond(epochSecond, nanoAdjustment)
.atZone(ZoneId.of("UTC"))
.toInstant();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -426,8 +426,12 @@ public static OptionalInt getLength(DataType dataType) {
return dataType.accept(LENGTH_EXTRACTOR);
}

private static final PrecisionExtractor PRECISION_EXTRACTOR = new PrecisionExtractor();
public static OptionalInt getScale(DataType dataType) {
return dataType.accept(SCALE_EXTRACTOR);
}

private static final PrecisionExtractor PRECISION_EXTRACTOR = new PrecisionExtractor();
private static final ScaleExtractor SCALE_EXTRACTOR = new ScaleExtractor();
private static final LengthExtractor LENGTH_EXTRACTOR = new LengthExtractor();

private static class PrecisionExtractor extends DataTypeDefaultVisitor<OptionalInt> {
Expand Down Expand Up @@ -463,6 +467,18 @@ protected OptionalInt defaultMethod(DataType dataType) {
}
}

private static class ScaleExtractor extends DataTypeDefaultVisitor<OptionalInt> {
@Override
public OptionalInt visit(DecimalType decimalType) {
return OptionalInt.of(decimalType.getScale());
}

@Override
protected OptionalInt defaultMethod(DataType dataType) {
return OptionalInt.empty();
}
}

private static class LengthExtractor extends DataTypeDefaultVisitor<OptionalInt> {

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@

package com.ververica.cdc.common.types.utils;

import org.apache.flink.util.CollectionUtil;

import com.ververica.cdc.common.data.ArrayData;
import com.ververica.cdc.common.data.DecimalData;
import com.ververica.cdc.common.data.MapData;
Expand All @@ -24,6 +26,10 @@
import com.ververica.cdc.common.data.TimestampData;
import com.ververica.cdc.common.data.ZonedTimestampData;
import com.ververica.cdc.common.types.DataType;
import com.ververica.cdc.common.types.DataTypes;
import com.ververica.cdc.common.utils.Preconditions;

import java.util.List;

/** Utilities for handling {@link DataType}s. */
public class DataTypeUtils {
Expand Down Expand Up @@ -73,4 +79,65 @@ public static Class<?> toInternalConversionClass(DataType type) {
throw new IllegalArgumentException("Illegal type: " + type);
}
}

/**
* Convert CDC's {@link DataType} to Flink's internal {@link
* org.apache.flink.table.types.DataType}.
*/
public static org.apache.flink.table.types.DataType toFlinkDataType(DataType type) {
// ordered by type root definition
List<DataType> children = type.getChildren();
int length = DataTypes.getLength(type).orElse(0);
int precision = DataTypes.getPrecision(type).orElse(0);
int scale = DataTypes.getScale(type).orElse(0);
switch (type.getTypeRoot()) {
case CHAR:
return org.apache.flink.table.api.DataTypes.CHAR(length);
case VARCHAR:
return org.apache.flink.table.api.DataTypes.VARCHAR(length);
case BOOLEAN:
return org.apache.flink.table.api.DataTypes.BOOLEAN();
case BINARY:
return org.apache.flink.table.api.DataTypes.BINARY(length);
case VARBINARY:
return org.apache.flink.table.api.DataTypes.VARBINARY(length);
case DECIMAL:
return org.apache.flink.table.api.DataTypes.DECIMAL(precision, scale);
case TINYINT:
return org.apache.flink.table.api.DataTypes.TINYINT();
case SMALLINT:
return org.apache.flink.table.api.DataTypes.SMALLINT();
case INTEGER:
return org.apache.flink.table.api.DataTypes.INT();
case DATE:
return org.apache.flink.table.api.DataTypes.DATE();
case TIME_WITHOUT_TIME_ZONE:
return org.apache.flink.table.api.DataTypes.TIME(length);
case BIGINT:
return org.apache.flink.table.api.DataTypes.BIGINT();
case FLOAT:
return org.apache.flink.table.api.DataTypes.FLOAT();
case DOUBLE:
return org.apache.flink.table.api.DataTypes.DOUBLE();
case TIMESTAMP_WITHOUT_TIME_ZONE:
return org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_TIME_ZONE(length);
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(length);
case TIMESTAMP_WITH_TIME_ZONE:
return org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_TIME_ZONE(length);
case ARRAY:
Preconditions.checkState(children != null && children.size() > 0);
return org.apache.flink.table.api.DataTypes.ARRAY(toFlinkDataType(children.get(0)));
case MAP:
Preconditions.checkState(children != null && children.size() > 1);
return org.apache.flink.table.api.DataTypes.MAP(
toFlinkDataType(children.get(0)), toFlinkDataType(children.get(1)));
case ROW:
Preconditions.checkState(!CollectionUtil.isNullOrEmpty(children));
return org.apache.flink.table.api.DataTypes.ROW(
children.toArray(new org.apache.flink.table.types.DataType[] {}));
default:
throw new IllegalArgumentException("Illegal type: " + type);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Copyright 2023 Ververica Inc.

Licensed 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>com.ververica</groupId>
<artifactId>flink-cdc-pipeline-connectors</artifactId>
<version>${revision}</version>
</parent>
<artifactId>flink-cdc-pipeline-connector-doris</artifactId>
<name>flink-cdc-pipeline-connector-doris</name>

<dependencies>
<dependency>
<groupId>com.ververica</groupId>
<artifactId>flink-cdc-composer</artifactId>
<version>${revision}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.doris</groupId>
<artifactId>flink-doris-connector-${flink.major.version}</artifactId>
<version>1.5.0</version>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<id>shade-flink</id>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<shadeTestJar>false</shadeTestJar>
<artifactSet>
<includes>
<include>org.apache.doris:*</include>
</includes>
</artifactSet>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,160 @@
/*
* Copyright 2023 Ververica Inc.
*
* Licensed 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 com.ververica.cdc.connectors.doris.factory;

import com.ververica.cdc.common.annotation.Internal;
import com.ververica.cdc.common.configuration.ConfigOption;
import com.ververica.cdc.common.configuration.Configuration;
import com.ververica.cdc.common.factories.DataSinkFactory;
import com.ververica.cdc.common.pipeline.PipelineOptions;
import com.ververica.cdc.common.sink.DataSink;
import com.ververica.cdc.connectors.doris.sink.DorisDataSink;
import com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions;
import org.apache.doris.flink.cfg.DorisExecutionOptions;
import org.apache.doris.flink.cfg.DorisOptions;
import org.apache.doris.flink.cfg.DorisReadOptions;

import java.time.ZoneId;
import java.util.HashSet;
import java.util.Map;
import java.util.Properties;
import java.util.Set;

import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.AUTO_REDIRECT;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.BENODES;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.FENODES;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.JDBC_URL;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.PASSWORD;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_BUFFER_COUNT;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_BUFFER_FLUSH_INTERVAL;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_BUFFER_FLUSH_MAX_BYTES;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_BUFFER_FLUSH_MAX_ROWS;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_BUFFER_SIZE;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_CHECK_INTERVAL;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_ENABLE_2PC;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_ENABLE_BATCH_MODE;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_ENABLE_DELETE;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_FLUSH_QUEUE_SIZE;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_IGNORE_UPDATE_BEFORE;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_LABEL_PREFIX;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_MAX_RETRIES;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.SINK_USE_CACHE;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.STREAM_LOAD_PROP_PREFIX;
import static com.ververica.cdc.connectors.doris.sink.DorisDataSinkOptions.USERNAME;

/** A dummy {@link DataSinkFactory} to create {@link DorisDataSink}. */
@Internal
public class DorisDataSinkFactory implements DataSinkFactory {
@Override
public DataSink createDataSink(Context context) {
Configuration config = context.getFactoryConfiguration();
DorisOptions.Builder optionsBuilder = DorisOptions.builder();
DorisExecutionOptions.Builder executionBuilder = DorisExecutionOptions.builder();
config.getOptional(FENODES).ifPresent(optionsBuilder::setFenodes);
config.getOptional(BENODES).ifPresent(optionsBuilder::setBenodes);
config.getOptional(USERNAME).ifPresent(optionsBuilder::setUsername);
config.getOptional(PASSWORD).ifPresent(optionsBuilder::setPassword);
config.getOptional(JDBC_URL).ifPresent(optionsBuilder::setJdbcUrl);
config.getOptional(AUTO_REDIRECT).ifPresent(optionsBuilder::setAutoRedirect);

config.getOptional(SINK_CHECK_INTERVAL).ifPresent(executionBuilder::setCheckInterval);
config.getOptional(SINK_MAX_RETRIES).ifPresent(executionBuilder::setMaxRetries);
config.getOptional(SINK_ENABLE_DELETE).ifPresent(executionBuilder::setDeletable);
config.getOptional(SINK_LABEL_PREFIX).ifPresent(executionBuilder::setLabelPrefix);
config.getOptional(SINK_BUFFER_SIZE).ifPresent(executionBuilder::setBufferSize);
config.getOptional(SINK_BUFFER_COUNT).ifPresent(executionBuilder::setBufferCount);
config.getOptional(SINK_BUFFER_FLUSH_MAX_ROWS)
.ifPresent(executionBuilder::setBufferFlushMaxRows);
config.getOptional(SINK_BUFFER_FLUSH_MAX_BYTES)
.ifPresent(executionBuilder::setBufferFlushMaxBytes);
config.getOptional(SINK_FLUSH_QUEUE_SIZE).ifPresent(executionBuilder::setFlushQueueSize);
config.getOptional(SINK_IGNORE_UPDATE_BEFORE)
.ifPresent(executionBuilder::setIgnoreUpdateBefore);
config.getOptional(SINK_USE_CACHE).ifPresent(executionBuilder::setUseCache);
config.getOptional(SINK_BUFFER_FLUSH_INTERVAL)
.ifPresent(v -> executionBuilder.setBufferFlushIntervalMs(v.toMillis()));
config.getOptional(SINK_ENABLE_2PC)
.ifPresent(
b -> {
if (b) {
executionBuilder.enable2PC();
} else {
executionBuilder.disable2PC();
}
});
// default batch mode
executionBuilder.setBatchMode(config.get(SINK_ENABLE_BATCH_MODE));

// set streamload properties
Properties properties = DorisExecutionOptions.defaultsProperties();
Map<String, String> streamLoadProp =
DorisDataSinkOptions.getPropertiesByPrefix(config, STREAM_LOAD_PROP_PREFIX);
properties.putAll(streamLoadProp);
executionBuilder.setStreamLoadProp(properties);

return new DorisDataSink(
optionsBuilder.build(),
DorisReadOptions.builder().build(),
executionBuilder.build(),
config,
ZoneId.of(
context.getPipelineConfiguration()
.get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE)));
}

@Override
public String identifier() {
return "doris";
}

@Override
public Set<ConfigOption<?>> requiredOptions() {
Set<ConfigOption<?>> options = new HashSet<>();
options.add(FENODES);
options.add(USERNAME);
return options;
}

@Override
public Set<ConfigOption<?>> optionalOptions() {
Set<ConfigOption<?>> options = new HashSet<>();
options.add(FENODES);
options.add(BENODES);
options.add(USERNAME);
options.add(PASSWORD);
options.add(JDBC_URL);
options.add(AUTO_REDIRECT);

options.add(SINK_CHECK_INTERVAL);
options.add(SINK_ENABLE_2PC);
options.add(SINK_MAX_RETRIES);
options.add(SINK_ENABLE_DELETE);
options.add(SINK_LABEL_PREFIX);
options.add(SINK_BUFFER_SIZE);
options.add(SINK_BUFFER_COUNT);

options.add(SINK_ENABLE_BATCH_MODE);
options.add(SINK_BUFFER_FLUSH_MAX_ROWS);
options.add(SINK_BUFFER_FLUSH_MAX_BYTES);
options.add(SINK_FLUSH_QUEUE_SIZE);
options.add(SINK_BUFFER_FLUSH_INTERVAL);
options.add(SINK_IGNORE_UPDATE_BEFORE);
options.add(SINK_USE_CACHE);

return options;
}
}
Loading