├── .github
└── workflows
│ └── maven.yml
├── .gitignore
├── LICENSE
├── README.md
├── pom.xml
└── src
├── main
└── java
│ └── ru
│ └── ivi
│ └── opensource
│ └── flinkclickhousesink
│ ├── ClickHouseSink.java
│ ├── ClickHouseSinkConverter.java
│ ├── applied
│ ├── ClickHouseSinkBuffer.java
│ ├── ClickHouseSinkManager.java
│ ├── ClickHouseSinkScheduledCheckerAndCleaner.java
│ ├── ClickHouseWriter.java
│ ├── ExceptionsThrowableSink.java
│ ├── Sink.java
│ └── UnexceptionableSink.java
│ ├── model
│ ├── ClickHouseClusterSettings.java
│ ├── ClickHouseRequestBlank.java
│ ├── ClickHouseSinkCommonParams.java
│ └── ClickHouseSinkConst.java
│ └── util
│ ├── ConfigUtil.java
│ ├── FutureUtil.java
│ └── ThreadUtil.java
└── test
├── java
└── ru
│ └── ivi
│ └── opensource
│ └── flinkclickhousesink
│ ├── ClickHouseWriterTest.java
│ └── applied
│ ├── ClickHouseSinkBufferTest.java
│ ├── ClickHouseSinkScheduledCheckerAndCleanerTest.java
│ ├── ExceptionsThrowableSinkTest.java
│ └── UnexceptionableSinkTest.java
└── resources
├── logback-test.xml
└── reference.conf
/.github/workflows/maven.yml:
--------------------------------------------------------------------------------
1 | # This workflow will build a Java project with Maven, and cache/restore any dependencies to improve the workflow execution time
2 | # For more information see: https://docs.github.com/en/actions/automating-builds-and-tests/building-and-testing-java-with-maven
3 |
4 | # This workflow uses actions that are not certified by GitHub.
5 | # They are provided by a third-party and are governed by
6 | # separate terms of service, privacy policy, and support
7 | # documentation.
8 |
9 | name: Java CI with Maven
10 |
11 | on: [push, pull_request]
12 |
13 | jobs:
14 | build:
15 |
16 | runs-on: ubuntu-latest
17 |
18 | steps:
19 | - uses: actions/checkout@v3
20 | - name: Set up JDK 8
21 | uses: actions/setup-java@v3
22 | with:
23 | java-version: '8'
24 | distribution: 'temurin'
25 | cache: maven
26 | - name: Build with Maven
27 | run: mvn clean package
28 |
29 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | # mvn
2 | target
3 |
4 | # idea
5 | *.iml
6 | .idea
7 | dependency-reduced-pom.xml
8 |
9 | # mac
10 | .DS_Store
11 |
12 | # ignore
13 | out
14 | logs
15 |
16 | # version plugin
17 | pom.xml-backup
18 |
19 | # python
20 | venv
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 | MIT License
2 |
3 | Copyright (c) 2018 ivi.ru
4 |
5 | Permission is hereby granted, free of charge, to any person obtaining a copy
6 | of this software and associated documentation files (the "Software"), to deal
7 | in the Software without restriction, including without limitation the rights
8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
9 | copies of the Software, and to permit persons to whom the Software is
10 | furnished to do so, subject to the following conditions:
11 |
12 | The above copyright notice and this permission notice shall be included in all
13 | copies or substantial portions of the Software.
14 |
15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
21 | SOFTWARE.
22 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 |
2 | # Flink-ClickHouse-Sink
3 |
4 | 
5 | [](https://maven-badges.herokuapp.com/maven-central/ru.ivi.opensource/flink-clickhouse-sink/)
6 |
7 | ## Description
8 |
9 | [Flink](https://github.com/apache/flink) sink for [ClickHouse](https://github.com/yandex/ClickHouse) database.
10 | Powered by [Async Http Client](https://github.com/AsyncHttpClient/async-http-client).
11 |
12 | High-performance library for loading data to ClickHouse.
13 |
14 | It has two triggers for loading data:
15 | _by timeout_ and _by buffer size_.
16 |
17 | ##### Version map
18 | |flink | flink-clickhouse-sink |
19 | |:-------:|:---------------------:|
20 | |1.3.* | 1.0.0 |
21 | |1.9.* | 1.3.4 |
22 | |1.9.* | 1.4.* |
23 |
24 | ### Install
25 |
26 | ##### Maven Central
27 |
28 | ```xml
29 |
30 | ru.ivi.opensource
31 | flink-clickhouse-sink
32 | 1.4.0
33 |
34 | ```
35 |
36 | ## Usage
37 | ### Properties
38 | The flink-clickhouse-sink uses two parts of configuration properties:
39 | common and for each sink in you operators chain.
40 |
41 | **The common part** (use like global):
42 |
43 | `clickhouse.sink.num-writers` - number of writers, which build and send requests,
44 |
45 | `clickhouse.sink.queue-max-capacity` - max capacity (batches) of blank's queue,
46 |
47 | `clickhouse.sink.timeout-sec` - timeout for loading data,
48 |
49 | `clickhouse.sink.retries` - max number of retries,
50 |
51 | `clickhouse.sink.failed-records-path`- path for failed records,
52 |
53 | `clickhouse.sink.ignoring-clickhouse-sending-exception-enabled` - required boolean parameter responsible for raising (false) or not (true) ClickHouse sending exception in main thread.
54 | if `ignoring-clickhouse-sending-exception-enabled` is true, exception while clickhouse sending is ignored and failed data automatically goes to the disk.
55 | if `ignoring-clickhouse-sending-exception-enabled` is false, clickhouse sending exception thrown in "main" thread (thread which called ClickhHouseSink::invoke) and data also goes to the disk.
56 |
57 | **The sink part** (use in chain):
58 |
59 | `clickhouse.sink.target-table` - target table in ClickHouse,
60 |
61 | `clickhouse.sink.max-buffer-size`- buffer size.
62 |
63 | ### In code
64 |
65 | #### Configuration: global parameters
66 |
67 | At first, you add global parameters for the Flink environment:
68 | ```java
69 | StreamExecutionEnvironment environment = StreamExecutionEnvironment.createLocalEnvironment();
70 | Map globalParameters = new HashMap<>();
71 |
72 | // ClickHouse cluster properties
73 | globalParameters.put(ClickHouseClusterSettings.CLICKHOUSE_HOSTS, ...);
74 | globalParameters.put(ClickHouseClusterSettings.CLICKHOUSE_USER, ...);
75 | globalParameters.put(ClickHouseClusterSettings.CLICKHOUSE_PASSWORD, ...);
76 |
77 | // sink common
78 | globalParameters.put(ClickHouseSinkConst.TIMEOUT_SEC, ...);
79 | globalParameters.put(ClickHouseSinkConst.FAILED_RECORDS_PATH, ...);
80 | globalParameters.put(ClickHouseSinkConst.NUM_WRITERS, ...);
81 | globalParameters.put(ClickHouseSinkConst.NUM_RETRIES, ...);
82 | globalParameters.put(ClickHouseSinkConst.QUEUE_MAX_CAPACITY, ...);
83 | globalParameters.put(ClickHouseSinkConst.IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED, ...);
84 |
85 | // set global paramaters
86 | ParameterTool parameters = ParameterTool.fromMap(buildGlobalParameters(config));
87 | environment.getConfig().setGlobalJobParameters(parameters);
88 |
89 | ```
90 |
91 | #### Converter
92 |
93 | The main thing: the clickhouse-sink works with events in _String_
94 | (ClickHouse insert format, like CSV) format.
95 | You have to convert your event to csv format (like usual insert into a database).
96 |
97 | For example, you have an event-pojo:
98 | ```java
99 | class A {
100 | public final String str;
101 | public final int integer;
102 |
103 | public A(String str, int i){
104 | this.str = str;
105 | this.integer = i;
106 | }
107 | }
108 | ```
109 | You have to implement a converter to csv, using
110 | ```java
111 |
112 | public interface ClickHouseSinkConverter {
113 | ...
114 | }
115 | ```
116 |
117 | You convert the pojo like this:
118 |
119 | ```java
120 | import ru.ivi.opensource.flinkclickhousesink.ClickHouseSinkConverter;
121 |
122 | public class YourEventConverter implements ClickHouseSinkConverter{
123 |
124 | @Override
125 | public String convert(A record){
126 | StringBuilder builder = new StringBuilder();
127 | builder.append("(");
128 |
129 | // add a.str
130 | builder.append("'");
131 | builder.append(a.str);
132 | builder.append("', ");
133 |
134 | // add a.integer
135 | builder.append(String.valueOf(a.integer));
136 | builder.append(" )");
137 | return builder.toString();
138 | }
139 | }
140 | ```
141 |
142 | And then add your sink to the chain:
143 | ```java
144 |
145 | // create table props for sink
146 | Properties props = new Properties();
147 | props.put(ClickHouseSinkConst.TARGET_TABLE_NAME, "your_table");
148 | props.put(ClickHouseSinkConst.MAX_BUFFER_SIZE, "10000");
149 |
150 | // converter
151 | YourEventConverter converter = new YourEventConverter();
152 |
153 | // build chain
154 | DataStream dataStream = ...;
155 | dataStream.addSink(new ClickHouseSink(props, converter))
156 | .name("your_table ClickHouse sink);
157 | ```
158 |
159 | ## Roadmap
160 | - [ ] reading files from "failed-records-path"
161 | - [ ] migrate to gradle
162 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
5 | 4.0.0
6 |
7 |
8 | org.sonatype.oss
9 | oss-parent
10 | 9
11 |
12 |
13 | ru.ivi.opensource
14 | flink-clickhouse-sink
15 | 1.4.1-SNAPSHOT
16 | jar
17 |
18 | Flink ClickHouse sink
19 |
20 |
21 | Flink sink for ClickHouse database. Powered by Async Http Client.
22 | High-performance library for loading data to ClickHouse.
23 |
24 | https://github.com/ivi-ru/flink-clickhouse-sink.git
25 |
26 |
27 | UTF-8
28 |
29 | 1.8
30 | 2.11
31 | 1.9.0
32 |
33 |
34 | 2.10.4
35 | 1.2.3
36 | 23.0
37 | 1.3.3
38 | 0.1.40
39 |
40 |
41 | 4.13.1
42 | 2.8.9
43 | 3.1.2
44 | 1.17.1
45 | 2.3.8
46 | 2.9.0
47 | 2.13.2
48 |
49 |
50 | 3.7.0
51 |
52 |
53 |
54 | scm:git:git@github.com:ivi-ru/flink-clickhouse-sink.git
55 | scm:git:git@github.com:ivi-ru/flink-clickhouse-sink.git
56 | https://github.com/ivi-ru/flink-clickhouse-sink.git
57 |
58 |
59 |
60 |
61 | sonatype-nexus-snapshots
62 | Sonatype Nexus snapshot repository
63 | https://oss.sonatype.org/content/repositories/snapshots
64 |
65 |
66 | sonatype-nexus-staging
67 | Sonatype Nexus release repository
68 | https://oss.sonatype.org/service/local/staging/deploy/maven2/
69 |
70 |
71 |
72 |
73 |
74 | mchernyakov
75 | Mikhail Chernyakov
76 |
77 |
78 | aleksanchezz
79 | Alexander Lanko
80 |
81 |
82 | ashulenko
83 | Andrey Shulenko
84 |
85 |
86 | sserdyukov
87 | Stanislav Serdyukov
88 |
89 |
90 |
91 |
92 | https://github.com/ivi-ru/flink-clickhouse-sink/issues
93 | GitHub Issues
94 |
95 |
96 |
97 |
98 | MIT License
99 | http://www.opensource.org/licenses/mit-license.php
100 | repo
101 |
102 |
103 |
104 |
105 |
106 |
107 | com.typesafe
108 | config
109 | ${typesafe.config.version}
110 |
111 |
112 |
113 | com.google.guava
114 | guava
115 | ${guava.version}
116 |
117 |
118 |
119 | ch.qos.logback
120 | logback-classic
121 | ${logback.version}
122 |
123 |
124 |
125 | org.asynchttpclient
126 | async-http-client
127 | ${async.client.version}
128 |
129 |
130 |
131 | org.apache.flink
132 | flink-streaming-java_${scala.version}
133 | ${flink.version}
134 |
135 |
136 |
137 | junit
138 | junit
139 | ${junit.version}
140 | test
141 |
142 |
143 |
144 | ru.yandex.clickhouse
145 | clickhouse-jdbc
146 | ${ch.driver.version}
147 | test
148 |
149 |
150 |
151 | org.awaitility
152 | awaitility
153 | ${awaitility.version}
154 | test
155 |
156 |
157 |
158 | org.jmock
159 | jmock-junit4
160 | ${jmock.junit4.version}
161 | test
162 |
163 |
164 |
165 | com.zaxxer
166 | HikariCP
167 | ${hikari.version}
168 | test
169 |
170 |
171 |
172 | org.testcontainers
173 | clickhouse
174 | ${testcontainers.version}
175 | test
176 |
177 |
178 |
179 | com.fasterxml.jackson.core
180 | jackson-annotations
181 | ${jackson.version}
182 | test
183 |
184 |
185 |
186 | org.mockito
187 | mockito-core
188 | ${mockito.version}
189 | test
190 |
191 |
192 |
193 |
194 |
195 |
196 |
197 | org.apache.maven.plugins
198 | maven-compiler-plugin
199 | ${mvn.compiler.version}
200 |
201 | ${java.version}
202 | ${java.version}
203 |
204 |
205 |
206 |
207 |
208 |
209 |
210 | release
211 |
212 | false
213 |
214 |
215 |
216 |
217 | maven-enforcer-plugin
218 | 1.4.1
219 |
220 |
221 | enforce-versions
222 |
223 | enforce
224 |
225 |
226 |
227 |
228 | ${java.version}
229 |
230 |
231 |
232 |
233 |
234 |
235 |
236 |
237 | org.apache.maven.plugins
238 | maven-release-plugin
239 | 2.5.3
240 |
241 | true
242 | false
243 | release
244 | deploy
245 |
246 |
247 |
248 |
249 | org.apache.maven.plugins
250 | maven-source-plugin
251 | 3.0.1
252 |
253 |
254 | attach-sources
255 |
256 | jar-no-fork
257 |
258 |
259 |
260 |
261 |
262 |
263 | org.apache.maven.plugins
264 | maven-javadoc-plugin
265 | 3.0.1
266 |
267 |
268 | attach-javadocs
269 |
270 | jar
271 |
272 |
273 |
274 |
275 |
276 |
277 | org.apache.maven.plugins
278 | maven-gpg-plugin
279 | 1.6
280 |
281 |
282 | sign-artifacts
283 | verify
284 |
285 | sign
286 |
287 |
288 |
289 |
290 |
291 |
292 |
293 |
294 |
295 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/ClickHouseSink.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink;
2 |
3 | import org.apache.flink.configuration.Configuration;
4 | import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
5 | import org.slf4j.Logger;
6 | import org.slf4j.LoggerFactory;
7 | import ru.ivi.opensource.flinkclickhousesink.applied.ClickHouseSinkManager;
8 | import ru.ivi.opensource.flinkclickhousesink.applied.Sink;
9 |
10 | import java.util.Map;
11 | import java.util.Properties;
12 |
13 | public class ClickHouseSink extends RichSinkFunction {
14 |
15 | private static final Logger logger = LoggerFactory.getLogger(ClickHouseSink.class);
16 |
17 | private static final Object DUMMY_LOCK = new Object();
18 |
19 | private final Properties localProperties;
20 | private final ClickHouseSinkConverter clickHouseSinkConverter;
21 |
22 | private volatile static transient ClickHouseSinkManager sinkManager;
23 | private transient Sink sink;
24 |
25 | public ClickHouseSink(Properties properties,
26 | ClickHouseSinkConverter clickHouseSinkConverter) {
27 | this.localProperties = properties;
28 | this.clickHouseSinkConverter = clickHouseSinkConverter;
29 | }
30 |
31 | @Override
32 | public void open(Configuration config) {
33 | if (sinkManager == null) {
34 | synchronized (DUMMY_LOCK) {
35 | if (sinkManager == null) {
36 | Map params = getRuntimeContext()
37 | .getExecutionConfig()
38 | .getGlobalJobParameters()
39 | .toMap();
40 |
41 | sinkManager = new ClickHouseSinkManager(params);
42 | }
43 | }
44 | }
45 |
46 | sink = sinkManager.buildSink(localProperties);
47 | }
48 |
49 | /**
50 | * Add a record to sink
51 | *
52 | * @param record record, which will be converted to csv, using {@link ClickHouseSinkConverter}
53 | * @param context ctx
54 | */
55 | @Override
56 | public void invoke(T record, Context context) {
57 | try {
58 | String recordAsCSV = clickHouseSinkConverter.convert(record);
59 | sink.put(recordAsCSV);
60 | } catch (Exception e) {
61 | logger.error("Error while sending data to ClickHouse, record = {}", record, e);
62 | throw new RuntimeException(e);
63 | }
64 | }
65 |
66 | @Override
67 | public void close() throws Exception {
68 | if (sink != null) {
69 | sink.close();
70 | }
71 |
72 | if (sinkManager != null && !sinkManager.isClosed()) {
73 | synchronized (DUMMY_LOCK) {
74 | if (sinkManager != null && !sinkManager.isClosed()) {
75 | sinkManager.close();
76 | sinkManager = null;
77 | }
78 | }
79 | }
80 |
81 | super.close();
82 | }
83 | }
84 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/ClickHouseSinkConverter.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink;
2 |
3 | import java.io.Serializable;
4 |
5 | @FunctionalInterface
6 | public interface ClickHouseSinkConverter extends Serializable {
7 | String convert(T record);
8 | }
9 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/applied/ClickHouseSinkBuffer.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import com.google.common.base.Preconditions;
4 | import org.slf4j.Logger;
5 | import org.slf4j.LoggerFactory;
6 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseRequestBlank;
7 | import ru.ivi.opensource.flinkclickhousesink.util.FutureUtil;
8 |
9 | import java.util.ArrayList;
10 | import java.util.Collections;
11 | import java.util.List;
12 | import java.util.concurrent.CompletableFuture;
13 | import java.util.concurrent.ExecutionException;
14 | import java.util.concurrent.TimeUnit;
15 |
16 | public class ClickHouseSinkBuffer implements AutoCloseable {
17 | private static final Logger logger = LoggerFactory.getLogger(ClickHouseSinkBuffer.class);
18 |
19 | private final ClickHouseWriter writer;
20 | private final String targetTable;
21 | private final int maxFlushBufferSize;
22 | private final long timeoutMillis;
23 | private final List localValues;
24 | private final List> futures;
25 |
26 | private volatile long lastAddTimeMillis = System.currentTimeMillis();
27 |
28 | private ClickHouseSinkBuffer(
29 | ClickHouseWriter chWriter,
30 | long timeout,
31 | int maxBuffer,
32 | String table,
33 | List> futures
34 | ) {
35 | writer = chWriter;
36 | localValues = new ArrayList<>();
37 | timeoutMillis = timeout;
38 | maxFlushBufferSize = maxBuffer;
39 | targetTable = table;
40 |
41 | this.futures = futures;
42 |
43 | logger.info("Instance ClickHouse Sink, target table = {}, buffer size = {}", this.targetTable, this.maxFlushBufferSize);
44 | }
45 |
46 | String getTargetTable() {
47 | return targetTable;
48 | }
49 |
50 | public void put(String recordAsCSV) {
51 | tryAddToQueue();
52 | localValues.add(recordAsCSV);
53 | }
54 |
55 | synchronized void tryAddToQueue() {
56 | if (flushCondition()) {
57 | addToQueue();
58 | lastAddTimeMillis = System.currentTimeMillis();
59 | }
60 | }
61 |
62 | private void addToQueue() {
63 | List deepCopy = buildDeepCopy(localValues);
64 | ClickHouseRequestBlank params = ClickHouseRequestBlank.Builder
65 | .aBuilder()
66 | .withValues(deepCopy)
67 | .withTargetTable(targetTable)
68 | .build();
69 |
70 | logger.debug("Build blank with params: buffer size = {}, target table = {}", params.getValues().size(), params.getTargetTable());
71 | writer.put(params);
72 |
73 | localValues.clear();
74 | }
75 |
76 | private boolean flushCondition() {
77 | return localValues.size() > 0 && (checkSize() || checkTime());
78 | }
79 |
80 | private boolean checkSize() {
81 | return localValues.size() >= maxFlushBufferSize;
82 | }
83 |
84 | private boolean checkTime() {
85 | long current = System.currentTimeMillis();
86 | return current - lastAddTimeMillis > timeoutMillis;
87 | }
88 |
89 | private static List buildDeepCopy(List original) {
90 | return Collections.unmodifiableList(new ArrayList<>(original));
91 | }
92 |
93 | public void assertFuturesNotFailedYet() throws ExecutionException, InterruptedException {
94 | CompletableFuture future = FutureUtil.allOf(futures);
95 | //nonblocking operation
96 | if (future.isCompletedExceptionally()) {
97 | future.get();
98 | }
99 | }
100 |
101 | @Override
102 | public void close() {
103 | logger.info("ClickHouse sink buffer is shutting down.");
104 | if (localValues != null && localValues.size() > 0) {
105 | addToQueue();
106 | }
107 | logger.info("ClickHouse sink buffer shutdown complete.");
108 | }
109 |
110 | public static final class Builder {
111 | private String targetTable;
112 | private int maxFlushBufferSize;
113 | private int timeoutSec;
114 | private List> futures;
115 |
116 | private Builder() {
117 | }
118 |
119 | public static Builder aClickHouseSinkBuffer() {
120 | return new Builder();
121 | }
122 |
123 | public Builder withTargetTable(String targetTable) {
124 | this.targetTable = targetTable;
125 | return this;
126 | }
127 |
128 | public Builder withMaxFlushBufferSize(int maxFlushBufferSize) {
129 | this.maxFlushBufferSize = maxFlushBufferSize;
130 | return this;
131 | }
132 |
133 | public Builder withTimeoutSec(int timeoutSec) {
134 | this.timeoutSec = timeoutSec;
135 | return this;
136 | }
137 |
138 | public Builder withFutures(List> futures) {
139 | this.futures = futures;
140 | return this;
141 | }
142 |
143 | public ClickHouseSinkBuffer build(ClickHouseWriter writer) {
144 |
145 | Preconditions.checkNotNull(targetTable);
146 | Preconditions.checkArgument(maxFlushBufferSize > 0);
147 | Preconditions.checkArgument(timeoutSec > 0);
148 |
149 | return new ClickHouseSinkBuffer(
150 | writer,
151 | TimeUnit.SECONDS.toMillis(this.timeoutSec),
152 | this.maxFlushBufferSize,
153 | this.targetTable,
154 | this.futures
155 | );
156 | }
157 | }
158 | }
159 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/applied/ClickHouseSinkManager.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import com.google.common.base.Preconditions;
4 | import org.slf4j.Logger;
5 | import org.slf4j.LoggerFactory;
6 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkCommonParams;
7 |
8 | import java.util.*;
9 | import java.util.concurrent.CompletableFuture;
10 |
11 | import static ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst.MAX_BUFFER_SIZE;
12 | import static ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst.TARGET_TABLE_NAME;
13 |
14 | public class ClickHouseSinkManager implements AutoCloseable {
15 | private static final Logger logger = LoggerFactory.getLogger(ClickHouseSinkManager.class);
16 |
17 | private final ClickHouseWriter clickHouseWriter;
18 | private final ClickHouseSinkScheduledCheckerAndCleaner clickHouseSinkScheduledCheckerAndCleaner;
19 | private final ClickHouseSinkCommonParams sinkParams;
20 | private final List> futures = Collections.synchronizedList(new LinkedList<>());
21 |
22 | private volatile boolean isClosed = false;
23 |
24 | public ClickHouseSinkManager(Map globalParams) {
25 | sinkParams = new ClickHouseSinkCommonParams(globalParams);
26 | clickHouseWriter = new ClickHouseWriter(sinkParams, futures);
27 | clickHouseSinkScheduledCheckerAndCleaner = new ClickHouseSinkScheduledCheckerAndCleaner(sinkParams, futures);
28 | logger.info("Build sink writer's manager. params = {}", sinkParams);
29 | }
30 |
31 | public Sink buildSink(Properties localProperties) {
32 | String targetTable = localProperties.getProperty(TARGET_TABLE_NAME);
33 | int maxFlushBufferSize = Integer.parseInt(localProperties.getProperty(MAX_BUFFER_SIZE));
34 |
35 | return buildSink(targetTable, maxFlushBufferSize);
36 | }
37 |
38 | public Sink buildSink(String targetTable, int maxBufferSize) {
39 | Preconditions.checkNotNull(clickHouseSinkScheduledCheckerAndCleaner);
40 | Preconditions.checkNotNull(clickHouseWriter);
41 |
42 | ClickHouseSinkBuffer clickHouseSinkBuffer = ClickHouseSinkBuffer.Builder
43 | .aClickHouseSinkBuffer()
44 | .withTargetTable(targetTable)
45 | .withMaxFlushBufferSize(maxBufferSize)
46 | .withTimeoutSec(sinkParams.getTimeout())
47 | .withFutures(futures)
48 | .build(clickHouseWriter);
49 |
50 | clickHouseSinkScheduledCheckerAndCleaner.addSinkBuffer(clickHouseSinkBuffer);
51 |
52 | if (sinkParams.isIgnoringClickHouseSendingExceptionEnabled()) {
53 | return new UnexceptionableSink(clickHouseSinkBuffer);
54 | } else {
55 | return new ExceptionsThrowableSink(clickHouseSinkBuffer);
56 | }
57 |
58 | }
59 |
60 | public boolean isClosed() {
61 | return isClosed;
62 | }
63 |
64 | @Override
65 | public void close() throws Exception {
66 | logger.info("ClickHouse sink manager is shutting down.");
67 | clickHouseSinkScheduledCheckerAndCleaner.close();
68 | clickHouseWriter.close();
69 | isClosed = true;
70 | logger.info("ClickHouse sink manager shutdown complete.");
71 | }
72 | }
73 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/applied/ClickHouseSinkScheduledCheckerAndCleaner.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import org.slf4j.Logger;
4 | import org.slf4j.LoggerFactory;
5 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkCommonParams;
6 | import ru.ivi.opensource.flinkclickhousesink.util.ThreadUtil;
7 |
8 | import java.util.ArrayList;
9 | import java.util.List;
10 | import java.util.concurrent.CompletableFuture;
11 | import java.util.concurrent.Executors;
12 | import java.util.concurrent.ScheduledExecutorService;
13 | import java.util.concurrent.ThreadFactory;
14 | import java.util.concurrent.TimeUnit;
15 | import java.util.function.Predicate;
16 |
17 | public class ClickHouseSinkScheduledCheckerAndCleaner implements AutoCloseable {
18 | private static final Logger logger = LoggerFactory.getLogger(ClickHouseSinkScheduledCheckerAndCleaner.class);
19 |
20 | private final ScheduledExecutorService scheduledExecutorService;
21 | private final List clickHouseSinkBuffers = new ArrayList<>();
22 | private final List> futures;
23 | private final Predicate> filter;
24 |
25 | public ClickHouseSinkScheduledCheckerAndCleaner(ClickHouseSinkCommonParams props, List> futures) {
26 | this.futures = futures;
27 | this.filter = getFuturesFilter(props.isIgnoringClickHouseSendingExceptionEnabled());
28 | ThreadFactory factory = ThreadUtil.threadFactory("clickhouse-writer-checker-and-cleaner");
29 | scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(factory);
30 | scheduledExecutorService.scheduleWithFixedDelay(getTask(), props.getTimeout(), props.getTimeout(), TimeUnit.SECONDS);
31 | logger.info("Build Sink scheduled checker, timeout (sec) = {}", props.getTimeout());
32 | }
33 |
34 | public void addSinkBuffer(ClickHouseSinkBuffer clickHouseSinkBuffer) {
35 | synchronized (this) {
36 | clickHouseSinkBuffers.add(clickHouseSinkBuffer);
37 | }
38 | logger.debug("Add sinkBuffer, target table = {}", clickHouseSinkBuffer.getTargetTable());
39 | }
40 |
41 | private Runnable getTask() {
42 | return () -> {
43 | synchronized (this) {
44 | logger.debug("Start checking buffers and cleanup futures: Before cleanup = {}.", futures.size());
45 | futures.removeIf(filter);
46 | clickHouseSinkBuffers.forEach(ClickHouseSinkBuffer::tryAddToQueue);
47 | }
48 | };
49 | }
50 |
51 | private static Predicate> getFuturesFilter(boolean ignoringExceptionEnabled) {
52 | if (ignoringExceptionEnabled) {
53 | return CompletableFuture::isDone;
54 | } else {
55 | return f -> f.isDone() && !f.isCompletedExceptionally();
56 | }
57 | }
58 |
59 | @Override
60 | public void close() throws Exception {
61 | logger.info("ClickHouseSinkScheduledCheckerAndCleaner is shutting down.");
62 | ThreadUtil.shutdownExecutorService(scheduledExecutorService);
63 | logger.info("ClickHouseSinkScheduledCheckerAndCleaner shutdown complete.");
64 | }
65 | }
66 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/applied/ClickHouseWriter.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import com.google.common.collect.Lists;
4 | import io.netty.handler.codec.http.HttpHeaderNames;
5 | import org.asynchttpclient.AsyncHttpClient;
6 | import org.asynchttpclient.BoundRequestBuilder;
7 | import org.asynchttpclient.Dsl;
8 | import org.asynchttpclient.ListenableFuture;
9 | import org.asynchttpclient.Request;
10 | import org.asynchttpclient.Response;
11 | import org.slf4j.Logger;
12 | import org.slf4j.LoggerFactory;
13 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseRequestBlank;
14 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkCommonParams;
15 | import ru.ivi.opensource.flinkclickhousesink.util.FutureUtil;
16 | import ru.ivi.opensource.flinkclickhousesink.util.ThreadUtil;
17 |
18 | import java.io.IOException;
19 | import java.io.PrintWriter;
20 | import java.nio.file.Files;
21 | import java.nio.file.Path;
22 | import java.nio.file.Paths;
23 | import java.util.List;
24 | import java.util.concurrent.BlockingQueue;
25 | import java.util.concurrent.CompletableFuture;
26 | import java.util.concurrent.ExecutorService;
27 | import java.util.concurrent.Executors;
28 | import java.util.concurrent.LinkedBlockingQueue;
29 | import java.util.concurrent.ThreadFactory;
30 | import java.util.concurrent.TimeUnit;
31 | import java.util.concurrent.atomic.AtomicLong;
32 |
33 | public class ClickHouseWriter implements AutoCloseable {
34 | private static final Logger logger = LoggerFactory.getLogger(ClickHouseWriter.class);
35 |
36 | private final BlockingQueue commonQueue;
37 | private final AtomicLong unprocessedRequestsCounter = new AtomicLong();
38 | private final AsyncHttpClient asyncHttpClient;
39 | private final List> futures;
40 | private final ClickHouseSinkCommonParams sinkParams;
41 |
42 | private ExecutorService service;
43 | private ExecutorService callbackService;
44 | private List tasks;
45 |
46 | public ClickHouseWriter(ClickHouseSinkCommonParams sinkParams, List> futures) {
47 | this(sinkParams, futures, Dsl.asyncHttpClient());
48 | }
49 |
50 | public ClickHouseWriter(ClickHouseSinkCommonParams sinkParams, List> futures, AsyncHttpClient asyncHttpClient) {
51 | this.sinkParams = sinkParams;
52 | this.futures = futures;
53 | this.commonQueue = new LinkedBlockingQueue<>(sinkParams.getQueueMaxCapacity());
54 | this.asyncHttpClient = asyncHttpClient;
55 | initDirAndExecutors();
56 | }
57 |
58 | private void initDirAndExecutors() {
59 | try {
60 | initDir(sinkParams.getFailedRecordsPath());
61 | buildComponents();
62 | } catch (Exception e) {
63 | logger.error("Error while starting CH writer", e);
64 | throw new RuntimeException(e);
65 | }
66 | }
67 |
68 | private static void initDir(String pathName) throws IOException {
69 | Path path = Paths.get(pathName);
70 | Files.createDirectories(path);
71 | }
72 |
73 | private void buildComponents() {
74 | logger.info("Building components");
75 |
76 | ThreadFactory threadFactory = ThreadUtil.threadFactory("clickhouse-writer");
77 | service = Executors.newFixedThreadPool(sinkParams.getNumWriters(), threadFactory);
78 |
79 | ThreadFactory callbackServiceFactory = ThreadUtil.threadFactory("clickhouse-writer-callback-executor");
80 | callbackService = Executors.newCachedThreadPool(callbackServiceFactory);
81 |
82 | int numWriters = sinkParams.getNumWriters();
83 | tasks = Lists.newArrayListWithCapacity(numWriters);
84 | for (int i = 0; i < numWriters; i++) {
85 | WriterTask task = new WriterTask(i, asyncHttpClient, commonQueue, sinkParams, callbackService, futures, unprocessedRequestsCounter);
86 | tasks.add(task);
87 | service.submit(task);
88 | }
89 | }
90 |
91 | public void put(ClickHouseRequestBlank params) {
92 | try {
93 | unprocessedRequestsCounter.incrementAndGet();
94 | commonQueue.put(params);
95 | } catch (InterruptedException e) {
96 | logger.error("Interrupted error while putting data to queue", e);
97 | Thread.currentThread().interrupt();
98 | throw new RuntimeException(e);
99 | }
100 | }
101 |
102 | private void waitUntilAllFuturesDone() {
103 | logger.info("Wait until all futures are done or completed exceptionally. Futures size: {}", futures.size());
104 | try {
105 | while (unprocessedRequestsCounter.get() > 0 || !futures.isEmpty()) {
106 | if (logger.isDebugEnabled()) {
107 | logger.debug("Futures size: {}.", futures.size());
108 | }
109 | CompletableFuture future = FutureUtil.allOf(futures);
110 | try {
111 | future.get();
112 | futures.removeIf(f -> f.isDone() && !f.isCompletedExceptionally());
113 | if (logger.isDebugEnabled()) {
114 | logger.debug("Futures size after removing: {}", futures.size());
115 | }
116 | } catch (Exception e) {
117 | throw new RuntimeException(e);
118 | }
119 | }
120 | } finally {
121 | stopWriters();
122 | futures.clear();
123 | }
124 | }
125 |
126 | private void stopWriters() {
127 | logger.info("Stopping writers.");
128 | if (tasks != null && tasks.size() > 0) {
129 | tasks.forEach(WriterTask::setStopWorking);
130 | }
131 | logger.info("Writers stopped.");
132 | }
133 |
134 | @Override
135 | public void close() throws Exception {
136 | logger.info("ClickHouseWriter is shutting down.");
137 | try {
138 | waitUntilAllFuturesDone();
139 | } finally {
140 | ThreadUtil.shutdownExecutorService(service);
141 | ThreadUtil.shutdownExecutorService(callbackService);
142 | asyncHttpClient.close();
143 | logger.info("{} shutdown complete.", ClickHouseWriter.class.getSimpleName());
144 | }
145 | }
146 |
147 | static class WriterTask implements Runnable {
148 | private static final Logger logger = LoggerFactory.getLogger(WriterTask.class);
149 |
150 | private static final int HTTP_OK = 200;
151 |
152 | private final BlockingQueue queue;
153 | private final AtomicLong queueCounter;
154 | private final ClickHouseSinkCommonParams sinkSettings;
155 | private final AsyncHttpClient asyncHttpClient;
156 | private final ExecutorService callbackService;
157 | private final List> futures;
158 |
159 | private final int id;
160 |
161 | private volatile boolean isWorking;
162 |
163 | WriterTask(int id,
164 | AsyncHttpClient asyncHttpClient,
165 | BlockingQueue queue,
166 | ClickHouseSinkCommonParams settings,
167 | ExecutorService callbackService,
168 | List> futures,
169 | AtomicLong queueCounter) {
170 | this.id = id;
171 | this.sinkSettings = settings;
172 | this.queue = queue;
173 | this.callbackService = callbackService;
174 | this.asyncHttpClient = asyncHttpClient;
175 | this.futures = futures;
176 | this.queueCounter = queueCounter;
177 | }
178 |
179 | @Override
180 | public void run() {
181 | try {
182 | isWorking = true;
183 |
184 | logger.info("Start writer task, id = {}", id);
185 | while (isWorking || queue.size() > 0) {
186 | ClickHouseRequestBlank blank = queue.poll(300, TimeUnit.MILLISECONDS);
187 | if (blank != null) {
188 | CompletableFuture future = new CompletableFuture<>();
189 | futures.add(future);
190 | send(blank, future);
191 | }
192 | }
193 | } catch (Exception e) {
194 | logger.error("Error while inserting data", e);
195 | throw new RuntimeException(e);
196 | } finally {
197 | logger.info("Task id = {} is finished", id);
198 | }
199 | }
200 |
201 | private void send(ClickHouseRequestBlank requestBlank, CompletableFuture future) {
202 | Request request = buildRequest(requestBlank);
203 | logger.info("Ready to load data to {}, size = {}", requestBlank.getTargetTable(), requestBlank.getValues().size());
204 | ListenableFuture whenResponse = asyncHttpClient.executeRequest(request);
205 | Runnable callback = responseCallback(whenResponse, requestBlank, future);
206 | whenResponse.addListener(callback, callbackService);
207 | }
208 |
209 | private Request buildRequest(ClickHouseRequestBlank requestBlank) {
210 | String resultCSV = String.join(" , ", requestBlank.getValues());
211 | String query = String.format("INSERT INTO %s VALUES %s", requestBlank.getTargetTable(), resultCSV);
212 | String host = sinkSettings.getClickHouseClusterSettings().getRandomHostUrl();
213 |
214 | BoundRequestBuilder builder = asyncHttpClient
215 | .preparePost(host)
216 | .setHeader(HttpHeaderNames.CONTENT_TYPE, "text/plain; charset=utf-8")
217 | .setBody(query);
218 |
219 | if (sinkSettings.getClickHouseClusterSettings().isAuthorizationRequired()) {
220 | builder.setHeader(HttpHeaderNames.AUTHORIZATION, "Basic " + sinkSettings.getClickHouseClusterSettings().getCredentials());
221 | }
222 |
223 | return builder.build();
224 | }
225 |
226 | private Runnable responseCallback(ListenableFuture whenResponse, ClickHouseRequestBlank requestBlank, CompletableFuture future) {
227 | return () -> {
228 | Response response = null;
229 | try {
230 | response = whenResponse.get();
231 |
232 | if (response.getStatusCode() != HTTP_OK) {
233 | handleUnsuccessfulResponse(response, requestBlank, future);
234 | } else {
235 | logger.info("Successful send data to ClickHouse, batch size = {}, target table = {}, current attempt = {}",
236 | requestBlank.getValues().size(),
237 | requestBlank.getTargetTable(),
238 | requestBlank.getAttemptCounter());
239 | future.complete(true);
240 | }
241 | } catch (Exception e) {
242 | logger.error("Error while executing callback, params = {}", sinkSettings, e);
243 | requestBlank.setException(e);
244 | try {
245 | handleUnsuccessfulResponse(response, requestBlank, future);
246 | } catch (Exception error) {
247 | logger.error("Error while handle unsuccessful response", error);
248 | future.completeExceptionally(error);
249 | }
250 | } finally {
251 | queueCounter.decrementAndGet();
252 | }
253 | };
254 | }
255 |
256 | private void handleUnsuccessfulResponse(Response response, ClickHouseRequestBlank requestBlank, CompletableFuture future) throws Exception {
257 | int currentCounter = requestBlank.getAttemptCounter();
258 | if (currentCounter >= sinkSettings.getMaxRetries()) {
259 | logger.warn("Failed to send data to ClickHouse, cause: limit of attempts is exceeded." +
260 | " ClickHouse response = {}. Ready to flush data on disk.", response, requestBlank.getException());
261 | logFailedRecords(requestBlank);
262 | future.completeExceptionally(new RuntimeException(String.format("Failed to send data to ClickHouse, cause: limit of attempts is exceeded." +
263 | " ClickHouse response: %s. Cause: %s", response != null ? response.getResponseBody() : null, requestBlank.getException())));
264 | } else {
265 | requestBlank.incrementCounter();
266 | logger.warn("Next attempt to send data to ClickHouse, table = {}, buffer size = {}, current attempt num = {}, max attempt num = {}, response = {}",
267 | requestBlank.getTargetTable(),
268 | requestBlank.getValues().size(),
269 | requestBlank.getAttemptCounter(),
270 | sinkSettings.getMaxRetries(),
271 | response);
272 | queueCounter.incrementAndGet();
273 | queue.put(requestBlank);
274 | future.complete(false);
275 | }
276 | }
277 |
278 | private void logFailedRecords(ClickHouseRequestBlank requestBlank) throws Exception {
279 | String filePath = String.format("%s/%s_%s",
280 | sinkSettings.getFailedRecordsPath(),
281 | requestBlank.getTargetTable(),
282 | System.currentTimeMillis());
283 |
284 | try (PrintWriter writer = new PrintWriter(filePath)) {
285 | List records = requestBlank.getValues();
286 | records.forEach(writer::println);
287 | writer.flush();
288 | }
289 | logger.info("Successful send data on disk, path = {}, size = {} ", filePath, requestBlank.getValues().size());
290 | }
291 |
292 | void setStopWorking() {
293 | isWorking = false;
294 | }
295 | }
296 | }
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/applied/ExceptionsThrowableSink.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import java.util.concurrent.ExecutionException;
4 |
5 | public class ExceptionsThrowableSink implements Sink {
6 | private final ClickHouseSinkBuffer clickHouseSinkBuffer;
7 |
8 | public ExceptionsThrowableSink(ClickHouseSinkBuffer buffer) {
9 | this.clickHouseSinkBuffer = buffer;
10 | }
11 |
12 | @Override
13 | public void put(String message) throws ExecutionException, InterruptedException {
14 | clickHouseSinkBuffer.put(message);
15 | clickHouseSinkBuffer.assertFuturesNotFailedYet();
16 | }
17 |
18 | @Override
19 | public void close() {
20 | if (clickHouseSinkBuffer != null) {
21 | clickHouseSinkBuffer.close();
22 | }
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/applied/Sink.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import java.util.concurrent.ExecutionException;
4 |
5 | public interface Sink extends AutoCloseable {
6 | void put(String message) throws ExecutionException, InterruptedException;
7 | }
8 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/applied/UnexceptionableSink.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | public class UnexceptionableSink implements Sink {
4 | private final ClickHouseSinkBuffer clickHouseSinkBuffer;
5 |
6 | public UnexceptionableSink(ClickHouseSinkBuffer buffer) {
7 | this.clickHouseSinkBuffer = buffer;
8 | }
9 |
10 | @Override
11 | public void put(String message) {
12 | clickHouseSinkBuffer.put(message);
13 | }
14 |
15 | @Override
16 | public void close() {
17 | if (clickHouseSinkBuffer != null) {
18 | clickHouseSinkBuffer.close();
19 | }
20 | }
21 | }
22 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/model/ClickHouseClusterSettings.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.model;
2 |
3 | import com.google.common.base.Preconditions;
4 | import org.apache.commons.lang3.StringUtils;
5 | import ru.ivi.opensource.flinkclickhousesink.util.ConfigUtil;
6 |
7 | import java.util.Arrays;
8 | import java.util.Base64;
9 | import java.util.List;
10 | import java.util.Map;
11 | import java.util.concurrent.ThreadLocalRandom;
12 | import java.util.stream.Collectors;
13 |
14 | public class ClickHouseClusterSettings {
15 |
16 | public static final String CLICKHOUSE_HOSTS = "clickhouse.access.hosts";
17 | public static final String CLICKHOUSE_USER = "clickhouse.access.user";
18 | public static final String CLICKHOUSE_PASSWORD = "clickhouse.access.password";
19 |
20 | private final List hostsWithPorts;
21 | private final String user;
22 | private final String password;
23 | private final String credentials;
24 | private final boolean authorizationRequired;
25 |
26 | private int currentHostId = 0;
27 |
28 | public ClickHouseClusterSettings(Map parameters) {
29 | Preconditions.checkNotNull(parameters);
30 |
31 | String hostsString = parameters.get(CLICKHOUSE_HOSTS);
32 | Preconditions.checkNotNull(hostsString);
33 |
34 | hostsWithPorts = buildHostsAndPort(hostsString);
35 | Preconditions.checkArgument(hostsWithPorts.size() > 0);
36 |
37 | String usr = parameters.get(CLICKHOUSE_USER);
38 | String pass = parameters.get(CLICKHOUSE_PASSWORD);
39 |
40 | if (StringUtils.isNotEmpty(usr) && StringUtils.isNotEmpty(pass)) {
41 | user = parameters.get(CLICKHOUSE_USER);
42 | password = parameters.get(CLICKHOUSE_PASSWORD);
43 |
44 | credentials = buildCredentials(user, password);
45 | authorizationRequired = true;
46 | } else {
47 | // avoid NPE
48 | credentials = "";
49 | password = "";
50 | user = "";
51 | authorizationRequired = false;
52 | }
53 | }
54 |
55 | private static List buildHostsAndPort(String hostsString) {
56 | String hosts = hostsString.replace(" ", "");
57 | return Arrays.stream(hosts
58 | .split(ConfigUtil.HOST_DELIMITER))
59 | .map(ClickHouseClusterSettings::checkHttpAndAdd)
60 | .collect(Collectors.toList());
61 | }
62 |
63 | private static String checkHttpAndAdd(String host) {
64 | String newHost = host.replace(" ", "");
65 | if (!newHost.contains("http")) {
66 | return "http://" + newHost;
67 | }
68 | return newHost;
69 | }
70 |
71 | private static String buildCredentials(String user, String password) {
72 | Base64.Encoder x = Base64.getEncoder();
73 | String credentials = String.join(":", user, password);
74 | return new String(x.encode(credentials.getBytes()));
75 | }
76 |
77 | public String getRandomHostUrl() {
78 | currentHostId = ThreadLocalRandom.current().nextInt(hostsWithPorts.size());
79 | return hostsWithPorts.get(currentHostId);
80 | }
81 |
82 | public String getNextHost() {
83 | if (currentHostId >= hostsWithPorts.size() - 1) {
84 | currentHostId = 0;
85 | } else {
86 | currentHostId += 1;
87 | }
88 | return hostsWithPorts.get(currentHostId);
89 | }
90 |
91 | public List getHostsWithPorts() {
92 | return hostsWithPorts;
93 | }
94 |
95 | public String getUser() {
96 | return user;
97 | }
98 |
99 | public String getPassword() {
100 | return password;
101 | }
102 |
103 | public String getCredentials() {
104 | return credentials;
105 | }
106 |
107 | public boolean isAuthorizationRequired() {
108 | return authorizationRequired;
109 | }
110 |
111 | @Override
112 | public String toString() {
113 | return "ClickHouseClusterSettings{" +
114 | "hostsWithPorts=" + hostsWithPorts +
115 | ", credentials='" + credentials + '\'' +
116 | ", authorizationRequired=" + authorizationRequired +
117 | ", currentHostId=" + currentHostId +
118 | '}';
119 | }
120 | }
121 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/model/ClickHouseRequestBlank.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.model;
2 |
3 | import java.util.List;
4 |
5 | public class ClickHouseRequestBlank {
6 | private final List values;
7 | private final String targetTable;
8 | private int attemptCounter;
9 |
10 | private Exception exception;
11 |
12 | public ClickHouseRequestBlank(List values, String targetTable, Exception exception) {
13 | this.values = values;
14 | this.targetTable = targetTable;
15 | this.attemptCounter = 0;
16 | this.exception = exception;
17 | }
18 |
19 | public List getValues() {
20 | return values;
21 | }
22 |
23 | public void incrementCounter() {
24 | this.attemptCounter++;
25 | }
26 |
27 | public int getAttemptCounter() {
28 | return attemptCounter;
29 | }
30 |
31 | public String getTargetTable() {
32 | return targetTable;
33 | }
34 |
35 | public Exception getException() {
36 | return exception;
37 | }
38 |
39 | public void setException(Exception exception) {
40 | this.exception = exception;
41 | }
42 |
43 | public static final class Builder {
44 | private List values;
45 | private String targetTable;
46 | private Exception exception;
47 |
48 | private Builder() {
49 | }
50 |
51 | public static Builder aBuilder() {
52 | return new Builder();
53 | }
54 |
55 | public Builder withValues(List values) {
56 | this.values = values;
57 | return this;
58 | }
59 |
60 | public Builder withTargetTable(String targetTable) {
61 | this.targetTable = targetTable;
62 | return this;
63 | }
64 |
65 | public Builder withException(Exception exception) {
66 | this.exception = exception;
67 | return this;
68 | }
69 |
70 | public ClickHouseRequestBlank build() {
71 | return new ClickHouseRequestBlank(values, targetTable, exception);
72 | }
73 | }
74 |
75 | @Override
76 | public String toString() {
77 | return "ClickHouseRequestBlank{" +
78 | "values=" + values +
79 | ", targetTable='" + targetTable + '\'' +
80 | ", attemptCounter=" + attemptCounter +
81 | ", exception=" + exception +
82 | '}';
83 | }
84 | }
85 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/model/ClickHouseSinkCommonParams.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.model;
2 |
3 | import com.google.common.base.Preconditions;
4 |
5 | import java.util.Map;
6 |
7 | import static ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst.FAILED_RECORDS_PATH;
8 | import static ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst.IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED;
9 | import static ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst.NUM_RETRIES;
10 | import static ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst.NUM_WRITERS;
11 | import static ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst.QUEUE_MAX_CAPACITY;
12 | import static ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst.TIMEOUT_SEC;
13 |
14 | public class ClickHouseSinkCommonParams {
15 |
16 | private final ClickHouseClusterSettings clickHouseClusterSettings;
17 |
18 | private final String failedRecordsPath;
19 | private final int numWriters;
20 | private final int queueMaxCapacity;
21 | private final boolean ignoringClickHouseSendingExceptionEnabled;
22 |
23 | private final int timeout;
24 | private final int maxRetries;
25 |
26 | public ClickHouseSinkCommonParams(Map params) {
27 | Preconditions.checkNotNull(params.get(IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED),
28 | "Parameter " + IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED + " must be initialized");
29 |
30 | this.clickHouseClusterSettings = new ClickHouseClusterSettings(params);
31 | this.numWriters = Integer.parseInt(params.get(NUM_WRITERS));
32 | this.queueMaxCapacity = Integer.parseInt(params.get(QUEUE_MAX_CAPACITY));
33 | this.maxRetries = Integer.parseInt(params.get(NUM_RETRIES));
34 | this.timeout = Integer.parseInt(params.get(TIMEOUT_SEC));
35 | this.failedRecordsPath = params.get(FAILED_RECORDS_PATH);
36 | this.ignoringClickHouseSendingExceptionEnabled = Boolean.parseBoolean(params.get(IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED));
37 |
38 | Preconditions.checkNotNull(failedRecordsPath);
39 | Preconditions.checkArgument(queueMaxCapacity > 0);
40 | Preconditions.checkArgument(numWriters > 0);
41 | Preconditions.checkArgument(timeout > 0);
42 | Preconditions.checkArgument(maxRetries > 0);
43 | }
44 |
45 | public int getNumWriters() {
46 | return numWriters;
47 | }
48 |
49 | public int getQueueMaxCapacity() {
50 | return queueMaxCapacity;
51 | }
52 |
53 | public boolean isIgnoringClickHouseSendingExceptionEnabled() {
54 | return ignoringClickHouseSendingExceptionEnabled;
55 | }
56 |
57 | public ClickHouseClusterSettings getClickHouseClusterSettings() {
58 | return clickHouseClusterSettings;
59 | }
60 |
61 | public int getTimeout() {
62 | return timeout;
63 | }
64 |
65 | public int getMaxRetries() {
66 | return maxRetries;
67 | }
68 |
69 | public String getFailedRecordsPath() {
70 | return failedRecordsPath;
71 | }
72 |
73 | @Override
74 | public String toString() {
75 | return "ClickHouseSinkCommonParams{" +
76 | "clickHouseClusterSettings=" + clickHouseClusterSettings +
77 | ", failedRecordsPath='" + failedRecordsPath + '\'' +
78 | ", numWriters=" + numWriters +
79 | ", queueMaxCapacity=" + queueMaxCapacity +
80 | ", ignoringClickHouseSendingExceptionEnabled=" + ignoringClickHouseSendingExceptionEnabled +
81 | ", timeout=" + timeout +
82 | ", maxRetries=" + maxRetries +
83 | '}';
84 | }
85 | }
86 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/model/ClickHouseSinkConst.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.model;
2 |
3 | public final class ClickHouseSinkConst {
4 | private ClickHouseSinkConst() {
5 | }
6 |
7 | public static final String TARGET_TABLE_NAME = "clickhouse.sink.target-table";
8 | public static final String MAX_BUFFER_SIZE = "clickhouse.sink.max-buffer-size";
9 |
10 | public static final String NUM_WRITERS = "clickhouse.sink.num-writers";
11 | public static final String QUEUE_MAX_CAPACITY = "clickhouse.sink.queue-max-capacity";
12 | public static final String TIMEOUT_SEC = "clickhouse.sink.timeout-sec";
13 | public static final String NUM_RETRIES = "clickhouse.sink.retries";
14 | public static final String FAILED_RECORDS_PATH = "clickhouse.sink.failed-records-path";
15 | public static final String IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED = "clickhouse.sink.ignoring-clickhouse-sending-exception-enabled";
16 | }
17 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/util/ConfigUtil.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.util;
2 |
3 | import com.typesafe.config.Config;
4 | import com.typesafe.config.ConfigValue;
5 |
6 | import java.util.Arrays;
7 | import java.util.HashMap;
8 | import java.util.List;
9 | import java.util.Map;
10 | import java.util.Properties;
11 |
12 | public final class ConfigUtil {
13 |
14 | public static final String HOST_DELIMITER = ",";
15 |
16 | private ConfigUtil() {
17 |
18 | }
19 |
20 | public static Properties toProperties(Config config) {
21 | Properties properties = new Properties();
22 | config.entrySet().forEach(e -> properties.put(e.getKey(), unwrapped(config.getValue(e.getKey()))));
23 | return properties;
24 | }
25 |
26 | public static Map toMap(Config config) {
27 | Map map = new HashMap<>();
28 | config.entrySet().forEach(e -> map.put(e.getKey(), unwrapped(e.getValue())));
29 | return map;
30 | }
31 |
32 | private static String unwrapped(ConfigValue configValue) {
33 | Object object = configValue.unwrapped();
34 | return object.toString();
35 | }
36 |
37 | public static String buildStringFromList(List list) {
38 | return String.join(HOST_DELIMITER, list);
39 | }
40 |
41 | public static List buildListFromString(String string) {
42 | return Arrays.asList(string.split(" "));
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/util/FutureUtil.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.util;
2 |
3 | import java.util.List;
4 | import java.util.concurrent.CompletableFuture;
5 |
6 | public final class FutureUtil {
7 |
8 | private FutureUtil() {
9 |
10 | }
11 |
12 | public static CompletableFuture allOf(List> futures) {
13 | return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]));
14 | }
15 | }
16 |
--------------------------------------------------------------------------------
/src/main/java/ru/ivi/opensource/flinkclickhousesink/util/ThreadUtil.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.util;
2 |
3 | import com.google.common.util.concurrent.ThreadFactoryBuilder;
4 |
5 | import java.util.concurrent.ExecutorService;
6 | import java.util.concurrent.ThreadFactory;
7 | import java.util.concurrent.TimeUnit;
8 |
9 | public final class ThreadUtil {
10 |
11 | private ThreadUtil() {
12 |
13 | }
14 |
15 | public static ThreadFactory threadFactory(String threadName, boolean isDaemon) {
16 | return new ThreadFactoryBuilder()
17 | .setNameFormat(threadName + "-%d")
18 | .setDaemon(isDaemon)
19 | .build();
20 | }
21 |
22 | public static ThreadFactory threadFactory(String threadName) {
23 | return threadFactory(threadName, true);
24 | }
25 |
26 | public static void shutdownExecutorService(ExecutorService executorService) throws InterruptedException {
27 | shutdownExecutorService(executorService, 5);
28 | }
29 |
30 | public static void shutdownExecutorService(ExecutorService executorService, int timeoutS) throws InterruptedException {
31 | if (executorService != null && !executorService.isShutdown()) {
32 | executorService.shutdown();
33 | if (!executorService.awaitTermination(timeoutS, TimeUnit.SECONDS)) {
34 | executorService.shutdownNow();
35 | executorService.awaitTermination(timeoutS, TimeUnit.SECONDS);
36 | }
37 | }
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/src/test/java/ru/ivi/opensource/flinkclickhousesink/ClickHouseWriterTest.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink;
2 |
3 | import com.google.common.collect.Lists;
4 | import com.google.common.util.concurrent.Striped;
5 | import com.typesafe.config.Config;
6 | import com.typesafe.config.ConfigFactory;
7 | import com.zaxxer.hikari.HikariConfig;
8 | import com.zaxxer.hikari.HikariDataSource;
9 | import org.apache.flink.api.common.restartstrategy.RestartStrategies;
10 | import org.apache.flink.api.java.utils.ParameterTool;
11 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
12 | import org.asynchttpclient.AsyncHttpClient;
13 | import org.asynchttpclient.Dsl;
14 | import org.asynchttpclient.ListenableFuture;
15 | import org.asynchttpclient.Request;
16 | import org.hamcrest.CoreMatchers;
17 | import org.jmock.lib.concurrent.Blitzer;
18 | import org.junit.After;
19 | import org.junit.Before;
20 | import org.junit.Rule;
21 | import org.junit.Test;
22 | import org.mockito.Mockito;
23 | import org.testcontainers.containers.ClickHouseContainer;
24 | import org.testcontainers.shaded.com.google.common.collect.ImmutableList;
25 | import ru.ivi.opensource.flinkclickhousesink.applied.ClickHouseSinkManager;
26 | import ru.ivi.opensource.flinkclickhousesink.applied.ClickHouseWriter;
27 | import ru.ivi.opensource.flinkclickhousesink.applied.Sink;
28 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseClusterSettings;
29 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseRequestBlank;
30 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkCommonParams;
31 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst;
32 | import ru.ivi.opensource.flinkclickhousesink.util.ConfigUtil;
33 |
34 | import java.sql.Connection;
35 | import java.sql.ResultSet;
36 | import java.sql.Statement;
37 | import java.util.ArrayList;
38 | import java.util.Collections;
39 | import java.util.LinkedList;
40 | import java.util.List;
41 | import java.util.Map;
42 | import java.util.Properties;
43 | import java.util.concurrent.ExecutionException;
44 | import java.util.concurrent.ThreadLocalRandom;
45 | import java.util.concurrent.locks.Lock;
46 |
47 | import static java.util.concurrent.TimeUnit.MILLISECONDS;
48 | import static org.awaitility.Awaitility.await;
49 | import static org.hamcrest.MatcherAssert.assertThat;
50 | import static org.junit.Assert.fail;
51 |
52 | public class ClickHouseWriterTest {
53 |
54 | private static final int HTTP_CLICKHOUSE_PORT = 8123;
55 |
56 | private static final String JDBC_DRIVER = "ru.yandex.clickhouse.ClickHouseDriver";
57 |
58 | private HikariDataSource hikariDataSource;
59 |
60 | @Rule
61 | public ClickHouseContainer clickHouse = new ClickHouseContainer();
62 |
63 | private ClickHouseSinkManager sinkManager;
64 |
65 | private Map params;
66 | private ClickHouseSinkCommonParams clickHouseSinkCommonParams;
67 |
68 | @Before
69 | public void setUp() throws Exception {
70 | Config config = ConfigFactory.load();
71 | params = ConfigUtil.toMap(config);
72 |
73 | params.put(ClickHouseClusterSettings.CLICKHOUSE_USER, "");
74 | params.put(ClickHouseClusterSettings.CLICKHOUSE_PASSWORD, "");
75 | int dockerActualPort = clickHouse.getMappedPort(HTTP_CLICKHOUSE_PORT);
76 |
77 | // container with CH is raised for every test -> we should config host and port every time
78 | params.put(ClickHouseClusterSettings.CLICKHOUSE_HOSTS, "http://" + clickHouse.getContainerIpAddress() + ":" + dockerActualPort);
79 | params.put(ClickHouseSinkConst.IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED, "true");
80 | clickHouseSinkCommonParams = new ClickHouseSinkCommonParams(params);
81 |
82 | try {
83 | Class.forName(JDBC_DRIVER);
84 | } catch (ClassNotFoundException e) {
85 | throw new RuntimeException(e);
86 | }
87 |
88 | HikariConfig hikariConfig = new HikariConfig();
89 | hikariConfig.setJdbcUrl(clickHouse.getJdbcUrl());
90 | hikariConfig.setUsername(clickHouse.getUsername());
91 | hikariConfig.setPassword(clickHouse.getPassword());
92 | hikariDataSource = new HikariDataSource(hikariConfig);
93 |
94 | try (Connection connection = hikariDataSource.getConnection();
95 | Statement statement = connection.createStatement()) {
96 | statement.executeQuery("CREATE DATABASE IF NOT EXISTS test;");
97 |
98 | statement.executeQuery("DROP TABLE IF EXISTS test.test0;");
99 | statement.executeQuery("CREATE TABLE test.test0 (" +
100 | "id UInt64," +
101 | "title String," +
102 | "container String," +
103 | "drm String," +
104 | "quality String)" +
105 | "ENGINE = Log;");
106 |
107 | statement.executeQuery("DROP TABLE IF EXISTS test.test1;");
108 | statement.executeQuery("CREATE TABLE test.test1 (" +
109 | "id UInt64," +
110 | "title String," +
111 | "num UInt64)" +
112 | "ENGINE = Log;");
113 | }
114 |
115 | sinkManager = new ClickHouseSinkManager(params);
116 | }
117 |
118 | @After
119 | public void down() throws Exception {
120 | sinkManager.close();
121 | }
122 |
123 | private int getCount(String tableName) throws Exception {
124 | int res = 0;
125 | try (Connection connection = hikariDataSource.getConnection();
126 | Statement statement = connection.createStatement()) {
127 | ResultSet resultSet = statement.executeQuery("select count (*) as total from " + tableName);
128 | while (resultSet.next()) {
129 | int count = resultSet.getInt("total");
130 | System.out.println("Count " + tableName + " = " + count);
131 | res = count;
132 | }
133 | }
134 | return res;
135 | }
136 |
137 | @Test
138 | public void highConcurrentTest() throws Exception {
139 |
140 | final int numBuffers = 4;
141 | Striped striped = Striped.lock(numBuffers);
142 |
143 | List buffers = new ArrayList<>();
144 | ThreadLocalRandom random = ThreadLocalRandom.current();
145 | for (int i = 0; i < 4; i++) {
146 | String targetTable;
147 | if (i % 2 != 0) {
148 | targetTable = "test.test0";
149 | } else targetTable = "test.test1";
150 |
151 | int maxBuffer = random.nextInt(1_000, 100_000);
152 | Sink sink = sinkManager.buildSink(targetTable, maxBuffer);
153 | buffers.add(sink);
154 | }
155 |
156 | final int attempts = 2_000_000;
157 | Blitzer blitzer = new Blitzer(attempts, numBuffers);
158 | blitzer.blitz(() -> {
159 | int id = ThreadLocalRandom.current().nextInt(0, numBuffers);
160 | Lock lock = striped.get(id);
161 | lock.lock();
162 | try {
163 | Sink sink = buffers.get(id);
164 | String csv;
165 | if (id % 2 != 0) {
166 | csv = "(10, 'title', 'container', 'drm', 'quality')";
167 | } else {
168 | csv = "(11, 'title', 111)";
169 | }
170 | sink.put(csv);
171 | } catch (InterruptedException | ExecutionException e) {
172 | e.printStackTrace();
173 | } finally {
174 | lock.unlock();
175 | }
176 | });
177 |
178 | await()
179 | .atMost(10000, MILLISECONDS)
180 | .with()
181 | .pollInterval(500, MILLISECONDS)
182 | .until(() -> getCount("test.test0") + getCount("test.test1") == attempts);
183 | }
184 |
185 | @Test
186 | public void testInvalidRequestException() throws Exception {
187 | ClickHouseWriter clickHouseWriter = new ClickHouseWriter(clickHouseSinkCommonParams,
188 | Collections.synchronizedList(new LinkedList<>()));
189 | clickHouseWriter.put(ClickHouseRequestBlank.Builder
190 | .aBuilder()
191 | .withValues(ImmutableList.of("('10')"))
192 | .withTargetTable("test.test0")
193 | .build());
194 |
195 | try {
196 | clickHouseWriter.close();
197 | fail("Expected RuntimeException.");
198 | } catch (RuntimeException expected) {
199 | }
200 | }
201 |
202 | @Test
203 | public void testWaitLastRequestSuccess() throws Exception {
204 | AsyncHttpClient asyncHttpClient = Mockito.spy(Dsl.asyncHttpClient());
205 |
206 | ClickHouseWriter clickHouseWriter = new ClickHouseWriter(clickHouseSinkCommonParams,
207 | Collections.synchronizedList(Lists.newLinkedList()), asyncHttpClient);
208 | clickHouseWriter.put(ClickHouseRequestBlank.Builder
209 | .aBuilder()
210 | .withValues(ImmutableList.of("(10, 'title', 'container', 'drm', 'quality')"))
211 | .withTargetTable("test.test0")
212 | .build());
213 |
214 | clickHouseWriter.close();
215 |
216 | await()
217 | .atMost(10000, MILLISECONDS)
218 | .with()
219 | .pollInterval(500, MILLISECONDS)
220 | .until(() -> getCount("test.test0") == 1);
221 |
222 | Mockito.verify(asyncHttpClient, Mockito.times(1))
223 | .executeRequest(Mockito.any(Request.class));
224 | }
225 |
226 | @Test
227 | public void testMaxRetries() throws Exception {
228 | int maxRetries = clickHouseSinkCommonParams.getMaxRetries();
229 | AsyncHttpClient asyncHttpClient = Mockito.spy(Dsl.asyncHttpClient());
230 |
231 | ClickHouseWriter clickHouseWriter = new ClickHouseWriter(clickHouseSinkCommonParams,
232 | Collections.synchronizedList(Lists.newLinkedList()),
233 | asyncHttpClient);
234 | clickHouseWriter.put(ClickHouseRequestBlank.Builder
235 | .aBuilder()
236 | .withValues(ImmutableList.of("('10')"))
237 | .withTargetTable("test.test0")
238 | .build());
239 |
240 | try {
241 | clickHouseWriter.close();
242 | fail("Expected RuntimeException.");
243 | } catch (RuntimeException expected) {
244 | }
245 |
246 | Mockito.verify(asyncHttpClient, Mockito.times(maxRetries + 1))
247 | .executeRequest(Mockito.any(Request.class));
248 | }
249 |
250 | @Test
251 | public void testExceptionInCallback() throws Exception {
252 | int maxRetries = clickHouseSinkCommonParams.getMaxRetries();
253 | AsyncHttpClient asyncHttpClient = Mockito.spy(Dsl.asyncHttpClient());
254 |
255 | Mockito.when(asyncHttpClient.executeRequest(Mockito.any(Request.class)))
256 | .thenReturn(new ListenableFuture.CompletedFailure<>(new NullPointerException("NPE")));
257 |
258 | ClickHouseWriter clickHouseWriter = new ClickHouseWriter(clickHouseSinkCommonParams,
259 | Collections.synchronizedList(Lists.newLinkedList()),
260 | asyncHttpClient);
261 | clickHouseWriter.put(ClickHouseRequestBlank.Builder.aBuilder()
262 | .withValues(ImmutableList.of("(10, 'title', 'container', 'drm', 'quality')"))
263 | .withTargetTable("test.test0")
264 | .build());
265 |
266 | try {
267 | clickHouseWriter.close();
268 | fail("Expected RuntimeException.");
269 | } catch (RuntimeException expected) {
270 | assertThat(expected.getMessage(), CoreMatchers.containsString("NPE"));
271 | }
272 |
273 | Mockito.verify(asyncHttpClient, Mockito.times(maxRetries + 1)).executeRequest(Mockito.any(Request.class));
274 | }
275 |
276 | @Test
277 | public void flinkPipelineTest() throws Exception {
278 | StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
279 | env.setParallelism(2);
280 | ParameterTool parameters = ParameterTool.fromMap(params);
281 | env.getConfig().setGlobalJobParameters(parameters);
282 | env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
283 | env.getConfig().disableSysoutLogging();
284 |
285 | Properties props = new Properties();
286 | props.put(ClickHouseSinkConst.TARGET_TABLE_NAME, "test.test1");
287 | props.put(ClickHouseSinkConst.MAX_BUFFER_SIZE, "100");
288 |
289 | List l = new ArrayList<>();
290 | l.add(new Record(10, "super-movie-0", 100));
291 | l.add(new Record(20, "super-movie-1", 200));
292 | l.add(new Record(30, "super-movie-2", 300));
293 | int size = l.size();
294 |
295 | ClickHouseSinkConverter clickHouseSinkConverter = record -> "(" +
296 | record.id +
297 | ", " +
298 | "'" +
299 | record.title +
300 | "', " +
301 | record.num +
302 | ")";
303 |
304 | env.fromCollection(l)
305 | .addSink(new ClickHouseSink<>(props, clickHouseSinkConverter));
306 |
307 | env.execute("Flink test");
308 |
309 | await()
310 | .atMost(10000, MILLISECONDS)
311 | .with()
312 | .pollInterval(500, MILLISECONDS)
313 | .until(() -> getCount("test.test1") == size);
314 | }
315 |
316 | // for test.test1 table
317 | private static class Record {
318 | final long id;
319 | final String title;
320 | final long num;
321 |
322 | private Record(long id, String title, long num) {
323 | this.id = id;
324 | this.title = title;
325 | this.num = num;
326 | }
327 | }
328 | }
--------------------------------------------------------------------------------
/src/test/java/ru/ivi/opensource/flinkclickhousesink/applied/ClickHouseSinkBufferTest.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import com.typesafe.config.Config;
4 | import com.typesafe.config.ConfigFactory;
5 | import org.junit.After;
6 | import org.junit.Before;
7 | import org.junit.Test;
8 | import org.junit.runner.RunWith;
9 | import org.mockito.Mockito;
10 | import org.mockito.MockitoAnnotations;
11 | import org.mockito.junit.MockitoJUnitRunner;
12 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseClusterSettings;
13 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseRequestBlank;
14 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkCommonParams;
15 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst;
16 | import ru.ivi.opensource.flinkclickhousesink.util.ConfigUtil;
17 |
18 | import java.util.Collections;
19 | import java.util.LinkedList;
20 | import java.util.List;
21 | import java.util.Map;
22 | import java.util.concurrent.CompletableFuture;
23 | import java.util.concurrent.atomic.AtomicBoolean;
24 |
25 | import static java.util.concurrent.TimeUnit.MILLISECONDS;
26 | import static java.util.concurrent.TimeUnit.SECONDS;
27 | import static org.awaitility.Awaitility.await;
28 | import static org.junit.Assert.assertEquals;
29 | import static org.junit.Assert.assertTrue;
30 |
31 | @RunWith(MockitoJUnitRunner.Silent.class)
32 | public class ClickHouseSinkBufferTest {
33 |
34 | private static final int BUFFER_SIZE_10000 = 10000;
35 | private static final int BUFFER_SIZE_10 = 10;
36 | private static final int TIMEOUT_SEC = 10;
37 |
38 | private ClickHouseSinkBuffer bufferTimeTrigger;
39 | private ClickHouseSinkBuffer bufferSizeTrigger;
40 | private ClickHouseWriter writer;
41 | private final List> futures = Collections.synchronizedList(new LinkedList<>());
42 |
43 | @Before
44 | public void setUp() {
45 | writer = Mockito.mock(ClickHouseWriter.class);
46 | bufferTimeTrigger = ClickHouseSinkBuffer.Builder
47 | .aClickHouseSinkBuffer()
48 | .withTargetTable("table")
49 | .withMaxFlushBufferSize(BUFFER_SIZE_10000)
50 | .withTimeoutSec(TIMEOUT_SEC)
51 | .withFutures(futures)
52 | .build(writer);
53 |
54 | bufferSizeTrigger = ClickHouseSinkBuffer.Builder
55 | .aClickHouseSinkBuffer()
56 | .withTargetTable("table")
57 | .withMaxFlushBufferSize(BUFFER_SIZE_10)
58 | .withTimeoutSec(TIMEOUT_SEC)
59 | .withFutures(futures)
60 | .build(writer);
61 |
62 | MockitoAnnotations.initMocks(this);
63 | }
64 |
65 | @After
66 | public void tearDown() {
67 | }
68 |
69 | @Test
70 | public void simplePut() {
71 | Mockito.doAnswer(invocationOnMock -> {
72 | ClickHouseRequestBlank blank = invocationOnMock.getArgument(0);
73 | System.out.println(blank);
74 | assertEquals(BUFFER_SIZE_10000, blank.getValues().size());
75 | assertEquals("table", blank.getTargetTable());
76 | return invocationOnMock;
77 | }).when(writer).put(Mockito.any());
78 |
79 | for (int i = 0; i < 100; i++) {
80 | bufferTimeTrigger.put("csv");
81 | }
82 | }
83 |
84 | private ClickHouseSinkScheduledCheckerAndCleaner initChecker() {
85 | Config config = ConfigFactory.load();
86 | Map params = ConfigUtil.toMap(config);
87 | params.put(ClickHouseClusterSettings.CLICKHOUSE_USER, "");
88 | params.put(ClickHouseClusterSettings.CLICKHOUSE_PASSWORD, "");
89 | params.put(ClickHouseClusterSettings.CLICKHOUSE_HOSTS, "http://localhost:8123");
90 | params.put(ClickHouseSinkConst.TIMEOUT_SEC, String.valueOf(TIMEOUT_SEC));
91 | params.put(ClickHouseSinkConst.IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED, "true");
92 |
93 | ClickHouseSinkCommonParams commonParams = new ClickHouseSinkCommonParams(params);
94 | return new ClickHouseSinkScheduledCheckerAndCleaner(commonParams, futures);
95 | }
96 |
97 | @Test
98 | public void testAddToQueueByTimeTrigger() {
99 | ClickHouseSinkScheduledCheckerAndCleaner checker = initChecker();
100 | checker.addSinkBuffer(bufferTimeTrigger);
101 |
102 | AtomicBoolean flag = new AtomicBoolean();
103 | Mockito.doAnswer(invocationOnMock -> {
104 | ClickHouseRequestBlank blank = invocationOnMock.getArgument(0);
105 |
106 | assertTrue(BUFFER_SIZE_10000 > blank.getValues().size());
107 | assertEquals("table", blank.getTargetTable());
108 | flag.set(true);
109 | return invocationOnMock;
110 | }).when(writer).put(Mockito.any());
111 |
112 | for (int i = 0; i < 800; i++) {
113 | bufferTimeTrigger.put("csv");
114 | }
115 |
116 | await()
117 | .atMost(15, SECONDS)
118 | .with()
119 | .pollInterval(200, MILLISECONDS)
120 | .until(flag::get);
121 | }
122 |
123 | @Test
124 | public void testAddToQueueBySizeTrigger() {
125 | ClickHouseSinkScheduledCheckerAndCleaner checker = initChecker();
126 | checker.addSinkBuffer(bufferSizeTrigger);
127 |
128 | AtomicBoolean flag = new AtomicBoolean();
129 | Mockito.doAnswer(invocationOnMock -> {
130 | ClickHouseRequestBlank blank = invocationOnMock.getArgument(0);
131 |
132 | assertEquals(BUFFER_SIZE_10, blank.getValues().size());
133 | assertEquals("table", blank.getTargetTable());
134 | flag.set(true);
135 | return invocationOnMock;
136 | }).when(writer).put(Mockito.any());
137 |
138 | for (int i = 0; i < 800; i++) {
139 | bufferSizeTrigger.put("csv");
140 | }
141 |
142 | await()
143 | .atMost(TIMEOUT_SEC, SECONDS)
144 | .with()
145 | .pollInterval(200, MILLISECONDS)
146 | .until(flag::get);
147 | }
148 | }
--------------------------------------------------------------------------------
/src/test/java/ru/ivi/opensource/flinkclickhousesink/applied/ClickHouseSinkScheduledCheckerAndCleanerTest.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import com.typesafe.config.Config;
4 | import com.typesafe.config.ConfigFactory;
5 | import org.junit.After;
6 | import org.junit.Before;
7 | import org.junit.Test;
8 | import org.junit.runner.RunWith;
9 | import org.mockito.Mockito;
10 | import org.mockito.MockitoAnnotations;
11 | import org.mockito.junit.MockitoJUnitRunner;
12 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseClusterSettings;
13 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkCommonParams;
14 | import ru.ivi.opensource.flinkclickhousesink.model.ClickHouseSinkConst;
15 | import ru.ivi.opensource.flinkclickhousesink.util.ConfigUtil;
16 |
17 | import java.util.Collections;
18 | import java.util.LinkedList;
19 | import java.util.List;
20 | import java.util.Map;
21 | import java.util.concurrent.CompletableFuture;
22 | import java.util.concurrent.atomic.AtomicInteger;
23 |
24 | import static java.util.concurrent.TimeUnit.MILLISECONDS;
25 | import static org.awaitility.Awaitility.await;
26 |
27 | @RunWith(MockitoJUnitRunner.Silent.class)
28 | public class ClickHouseSinkScheduledCheckerAndCleanerTest {
29 |
30 | private ClickHouseSinkScheduledCheckerAndCleaner checker;
31 | private final AtomicInteger counter = new AtomicInteger(0);
32 | private final List> futures = Collections.synchronizedList(new LinkedList<>());
33 |
34 | @Before
35 | public void setUp() {
36 | Config config = ConfigFactory.load();
37 | Map params = ConfigUtil.toMap(config);
38 | params.put(ClickHouseClusterSettings.CLICKHOUSE_USER, "");
39 | params.put(ClickHouseClusterSettings.CLICKHOUSE_PASSWORD, "");
40 | params.put(ClickHouseClusterSettings.CLICKHOUSE_HOSTS, "http://localhost:8123");
41 | params.put(ClickHouseSinkConst.IGNORING_CLICKHOUSE_SENDING_EXCEPTION_ENABLED, "true");
42 |
43 | ClickHouseSinkCommonParams commonParams = new ClickHouseSinkCommonParams(params);
44 | checker = new ClickHouseSinkScheduledCheckerAndCleaner(commonParams, futures);
45 |
46 | MockitoAnnotations.initMocks(this);
47 | }
48 |
49 | @After
50 | public void tearDown() throws Exception {
51 | checker.close();
52 | }
53 |
54 | @Test
55 | public void addSinkBuffer() {
56 | test(3, 3);
57 | }
58 |
59 | private void test(int numBuffers, int target) {
60 | for (int i = 0; i < numBuffers; i++) {
61 | ClickHouseSinkBuffer buffer = buildMockBuffer();
62 | checker.addSinkBuffer(buffer);
63 | }
64 |
65 | await()
66 | .atMost(2000, MILLISECONDS)
67 | .with()
68 | .pollInterval(200, MILLISECONDS)
69 | .until(() -> {
70 | System.out.println(counter.get());
71 | return counter.get() == target;
72 | });
73 | }
74 |
75 | private ClickHouseSinkBuffer buildMockBuffer() {
76 | ClickHouseSinkBuffer buffer = Mockito.mock(ClickHouseSinkBuffer.class);
77 | Mockito.doAnswer(invocationOnMock -> {
78 | counter.incrementAndGet();
79 | return invocationOnMock;
80 | }).when(buffer).tryAddToQueue();
81 | return buffer;
82 | }
83 |
84 | @Test
85 | public void checkBuffersAfterAttempt() {
86 | int first = 4;
87 | int second = 1;
88 | test(first, first);
89 | test(second, first * 2 + second);
90 | }
91 | }
--------------------------------------------------------------------------------
/src/test/java/ru/ivi/opensource/flinkclickhousesink/applied/ExceptionsThrowableSinkTest.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import org.hamcrest.CoreMatchers;
4 | import org.junit.Before;
5 | import org.junit.Test;
6 | import org.mockito.Mockito;
7 | import org.mockito.MockitoAnnotations;
8 |
9 | import java.util.concurrent.ExecutionException;
10 |
11 | import static org.hamcrest.MatcherAssert.assertThat;
12 | import static org.junit.Assert.assertEquals;
13 | import static org.mockito.Mockito.doAnswer;
14 | import static org.mockito.Mockito.doThrow;
15 | import static org.mockito.Mockito.times;
16 | import static org.mockito.Mockito.verify;
17 |
18 | public class ExceptionsThrowableSinkTest {
19 |
20 | private Sink exceptionsThrowableSink;
21 | private ClickHouseSinkBuffer buffer;
22 |
23 | @Before
24 | public void setUp() {
25 | buffer = Mockito.mock(ClickHouseSinkBuffer.class);
26 | exceptionsThrowableSink = new ExceptionsThrowableSink(buffer);
27 | MockitoAnnotations.initMocks(this);
28 | }
29 |
30 | @Test
31 | public void putWithException() throws ExecutionException, InterruptedException {
32 | String actual = "csv";
33 |
34 | doThrow(new InterruptedException("test Exception message"))
35 | .when(buffer)
36 | .assertFuturesNotFailedYet();
37 | doAnswer(invocation -> {
38 | String expected = (String) invocation.getArguments()[0];
39 | assertEquals(expected, actual);
40 | return invocation;
41 | }).when(buffer).put(Mockito.anyString());
42 |
43 | try {
44 | exceptionsThrowableSink.put(actual);
45 | } catch (InterruptedException expectedException) {
46 | assertThat(expectedException.getMessage(), CoreMatchers.containsString("test Exception message"));
47 | }
48 | verify(buffer, times(1)).assertFuturesNotFailedYet();
49 | }
50 | }
--------------------------------------------------------------------------------
/src/test/java/ru/ivi/opensource/flinkclickhousesink/applied/UnexceptionableSinkTest.java:
--------------------------------------------------------------------------------
1 | package ru.ivi.opensource.flinkclickhousesink.applied;
2 |
3 | import org.junit.Before;
4 | import org.junit.Test;
5 | import org.mockito.Mockito;
6 | import org.mockito.MockitoAnnotations;
7 |
8 | import java.util.concurrent.ExecutionException;
9 |
10 | import static org.junit.Assert.assertEquals;
11 | import static org.mockito.Mockito.doAnswer;
12 | import static org.mockito.Mockito.times;
13 | import static org.mockito.Mockito.verify;
14 |
15 | public class UnexceptionableSinkTest {
16 |
17 | private Sink unexceptionableSink;
18 | private ClickHouseSinkBuffer buffer;
19 |
20 | @Before
21 | public void setUp() {
22 | buffer = Mockito.mock(ClickHouseSinkBuffer.class);
23 | unexceptionableSink = new UnexceptionableSink(buffer);
24 | MockitoAnnotations.initMocks(this);
25 | }
26 |
27 | @Test
28 | public void put() throws ExecutionException, InterruptedException {
29 | String actual = "csv";
30 |
31 | doAnswer(invocation -> {
32 | String expected = (String) invocation.getArguments()[0];
33 | assertEquals(expected, actual);
34 | return invocation;
35 | }).when(buffer).put(Mockito.anyString());
36 |
37 | unexceptionableSink.put(actual);
38 |
39 | verify(buffer, times(0)).assertFuturesNotFailedYet();
40 | }
41 | }
--------------------------------------------------------------------------------
/src/test/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 | %d{yyyy-MM-dd HH:mm:ss.SSS zzz} %highlight(%-5p) %t %c{2} - %m%n
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
--------------------------------------------------------------------------------
/src/test/resources/reference.conf:
--------------------------------------------------------------------------------
1 | clickhouse {
2 | sink {
3 | num-writers = 3
4 | timeout-sec = 1
5 | retries = 10
6 | queue-max-capacity = 1000
7 | failed-records-path = "/tmp/failed_records"
8 | ignoring-exception-enabled = "true"
9 | }
10 | }
11 |
12 |
--------------------------------------------------------------------------------