/
StatelessNiFiSinkTask.java
289 lines (243 loc) · 12.1 KB
/
StatelessNiFiSinkTask.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.kafka.connect;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.connect.errors.RetriableException;
import org.apache.kafka.connect.header.Header;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.stateless.flow.DataflowTrigger;
import org.apache.nifi.stateless.flow.StatelessDataflow;
import org.apache.nifi.stateless.flow.TriggerResult;
import org.apache.nifi.util.FormatUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
public class StatelessNiFiSinkTask extends SinkTask {
private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSinkTask.class);
private StatelessDataflow dataflow;
private String inputPortName;
private Set<String> failurePortNames;
private long timeoutMillis;
private Pattern headerNameRegex;
private String headerNamePrefix;
private int batchSize;
private long batchBytes;
private QueueSize queueSize;
private String dataflowName;
private long backoffMillis = 0L;
@Override
public String version() {
return StatelessKafkaConnectorUtil.getVersion();
}
@Override
public void start(final Map<String, String> properties) {
logger.info("Starting Sink Task with properties {}", StatelessKafkaConnectorUtil.getLoggableProperties(properties));
final String timeout = properties.getOrDefault(StatelessKafkaConnectorUtil.DATAFLOW_TIMEOUT, StatelessKafkaConnectorUtil.DEFAULT_DATAFLOW_TIMEOUT);
timeoutMillis = (long) FormatUtils.getPreciseTimeDuration(timeout, TimeUnit.MILLISECONDS);
dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
final String regex = properties.get(StatelessNiFiSinkConnector.HEADERS_AS_ATTRIBUTES_REGEX);
headerNameRegex = regex == null ? null : Pattern.compile(regex);
headerNamePrefix = properties.getOrDefault(StatelessNiFiSinkConnector.HEADER_ATTRIBUTE_NAME_PREFIX, "");
batchSize = Integer.parseInt(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_COUNT, "0"));
batchBytes = Long.parseLong(properties.getOrDefault(StatelessNiFiSinkConnector.BATCH_SIZE_BYTES, "0"));
dataflow = StatelessKafkaConnectorUtil.createDataflow(properties);
// Determine input port name. If input port is explicitly set, use the value given. Otherwise, if only one port exists, use that. Otherwise, throw ConfigException.
final String dataflowName = properties.get(StatelessKafkaConnectorUtil.DATAFLOW_NAME);
inputPortName = properties.get(StatelessNiFiSinkConnector.INPUT_PORT_NAME);
if (inputPortName == null) {
final Set<String> inputPorts = dataflow.getInputPortNames();
if (inputPorts.isEmpty()) {
throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have an Input Port at the root level. Dataflows used for a Kafka Connect Sink Task "
+ "must have at least one Input Port at the root level.");
}
if (inputPorts.size() > 1) {
throw new ConfigException("The dataflow specified for <" + dataflowName + "> has multiple Input Ports at the root level (" + inputPorts.toString()
+ "). The " + StatelessNiFiSinkConnector.INPUT_PORT_NAME + " property must be set to indicate which of these Ports Kafka records should be sent to.");
}
inputPortName = inputPorts.iterator().next();
}
// Validate the input port
if (!dataflow.getInputPortNames().contains(inputPortName)) {
throw new ConfigException("The dataflow specified for <" + dataflowName + "> does not have Input Port with name <" + inputPortName + "> at the root level. Existing Input Port names are "
+ dataflow.getInputPortNames());
}
// Determine the failure Ports, if any are given.
final String failurePortList = properties.get(StatelessNiFiSinkConnector.FAILURE_PORTS);
if (failurePortList == null || failurePortList.trim().isEmpty()) {
failurePortNames = Collections.emptySet();
} else {
failurePortNames = new HashSet<>();
final String[] names = failurePortList.split(",");
for (final String name : names) {
final String trimmed = name.trim();
failurePortNames.add(trimmed);
}
}
// Validate the failure ports
final Set<String> outputPortNames = dataflow.getOutputPortNames();
for (final String failurePortName : failurePortNames) {
if (!outputPortNames.contains(failurePortName)) {
throw new ConfigException("Dataflow was configured with a Failure Port of " + failurePortName
+ " but there is no Port with that name in the dataflow. Valid Port names are " + outputPortNames);
}
}
}
@Override
public void put(final Collection<SinkRecord> records) {
if (backoffMillis > 0) {
logger.debug("Due to previous failure, will wait {} millis before executing dataflow", backoffMillis);
try {
Thread.sleep(backoffMillis);
} catch (final InterruptedException ie) {
Thread.currentThread().interrupt();
throw new RuntimeException("Interrupted while waiting to enqueue data", ie);
}
}
logger.debug("Enqueuing {} Kafka messages", records.size());
for (final SinkRecord record : records) {
final Map<String, String> attributes = createAttributes(record);
final byte[] contents = getContents(record.value());
queueSize = dataflow.enqueue(contents, attributes, inputPortName);
}
if (queueSize == null || queueSize.getObjectCount() < batchSize) {
return;
}
if (queueSize.getByteCount() < batchBytes) {
return;
}
logger.debug("Triggering dataflow");
try {
triggerDataflow();
resetBackoff();
} catch (final RetriableException re) {
backoff();
throw re;
}
}
private void backoff() {
// If no backoff period has been set, set it to 1 second. Otherwise, double the amount of time to backoff, up to 10 seconds.
if (backoffMillis == 0L) {
backoffMillis = 1000L;
}
backoffMillis = Math.min(backoffMillis * 2, 10_000L);
}
private void resetBackoff() {
backoffMillis = 0L;
}
private void triggerDataflow() {
final long start = System.nanoTime();
while (dataflow.isFlowFileQueued()) {
final DataflowTrigger trigger = dataflow.trigger();
try {
final Optional<TriggerResult> resultOptional = trigger.getResult(timeoutMillis, TimeUnit.MILLISECONDS);
if (resultOptional.isPresent()) {
final TriggerResult result = resultOptional.get();
if (result.isSuccessful()) {
// Verify that data was only transferred to the expected Input Port
verifyOutputPortContents(trigger, result);
// Acknowledge the data so that the session can be committed
result.acknowledge();
} else {
logger.error("Dataflow {} failed to execute properly", dataflowName, result.getFailureCause().orElse(null));
trigger.cancel();
throw new RetriableException("Dataflow failed to execute properly", result.getFailureCause().orElse(null));
}
} else {
trigger.cancel();
throw new RetriableException("Timed out waiting for the dataflow to complete");
}
} catch (final InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException("Interrupted while waiting for dataflow to complete", e);
}
}
final long nanos = System.nanoTime() - start;
logger.debug("Ran dataflow with {} messages ({}) in {} nanos", queueSize.getObjectCount(), FormatUtils.formatDataSize(queueSize.getByteCount()), nanos);
}
private void verifyOutputPortContents(final DataflowTrigger trigger, final TriggerResult result) {
for (final String failurePort : failurePortNames) {
final List<FlowFile> flowFiles = result.getOutputFlowFiles(failurePort);
if (flowFiles != null && !flowFiles.isEmpty()) {
logger.error("Dataflow transferred FlowFiles to Port {}, which is configured as a Failure Port. Rolling back session.", failurePort);
trigger.cancel();
throw new RetriableException("Data was transferred to Failure Port " + failurePort);
}
}
}
@Override
public void flush(final Map<TopicPartition, OffsetAndMetadata> currentOffsets) {
super.flush(currentOffsets);
if (queueSize != null && queueSize.getObjectCount() > 0) {
triggerDataflow();
}
}
private byte[] getContents(final Object value) {
if (value == null) {
return new byte[0];
}
if (value instanceof String) {
return ((String) value).getBytes(StandardCharsets.UTF_8);
}
if (value instanceof byte[]) {
return (byte[]) value;
}
throw new IllegalArgumentException("Unsupported message type: the Message value was " + value + " but was expected to be a byte array or a String");
}
private Map<String, String> createAttributes(final SinkRecord record) {
final Map<String, String> attributes = new HashMap<>();
attributes.put("kafka.topic", record.topic());
attributes.put("kafka.offset", String.valueOf(record.kafkaOffset()));
attributes.put("kafka.partition", String.valueOf(record.kafkaPartition()));
attributes.put("kafka.timestamp", String.valueOf(record.timestamp()));
final Object key = record.key();
if (key instanceof String) {
attributes.put("kafka.key", (String) key);
}
if (headerNameRegex != null) {
for (final Header header : record.headers()) {
if (headerNameRegex.matcher(header.key()).matches()) {
final String attributeName = headerNamePrefix + header.key();
final String attributeValue = String.valueOf(header.value());
attributes.put(attributeName, attributeValue);
}
}
}
return attributes;
}
@Override
public void stop() {
logger.info("Shutting down Sink Task");
if (dataflow != null) {
dataflow.shutdown();
}
}
}