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

[Feature][Connector-V2] Starrocks sink connector #3164

Merged
merged 15 commits into from
Nov 7, 2022
120 changes: 120 additions & 0 deletions docs/en/connector-v2/sink/StarRocks.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,120 @@
# StarRocks

> StarRocks sink connector

## Description
Used to send data to StarRocks. Both support streaming and batch mode.
The internal implementation of StarRocks 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 | no | - |
| 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.* | starrocks stream load config | no | - |

### node_urls [list]

`StarRocks` cluster address, the format is `["fe_ip:fe_http_port", ...]`

### username [string]

`StarRocks` user username

### password [string]

`StarRocks` user password

### database [string]

The name of StarRocks database

### table [string]

The name of StarRocks table

### labelPrefix [string]

the prefix of StarRocks stream load label
531651225 marked this conversation as resolved.
Show resolved Hide resolved
### batch_max_rows [string]

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 StarRocks

### batch_max_bytes [string]

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 StarRocks

### batch_interval_ms [string]

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 StarRocks

### max_retries [string]

The number of retries to flush failed

### retry_backoff_multiplier_ms [string]

Using as a multiplier for generating the next delay for backoff

### max_retry_backoff_ms [string]

The amount of time to wait before attempting to retry a request to `StarRocks`

### sink.properties.* [starrocks stream load config]
531651225 marked this conversation as resolved.
Show resolved Hide resolved
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
531651225 marked this conversation as resolved.
Show resolved Hide resolved
The supported formats include CSV and JSON. Default value: CSV


## Example
Use JSON format to import data
```
sink {
StarRocks {
nodeUrls = ["e2e_starRocksdb:8030"]
username = root
password = ""
database = "test"
table = "e2e_table_sink"
batch_max_rows = 10
sink.properties.format = "JSON"
sink.properties.strip_outer_array = true
}
}

```

Use CSV format to import data
```
sink {
StarRocks {
nodeUrls = ["e2e_starRocksdb:8030"]
username = root
password = ""
database = "test"
table = "e2e_table_sink"
batch_max_rows = 10
sink.properties.format = "CSV"
sink.properties.column_separator = "\\x01",
sink.properties.row_delimiter = "\\x02"
}
}
```
531651225 marked this conversation as resolved.
Show resolved Hide resolved
3 changes: 2 additions & 1 deletion plugin-mapping.properties
Original file line number Diff line number Diff line change
Expand Up @@ -135,4 +135,5 @@ seatunnel.sink.MongoDB = connector-mongodb
seatunnel.source.Iceberg = connector-iceberg
seatunnel.source.InfluxDB = connector-influxdb
seatunnel.source.S3File = connector-file-s3
seatunnel.sink.S3File = connector-file-s3
seatunnel.sink.S3File = connector-file-s3
seatunnel.sink.StarRocks = connector-starrocks
59 changes: 59 additions & 0 deletions seatunnel-connectors-v2/connector-starrocks/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--

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.

-->
<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">
<parent>
<artifactId>seatunnel-connectors-v2</artifactId>
<groupId>org.apache.seatunnel</groupId>
<version>${revision}</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>connector-starrocks</artifactId>

<properties>
<httpclient.version>4.5.13</httpclient.version>
<httpcore.version>4.4.4</httpcore.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-api</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>connector-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
<version>${httpclient.version}</version>
</dependency>

531651225 marked this conversation as resolved.
Show resolved Hide resolved
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId>
<version>${httpcore.version}</version>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -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.
*/

package org.apache.seatunnel.connectors.seatunnel.starrocks.client;

import java.util.List;

public class StarRocksFlushTuple {
531651225 marked this conversation as resolved.
Show resolved Hide resolved
private String label;
private Long bytes;
private List<byte[]> rows;

public StarRocksFlushTuple(String label, Long bytes, List<byte[]> rows) {
this.label = label;
this.bytes = bytes;
this.rows = rows;
}

public String getLabel() {
return label;
}

public void setLabel(String label) {
this.label = label;
}

public Long getBytes() {
return bytes;
}

public List<byte[]> getRows() {
return rows;
}
531651225 marked this conversation as resolved.
Show resolved Hide resolved
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,154 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.seatunnel.connectors.seatunnel.starrocks.client;

import org.apache.seatunnel.connectors.seatunnel.starrocks.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 StarRocksSinkManager {

private final SinkConfig sinkConfig;
private final List<byte[]> batchList;

private StarRocksStreamLoadVisitor starrocksStreamLoadVisitor;
private ScheduledExecutorService scheduler;
private ScheduledFuture<?> scheduledFuture;
private volatile boolean initialize;
private volatile Exception flushException;
private int batchRowCount = 0;
private long batchBytesSize = 0;

public StarRocksSinkManager(SinkConfig sinkConfig, List<String> fileNames) {
this.sinkConfig = sinkConfig;
this.batchList = new ArrayList<>();
starrocksStreamLoadVisitor = new StarRocksStreamLoadVisitor(sinkConfig, fileNames);
}

private void tryInit() throws IOException {
if (initialize) {
return;
}

if (sinkConfig.getBatchIntervalMs() != null) {
531651225 marked this conversation as resolved.
Show resolved Hide resolved
scheduler = Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder().setNameFormat("StarRocks-sink-output-%s").build());
scheduledFuture = scheduler.scheduleAtFixedRate(
() -> {
try {
flush();
} catch (IOException e) {
flushException = e;
}
},
sinkConfig.getBatchIntervalMs(),
sinkConfig.getBatchIntervalMs(),
TimeUnit.MILLISECONDS);
}
initialize = true;
531651225 marked this conversation as resolved.
Show resolved Hide resolved
}

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;
}
StarRocksFlushTuple tuple = null;
for (int i = 0; i <= sinkConfig.getMaxRetries(); i++) {
try {
String label = createBatchLabel();
tuple = new StarRocksFlushTuple(label, batchBytesSize, new ArrayList<>(batchList));
starrocksStreamLoadVisitor.doStreamLoad(tuple);
} catch (Exception e) {

log.error("Writing records to StarRocks failed, retry times = {}", i, e);
if (i >= sinkConfig.getMaxRetries()) {
throw new IOException("Writing records to StarRocks failed.", e);
}

if (e instanceof StarRocksStreamLoadFailedException && ((StarRocksStreamLoadFailedException) 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 StarRocks 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();
}
}
Loading