-
Notifications
You must be signed in to change notification settings - Fork 13.6k
/
HeaderFrom.java
277 lines (228 loc) · 9.93 KB
/
HeaderFrom.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
/*
* 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.kafka.connect.transforms;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Validator;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.header.ConnectHeaders;
import org.apache.kafka.connect.header.Headers;
import org.apache.kafka.connect.transforms.util.SimpleConfig;
import java.util.AbstractMap.SimpleImmutableEntry;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static org.apache.kafka.connect.transforms.util.Requirements.requireStructOrNull;
public abstract class HeaderFrom<R extends ConnectRecord<R>> implements Transformation<R> {
public static final String OVERVIEW_DOC =
"Moves or copies fields in the key/value on a record into that record's headers"
+ "<p/>Use the concrete transformation type designed for the record key"
+ "(<code>" + InsertField.Key.class.getName() + "</code>) or value "
+ "(<code>" + InsertField.Value.class.getName() + "</code>).";
public static String[] trimAll(String[] array) {
return Arrays
.stream(array)
.map(String::trim)
.toArray(String[]::new);
}
private static final String DEFAULT_OPERATION = "copy";
private static final String PURPOSE = "Header creation from key/value";
private static final Validator OPERATION_VALIDATOR = (name, value) -> {
try {
Operation.valueOf(value.toString().toLowerCase(Locale.ROOT));
} catch (IllegalArgumentException exception) {
throw new ConfigException(
String.format("Only two operations are supported: copy and move (found: %s)", value.toString())
);
}
};
private static final Validator LIST_VALIDATOR = (name, value) -> {
if (!Pattern.compile("(\\w+)(,\\w+)*").matcher(value.toString().replaceAll("\\s+", "")).matches()) {
throw new ConfigException(
String.format(
"Config %s is suppose to be a comma-separated list of fields (found: %s)",
name,
value.toString()
)
);
}
};
private static final ConfigDef CONFIG_DEF = new ConfigDef()
.define(ConfigName.FIELDS,
ConfigDef.Type.STRING,
"transformation, not, configured",
LIST_VALIDATOR,
ConfigDef.Importance.HIGH,
"Field names whose values are to be copied/moved to headers.")
.define(ConfigName.HEADERS,
ConfigDef.Type.STRING,
"transformation, not, configured",
LIST_VALIDATOR,
ConfigDef.Importance.HIGH,
"Header names, in the same order as the field names listed in the fields configuration property")
.define(ConfigName.OPERATION,
ConfigDef.Type.STRING,
DEFAULT_OPERATION,
OPERATION_VALIDATOR,
ConfigDef.Importance.HIGH,
"Operation applied on the filed (can be either move or copy)");
private static Struct structWithRemovedFields(Struct struct, String[] fieldsToRemove) {
List<String> fieldsToRemoveList = Arrays.asList(fieldsToRemove);
SchemaBuilder newSchema = SchemaBuilder.struct();
newSchema.doc(struct.schema().doc());
struct.schema().fields().forEach(filed -> {
if (!fieldsToRemoveList.contains(filed.name())) newSchema.field(filed.name(), filed.schema());
});
Struct newStruct = new Struct(newSchema.build());
struct.schema().fields().forEach(filed -> {
if (!fieldsToRemoveList.contains(filed.name())) newStruct.put(filed.name(), struct.get(filed));
});
return newStruct;
}
private static Headers addHeaders(Struct struct,
Schema schema,
Headers headers,
List<Map.Entry<String, String>> fieldsHeadersZipped) {
Headers newHeaders = new ConnectHeaders(headers);
fieldsHeadersZipped.forEach(pair -> {
Object fieldValue = struct.get(pair.getKey());
Schema fieldSchema = schema.field(pair.getKey()).schema();
newHeaders.add(pair.getValue(), new SchemaAndValue(fieldSchema, fieldValue));
});
return newHeaders;
}
private enum Operation {
move,
copy
}
private interface ConfigName {
String FIELDS = "fields";
String HEADERS = "headers";
String OPERATION = "operation";
}
protected String[] fields;
protected String[] headers;
protected Operation operation;
protected List<Map.Entry<String, String>> fieldsHeadersZipped;
@Override
public R apply(R record) {
if (record == null || operatingSchema(record) == null) return record;
Schema recordSchema = operatingSchema(record);
Struct recordStruct = requireStructOrNull(operatingValue(record), PURPOSE);
Headers newHeaders = addHeaders(recordStruct, recordSchema, record.headers(), fieldsHeadersZipped);
R operatedRecord = operation == Operation.move ? removeFields(record) : record;
return record.newRecord(
record.topic(),
record.kafkaPartition(),
operatedRecord.keySchema(),
operatedRecord.key(),
operatedRecord.valueSchema(),
operatedRecord.value(),
record.timestamp(),
newHeaders
);
}
@Override
public void close() {
}
@Override
public ConfigDef config() {
return CONFIG_DEF;
}
@Override
public void configure(Map<String, ?> configs) {
final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs);
fields = trimAll(config.getString(ConfigName.FIELDS).split(","));
headers = trimAll(config.getString(ConfigName.HEADERS).split(","));
operation = Operation.valueOf(config.getString(ConfigName.OPERATION).toLowerCase(Locale.ROOT));
if (fields.length != headers.length) {
throw new ConfigException(
String.format(
"The fields and headers should have the same number of elements. " +
"Found: %s fields and %s headers",
fields.length,
headers.length
)
);
}
fieldsHeadersZipped = IntStream
.range(0, fields.length)
.mapToObj(i -> new SimpleImmutableEntry<>(fields[i], headers[i]))
.collect(Collectors.toList());
}
protected abstract R removeFields(R record);
protected abstract Schema operatingSchema(R record);
protected abstract Object operatingValue(R record);
public static class Key<R extends ConnectRecord<R>> extends HeaderFrom<R> {
@Override
protected Schema operatingSchema(R record) {
return record.keySchema();
}
@Override
protected Object operatingValue(R record) {
return record.key();
}
@Override
protected R removeFields(R record) {
Struct oldKey = requireStructOrNull(operatingValue(record), PURPOSE);
Struct newKey = structWithRemovedFields(oldKey, fields);
return record.newRecord(
record.topic(),
record.kafkaPartition(),
newKey.schema(),
newKey,
record.valueSchema(),
record.value(),
record.timestamp(),
record.headers()
);
}
}
public static class Value<R extends ConnectRecord<R>> extends HeaderFrom<R> {
@Override
protected Schema operatingSchema(R record) {
return record.valueSchema();
}
@Override
protected Object operatingValue(R record) {
return record.value();
}
@Override
protected R removeFields(R record) {
Struct oldValue = requireStructOrNull(operatingValue(record), PURPOSE);
Struct newValue = structWithRemovedFields(oldValue, fields);
return record.newRecord(
record.topic(),
record.kafkaPartition(),
record.keySchema(),
record.key(),
newValue.schema(),
newValue,
record.timestamp(),
record.headers()
);
}
}
}