|
| 1 | +/* |
| 2 | + * Copyright (c) 2025 Alibaba Group Holding Ltd. |
| 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 | + |
| 17 | +package com.alibaba.fluss.flink.source.deserializer; |
| 18 | + |
| 19 | +import com.alibaba.fluss.annotation.PublicEvolving; |
| 20 | +import com.alibaba.fluss.record.LogRecord; |
| 21 | +import com.alibaba.fluss.types.RowType; |
| 22 | + |
| 23 | +import org.apache.flink.api.common.typeinfo.TypeInformation; |
| 24 | +import org.apache.flink.api.common.typeinfo.Types; |
| 25 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; |
| 26 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializationFeature; |
| 27 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; |
| 28 | + |
| 29 | +import java.util.LinkedHashMap; |
| 30 | +import java.util.Map; |
| 31 | + |
| 32 | +/** |
| 33 | + * A deserialization schema that converts {@link LogRecord} objects to JSON strings. |
| 34 | + * |
| 35 | + * <p>This implementation serializes Fluss records into JSON strings, making it useful for |
| 36 | + * debugging, logging, or when the downstream processing requires string-based JSON data. The schema |
| 37 | + * preserves important metadata such as offset, timestamp, and change type along with the actual row |
| 38 | + * data. |
| 39 | + * |
| 40 | + * <p>The resulting JSON has the following structure: |
| 41 | + * |
| 42 | + * <pre>{@code |
| 43 | + * { |
| 44 | + * "offset": <record_offset>, |
| 45 | + * "timestamp": <record_timestamp>, |
| 46 | + * "changeType": <APPEND_ONLY|INSERT|UPDATE_BEFORE|UPDATE_AFTER|DELETE>, |
| 47 | + * "row": <string_representation_of_row> |
| 48 | + * } |
| 49 | + * }</pre> |
| 50 | + * |
| 51 | + * <p>Usage example: |
| 52 | + * |
| 53 | + * <pre>{@code |
| 54 | + * FlussSource<String> source = FlussSource.builder() |
| 55 | + * .setDeserializationSchema(new JsonStringDeserializationSchema()) |
| 56 | + * .build(); |
| 57 | + * }</pre> |
| 58 | + * |
| 59 | + * @since 0.7 |
| 60 | + */ |
| 61 | +@PublicEvolving |
| 62 | +public class JsonStringDeserializationSchema implements FlussDeserializationSchema<String> { |
| 63 | + private static final long serialVersionUID = 1L; |
| 64 | + |
| 65 | + /** |
| 66 | + * Jackson ObjectMapper used for JSON serialization. Marked as transient because ObjectMapper is |
| 67 | + * not serializable and needs to be recreated in the open method. |
| 68 | + */ |
| 69 | + private transient ObjectMapper objectMapper = new ObjectMapper(); |
| 70 | + |
| 71 | + /** |
| 72 | + * Reusable map for building the record representation before serializing to JSON. This avoids |
| 73 | + * creating a new Map for each record. Using LinkedHashMap to ensure a stable order of fields in |
| 74 | + * the JSON output. |
| 75 | + */ |
| 76 | + private final Map<String, Object> recordMap = new LinkedHashMap<>(4); |
| 77 | + |
| 78 | + /** |
| 79 | + * Initializes the JSON serialization mechanism. |
| 80 | + * |
| 81 | + * <p>This method creates a new ObjectMapper instance and configures it with: |
| 82 | + * |
| 83 | + * <ul> |
| 84 | + * <li>JavaTimeModule for proper serialization of date/time objects |
| 85 | + * <li>Configuration to render dates in ISO-8601 format rather than timestamps |
| 86 | + * </ul> |
| 87 | + * |
| 88 | + * @param context Contextual information for initialization (not used in this implementation) |
| 89 | + * @throws Exception if initialization fails |
| 90 | + */ |
| 91 | + @Override |
| 92 | + public void open(InitializationContext context) throws Exception { |
| 93 | + objectMapper = new ObjectMapper(); |
| 94 | + |
| 95 | + objectMapper.registerModule(new JavaTimeModule()); |
| 96 | + objectMapper.configure(SerializationFeature.WRITE_DATES_AS_TIMESTAMPS, false); |
| 97 | + } |
| 98 | + |
| 99 | + /** |
| 100 | + * Deserializes a {@link LogRecord} into a JSON {@link String}. |
| 101 | + * |
| 102 | + * <p>The method extracts key information from the record (offset, timestamp, change type, and |
| 103 | + * row data) and serializes it as a JSON string. |
| 104 | + * |
| 105 | + * @param record The Fluss LogRecord to deserialize |
| 106 | + * @return JSON string representation of the record |
| 107 | + * @throws Exception If JSON serialization fails |
| 108 | + */ |
| 109 | + @Override |
| 110 | + public String deserialize(LogRecord record) throws Exception { |
| 111 | + recordMap.put("offset", record.logOffset()); |
| 112 | + recordMap.put("timestamp", record.timestamp()); |
| 113 | + recordMap.put("change_type", record.getChangeType().toString()); |
| 114 | + // TODO: convert row into JSON https://github.com/alibaba/fluss/issues/678 |
| 115 | + recordMap.put("row", record.getRow().toString()); |
| 116 | + |
| 117 | + return objectMapper.writeValueAsString(recordMap); |
| 118 | + } |
| 119 | + |
| 120 | + /** |
| 121 | + * Returns the TypeInformation for the produced {@link String} type. |
| 122 | + * |
| 123 | + * @return TypeInformation for String class |
| 124 | + */ |
| 125 | + @Override |
| 126 | + public TypeInformation<String> getProducedType(RowType rowSchema) { |
| 127 | + return Types.STRING; |
| 128 | + } |
| 129 | +} |
0 commit comments