Skip to content

Commit 5f879c6

Browse files
authored
Merge pull request #158 from /issues/156
[fix]: Ensure checkpointing triggers records flush to ClickHouse #156
2 parents 165a969 + ea5d1ec commit 5f879c6

File tree

2 files changed

+76
-2
lines changed

2 files changed

+76
-2
lines changed

flink-connector-clickhouse/src/main/java/org/apache/flink/connector/clickhouse/ClickHouseDynamicTableSink.java

+4-2
Original file line numberDiff line numberDiff line change
@@ -18,10 +18,11 @@
1818
package org.apache.flink.connector.clickhouse;
1919

2020
import org.apache.flink.connector.clickhouse.internal.AbstractClickHouseOutputFormat;
21+
import org.apache.flink.connector.clickhouse.internal.ClickHouseRowDataSinkFunction;
2122
import org.apache.flink.connector.clickhouse.internal.options.ClickHouseDmlOptions;
2223
import org.apache.flink.table.connector.ChangelogMode;
2324
import org.apache.flink.table.connector.sink.DynamicTableSink;
24-
import org.apache.flink.table.connector.sink.OutputFormatProvider;
25+
import org.apache.flink.table.connector.sink.SinkFunctionProvider;
2526
import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning;
2627
import org.apache.flink.table.types.DataType;
2728
import org.apache.flink.types.RowKind;
@@ -98,7 +99,8 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
9899
.withPrimaryKey(primaryKeys)
99100
.withPartitionKey(partitionKeys)
100101
.build();
101-
return OutputFormatProvider.of(outputFormat, options.getParallelism());
102+
return SinkFunctionProvider.of(
103+
new ClickHouseRowDataSinkFunction(outputFormat), options.getParallelism());
102104
}
103105

104106
@Override
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,72 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.flink.connector.clickhouse.internal;
19+
20+
import org.apache.flink.annotation.Internal;
21+
import org.apache.flink.api.common.functions.RuntimeContext;
22+
import org.apache.flink.configuration.Configuration;
23+
import org.apache.flink.runtime.state.FunctionInitializationContext;
24+
import org.apache.flink.runtime.state.FunctionSnapshotContext;
25+
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
26+
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
27+
import org.apache.flink.table.data.RowData;
28+
import org.apache.flink.util.Preconditions;
29+
30+
import javax.annotation.Nonnull;
31+
32+
import java.io.IOException;
33+
34+
/** A rich sink function to write {@link RowData} records into ClickHouse. */
35+
@Internal
36+
public class ClickHouseRowDataSinkFunction extends RichSinkFunction<RowData>
37+
implements CheckpointedFunction {
38+
39+
private final AbstractClickHouseOutputFormat outputFormat;
40+
41+
public ClickHouseRowDataSinkFunction(@Nonnull AbstractClickHouseOutputFormat outputFormat) {
42+
this.outputFormat = Preconditions.checkNotNull(outputFormat);
43+
}
44+
45+
@Override
46+
public void open(Configuration parameters) throws Exception {
47+
outputFormat.configure(parameters);
48+
RuntimeContext runtimeContext = getRuntimeContext();
49+
outputFormat.setRuntimeContext(runtimeContext);
50+
outputFormat.open(
51+
runtimeContext.getIndexOfThisSubtask(),
52+
runtimeContext.getNumberOfParallelSubtasks());
53+
}
54+
55+
@Override
56+
public void invoke(RowData value, Context context) throws IOException {
57+
outputFormat.writeRecord(value);
58+
}
59+
60+
@Override
61+
public void initializeState(FunctionInitializationContext context) {}
62+
63+
@Override
64+
public void snapshotState(FunctionSnapshotContext context) throws Exception {
65+
outputFormat.flush();
66+
}
67+
68+
@Override
69+
public void close() {
70+
outputFormat.close();
71+
}
72+
}

0 commit comments

Comments
 (0)