|
| 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 | +package com.apache.flink.streaming.connectors.clickhouse; |
| 18 | +import org.apache.flink.configuration.Configuration; |
| 19 | +import org.apache.flink.runtime.state.FunctionInitializationContext; |
| 20 | +import org.apache.flink.runtime.state.FunctionSnapshotContext; |
| 21 | +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; |
| 22 | +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; |
| 23 | +import org.apache.flink.types.Row; |
| 24 | +import org.slf4j.Logger; |
| 25 | +import org.slf4j.LoggerFactory; |
| 26 | +import ru.yandex.clickhouse.BalancedClickhouseDataSource; |
| 27 | +import ru.yandex.clickhouse.settings.ClickHouseProperties; |
| 28 | + |
| 29 | +import java.sql.Connection; |
| 30 | +import java.sql.PreparedStatement; |
| 31 | +import java.util.Properties; |
| 32 | + |
| 33 | +public class ClickHouseAppendSinkFunction extends RichSinkFunction<Row> implements CheckpointedFunction { |
| 34 | + private static final String USERNAME = "user"; |
| 35 | + private static final String PASSWORD = "password"; |
| 36 | + |
| 37 | + private static final Logger log = LoggerFactory.getLogger(ClickHouseAppendSinkFunction.class); |
| 38 | + private static final long serialVersionUID = 1L; |
| 39 | + |
| 40 | + private Connection connection; |
| 41 | + private BalancedClickhouseDataSource dataSource; |
| 42 | + private PreparedStatement pstat; |
| 43 | + |
| 44 | + private String address; |
| 45 | + private String username; |
| 46 | + private String password; |
| 47 | + |
| 48 | + private String prepareStatement; |
| 49 | + private Integer batchSize; |
| 50 | + private Long commitPadding; |
| 51 | + |
| 52 | + private Integer retries; |
| 53 | + private Long retryInterval; |
| 54 | + |
| 55 | + private Boolean ignoreInsertError; |
| 56 | + |
| 57 | + private Integer currentSize; |
| 58 | + private Long lastExecuteTime; |
| 59 | + |
| 60 | + public ClickHouseAppendSinkFunction(String address, String username, String password, String prepareStatement, Integer batchSize, Long commitPadding, Integer retries, Long retryInterval, Boolean ignoreInsertError) { |
| 61 | + this.address = address; |
| 62 | + this.username = username; |
| 63 | + this.password = password; |
| 64 | + this.prepareStatement = prepareStatement; |
| 65 | + this.batchSize = batchSize; |
| 66 | + this.commitPadding = commitPadding; |
| 67 | + this.retries = retries; |
| 68 | + this.retryInterval = retryInterval; |
| 69 | + this.ignoreInsertError = ignoreInsertError; |
| 70 | + } |
| 71 | + |
| 72 | + @Override |
| 73 | + public void open(Configuration parameters) throws Exception { |
| 74 | + super.open(parameters); |
| 75 | + Properties properties = new Properties(); |
| 76 | + properties.setProperty(USERNAME, username); |
| 77 | + properties.setProperty(PASSWORD, password); |
| 78 | + ClickHouseProperties clickHouseProperties = new ClickHouseProperties(properties); |
| 79 | + dataSource = new BalancedClickhouseDataSource(address, clickHouseProperties); |
| 80 | + connection = dataSource.getConnection(); |
| 81 | + pstat = connection.prepareStatement(prepareStatement); |
| 82 | + lastExecuteTime = System.currentTimeMillis(); |
| 83 | + currentSize = 0; |
| 84 | + |
| 85 | + } |
| 86 | + |
| 87 | + @Override |
| 88 | + public void invoke(Row value, Context context) throws Exception { |
| 89 | + for (int i = 0; i < value.getArity(); i++) { |
| 90 | + pstat.setObject(i + 1, value.getField(i)); |
| 91 | + } |
| 92 | + pstat.addBatch(); |
| 93 | + currentSize++; |
| 94 | + if (currentSize >= batchSize || (System.currentTimeMillis() - lastExecuteTime) > commitPadding) { |
| 95 | + try { |
| 96 | + doExecuteRetries(retries, retryInterval); |
| 97 | + } catch (Exception e) { |
| 98 | + log.error("clickhouse-insert-error ( maxRetries:" + retries + " , retryInterval : " + retryInterval + " millisecond )" + e.getMessage()); |
| 99 | + } finally { |
| 100 | + pstat.clearBatch(); |
| 101 | + currentSize = 0; |
| 102 | + lastExecuteTime = System.currentTimeMillis(); |
| 103 | + } |
| 104 | + } |
| 105 | + } |
| 106 | + |
| 107 | + public void doExecuteRetries(int count, long retryInterval) throws Exception { |
| 108 | + |
| 109 | + int retrySize = 0; |
| 110 | + Exception resultException = null; |
| 111 | + for (int i = 0; i < count; i++) { |
| 112 | + try { |
| 113 | + pstat.executeBatch(); |
| 114 | + break; |
| 115 | + } catch (Exception e) { |
| 116 | + retrySize++; |
| 117 | + resultException = e; |
| 118 | + } |
| 119 | + try { |
| 120 | + Thread.sleep(retryInterval); |
| 121 | + } catch (InterruptedException e) { |
| 122 | + log.error("clickhouse retry interval exception : ",e); |
| 123 | + } |
| 124 | + } |
| 125 | + if (retrySize == count && !ignoreInsertError) { |
| 126 | + throw resultException; |
| 127 | + } |
| 128 | + } |
| 129 | + |
| 130 | + @Override |
| 131 | + public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception { |
| 132 | + doExecuteRetries(retries, retryInterval); |
| 133 | + } |
| 134 | + |
| 135 | + @Override |
| 136 | + public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception { |
| 137 | + |
| 138 | + } |
| 139 | + |
| 140 | +} |
0 commit comments