|
| 1 | +/* |
| 2 | + * Copyright 2021 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 io.aiven.kafka.connect.transforms; |
| 18 | + |
| 19 | +import java.util.HashMap; |
| 20 | +import java.util.Map; |
| 21 | +import java.util.StringJoiner; |
| 22 | + |
| 23 | +import org.apache.kafka.common.config.ConfigDef; |
| 24 | +import org.apache.kafka.connect.connector.ConnectRecord; |
| 25 | +import org.apache.kafka.connect.data.Schema; |
| 26 | +import org.apache.kafka.connect.data.SchemaAndValue; |
| 27 | +import org.apache.kafka.connect.data.SchemaBuilder; |
| 28 | +import org.apache.kafka.connect.data.Struct; |
| 29 | +import org.apache.kafka.connect.errors.DataException; |
| 30 | +import org.apache.kafka.connect.transforms.Transformation; |
| 31 | + |
| 32 | +import org.slf4j.Logger; |
| 33 | +import org.slf4j.LoggerFactory; |
| 34 | + |
| 35 | +public abstract class ConcatFields<R extends ConnectRecord<R>> implements Transformation<R> { |
| 36 | + private ConcatFieldsConfig config; |
| 37 | + private static final Logger log = LoggerFactory.getLogger(ConcatFields.class); |
| 38 | + |
| 39 | + protected abstract String dataPlace(); |
| 40 | + |
| 41 | + protected abstract SchemaAndValue getSchemaAndValue(final R record); |
| 42 | + |
| 43 | + protected abstract R createNewRecord(final R record, final Schema newSchema, final Object newValue); |
| 44 | + |
| 45 | + @Override |
| 46 | + public ConfigDef config() { |
| 47 | + return ConcatFieldsConfig.config(); |
| 48 | + } |
| 49 | + |
| 50 | + @Override |
| 51 | + public void configure(final Map<String, ?> configs) { |
| 52 | + this.config = new ConcatFieldsConfig(configs); |
| 53 | + } |
| 54 | + |
| 55 | + @Override |
| 56 | + public R apply(final R record) { |
| 57 | + final SchemaAndValue schemaAndValue = getSchemaAndValue(record); |
| 58 | + final SchemaBuilder newSchema = SchemaBuilder.struct(); |
| 59 | + |
| 60 | + if (schemaAndValue.value() == null) { |
| 61 | + throw new DataException(dataPlace() + " Value can't be null: " + record); |
| 62 | + } |
| 63 | + |
| 64 | + final R newRecord; |
| 65 | + |
| 66 | + if (schemaAndValue.value() instanceof Struct) { |
| 67 | + final Struct struct = (Struct) schemaAndValue.value(); |
| 68 | + final StringJoiner outputValue = new StringJoiner(config.delimiter()); |
| 69 | + |
| 70 | + if (schemaAndValue.schema() != null) { |
| 71 | + schemaAndValue.schema().fields().forEach(field -> newSchema.field(field.name(), field.schema())); |
| 72 | + } else { |
| 73 | + struct.schema().fields().forEach(field -> newSchema.field(field.name(), field.schema())); |
| 74 | + } |
| 75 | + newSchema.field(config.outputFieldName(), Schema.OPTIONAL_STRING_SCHEMA); |
| 76 | + final Struct newStruct = new Struct(newSchema.build()); |
| 77 | + struct.schema().fields().forEach(field -> { |
| 78 | + newStruct.put(field.name(), struct.get(field)); |
| 79 | + }); |
| 80 | + config.fieldNames().forEach(field -> { |
| 81 | + try { |
| 82 | + if (struct.get(field) == null) { |
| 83 | + outputValue.add(config.fieldReplaceMissing()); |
| 84 | + } else { |
| 85 | + outputValue.add(struct.get(field).toString()); |
| 86 | + } |
| 87 | + } catch (final DataException e) { |
| 88 | + log.debug("{} is missing, concat will use {}", field, config.fieldReplaceMissing()); |
| 89 | + outputValue.add(config.fieldReplaceMissing()); |
| 90 | + } |
| 91 | + }); |
| 92 | + newStruct.put(config.outputFieldName(), outputValue.toString()); |
| 93 | + newRecord = createNewRecord(record, newSchema.build(), newStruct); |
| 94 | + } else if (schemaAndValue.value() instanceof Map) { |
| 95 | + final Map newValue = new HashMap<>((Map<?, ?>) schemaAndValue.value()); |
| 96 | + final StringJoiner outputValue = new StringJoiner(config.delimiter()); |
| 97 | + config.fieldNames().forEach(field -> { |
| 98 | + if (newValue.get(field) == null) { |
| 99 | + outputValue.add(config.fieldReplaceMissing()); |
| 100 | + } else { |
| 101 | + outputValue.add(newValue.get(field).toString()); |
| 102 | + } |
| 103 | + }); |
| 104 | + newValue.put(config.outputFieldName(), outputValue.toString()); |
| 105 | + |
| 106 | + //if we have a schema, we can add the new field to it, otherwise just keep the schema null |
| 107 | + if (schemaAndValue.schema() != null) { |
| 108 | + schemaAndValue.schema().fields().forEach(field -> newSchema.field(field.name(), field.schema())); |
| 109 | + newSchema.field(config.outputFieldName(), Schema.OPTIONAL_STRING_SCHEMA); |
| 110 | + newRecord = createNewRecord(record, newSchema.build(), newValue); |
| 111 | + } else { |
| 112 | + newRecord = createNewRecord(record, null, newValue); |
| 113 | + } |
| 114 | + } else { |
| 115 | + throw new DataException("Value type must be STRUCT or MAP: " + record); |
| 116 | + } |
| 117 | + |
| 118 | + return newRecord; |
| 119 | + } |
| 120 | + |
| 121 | + public static class Key<R extends ConnectRecord<R>> extends ConcatFields<R> { |
| 122 | + @Override |
| 123 | + protected SchemaAndValue getSchemaAndValue(final R record) { |
| 124 | + return new SchemaAndValue(record.keySchema(), record.key()); |
| 125 | + } |
| 126 | + |
| 127 | + @Override |
| 128 | + protected R createNewRecord(final R record, final Schema newSchema, final Object newValue) { |
| 129 | + return record.newRecord( |
| 130 | + record.topic(), |
| 131 | + record.kafkaPartition(), |
| 132 | + newSchema, |
| 133 | + newValue, |
| 134 | + record.valueSchema(), |
| 135 | + record.value(), |
| 136 | + record.timestamp(), |
| 137 | + record.headers() |
| 138 | + ); |
| 139 | + } |
| 140 | + |
| 141 | + @Override |
| 142 | + protected String dataPlace() { |
| 143 | + return "key"; |
| 144 | + } |
| 145 | + } |
| 146 | + |
| 147 | + public static class Value<R extends ConnectRecord<R>> extends ConcatFields<R> { |
| 148 | + @Override |
| 149 | + protected SchemaAndValue getSchemaAndValue(final R record) { |
| 150 | + return new SchemaAndValue(record.valueSchema(), record.value()); |
| 151 | + } |
| 152 | + |
| 153 | + @Override |
| 154 | + protected R createNewRecord(final R record, final Schema newSchema, final Object newValue) { |
| 155 | + return record.newRecord( |
| 156 | + record.topic(), |
| 157 | + record.kafkaPartition(), |
| 158 | + record.keySchema(), |
| 159 | + record.key(), |
| 160 | + newSchema, |
| 161 | + newValue, |
| 162 | + record.timestamp(), |
| 163 | + record.headers() |
| 164 | + ); |
| 165 | + } |
| 166 | + |
| 167 | + @Override |
| 168 | + protected String dataPlace() { |
| 169 | + return "value"; |
| 170 | + } |
| 171 | + } |
| 172 | + |
| 173 | + @Override |
| 174 | + public void close() { |
| 175 | + } |
| 176 | +} |
0 commit comments