forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 2
/
CanalJsonFormatFactory.java
156 lines (135 loc) · 6.77 KB
/
CanalJsonFormatFactory.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
/*
* 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.flink.formats.json.canal;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.formats.common.TimestampFormat;
import org.apache.flink.formats.json.JsonFormatOptions;
import org.apache.flink.formats.json.JsonFormatOptionsUtil;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.format.EncodingFormat;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.factories.DeserializationFormatFactory;
import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.factories.SerializationFormatFactory;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.types.RowKind;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER;
import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_IGNORE_NULL_FIELDS;
import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.DATABASE_INCLUDE;
import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.IGNORE_PARSE_ERRORS;
import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL;
import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.JSON_MAP_NULL_KEY_MODE;
import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.TABLE_INCLUDE;
import static org.apache.flink.formats.json.canal.CanalJsonFormatOptions.TIMESTAMP_FORMAT;
/**
* Format factory for providing configured instances of Canal JSON to RowData {@link
* DeserializationSchema}.
*/
@Internal
public class CanalJsonFormatFactory
implements DeserializationFormatFactory, SerializationFormatFactory {
public static final String IDENTIFIER = "canal-json";
@Override
public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(
DynamicTableFactory.Context context, ReadableConfig formatOptions) {
FactoryUtil.validateFactoryOptions(this, formatOptions);
validateDecodingFormatOptions(formatOptions);
final String database = formatOptions.getOptional(DATABASE_INCLUDE).orElse(null);
final String table = formatOptions.getOptional(TABLE_INCLUDE).orElse(null);
final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS);
final TimestampFormat timestampFormat =
JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
return new CanalJsonDecodingFormat(database, table, ignoreParseErrors, timestampFormat);
}
@Override
public EncodingFormat<SerializationSchema<RowData>> createEncodingFormat(
DynamicTableFactory.Context context, ReadableConfig formatOptions) {
FactoryUtil.validateFactoryOptions(this, formatOptions);
validateEncodingFormatOptions(formatOptions);
TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
JsonFormatOptions.MapNullKeyMode mapNullKeyMode =
JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions);
String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL);
final boolean encodeDecimalAsPlainNumber =
formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
final boolean ignoreNullFields = formatOptions.get(ENCODE_IGNORE_NULL_FIELDS);
return new EncodingFormat<SerializationSchema<RowData>>() {
@Override
public ChangelogMode getChangelogMode() {
return ChangelogMode.newBuilder()
.addContainedKind(RowKind.INSERT)
.addContainedKind(RowKind.UPDATE_BEFORE)
.addContainedKind(RowKind.UPDATE_AFTER)
.addContainedKind(RowKind.DELETE)
.build();
}
@Override
public SerializationSchema<RowData> createRuntimeEncoder(
DynamicTableSink.Context context, DataType consumedDataType) {
final RowType rowType = (RowType) consumedDataType.getLogicalType();
return new CanalJsonSerializationSchema(
rowType,
timestampFormat,
mapNullKeyMode,
mapNullKeyLiteral,
encodeDecimalAsPlainNumber,
ignoreNullFields);
}
};
}
@Override
public String factoryIdentifier() {
return IDENTIFIER;
}
@Override
public Set<ConfigOption<?>> requiredOptions() {
return Collections.emptySet();
}
@Override
public Set<ConfigOption<?>> optionalOptions() {
Set<ConfigOption<?>> options = new HashSet<>();
options.add(IGNORE_PARSE_ERRORS);
options.add(TIMESTAMP_FORMAT);
options.add(DATABASE_INCLUDE);
options.add(TABLE_INCLUDE);
options.add(JSON_MAP_NULL_KEY_MODE);
options.add(JSON_MAP_NULL_KEY_LITERAL);
options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
options.add(ENCODE_IGNORE_NULL_FIELDS);
return options;
}
/** Validator for canal decoding format. */
private static void validateDecodingFormatOptions(ReadableConfig tableOptions) {
JsonFormatOptionsUtil.validateDecodingFormatOptions(tableOptions);
}
/** Validator for canal encoding format. */
private static void validateEncodingFormatOptions(ReadableConfig tableOptions) {
JsonFormatOptionsUtil.validateEncodingFormatOptions(tableOptions);
}
}