|
| 1 | +/* |
| 2 | + * Copyright © 2024 DataSQRL (contact@datasqrl.com) |
| 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 | +package com.datasqrl.flink.format.json; |
| 17 | + |
| 18 | +import static org.apache.flink.formats.json.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER; |
| 19 | +import static org.apache.flink.formats.json.JsonFormatOptions.MAP_NULL_KEY_LITERAL; |
| 20 | + |
| 21 | +import java.util.Set; |
| 22 | +import lombok.SneakyThrows; |
| 23 | +import org.apache.flink.api.common.serialization.DeserializationSchema; |
| 24 | +import org.apache.flink.api.common.serialization.SerializationSchema; |
| 25 | +import org.apache.flink.api.common.typeinfo.TypeInformation; |
| 26 | +import org.apache.flink.configuration.ConfigOption; |
| 27 | +import org.apache.flink.configuration.ReadableConfig; |
| 28 | +import org.apache.flink.formats.common.TimestampFormat; |
| 29 | +import org.apache.flink.formats.json.JsonFormatFactory; |
| 30 | +import org.apache.flink.formats.json.JsonFormatOptions; |
| 31 | +import org.apache.flink.formats.json.JsonFormatOptionsUtil; |
| 32 | +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; |
| 33 | +import org.apache.flink.table.connector.ChangelogMode; |
| 34 | +import org.apache.flink.table.connector.Projection; |
| 35 | +import org.apache.flink.table.connector.format.DecodingFormat; |
| 36 | +import org.apache.flink.table.connector.format.EncodingFormat; |
| 37 | +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; |
| 38 | +import org.apache.flink.table.connector.sink.DynamicTableSink; |
| 39 | +import org.apache.flink.table.connector.source.DynamicTableSource; |
| 40 | +import org.apache.flink.table.data.RowData; |
| 41 | +import org.apache.flink.table.factories.DeserializationFormatFactory; |
| 42 | +import org.apache.flink.table.factories.DynamicTableFactory.Context; |
| 43 | +import org.apache.flink.table.factories.FactoryUtil; |
| 44 | +import org.apache.flink.table.factories.SerializationFormatFactory; |
| 45 | +import org.apache.flink.table.types.DataType; |
| 46 | +import org.apache.flink.table.types.logical.RowType; |
| 47 | + |
| 48 | +public class FlexibleJsonFormat |
| 49 | + implements DeserializationFormatFactory, SerializationFormatFactory { |
| 50 | + |
| 51 | + public static final String FORMAT_NAME = "flexible-json"; |
| 52 | + |
| 53 | + /** |
| 54 | + * This just delegates to the "standard" json format in Flink |
| 55 | + * |
| 56 | + * @param context |
| 57 | + * @param formatOptions |
| 58 | + * @return |
| 59 | + */ |
| 60 | + @Override |
| 61 | + public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat( |
| 62 | + Context context, ReadableConfig formatOptions) { |
| 63 | + FactoryUtil.validateFactoryOptions(this, formatOptions); |
| 64 | + |
| 65 | + return new ProjectableDecodingFormat<DeserializationSchema<RowData>>() { |
| 66 | + @SneakyThrows |
| 67 | + @Override |
| 68 | + public DeserializationSchema<RowData> createRuntimeDecoder( |
| 69 | + DynamicTableSource.Context context, DataType physicalDataType, int[][] projections) { |
| 70 | + final DataType producedDataType = Projection.of(projections).project(physicalDataType); |
| 71 | + final RowType rowType = (RowType) producedDataType.getLogicalType(); |
| 72 | + final TypeInformation<RowData> rowDataTypeInfo = |
| 73 | + context.createTypeInformation(producedDataType); |
| 74 | + JsonRowDataDeserializationSchema jsonRowDataDeserializationSchema = |
| 75 | + new JsonRowDataDeserializationSchema( |
| 76 | + rowType, rowDataTypeInfo, false, false, TimestampFormat.ISO_8601); |
| 77 | + return jsonRowDataDeserializationSchema; |
| 78 | + } |
| 79 | + |
| 80 | + @Override |
| 81 | + public ChangelogMode getChangelogMode() { |
| 82 | + return ChangelogMode.insertOnly(); |
| 83 | + } |
| 84 | + }; |
| 85 | + } |
| 86 | + |
| 87 | + /** |
| 88 | + * This uses a SQRL specific encoding format so that we can add support for SQRL types |
| 89 | + * |
| 90 | + * @param context |
| 91 | + * @param formatOptions |
| 92 | + * @return |
| 93 | + */ |
| 94 | + @Override |
| 95 | + public EncodingFormat<SerializationSchema<RowData>> createEncodingFormat( |
| 96 | + Context context, ReadableConfig formatOptions) { |
| 97 | + FactoryUtil.validateFactoryOptions(this, formatOptions); |
| 98 | + JsonFormatOptionsUtil.validateEncodingFormatOptions(formatOptions); |
| 99 | + |
| 100 | + TimestampFormat timestampOption = JsonFormatOptionsUtil.getTimestampFormat(formatOptions); |
| 101 | + JsonFormatOptions.MapNullKeyMode mapNullKeyMode = |
| 102 | + JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions); |
| 103 | + String mapNullKeyLiteral = formatOptions.get(MAP_NULL_KEY_LITERAL); |
| 104 | + |
| 105 | + final boolean encodeDecimalAsPlainNumber = formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER); |
| 106 | + |
| 107 | + return new EncodingFormat<SerializationSchema<RowData>>() { |
| 108 | + @Override |
| 109 | + public SerializationSchema<RowData> createRuntimeEncoder( |
| 110 | + DynamicTableSink.Context context, DataType consumedDataType) { |
| 111 | + final RowType rowType = (RowType) consumedDataType.getLogicalType(); |
| 112 | + return new SqrlJsonRowDataSerializationSchema( |
| 113 | + rowType, |
| 114 | + timestampOption, |
| 115 | + mapNullKeyMode, |
| 116 | + mapNullKeyLiteral, |
| 117 | + encodeDecimalAsPlainNumber); |
| 118 | + } |
| 119 | + |
| 120 | + @Override |
| 121 | + public ChangelogMode getChangelogMode() { |
| 122 | + return ChangelogMode.insertOnly(); |
| 123 | + } |
| 124 | + }; |
| 125 | + } |
| 126 | + |
| 127 | + @Override |
| 128 | + public String factoryIdentifier() { |
| 129 | + return FORMAT_NAME; |
| 130 | + } |
| 131 | + |
| 132 | + @Override |
| 133 | + public Set<ConfigOption<?>> requiredOptions() { |
| 134 | + return new JsonFormatFactory().requiredOptions(); |
| 135 | + } |
| 136 | + |
| 137 | + @Override |
| 138 | + public Set<ConfigOption<?>> optionalOptions() { |
| 139 | + return new JsonFormatFactory().optionalOptions(); |
| 140 | + } |
| 141 | +} |
0 commit comments