30
30
import org .apache .flink .types .RowKind ;
31
31
import org .apache .flink .util .Collector ;
32
32
33
- import org .apache .flink .shaded .jackson2 .com .fasterxml .jackson .databind .JsonNode ;
34
-
35
33
import com .ververica .cdc .debezium .DebeziumDeserializationSchema ;
36
34
import com .ververica .cdc .debezium .utils .TemporalConversions ;
37
35
import io .debezium .data .Envelope ;
55
53
import java .time .LocalDateTime ;
56
54
import java .time .ZoneId ;
57
55
56
+ import static org .apache .flink .util .Preconditions .checkNotNull ;
57
+
58
58
/**
59
59
* Deserialization schema from Debezium object to Flink Table/SQL internal data structure {@link
60
60
* RowData}.
61
61
*/
62
62
public final class RowDataDebeziumDeserializeSchema
63
63
implements DebeziumDeserializationSchema <RowData > {
64
- private static final long serialVersionUID = - 4852684966051743776L ;
64
+ private static final long serialVersionUID = 2L ;
65
65
66
66
/** Custom validator to validate the row value. */
67
67
public interface ValueValidator extends Serializable {
@@ -72,26 +72,42 @@ public interface ValueValidator extends Serializable {
72
72
private final TypeInformation <RowData > resultTypeInfo ;
73
73
74
74
/**
75
- * Runtime converter that converts {@link JsonNode }s into objects of Flink SQL internal data
76
- * structures. *
75
+ * Runtime converter that converts Kafka {@link SourceRecord }s into {@link RowData} consisted of
76
+ * physical column values.
77
77
*/
78
- private final DeserializationRuntimeConverter runtimeConverter ;
78
+ private final DeserializationRuntimeConverter physicalConverter ;
79
+
80
+ /** Whether the deserializer needs to handle metadata columns. */
81
+ private final boolean hasMetadata ;
82
+
83
+ /**
84
+ * A wrapped output collector which is used to append metadata columns after physical columns.
85
+ */
86
+ private final AppendMetadataCollector appendMetadataCollector ;
79
87
80
88
/** Time zone of the database server. */
81
89
private final ZoneId serverTimeZone ;
82
90
83
91
/** Validator to validate the row value. */
84
92
private final ValueValidator validator ;
85
93
86
- public RowDataDebeziumDeserializeSchema (
87
- RowType rowType ,
94
+ /** Returns a builder to build {@link RowDataDebeziumDeserializeSchema}. */
95
+ public static Builder newBuilder () {
96
+ return new Builder ();
97
+ }
98
+
99
+ RowDataDebeziumDeserializeSchema (
100
+ RowType physicalDataType ,
101
+ MetadataConverter [] metadataConverters ,
88
102
TypeInformation <RowData > resultTypeInfo ,
89
103
ValueValidator validator ,
90
104
ZoneId serverTimeZone ) {
91
- this .runtimeConverter = createConverter (rowType );
92
- this .resultTypeInfo = resultTypeInfo ;
93
- this .validator = validator ;
94
- this .serverTimeZone = serverTimeZone ;
105
+ this .hasMetadata = checkNotNull (metadataConverters ).length > 0 ;
106
+ this .appendMetadataCollector = new AppendMetadataCollector (metadataConverters );
107
+ this .physicalConverter = createConverter (checkNotNull (physicalDataType ));
108
+ this .resultTypeInfo = checkNotNull (resultTypeInfo );
109
+ this .validator = checkNotNull (validator );
110
+ this .serverTimeZone = checkNotNull (serverTimeZone );
95
111
}
96
112
97
113
@ Override
@@ -103,42 +119,96 @@ public void deserialize(SourceRecord record, Collector<RowData> out) throws Exce
103
119
GenericRowData insert = extractAfterRow (value , valueSchema );
104
120
validator .validate (insert , RowKind .INSERT );
105
121
insert .setRowKind (RowKind .INSERT );
106
- out . collect ( insert );
122
+ emit ( record , insert , out );
107
123
} else if (op == Envelope .Operation .DELETE ) {
108
124
GenericRowData delete = extractBeforeRow (value , valueSchema );
109
125
validator .validate (delete , RowKind .DELETE );
110
126
delete .setRowKind (RowKind .DELETE );
111
- out . collect ( delete );
127
+ emit ( record , delete , out );
112
128
} else {
113
129
GenericRowData before = extractBeforeRow (value , valueSchema );
114
130
validator .validate (before , RowKind .UPDATE_BEFORE );
115
131
before .setRowKind (RowKind .UPDATE_BEFORE );
116
- out . collect ( before );
132
+ emit ( record , before , out );
117
133
118
134
GenericRowData after = extractAfterRow (value , valueSchema );
119
135
validator .validate (after , RowKind .UPDATE_AFTER );
120
136
after .setRowKind (RowKind .UPDATE_AFTER );
121
- out . collect ( after );
137
+ emit ( record , after , out );
122
138
}
123
139
}
124
140
125
141
private GenericRowData extractAfterRow (Struct value , Schema valueSchema ) throws Exception {
126
142
Schema afterSchema = valueSchema .field (Envelope .FieldName .AFTER ).schema ();
127
143
Struct after = value .getStruct (Envelope .FieldName .AFTER );
128
- return (GenericRowData ) runtimeConverter .convert (after , afterSchema );
144
+ return (GenericRowData ) physicalConverter .convert (after , afterSchema );
129
145
}
130
146
131
147
private GenericRowData extractBeforeRow (Struct value , Schema valueSchema ) throws Exception {
132
148
Schema beforeSchema = valueSchema .field (Envelope .FieldName .BEFORE ).schema ();
133
149
Struct before = value .getStruct (Envelope .FieldName .BEFORE );
134
- return (GenericRowData ) runtimeConverter .convert (before , beforeSchema );
150
+ return (GenericRowData ) physicalConverter .convert (before , beforeSchema );
151
+ }
152
+
153
+ private void emit (SourceRecord inRecord , RowData physicalRow , Collector <RowData > collector ) {
154
+ if (!hasMetadata ) {
155
+ collector .collect (physicalRow );
156
+ return ;
157
+ }
158
+
159
+ appendMetadataCollector .inputRecord = inRecord ;
160
+ appendMetadataCollector .outputCollector = collector ;
161
+ appendMetadataCollector .collect (physicalRow );
135
162
}
136
163
137
164
@ Override
138
165
public TypeInformation <RowData > getProducedType () {
139
166
return resultTypeInfo ;
140
167
}
141
168
169
+ // -------------------------------------------------------------------------------------
170
+ // Builder
171
+ // -------------------------------------------------------------------------------------
172
+
173
+ /** Builder of {@link RowDataDebeziumDeserializeSchema}. */
174
+ public static class Builder {
175
+ private RowType physicalRowType ;
176
+ private TypeInformation <RowData > resultTypeInfo ;
177
+ private MetadataConverter [] metadataConverters = new MetadataConverter [0 ];
178
+ private ValueValidator validator = (rowData , rowKind ) -> {};
179
+ private ZoneId serverTimeZone = ZoneId .of ("UTC" );
180
+
181
+ public Builder setPhysicalRowType (RowType physicalRowType ) {
182
+ this .physicalRowType = physicalRowType ;
183
+ return this ;
184
+ }
185
+
186
+ public Builder setMetadataConverters (MetadataConverter [] metadataConverters ) {
187
+ this .metadataConverters = metadataConverters ;
188
+ return this ;
189
+ }
190
+
191
+ public Builder setResultTypeInfo (TypeInformation <RowData > resultTypeInfo ) {
192
+ this .resultTypeInfo = resultTypeInfo ;
193
+ return this ;
194
+ }
195
+
196
+ public Builder setValueValidator (ValueValidator validator ) {
197
+ this .validator = validator ;
198
+ return this ;
199
+ }
200
+
201
+ public Builder setServerTimeZone (ZoneId serverTimeZone ) {
202
+ this .serverTimeZone = serverTimeZone ;
203
+ return this ;
204
+ }
205
+
206
+ public RowDataDebeziumDeserializeSchema build () {
207
+ return new RowDataDebeziumDeserializeSchema (
208
+ physicalRowType , metadataConverters , resultTypeInfo , validator , serverTimeZone );
209
+ }
210
+ }
211
+
142
212
// -------------------------------------------------------------------------------------
143
213
// Runtime Converters
144
214
// -------------------------------------------------------------------------------------
0 commit comments