Skip to content
This repository was archived by the owner on Feb 16, 2024. It is now read-only.
Open
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
1 change: 0 additions & 1 deletion dev/checkstyle.xml
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
<!DOCTYPE module PUBLIC
"-//Puppy Crawl//DTD Check Configuration 1.3//EN"
"http://www.puppycrawl.com/dtds/configuration_1_3.dtd">

<!--

Checkstyle configuration based on the Google coding conventions from:
Expand Down
111 changes: 111 additions & 0 deletions flink-connector-clickhouse/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
<?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/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.bahir</groupId>
<artifactId>bahir-flink-parent_2.11</artifactId>
<version>1.1-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

<artifactId>flink-connector-clickhouse_${scala.binary.version}</artifactId>
<name>flink-connector-clickhouse</name>
<packaging>jar</packaging>

<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-scala_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>ru.yandex.clickhouse</groupId>
<artifactId>clickhouse-jdbc</artifactId>
<version>${clickhouse.version}</version>
</dependency>

<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<version>${slf4j.version}</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<version>${log4j.version}</version>
<scope>runtime</scope>
</dependency>

<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>4.11</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this are in parent pom no?

<pluginManagement>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<shadeTestJar>false</shadeTestJar>
<artifactSet>
<excludes>
<exclude>junit:junit</exclude>
</excludes>
<includes>
<include>*:*</include>
</includes>
</artifactSet>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</pluginManagement>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/*
* 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 com.apache.flink.streaming.connectors.clickhouse;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.flink.types.Row;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import ru.yandex.clickhouse.BalancedClickhouseDataSource;
import ru.yandex.clickhouse.settings.ClickHouseProperties;

import java.sql.Connection;
import java.sql.PreparedStatement;
import java.util.Properties;

public class ClickHouseAppendSinkFunction extends RichSinkFunction<Row> implements CheckpointedFunction {
private static final String USERNAME = "user";
private static final String PASSWORD = "password";

private static final Logger log = LoggerFactory.getLogger(ClickHouseAppendSinkFunction.class);
private static final long serialVersionUID = 1L;

private Connection connection;
private BalancedClickhouseDataSource dataSource;
private PreparedStatement pstat;

private String address;
private String username;
private String password;

private String prepareStatement;
private Integer batchSize;
private Long commitPadding;

private Integer retries;
private Long retryInterval;

private Boolean ignoreInsertError;

private Integer currentSize;
private Long lastExecuteTime;

public ClickHouseAppendSinkFunction(String address, String username, String password, String prepareStatement, Integer batchSize, Long commitPadding, Integer retries, Long retryInterval, Boolean ignoreInsertError) {
this.address = address;
this.username = username;
this.password = password;
this.prepareStatement = prepareStatement;
this.batchSize = batchSize;
this.commitPadding = commitPadding;
this.retries = retries;
this.retryInterval = retryInterval;
this.ignoreInsertError = ignoreInsertError;
}

@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
Properties properties = new Properties();
properties.setProperty(USERNAME, username);
properties.setProperty(PASSWORD, password);
ClickHouseProperties clickHouseProperties = new ClickHouseProperties(properties);
dataSource = new BalancedClickhouseDataSource(address, clickHouseProperties);
connection = dataSource.getConnection();
pstat = connection.prepareStatement(prepareStatement);
lastExecuteTime = System.currentTimeMillis();
currentSize = 0;

}

@Override
public void invoke(Row value, Context context) throws Exception {
for (int i = 0; i < value.getArity(); i++) {
pstat.setObject(i + 1, value.getField(i));
}
pstat.addBatch();
currentSize++;
if (currentSize >= batchSize || (System.currentTimeMillis() - lastExecuteTime) > commitPadding) {
try {
doExecuteRetries(retries, retryInterval);
} catch (Exception e) {
log.error("clickhouse-insert-error ( maxRetries:" + retries + " , retryInterval : " + retryInterval + " millisecond )" + e.getMessage());
} finally {
pstat.clearBatch();
currentSize = 0;
lastExecuteTime = System.currentTimeMillis();
}
}
}

public void doExecuteRetries(int count, long retryInterval) throws Exception {

int retrySize = 0;
Exception resultException = null;
for (int i = 0; i < count; i++) {
try {
pstat.executeBatch();
break;
} catch (Exception e) {
retrySize++;
resultException = e;
}
try {
Thread.sleep(retryInterval);
} catch (InterruptedException e) {
log.error("clickhouse retry interval exception : ",e);
}
}
if (retrySize == count && !ignoreInsertError) {
throw resultException;
}
}

@Override
public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception {
doExecuteRetries(retries, retryInterval);
}

@Override
public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {

}

}
Loading