34
34
35
35
import static org .apache .kafka .connect .data .Schema .Type .STRING ;
36
36
37
- public class FilterByFieldValue <R extends ConnectRecord <R >> implements Transformation <R > {
37
+ public abstract class FilterByFieldValue <R extends ConnectRecord <R >> implements Transformation <R > {
38
38
39
39
private String fieldName ;
40
40
private Optional <String > fieldExpectedValue ;
@@ -43,21 +43,29 @@ public class FilterByFieldValue<R extends ConnectRecord<R>> implements Transform
43
43
44
44
@ Override
45
45
public R apply (final R record ) {
46
- if (record .value () instanceof Struct ) {
47
- return handleStruct (record );
48
- } else if (record .value () instanceof Map ) {
49
- return handleMap (record );
46
+ if (operatingValue (record ) == null ) {
47
+ return record ;
48
+ }
49
+
50
+ if (operatingSchema (record ) == null ) {
51
+ return applySchemaless (record );
52
+ } else {
53
+ return applyWithSchema (record );
50
54
}
51
- return record ; // if record is other than map or struct, pass-by
52
55
}
53
56
54
- private R handleStruct (final R record ) {
55
- final Struct struct = (Struct ) record .value ();
56
- final Optional <String > fieldValue = extractStructFieldValue (struct , fieldName );
57
+
58
+ protected abstract Schema operatingSchema (R record );
59
+
60
+ protected abstract Object operatingValue (R record );
61
+
62
+ private R applyWithSchema (final R record ) {
63
+ final Struct struct = (Struct ) operatingValue (record );
64
+ final Optional <String > fieldValue = getStructFieldValue (struct , fieldName );
57
65
return filterCondition .test (fieldValue ) ? record : null ;
58
66
}
59
67
60
- private Optional <String > extractStructFieldValue (final Struct struct , final String fieldName ) {
68
+ private Optional <String > getStructFieldValue (final Struct struct , final String fieldName ) {
61
69
final Schema schema = struct .schema ();
62
70
final Field field = schema .field (fieldName );
63
71
final Object fieldValue = struct .get (field );
@@ -72,19 +80,19 @@ private Optional<String> extractStructFieldValue(final Struct struct, final Stri
72
80
}
73
81
74
82
@ SuppressWarnings ("unchecked" )
75
- private R handleMap (final R record ) {
76
- final Map <String , Object > map = (Map <String , Object >) record .value ();
77
- final Optional <String > fieldValue = extractMapFieldValue (map , fieldName );
78
- return filterCondition .test (fieldValue ) ? record : null ;
79
- }
80
-
81
- private Optional <String > extractMapFieldValue (final Map <String , Object > map , final String fieldName ) {
82
- if (!map .containsKey (fieldName )) {
83
- return Optional .empty ();
83
+ private R applySchemaless (final R record ) {
84
+ if (fieldName == null || fieldName .isEmpty ()) {
85
+ final Optional <String > value = getSchemalessFieldValue (operatingValue (record ));
86
+ return filterCondition .test (value ) ? record : null ;
87
+ } else {
88
+ final Map <String , Object > map = (Map <String , Object >) operatingValue (record );
89
+ final Optional <String > fieldValue = getSchemalessFieldValue (map .get (fieldName ));
90
+ return filterCondition .test (fieldValue ) ? record : null ;
84
91
}
92
+ }
85
93
86
- final Object fieldValue = map . get ( fieldName );
87
-
94
+ private Optional < String > getSchemalessFieldValue ( final Object fieldValue ) {
95
+ if ( fieldValue == null ) return Optional . empty ();
88
96
Optional <String > text = Optional .empty ();
89
97
if (isSupportedType (fieldValue )) {
90
98
text = Optional .of (fieldValue .toString ());
@@ -108,18 +116,24 @@ public ConfigDef config() {
108
116
return new ConfigDef ()
109
117
.define ("field.name" ,
110
118
ConfigDef .Type .STRING ,
119
+ null ,
111
120
ConfigDef .Importance .HIGH ,
112
- "The field name to filter by" )
121
+ "The field name to filter by." +
122
+ "Schema-based records (Avro), schemaless (e.g. JSON), and raw values are supported." +
123
+ "If empty, the whole key/value record will be filtered." )
113
124
.define ("field.value" ,
114
- ConfigDef .Type .STRING , null ,
125
+ ConfigDef .Type .STRING ,
126
+ null ,
115
127
ConfigDef .Importance .HIGH ,
116
128
"Expected value to match. Either define this, or a regex pattern" )
117
129
.define ("field.value.pattern" ,
118
- ConfigDef .Type .STRING , null ,
130
+ ConfigDef .Type .STRING ,
131
+ null ,
119
132
ConfigDef .Importance .HIGH ,
120
133
"The pattern to match. Either define this, or an expected value" )
121
134
.define ("field.value.matches" ,
122
- ConfigDef .Type .BOOLEAN , true ,
135
+ ConfigDef .Type .BOOLEAN ,
136
+ true ,
123
137
ConfigDef .Importance .MEDIUM ,
124
138
"The filter mode, 'true' for matching or 'false' for non-matching" );
125
139
}
@@ -150,4 +164,31 @@ public void configure(final Map<String, ?> configs) {
150
164
? matchCondition
151
165
: (result -> !matchCondition .test (result ));
152
166
}
167
+
168
+
169
+ public static final class Key <R extends ConnectRecord <R >> extends FilterByFieldValue <R > {
170
+
171
+ @ Override
172
+ protected Schema operatingSchema (R record ) {
173
+ return record .keySchema ();
174
+ }
175
+
176
+ @ Override
177
+ protected Object operatingValue (R record ) {
178
+ return record .key ();
179
+ }
180
+ }
181
+
182
+ public static final class Value <R extends ConnectRecord <R >> extends FilterByFieldValue <R > {
183
+
184
+ @ Override
185
+ protected Schema operatingSchema (R record ) {
186
+ return record .valueSchema ();
187
+ }
188
+
189
+ @ Override
190
+ protected Object operatingValue (R record ) {
191
+ return record .value ();
192
+ }
193
+ }
153
194
}
0 commit comments