records, QueryIdentifier queryId, ErrorReporter errorReporter) {
39 | doInsert(records, queryId);
40 | }
41 |
42 | @Override
43 | public long recordsInserted() {
44 | return this.recordMap.size();
45 | }
46 |
47 |
48 | }
49 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/db/TableMappingRefresher.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.db;
2 |
3 | import org.slf4j.Logger;
4 | import org.slf4j.LoggerFactory;
5 |
6 | import java.util.TimerTask;
7 |
8 | public class TableMappingRefresher extends TimerTask {
9 | private static final Logger LOGGER = LoggerFactory.getLogger(TableMappingRefresher.class);
10 | private ClickHouseWriter chWriter = null;
11 | private String database = null;
12 |
13 | public TableMappingRefresher(String database, final ClickHouseWriter chWriter) {
14 | this.chWriter = chWriter;
15 | this.database = database;
16 | }
17 |
18 | @Override
19 | public void run() {
20 | try {
21 | chWriter.updateMapping(database);
22 | } catch (Exception e) {
23 | LOGGER.error("Update mapping Error: {}", e.getMessage());
24 | }
25 |
26 | }
27 | }
28 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/db/helper/ClickHouseFieldDescriptor.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.db.helper;
2 |
3 | import com.fasterxml.jackson.core.JsonProcessingException;
4 | import com.fasterxml.jackson.databind.ObjectMapper;
5 | import com.fasterxml.jackson.databind.PropertyNamingStrategies;
6 | import com.fasterxml.jackson.databind.annotation.JsonNaming;
7 | import lombok.Builder;
8 | import lombok.Data;
9 | import lombok.extern.jackson.Jacksonized;
10 |
11 | /**
12 | * Java object representation of one DESCRIBE TABLE result row.
13 | *
14 | * We use Jackson to instantiate it from JSON.
15 | */
16 | @Data
17 | @Builder
18 | @Jacksonized
19 | @JsonNaming(PropertyNamingStrategies.SnakeCaseStrategy.class)
20 | public class ClickHouseFieldDescriptor {
21 | private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
22 | private String name;
23 | private String type;
24 | private String defaultType;
25 | private String defaultExpression;
26 | private String comment;
27 | private String codecExpression;
28 | private String ttlExpression;
29 | private boolean isSubcolumn;
30 |
31 | public boolean isAlias() {
32 | return "ALIAS".equals(defaultType);
33 | }
34 |
35 | public boolean isMaterialized() {
36 | return "MATERIALIZED".equals(defaultType);
37 | }
38 |
39 | public boolean isEphemeral() {
40 | return "EPHEMERAL".equals(defaultType);
41 | }
42 |
43 | public boolean hasDefault() {
44 | return "DEFAULT".equals(defaultType);
45 | }
46 |
47 | public static ClickHouseFieldDescriptor fromJsonRow(String json) throws JsonProcessingException {
48 | return OBJECT_MAPPER.readValue(json.replace("\\", "\\\\").replace("\n", "\\n"), ClickHouseFieldDescriptor.class);
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/db/mapping/Table.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.db.mapping;
2 |
3 | import com.clickhouse.kafka.connect.util.Utils;
4 | import lombok.Getter;
5 | import lombok.Setter;
6 | import lombok.experimental.Accessors;
7 | import org.slf4j.Logger;
8 | import org.slf4j.LoggerFactory;
9 |
10 | import java.util.ArrayList;
11 | import java.util.HashMap;
12 | import java.util.List;
13 | import java.util.Map;
14 | import java.util.function.Predicate;
15 | import java.util.regex.Matcher;
16 | import java.util.regex.Pattern;
17 |
18 | @Getter
19 | public class Table {
20 | private static final Logger LOGGER = LoggerFactory.getLogger(Table.class);
21 | private static final Predicate SIZE_FIELD_MATCHER = Pattern.compile(".+\\.size[0-9]+$").asMatchPredicate();
22 | private static final Pattern MULTIPLE_MAP_VALUES_PATTERN = Pattern.compile("(\\.values)(?=((\\.values)+$))");
23 |
24 | private final String name;
25 | private final String database;
26 |
27 | private final List rootColumnsList;
28 | private final Map rootColumnsMap;
29 | private final List allColumnsList;
30 | private final Map allColumnsMap;
31 |
32 | @Setter
33 | @Accessors(fluent = true)
34 | private boolean hasDefaults;
35 |
36 | @Setter
37 | @Getter
38 | private int numColumns = 0;
39 |
40 | public Table(String database, String name) {
41 | this.database = database;
42 | this.name = name;
43 | this.rootColumnsList = new ArrayList<>();
44 | this.rootColumnsMap = new HashMap<>();
45 |
46 | this.allColumnsList = new ArrayList<>();
47 | this.allColumnsMap = new HashMap<>();
48 | }
49 |
50 | public Table(String database, String name, int numColumns) {
51 | this(database, name);
52 | this.numColumns = numColumns;
53 | }
54 |
55 | public String getCleanName() {
56 | return name;
57 | }
58 | public String getName() {
59 | return Utils.escapeName(name);
60 | }
61 |
62 | public String getFullName() {
63 | return Utils.escapeTableName(database, name);
64 | }
65 |
66 | private void registerValidColumn(Column column) {
67 | allColumnsMap.put(column.getName(), column);
68 | allColumnsList.add(column);
69 | }
70 |
71 | public void addColumn(Column column) {
72 | registerValidColumn(column);
73 |
74 | if (column.isSubColumn()) handleNonRoot(column);
75 | else {
76 | rootColumnsList.add(column);
77 | rootColumnsMap.put(column.getName(), column);
78 | }
79 | }
80 |
81 | private void handleNonRoot(Column column) {
82 | String parentName = column.getName().substring(0, column.getName().lastIndexOf("."));
83 | Column parent = allColumnsMap.getOrDefault(parentName, null);
84 | if (parent == null) {
85 | LOGGER.error("Got non-root column, but its parent was not found to be updated. {}", column);
86 | return;
87 | }
88 |
89 | updateParent(parent, column);
90 | }
91 |
92 | private void updateParent(Column parent, Column child) {
93 | switch (parent.getType()) {
94 | case VARIANT:
95 | // Variants are handled fully in the Column class because its types are always primitive. Let's ignore them here.
96 | return;
97 | case ARRAY:
98 | if (SIZE_FIELD_MATCHER.test(child.getName()))
99 | return;
100 |
101 | Column parentArrayType = parent.getArrayType();
102 | switch (parentArrayType.getType()) {
103 | case MAP:
104 | case TUPLE:
105 | updateParent(parent.getArrayType(), child.getArrayType());
106 | return;
107 | case ARRAY:
108 | do {
109 | child = child.getArrayType();
110 | parent = parent.getArrayType();
111 | } while (child.getType() == Type.ARRAY && parent.getType() == Type.ARRAY);
112 | updateParent(parent, child);
113 | return;
114 | case VARIANT:
115 | return;
116 | default:
117 | LOGGER.error("Unhandled complex type '{}' as a child of an array", parentArrayType.getType());
118 | return;
119 | }
120 | case MAP:
121 | // Keys are parsed fully in the Column class as its type is always primitive.
122 | if (child.getName().endsWith(".keys") || SIZE_FIELD_MATCHER.test(child.getName()))
123 | return;
124 |
125 | if (child.getType() == Type.ARRAY && child.getName().endsWith(".values")) {
126 | int depth = 1;
127 |
128 | Matcher matcher = MULTIPLE_MAP_VALUES_PATTERN.matcher(child.getName());
129 | while (matcher.find()) depth += 1;
130 |
131 | int remainingDepth = depth;
132 |
133 | // ClickHouse outputs nested maps values as nested array types
134 | while (remainingDepth-- > 0) {
135 | child = child.getArrayType();
136 | }
137 |
138 | child.setParent(parent);
139 |
140 | parent.setMapDepth(depth);
141 | parent.setMapValueType(child);
142 | registerValidColumn(child);
143 | }
144 | return;
145 | case TUPLE:
146 | Column parentOfParent = parent.getParent();
147 |
148 | if (parentOfParent != null) {
149 | boolean anyTransitionalParentIsMap = parentOfParent.getType() == Type.MAP;
150 |
151 | if (!anyTransitionalParentIsMap && parentOfParent.getType() == Type.ARRAY) {
152 | Column currentParent = parentOfParent.getParent();
153 |
154 | while (currentParent != null) {
155 | anyTransitionalParentIsMap = currentParent.getType() == Type.MAP;
156 |
157 | if (anyTransitionalParentIsMap)
158 | break;
159 |
160 | currentParent = currentParent.getParent();
161 | }
162 | }
163 |
164 | if (anyTransitionalParentIsMap) {
165 | int remainingDepth = getRemainingDepth(parent, parentOfParent);
166 |
167 | while (remainingDepth-- > 0) {
168 | child = child.getArrayType();
169 | }
170 | }
171 | }
172 | parent.getTupleFields().add(child);
173 | return;
174 | default:
175 | if (child.getName().endsWith(".null")) {
176 | LOGGER.debug("Ignoring complex column: {}", child);
177 | } else {
178 | LOGGER.warn("Unsupported complex parent type: {}", parent.getType());
179 | }
180 | }
181 | }
182 |
183 | private static int getRemainingDepth(Column parent, Column parentOfParent) {
184 | int compensationDepth = 0;
185 |
186 | // I don't really know why the ClickHouse describe table result wraps the type in an additional
187 | // array only when the parent is a map which is under array. But we have to deal with it.
188 | Matcher matcher = MULTIPLE_MAP_VALUES_PATTERN.matcher(parent.getName());
189 | while (matcher.find()) compensationDepth += 1;
190 |
191 | return parentOfParent.getMapDepth() + parentOfParent.getArrayDepth() - compensationDepth;
192 | }
193 | }
194 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/db/mapping/Type.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.db.mapping;
2 |
3 | public enum Type {
4 | UNKNOWN,
5 | INT8,
6 | INT16,
7 | INT32,
8 | INT64,
9 | INT128,
10 | INT256,
11 | STRING,
12 | FLOAT32,
13 | FLOAT64,
14 | BOOLEAN,
15 | ARRAY,
16 | MAP,
17 | TUPLE,
18 | VARIANT,
19 | Date,
20 | Date32,
21 | DateTime,
22 | DateTime64,
23 | UUID,
24 | UINT8,
25 | UINT16,
26 | UINT32,
27 | UINT64,
28 | UINT128,
29 | UINT256,
30 | Decimal,
31 | FIXED_STRING,
32 | Enum8,
33 | Enum16,
34 | }
35 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/dedup/DeDup.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.dedup;
2 |
3 | import com.clickhouse.kafka.connect.sink.kafka.OffsetContainer;
4 |
5 | public class DeDup {
6 |
7 | private OffsetContainer currentOffset;
8 | private OffsetContainer previousOffset;
9 | private DeDupStrategy deDupStrategy;
10 |
11 | public DeDup(DeDupStrategy deDupStrategy, OffsetContainer currentOffset) {
12 | this.currentOffset = currentOffset;
13 | this.deDupStrategy = deDupStrategy;
14 | previousOffset = null;
15 | }
16 |
17 | public boolean isNew(int recordOffset) {
18 | return true;
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/dedup/DeDupStrategy.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.dedup;
2 |
3 | public enum DeDupStrategy {
4 |
5 | OFF,
6 | PRIMARY_KEY,
7 | ALL_DATA,
8 | }
9 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/dlq/DuplicateException.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.dlq;
2 |
3 | public class DuplicateException extends Exception {
4 | public DuplicateException() {
5 |
6 | }
7 |
8 | public DuplicateException(String message) {
9 | super(message);
10 | }
11 |
12 | public DuplicateException(String message, Throwable cause) {
13 | super(message, cause);
14 | }
15 |
16 | public DuplicateException(Throwable cause) {
17 | super(cause);
18 | }
19 |
20 | public DuplicateException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
21 | super(message, cause, enableSuppression, writableStackTrace);
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/dlq/ErrorReporter.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.dlq;
2 |
3 | import org.apache.kafka.connect.sink.SinkRecord;
4 |
5 | public interface ErrorReporter {
6 | void report(SinkRecord record, Exception e);
7 | }
8 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/kafka/OffsetContainer.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.kafka;
2 |
3 | public class OffsetContainer extends TopicPartitionContainer {
4 | private long offset;
5 |
6 |
7 | public OffsetContainer(String topic, int partition, long offset) {
8 | super(topic, partition);
9 | this.offset = offset;
10 | }
11 |
12 | public long getOffset() {
13 | return offset;
14 | }
15 | }
16 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/kafka/RangeContainer.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.kafka;
2 |
3 | public class RangeContainer extends TopicPartitionContainer {
4 |
5 | private long maxOffset;
6 | private long minOffset;
7 |
8 | public RangeContainer(String topic, int partition) {
9 | super(topic, partition);
10 | this.maxOffset = -1;
11 | this.minOffset = Long.MAX_VALUE;
12 | }
13 |
14 | public RangeContainer(String topic, int partition, long maxOffset, long minOffset) {
15 | super(topic, partition);
16 | this.maxOffset = maxOffset;
17 | this.minOffset = minOffset;
18 | }
19 |
20 |
21 | /**
22 | * This method will set min/max values for offsets
23 | *
24 | * @param offset
25 | */
26 | public void defineInRange(long offset) {
27 | maxOffset = Long.max(maxOffset, offset);
28 | minOffset = Long.min(minOffset, offset);
29 | }
30 |
31 | public long getMaxOffset() {
32 | return maxOffset;
33 | }
34 |
35 | public long getMinOffset() {
36 | return minOffset;
37 | }
38 |
39 | public boolean isInRange(long offset) {
40 | if (offset >= minOffset && offset <= maxOffset)
41 | return true;
42 | return false;
43 | }
44 |
45 | /**
46 | * This compares the stored state with the actual state
47 | * @param rangeContainer A container with the actual state
48 | * @return The state of the comparison
49 | */
50 | public RangeState getOverLappingState(RangeContainer rangeContainer) {
51 | long actualMinOffset = rangeContainer.getMinOffset();
52 | long actualMaxOffset = rangeContainer.getMaxOffset();
53 |
54 | // SAME State [0, 10] Actual [0, 10]
55 | if (actualMaxOffset == maxOffset && actualMinOffset <= minOffset)
56 | return RangeState.SAME;
57 | // NEW State [0, 10] Actual [11, 20]
58 | if (actualMinOffset > maxOffset)
59 | return RangeState.NEW;
60 | // CONTAINS [0, 10] Actual [1, 10]
61 | if (actualMaxOffset <= maxOffset && actualMinOffset >= minOffset)
62 | return RangeState.CONTAINS;
63 | // OVER_LAPPING
64 | if (actualMaxOffset > maxOffset)
65 | return RangeState.OVER_LAPPING;
66 | // ZEROED [10, 20] Actual [0, 10]
67 | if (actualMinOffset == 0)
68 | return RangeState.ZERO;
69 | // PREVIOUS [10, 20] Actual [5, 8]
70 | if (actualMaxOffset < minOffset)
71 | return RangeState.PREVIOUS;
72 | // ERROR [10, 20] Actual [8, 19]
73 | return RangeState.ERROR;
74 | }
75 |
76 |
77 | public RangeContainer getRangeContainer() {
78 | return this;
79 | }
80 |
81 | public String toString() {
82 | return "Topic: " + getTopic() + " Partition: " + getPartition() + " MinOffset: " + minOffset + " MaxOffset: " + maxOffset;
83 | }
84 | }
85 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/kafka/RangeState.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.kafka;
2 |
3 | public enum RangeState {
4 | ZERO(0), //This is for when it seems like the topic has been deleted/recreated
5 | SAME(1),
6 | PREFIX(2),
7 | SUFFIX(3),
8 | CONTAINS(4),
9 | OVER_LAPPING(5),
10 | NEW(6),
11 | ERROR(7),
12 | PREVIOUS(8);
13 |
14 |
15 | private int rangeState;
16 |
17 | RangeState(int rangeState) {
18 | this.rangeState = rangeState;
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/kafka/TopicPartitionContainer.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.kafka;
2 |
3 | public class TopicPartitionContainer {
4 |
5 | protected String topic;
6 | protected int partition;
7 |
8 | public TopicPartitionContainer(String topic, int partition) {
9 | this.topic = topic;
10 | this.partition = partition;
11 | }
12 |
13 | public String getTopic() {
14 | return topic;
15 | }
16 |
17 | public int getPartition() {
18 | return partition;
19 | }
20 |
21 | public String getTopicAndPartitionKey() {
22 | return String.format("%s-%d", topic, partition);
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/state/State.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.state;
2 |
3 | public enum State {
4 | NONE(1),
5 | BEFORE_PROCESSING(2),
6 | IN_PROCESSING(3),
7 | AFTER_PROCESSING(4);
8 |
9 |
10 | private int state;
11 |
12 | State(int state) {
13 | this.state = state;
14 | }
15 | }
16 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/state/StateProvider.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.state;
2 |
3 | public interface StateProvider {
4 |
5 | public StateRecord getStateRecord(String topic, int partition );
6 |
7 |
8 | public void setStateRecord(StateRecord stateRecord);
9 |
10 | }
11 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/state/StateRecord.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.state;
2 |
3 | import com.clickhouse.kafka.connect.sink.kafka.RangeContainer;
4 |
5 | import java.util.Objects;
6 |
7 | public class StateRecord extends RangeContainer {
8 | private State state;
9 |
10 | public StateRecord(String topic, int partition , long maxOffset, long minOffset, State state) {
11 | super(topic, partition, maxOffset, minOffset);
12 | this.state = state;
13 | }
14 |
15 | public State getState() {
16 | return state;
17 | }
18 |
19 | public void setState(State state) {
20 | this.state = state;
21 | }
22 |
23 | public boolean equals(Object o) {
24 | if (this == o) return true;
25 | if (!(o instanceof StateRecord)) return false;
26 | //if (!super.equals(o)) return false; //If we overrode it there
27 |
28 | StateRecord that = (StateRecord) o;
29 |
30 | return Objects.equals(this.topic, that.topic)
31 | && this.partition == that.partition
32 | && this.state == that.state
33 | && this.getMinOffset() == that.getMinOffset()
34 | && this.getMaxOffset() == that.getMaxOffset();
35 | }
36 |
37 | public String toString() {
38 | return "StateRecord{" +
39 | "topic='" + topic + "'" +
40 | ", partition=" + partition +
41 | ", state='" + state + "'" +
42 | ", minOffset=" + getMinOffset() +
43 | ", maxOffset=" + getMaxOffset() +
44 | '}';
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/state/provider/InMemoryState.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.state.provider;
2 |
3 | import com.clickhouse.kafka.connect.sink.state.State;
4 | import com.clickhouse.kafka.connect.sink.state.StateProvider;
5 | import com.clickhouse.kafka.connect.sink.state.StateRecord;
6 |
7 | import java.util.HashMap;
8 | import java.util.Map;
9 |
10 | public class InMemoryState implements StateProvider {
11 |
12 | private Map stateDB = null;
13 | public InMemoryState() {
14 | this.stateDB = new HashMap<>(10);
15 | }
16 |
17 | private String genKey(String topic, int partition) {
18 | return String.format("%s-%d", topic, partition);
19 | }
20 | @Override
21 | public StateRecord getStateRecord(String topic, int partition) {
22 | String key = genKey(topic, partition);
23 | if ( !stateDB.containsKey(key))
24 | return new StateRecord(topic, partition, -1 , -1, State.NONE);
25 | return stateDB.get(key);
26 | }
27 |
28 | @Override
29 | public void setStateRecord(StateRecord stateRecord) {
30 | String key = genKey(stateRecord.getTopic(), stateRecord.getPartition());
31 | stateDB.put(key, stateRecord);
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/sink/state/provider/KeeperStateProvider.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.sink.state.provider;
2 |
3 | import com.clickhouse.client.ClickHouseClient;
4 | import com.clickhouse.client.ClickHouseException;
5 | import com.clickhouse.client.ClickHouseNode;
6 | import com.clickhouse.client.ClickHouseNodeSelector;
7 | import com.clickhouse.client.ClickHouseProtocol;
8 | import com.clickhouse.client.ClickHouseResponse;
9 | import com.clickhouse.client.api.query.Records;
10 | import com.clickhouse.data.ClickHouseFormat;
11 | import com.clickhouse.data.ClickHouseRecord;
12 | import com.clickhouse.kafka.connect.sink.ClickHouseSinkConfig;
13 | import com.clickhouse.kafka.connect.sink.db.helper.ClickHouseHelperClient;
14 | import com.clickhouse.kafka.connect.sink.state.State;
15 | import com.clickhouse.kafka.connect.sink.state.StateProvider;
16 | import com.clickhouse.kafka.connect.sink.state.StateRecord;
17 | import com.clickhouse.kafka.connect.util.Mask;
18 | import org.slf4j.Logger;
19 | import org.slf4j.LoggerFactory;
20 |
21 | import java.util.Map;
22 | import java.util.concurrent.ConcurrentHashMap;
23 |
24 | public class KeeperStateProvider implements StateProvider {
25 |
26 | private static final Logger LOGGER = LoggerFactory.getLogger(KeeperStateProvider.class);
27 | private ClickHouseNode server = null;
28 | private int pingTimeOut = 100;
29 |
30 |
31 | private ClickHouseHelperClient chc = null;
32 | private ClickHouseSinkConfig csc = null;
33 |
34 | private Map stateMap = null;
35 |
36 | public KeeperStateProvider(ClickHouseSinkConfig csc) {
37 | this.csc = csc;
38 | this.stateMap = new ConcurrentHashMap<>();
39 |
40 | String hostname = csc.getHostname();
41 | int port = csc.getPort();
42 | String database = csc.getDatabase();
43 | String username = csc.getUsername();
44 | String password = csc.getPassword();
45 | boolean sslEnabled = csc.isSslEnabled();
46 | String jdbcConnectionProperties = csc.getJdbcConnectionProperties();
47 | int timeout = csc.getTimeout();
48 | String clientVersion = csc.getClientVersion();
49 | boolean useClientV2 = clientVersion.equals("V1") ? false : true;
50 | LOGGER.info(String.format("hostname: [%s] port [%d] database [%s] username [%s] password [%s] sslEnabled [%s] timeout [%d]", hostname, port, database, username, Mask.passwordMask(password), sslEnabled, timeout));
51 |
52 | chc = new ClickHouseHelperClient.ClickHouseClientBuilder(hostname, port, csc.getProxyType(), csc.getProxyHost(), csc.getProxyPort())
53 | .setDatabase(database)
54 | .setUsername(username)
55 | .setPassword(password)
56 | .sslEnable(sslEnabled)
57 | .setJdbcConnectionProperties(jdbcConnectionProperties)
58 | .setTimeout(timeout)
59 | .setRetry(csc.getRetry())
60 | .useClientV2(useClientV2)
61 | .build();
62 |
63 | if (!chc.ping()) {
64 | LOGGER.error("Unable to ping Clickhouse server.");
65 | // TODO: exit
66 | }
67 | LOGGER.info("Ping is successful.");
68 | init();
69 | }
70 |
71 | public KeeperStateProvider(ClickHouseHelperClient chc) {
72 | if (!chc.ping())
73 | throw new RuntimeException("ping");
74 | this.chc = chc;
75 | init();
76 | }
77 |
78 | private void init() {
79 | String createTable = String.format("CREATE TABLE IF NOT EXISTS `%s`%s " +
80 | "(`key` String, minOffset BIGINT, maxOffset BIGINT, state String)" +
81 | " ENGINE=KeeperMap('%s') PRIMARY KEY `key`;",
82 | csc.getZkDatabase(),
83 | csc.getKeeperOnCluster().isEmpty() ? "" : " ON CLUSTER " + csc.getKeeperOnCluster(),
84 | csc.getZkPath());
85 | // TODO: exec instead of query
86 | if (chc.isUseClientV2()) {
87 | chc.queryV2(createTable);
88 | } else {
89 | ClickHouseResponse r = chc.queryV1(createTable);
90 | r.close();
91 | }
92 | }
93 |
94 | @Override
95 | public StateRecord getStateRecord(String topic, int partition) {
96 | String key = String.format("%s-%d", topic, partition);
97 | String selectStr = String.format("SELECT * FROM `%s` WHERE `key`= '%s'", csc.getZkDatabase(), key);
98 | try (ClickHouseClient client = ClickHouseClient.builder()
99 | .options(chc.getDefaultClientOptions())
100 | .nodeSelector(ClickHouseNodeSelector.of(ClickHouseProtocol.HTTP))
101 | .build();
102 | ClickHouseResponse response = client.read(chc.getServer()) // or client.connect(endpoints)
103 | .format(ClickHouseFormat.RowBinaryWithNamesAndTypes)
104 | .query(selectStr)
105 | .executeAndWait()) {
106 | LOGGER.debug("return size: {}", response.getSummary().getReadRows());
107 | long totalResultsFound = response.getSummary().getResultRows();
108 | if ( totalResultsFound == 0) {
109 | LOGGER.info("Read state record: topic {} partition {} with NONE state", topic, partition);
110 | return new StateRecord(topic, partition, 0, 0, State.NONE);
111 | } else if(totalResultsFound > 1){
112 | LOGGER.warn("There was more than 1 state records for query: {} ({} found)", selectStr, totalResultsFound);
113 | }
114 |
115 | ClickHouseRecord r = response.firstRecord();
116 | long minOffset = r.getValue(1).asLong();
117 | long maxOffset = r.getValue(2).asLong();
118 | State state = State.valueOf(r.getValue(3).asString());
119 | LOGGER.debug("read state record: topic {} partition {} with {} state max {} min {}", topic, partition, state, maxOffset, minOffset);
120 |
121 | StateRecord stateRecord = new StateRecord(topic, partition, maxOffset, minOffset, state);
122 | StateRecord storedRecord = stateMap.get(csc.getZkDatabase() + "-" + key);
123 | if (storedRecord != null && !stateRecord.equals(storedRecord)) {
124 | LOGGER.warn("State record is changed: {} -> {}", storedRecord, stateRecord);
125 | } else {
126 | LOGGER.debug("State record stored: {}", storedRecord);
127 | }
128 | return stateRecord;
129 | } catch (ClickHouseException e) {
130 | throw new RuntimeException(e);
131 | }
132 | }
133 |
134 | @Override
135 | public void setStateRecord(StateRecord stateRecord) {
136 | long minOffset = stateRecord.getMinOffset();
137 | long maxOffset = stateRecord.getMaxOffset();
138 | String key = stateRecord.getTopicAndPartitionKey();
139 | String state = stateRecord.getState().toString();
140 | String insertStr = String.format("INSERT INTO `%s` SETTINGS wait_for_async_insert=1 VALUES ('%s', %d, %d, '%s');", csc.getZkDatabase(), key, minOffset, maxOffset, state);
141 | LOGGER.info("Write state record: {}", stateRecord);
142 | if (chc.isUseClientV2()) {
143 | try (Records records = this.chc.queryV2(insertStr)) {
144 | LOGGER.debug("Number of written rows (V2) [{}]", records.getWrittenRows());
145 | } catch (Exception e) {
146 | LOGGER.error("Failed to write state record: {}", stateRecord, e);
147 | throw new RuntimeException(e);
148 | }
149 | } else {
150 | ClickHouseResponse response = this.chc.queryV1(insertStr);
151 | LOGGER.debug("Number of written rows (V1) [{}]", response.getSummary().getWrittenRows());
152 | response.close();
153 | }
154 |
155 | stateMap.put(csc.getZkDatabase() + "-" + key, stateRecord);
156 | }
157 | }
158 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/transforms/ExtractTopicConfig.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2019 Aiven Oy
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package com.clickhouse.kafka.connect.transforms;
18 |
19 | import org.apache.kafka.common.config.AbstractConfig;
20 | import org.apache.kafka.common.config.ConfigDef;
21 |
22 | import java.util.Map;
23 | import java.util.Optional;
24 |
25 | class ExtractTopicConfig extends AbstractConfig {
26 | public static final String FIELD_NAME_CONFIG = "field.name";
27 | private static final String FIELD_NAME_DOC =
28 | "The name of the field which should be used as the topic name. "
29 | + "If null or empty, the entire key or value is used (and assumed to be a string).";
30 |
31 | public static final String SKIP_MISSING_OR_NULL_CONFIG = "skip.missing.or.null";
32 | private static final String SKIP_MISSING_OR_NULL_DOC =
33 | "In case the source of the new topic name is null or missing, "
34 | + "should a record be silently passed without transformation.";
35 |
36 | ExtractTopicConfig(final Map, ?> originals) {
37 | super(config(), originals);
38 | }
39 |
40 | static ConfigDef config() {
41 | return new ConfigDef()
42 | .define(
43 | FIELD_NAME_CONFIG,
44 | ConfigDef.Type.STRING,
45 | null,
46 | ConfigDef.Importance.HIGH,
47 | FIELD_NAME_DOC)
48 | .define(
49 | SKIP_MISSING_OR_NULL_CONFIG,
50 | ConfigDef.Type.BOOLEAN,
51 | false,
52 | ConfigDef.Importance.LOW,
53 | SKIP_MISSING_OR_NULL_DOC);
54 | }
55 |
56 | Optional fieldName() {
57 | final String rawFieldName = getString(FIELD_NAME_CONFIG);
58 | if (null == rawFieldName || "".equals(rawFieldName)) {
59 | return Optional.empty();
60 | }
61 | return Optional.of(rawFieldName);
62 | }
63 |
64 | boolean skipMissingOrNull() {
65 | return getBoolean(SKIP_MISSING_OR_NULL_CONFIG);
66 | }
67 | }
68 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/transforms/KeyToValue.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.transforms;
2 |
3 | import org.apache.kafka.common.config.AbstractConfig;
4 | import org.apache.kafka.common.config.ConfigDef;
5 | import org.apache.kafka.connect.connector.ConnectRecord;
6 | import org.apache.kafka.connect.data.Schema;
7 | import org.apache.kafka.connect.data.SchemaBuilder;
8 | import org.apache.kafka.connect.data.Struct;
9 | import org.apache.kafka.connect.transforms.Transformation;
10 | import org.slf4j.Logger;
11 | import org.slf4j.LoggerFactory;
12 |
13 | import java.util.Map;
14 |
15 | public class KeyToValue> implements Transformation {
16 | private static final Logger LOGGER = LoggerFactory.getLogger(KeyToValue.class.getName());
17 | public static final ConfigDef CONFIG_DEF = new ConfigDef().define("field", ConfigDef.Type.STRING, "_key", ConfigDef.Importance.LOW,
18 | "Field name on the record value to extract the record key into.");
19 |
20 | private String keyFieldName;
21 | private Schema valueSchema;
22 |
23 | @Override
24 | public void configure(Map configs) {
25 | final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs);
26 | keyFieldName = config.getString("field");
27 | }
28 |
29 | @Override
30 | public R apply(R record) {
31 | LOGGER.debug("Old Key: {}, Old Value: {}", record.key(), record.value());
32 | if (record.valueSchema() == null) {
33 | return applySchemaless(record);
34 | } else {
35 | return applyWithSchema(record);
36 | }
37 | }
38 |
39 | private R applySchemaless(R record) {
40 | if (!(record.value() instanceof Map)) {
41 | throw new IllegalArgumentException("Schemaless record value must be a Map - make sure you're using the JSON Converter for value.");
42 | }
43 |
44 | final Map value = (Map) record.value();
45 | value.put(keyFieldName, record.key());
46 | LOGGER.debug("New schemaless value: {}", value);
47 | return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), record.valueSchema(), value, record.timestamp());
48 | }
49 |
50 | private R applyWithSchema(R record) {
51 | final Struct oldValue = (Struct) record.value();
52 |
53 | if (valueSchema == null) {
54 | final SchemaBuilder builder = SchemaBuilder.struct();
55 | builder.name(oldValue.schema().name());
56 | builder.version(oldValue.schema().version());
57 | builder.doc(oldValue.schema().doc());
58 | oldValue.schema().fields().forEach(f -> {
59 | builder.field(f.name(), f.schema());
60 | });
61 | builder.field(keyFieldName, record.keySchema() == null ? Schema.OPTIONAL_STRING_SCHEMA : record.keySchema());
62 | valueSchema = builder.build();
63 | valueSchema.schema().fields().forEach(f -> LOGGER.debug("Field: {}", f));
64 | }
65 |
66 | Struct newValue = new Struct(valueSchema);
67 | valueSchema.fields().forEach(f -> {
68 | if (f.name().equals(keyFieldName)) {
69 | newValue.put(f, record.key());
70 | } else {
71 | newValue.put(f, oldValue.get(f));
72 | }
73 | });
74 | LOGGER.debug("New schema value: {}", newValue);
75 | return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), valueSchema, newValue, record.timestamp());
76 | }
77 |
78 | @Override
79 | public ConfigDef config() {
80 | return CONFIG_DEF;
81 | }
82 |
83 | @Override
84 | public void close() {
85 | valueSchema = null;
86 | }
87 |
88 | public static class SimpleConfig extends AbstractConfig {
89 | public SimpleConfig(ConfigDef configDef, Map, ?> originals) {
90 | super(configDef, originals, false);
91 | }
92 | }
93 | }
94 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/util/Mask.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.util;
2 |
3 | public class Mask {
4 |
5 | public static String passwordMask(String password) {
6 | if (password.length() <= 6) {
7 | return "*".repeat(password.length());
8 | }
9 | String tmpPassword = "***" + password.substring(3, password.length() - 3) + "***";
10 | return tmpPassword;
11 | }
12 | }
13 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/util/QueryIdentifier.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.util;
2 |
3 | import org.slf4j.Logger;
4 | import org.slf4j.LoggerFactory;
5 |
6 | public class QueryIdentifier {
7 | private static final Logger LOGGER = LoggerFactory.getLogger(QueryIdentifier.class);
8 | private final String topic;
9 | private final int partition;
10 | private final long minOffset;
11 | private final long maxOffset;
12 | private final String queryId;
13 |
14 | public QueryIdentifier(String topic, String queryId) {
15 | this.topic = topic;
16 | this.queryId = queryId;
17 |
18 | int INVALID = -1;
19 | this.partition = INVALID;
20 | this.minOffset = INVALID;
21 | this.maxOffset = INVALID;
22 | }
23 | public QueryIdentifier(String topic, int partition, long minOffset, long maxOffset, String queryId) {
24 | this.topic = topic;
25 | this.partition = partition;
26 | this.minOffset = minOffset;
27 | this.maxOffset = maxOffset;
28 | this.queryId = queryId;
29 | }
30 |
31 | public String toString() {
32 | if (partition == -1) {
33 | return String.format("Topic: [%s], (QueryId: [%s])", topic, queryId);
34 | }
35 |
36 | return String.format("Topic: [%s], Partition: [%s], MinOffset: [%s], MaxOffset: [%s], (QueryId: [%s])",
37 | topic, partition, minOffset, maxOffset, queryId);
38 | }
39 |
40 | public String getQueryId() {
41 | return queryId;
42 | }
43 | public String getTopic() {
44 | return topic;
45 | }
46 | public int getPartition() {
47 | return partition;
48 | }
49 | public long getMinOffset() {
50 | return minOffset;
51 | }
52 | public long getMaxOffset() {
53 | return maxOffset;
54 | }
55 |
56 | public String getDeduplicationToken() {
57 | if (partition == -1) {
58 | return null;
59 | }
60 | return String.format("%s-%s-%s-%s", topic, partition, minOffset, maxOffset);
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/util/Utils.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.util;
2 |
3 | import com.clickhouse.client.ClickHouseException;
4 | import com.clickhouse.kafka.connect.sink.data.Record;
5 | import com.clickhouse.kafka.connect.sink.dlq.ErrorReporter;
6 | import org.apache.kafka.connect.errors.DataException;
7 | import org.apache.kafka.connect.errors.RetriableException;
8 | import org.apache.kafka.connect.sink.SinkRecord;
9 | import org.slf4j.Logger;
10 | import org.slf4j.LoggerFactory;
11 |
12 | import java.io.IOException;
13 | import java.net.SocketTimeoutException;
14 | import java.net.UnknownHostException;
15 | import java.util.ArrayList;
16 | import java.util.Collection;
17 | import java.util.List;
18 | import java.util.Map;
19 |
20 | public class Utils {
21 |
22 | public static String escapeName(String topic) {
23 | String cleanTopic = topic.replace("`", "");
24 | return String.format("`%s`", cleanTopic);
25 | }
26 |
27 | public static String escapeTableName(String database, String topicName) {
28 | return escapeName(database) + "." + escapeName(topicName);
29 | }
30 |
31 | private static final Logger LOGGER = LoggerFactory.getLogger(Utils.class);
32 |
33 | public static Exception getRootCause(Exception e) {
34 | return getRootCause(e, false);
35 | }
36 |
37 | /**
38 | * This will drill down to the first ClickHouseException in the exception chain
39 | *
40 | * @param e Exception to drill down
41 | * @return ClickHouseException or null if none found
42 | */
43 | public static Exception getRootCause(Exception e, Boolean prioritizeClickHouseException) {
44 | if (e == null)
45 | return null;
46 |
47 | Throwable runningException = e;//We have to use Throwable because of the getCause() signature
48 | while (runningException.getCause() != null &&
49 | (!prioritizeClickHouseException || !(runningException instanceof ClickHouseException))) {
50 | LOGGER.trace("Found exception: {}", runningException.getLocalizedMessage());
51 | runningException = runningException.getCause();
52 | }
53 |
54 | return runningException instanceof Exception ? (Exception) runningException : null;
55 | }
56 |
57 |
58 | /**
59 | * This method checks to see if we should retry, otherwise it just throws the exception again
60 | *
61 | * @param e Exception to check
62 | */
63 |
64 | public static void handleException(Exception e, boolean errorsTolerance, Collection records) {
65 | LOGGER.warn("Deciding how to handle exception: {}", e.getLocalizedMessage());
66 |
67 | //Let's check if we have a ClickHouseException to reference the error code
68 | //https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ErrorCodes.cpp
69 | Exception rootCause = Utils.getRootCause(e, true);
70 | if (rootCause instanceof ClickHouseException) {
71 | ClickHouseException clickHouseException = (ClickHouseException) rootCause;
72 | LOGGER.warn("ClickHouseException code: {}", clickHouseException.getErrorCode());
73 | switch (clickHouseException.getErrorCode()) {
74 | case 3: // UNEXPECTED_END_OF_FILE
75 | case 107: // FILE_DOESNT_EXIST
76 | case 159: // TIMEOUT_EXCEEDED
77 | case 164: // READONLY
78 | case 202: // TOO_MANY_SIMULTANEOUS_QUERIES
79 | case 203: // NO_FREE_CONNECTION
80 | case 209: // SOCKET_TIMEOUT
81 | case 210: // NETWORK_ERROR
82 | case 241: // MEMORY_LIMIT_EXCEEDED
83 | case 242: // TABLE_IS_READ_ONLY
84 | case 252: // TOO_MANY_PARTS
85 | case 285: // TOO_FEW_LIVE_REPLICAS
86 | case 319: // UNKNOWN_STATUS_OF_INSERT
87 | case 425: // SYSTEM_ERROR
88 | case 999: // KEEPER_EXCEPTION
89 | throw new RetriableException(e);
90 | default:
91 | LOGGER.error("Error code [{}] wasn't in the acceptable list.", clickHouseException.getErrorCode());
92 | break;
93 | }
94 | }
95 |
96 | //High-Level Explicit Exception Checking
97 | if (e instanceof DataException && !errorsTolerance) {
98 | LOGGER.warn("DataException thrown, wrapping exception: {}", e.getLocalizedMessage());
99 | throw (DataException) e;
100 | }
101 |
102 | //Otherwise use Root-Cause Exception Checking
103 | if (rootCause instanceof SocketTimeoutException) {
104 | LOGGER.warn("SocketTimeoutException thrown, wrapping exception: {}", e.getLocalizedMessage());
105 | throw new RetriableException(e);
106 | } else if (rootCause instanceof UnknownHostException) {
107 | LOGGER.warn("UnknownHostException thrown, wrapping exception: {}", e.getLocalizedMessage());
108 | throw new RetriableException(e);
109 | } else if (rootCause instanceof IOException) {
110 | final String msg = rootCause.getMessage();
111 | if (msg.indexOf(CLICKHOUSE_CLIENT_ERROR_READ_TIMEOUT_MSG) == 0 || msg.indexOf(CLICKHOUSE_CLIENT_ERROR_WRITE_TIMEOUT_MSG) == 0) {
112 | LOGGER.warn("IOException thrown, wrapping exception: {}", e.getLocalizedMessage());
113 | throw new RetriableException(e);
114 | }
115 | }
116 |
117 | if (errorsTolerance) {//Right now this is all exceptions - should we restrict to just ClickHouseExceptions?
118 | LOGGER.warn("Errors tolerance is enabled, ignoring exception: {}", e.getLocalizedMessage());
119 | } else {
120 | LOGGER.error("Errors tolerance is disabled, wrapping exception: {}", e.getLocalizedMessage());
121 | if (records != null) {
122 | throw new RuntimeException(String.format("Number of records: %d", records.size()), e);
123 | } else {
124 | throw new RuntimeException("Records was null", e);
125 | }
126 |
127 | }
128 | }
129 |
130 | private static final String CLICKHOUSE_CLIENT_ERROR_READ_TIMEOUT_MSG = "Read timed out after";
131 | private static final String CLICKHOUSE_CLIENT_ERROR_WRITE_TIMEOUT_MSG = "Write timed out after";
132 |
133 | public static void sendTODlq(ErrorReporter errorReporter, Record record, Exception exception) {
134 | sendTODlq(errorReporter, record.getSinkRecord(), exception);
135 | }
136 |
137 | public static void sendTODlq(ErrorReporter errorReporter, SinkRecord record, Exception exception) {
138 | if (errorReporter != null && record != null) {
139 | errorReporter.report(record, exception);
140 | }
141 | }
142 |
143 | public static String getTableName(String database, String topicName, Map topicToTableMap) {
144 | String tableName = topicToTableMap.get(topicName);
145 | LOGGER.debug("Topic name: {}, Table Name: {}", topicName, tableName);
146 | if (tableName == null) {
147 | tableName = topicName;
148 | }
149 |
150 | return escapeTableName(database, tableName);
151 | }
152 |
153 |
154 | public static String getOffsets(Collection records) {
155 | long minOffset = Long.MAX_VALUE;
156 | long maxOffset = -1;
157 |
158 | for (SinkRecord record : records) {
159 | if (record.kafkaOffset() > maxOffset) {
160 | maxOffset = record.kafkaOffset();
161 | }
162 | if (record.kafkaOffset() < minOffset) {
163 | minOffset = record.kafkaOffset();
164 | }
165 | }
166 |
167 | return String.format("minOffset: %d, maxOffset: %d", minOffset, maxOffset);
168 | }
169 |
170 | public static List splitIgnoringQuotes(String input, char separator) {
171 | List result = new ArrayList<>();
172 | StringBuilder sb = new StringBuilder();
173 | boolean inSingleQuotes = false;
174 | boolean inDoubleQuotes = false;
175 |
176 | for (char c : input.toCharArray()) {
177 | if (c == '\'' && !inDoubleQuotes) {
178 | inSingleQuotes = !inSingleQuotes;
179 | sb.append(c);
180 | } else if (c == '"' && !inSingleQuotes) {
181 | inDoubleQuotes = !inDoubleQuotes;
182 | sb.append(c);
183 | } else if (c == separator && !inSingleQuotes && !inDoubleQuotes) {
184 | result.add(sb.toString().trim());
185 | sb.setLength(0);
186 | } else {
187 | sb.append(c);
188 | }
189 | }
190 | result.add(sb.toString().trim());
191 |
192 | return result;
193 | }
194 | }
195 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/util/jmx/ExecutionTimer.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.util.jmx;
2 |
3 | public class ExecutionTimer {
4 | private final long startTime;
5 |
6 | private ExecutionTimer() {
7 | this.startTime = System.nanoTime();
8 | }
9 |
10 | public static ExecutionTimer start() {
11 | return new ExecutionTimer();
12 | }
13 |
14 | public long nanosElapsed() {
15 | return System.nanoTime() - this.startTime;
16 | }
17 | }
18 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/util/jmx/MBeanServerUtils.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.util.jmx;
2 |
3 | import org.slf4j.Logger;
4 | import org.slf4j.LoggerFactory;
5 |
6 | import javax.management.InstanceAlreadyExistsException;
7 | import javax.management.MBeanServer;
8 | import javax.management.ObjectName;
9 | import java.lang.management.ManagementFactory;
10 |
11 | public final class MBeanServerUtils {
12 | private static final Logger LOGGER = LoggerFactory.getLogger(MBeanServerUtils.class);
13 |
14 | private MBeanServerUtils() {
15 |
16 | }
17 |
18 | public static T registerMBean(final T mBean, final String mBeanName) {
19 | MBeanServer server = ManagementFactory.getPlatformMBeanServer();
20 | try {
21 | server.registerMBean(mBean, new ObjectName(mBeanName));
22 | return mBean;
23 | } catch (InstanceAlreadyExistsException e) {
24 | throw new RuntimeException(e);
25 | } catch (Exception e) {
26 | // JMX might not be available
27 | LOGGER.warn("Unable to register MBean " + mBeanName, e);
28 | return mBean;
29 | }
30 | }
31 | public static void unregisterMBean(final String mBeanName) {
32 | MBeanServer server = ManagementFactory.getPlatformMBeanServer();
33 | try {
34 | ObjectName objectName = new ObjectName(mBeanName);
35 | if (server.isRegistered(objectName)) {
36 | server.unregisterMBean(objectName);
37 | }
38 | } catch (Exception e) {
39 | // JMX might not be available
40 | LOGGER.warn("Unable to unregister MBean " + mBeanName, e);
41 | }
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/util/jmx/SinkTaskStatistics.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.util.jmx;
2 |
3 | public class SinkTaskStatistics implements SinkTaskStatisticsMBean {
4 | private volatile long receivedRecords;
5 | private volatile long recordProcessingTime;
6 | private volatile long taskProcessingTime;
7 | @Override
8 | public long getReceivedRecords() {
9 | return receivedRecords;
10 | }
11 |
12 | @Override
13 | public long getRecordProcessingTime() {
14 | return recordProcessingTime;
15 | }
16 |
17 | @Override
18 | public long getTaskProcessingTime() {
19 | return taskProcessingTime;
20 | }
21 |
22 | public void receivedRecords(final int n ) {
23 | this.receivedRecords += n;
24 | }
25 |
26 | public void recordProcessingTime(ExecutionTimer timer) {
27 | this.recordProcessingTime += timer.nanosElapsed();
28 | }
29 |
30 | public void taskProcessingTime(ExecutionTimer timer) {
31 | this.taskProcessingTime += timer.nanosElapsed();
32 | }
33 |
34 | }
35 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/util/jmx/SinkTaskStatisticsMBean.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.util.jmx;
2 |
3 | public interface SinkTaskStatisticsMBean {
4 |
5 | long getReceivedRecords();
6 |
7 | long getRecordProcessingTime();
8 |
9 | long getTaskProcessingTime();
10 |
11 | }
12 |
--------------------------------------------------------------------------------
/src/main/java/com/clickhouse/kafka/connect/util/reactor/function/Tuple2.java:
--------------------------------------------------------------------------------
1 | package com.clickhouse.kafka.connect.util.reactor.function;
2 |
3 | import java.io.Serializable;
4 | import java.util.Arrays;
5 | import java.util.Collections;
6 | import java.util.Iterator;
7 | import java.util.List;
8 | import java.util.Objects;
9 | import java.util.function.Function;
10 |
11 | /**
12 | * A tuple that holds two non-null values.
13 | *
14 | * @param The type of the first non-null value held by this tuple
15 | * @param The type of the second non-null value held by this tuple
16 | * @author Jon Brisbin
17 | * @author Stephane Maldini
18 | */
19 | @SuppressWarnings("rawtypes")
20 | public class Tuple2 implements Iterable