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,15 +43,23 @@ 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 ) {
57
+
58
+ protected abstract Schema operatingSchema (R record );
59
+
60
+ protected abstract Object operatingValue (R record );
61
+
62
+ private R applyWithSchema (final R record ) {
55
63
final Struct struct = (Struct ) record .value ();
56
64
final Optional <String > fieldValue = extractStructFieldValue (struct , fieldName );
57
65
return filterCondition .test (fieldValue ) ? record : null ;
@@ -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 .isEmpty ()) {
85
+ final Optional <String > value = extractSchemalessFieldValue (operatingValue (record ));
86
+ return filterCondition .test (value ) ? record : null ;
87
+ } else {
88
+ final Map <String , Object > map = (Map <String , Object >) record .value ();
89
+ final Optional <String > fieldValue = extractSchemalessFieldValue (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 > extractSchemalessFieldValue ( 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 ());
@@ -150,4 +158,31 @@ public void configure(final Map<String, ?> configs) {
150
158
? matchCondition
151
159
: (result -> !matchCondition .test (result ));
152
160
}
161
+
162
+
163
+ public static final class Key <R extends ConnectRecord <R >> extends FilterByFieldValue <R > {
164
+
165
+ @ Override
166
+ protected Schema operatingSchema (R record ) {
167
+ return record .keySchema ();
168
+ }
169
+
170
+ @ Override
171
+ protected Object operatingValue (R record ) {
172
+ return record .key ();
173
+ }
174
+ }
175
+
176
+ public static final class Value <R extends ConnectRecord <R >> extends FilterByFieldValue <R > {
177
+
178
+ @ Override
179
+ protected Schema operatingSchema (R record ) {
180
+ return record .valueSchema ();
181
+ }
182
+
183
+ @ Override
184
+ protected Object operatingValue (R record ) {
185
+ return record .value ();
186
+ }
187
+ }
153
188
}
0 commit comments