|
| 1 | +/* |
| 2 | + * Copyright 2023 Ververica Inc. |
| 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.ververica.cdc.connectors.starrocks.sink; |
| 18 | + |
| 19 | +import org.apache.flink.api.common.serialization.SerializationSchema; |
| 20 | + |
| 21 | +import com.starrocks.connector.flink.table.data.DefaultStarRocksRowData; |
| 22 | +import com.starrocks.connector.flink.table.data.StarRocksRowData; |
| 23 | +import com.starrocks.connector.flink.table.sink.v2.RecordSerializationSchema; |
| 24 | +import com.starrocks.connector.flink.table.sink.v2.StarRocksSinkContext; |
| 25 | +import com.starrocks.connector.flink.tools.JsonWrapper; |
| 26 | +import com.ververica.cdc.common.data.RecordData; |
| 27 | +import com.ververica.cdc.common.event.CreateTableEvent; |
| 28 | +import com.ververica.cdc.common.event.DataChangeEvent; |
| 29 | +import com.ververica.cdc.common.event.Event; |
| 30 | +import com.ververica.cdc.common.event.SchemaChangeEvent; |
| 31 | +import com.ververica.cdc.common.event.TableId; |
| 32 | +import com.ververica.cdc.common.schema.Column; |
| 33 | +import com.ververica.cdc.common.schema.Schema; |
| 34 | +import com.ververica.cdc.common.utils.Preconditions; |
| 35 | +import com.ververica.cdc.common.utils.SchemaUtils; |
| 36 | + |
| 37 | +import java.time.ZoneId; |
| 38 | +import java.util.HashMap; |
| 39 | +import java.util.List; |
| 40 | +import java.util.Map; |
| 41 | + |
| 42 | +import static com.ververica.cdc.connectors.starrocks.sink.StarRocksUtils.createFieldGetter; |
| 43 | + |
| 44 | +/** Serializer for the input {@link Event}. It will serialize a row to a json string. */ |
| 45 | +public class EventRecordSerializationSchema implements RecordSerializationSchema<Event> { |
| 46 | + |
| 47 | + private static final long serialVersionUID = 1L; |
| 48 | + |
| 49 | + /** |
| 50 | + * The local time zone used when converting from <code>TIMESTAMP WITH LOCAL TIME ZONE</code>. |
| 51 | + */ |
| 52 | + private final ZoneId zoneId; |
| 53 | + |
| 54 | + /** keep the relationship of TableId and table information. */ |
| 55 | + private transient Map<TableId, TableInfo> tableInfoMap; |
| 56 | + |
| 57 | + private transient DefaultStarRocksRowData reusableRowData; |
| 58 | + private transient JsonWrapper jsonWrapper; |
| 59 | + |
| 60 | + public EventRecordSerializationSchema(ZoneId zoneId) { |
| 61 | + this.zoneId = zoneId; |
| 62 | + } |
| 63 | + |
| 64 | + @Override |
| 65 | + public void open( |
| 66 | + SerializationSchema.InitializationContext context, StarRocksSinkContext sinkContext) { |
| 67 | + this.tableInfoMap = new HashMap<>(); |
| 68 | + this.reusableRowData = new DefaultStarRocksRowData(); |
| 69 | + this.jsonWrapper = new JsonWrapper(); |
| 70 | + } |
| 71 | + |
| 72 | + @Override |
| 73 | + public StarRocksRowData serialize(Event record) { |
| 74 | + if (record instanceof SchemaChangeEvent) { |
| 75 | + applySchemaChangeEvent((SchemaChangeEvent) record); |
| 76 | + return null; |
| 77 | + } else if (record instanceof DataChangeEvent) { |
| 78 | + return applyDataChangeEvent((DataChangeEvent) record); |
| 79 | + } else { |
| 80 | + throw new UnsupportedOperationException("Don't support event " + record); |
| 81 | + } |
| 82 | + } |
| 83 | + |
| 84 | + private void applySchemaChangeEvent(SchemaChangeEvent event) { |
| 85 | + TableId tableId = event.tableId(); |
| 86 | + Schema newSchema; |
| 87 | + if (event instanceof CreateTableEvent) { |
| 88 | + newSchema = ((CreateTableEvent) event).getSchema(); |
| 89 | + } else { |
| 90 | + TableInfo tableInfo = tableInfoMap.get(tableId); |
| 91 | + if (tableInfo == null) { |
| 92 | + throw new RuntimeException("schema of " + tableId + " is not existed."); |
| 93 | + } |
| 94 | + newSchema = SchemaUtils.applySchemaChangeEvent(tableInfo.schema, event); |
| 95 | + } |
| 96 | + TableInfo tableInfo = new TableInfo(); |
| 97 | + tableInfo.schema = newSchema; |
| 98 | + tableInfo.fieldGetters = new RecordData.FieldGetter[newSchema.getColumnCount()]; |
| 99 | + for (int i = 0; i < newSchema.getColumnCount(); i++) { |
| 100 | + tableInfo.fieldGetters[i] = |
| 101 | + createFieldGetter(newSchema.getColumns().get(i).getType(), i, zoneId); |
| 102 | + } |
| 103 | + tableInfoMap.put(tableId, tableInfo); |
| 104 | + } |
| 105 | + |
| 106 | + private StarRocksRowData applyDataChangeEvent(DataChangeEvent event) { |
| 107 | + TableInfo tableInfo = tableInfoMap.get(event.tableId()); |
| 108 | + Preconditions.checkNotNull(tableInfo, event.tableId() + " is not existed"); |
| 109 | + reusableRowData.setDatabase(event.tableId().getSchemaName()); |
| 110 | + reusableRowData.setTable(event.tableId().getTableName()); |
| 111 | + String value; |
| 112 | + switch (event.op()) { |
| 113 | + case INSERT: |
| 114 | + case UPDATE: |
| 115 | + case REPLACE: |
| 116 | + value = serializeRecord(tableInfo, event.after(), false); |
| 117 | + break; |
| 118 | + case DELETE: |
| 119 | + value = serializeRecord(tableInfo, event.before(), true); |
| 120 | + break; |
| 121 | + default: |
| 122 | + throw new UnsupportedOperationException( |
| 123 | + "Don't support operation type " + event.op()); |
| 124 | + } |
| 125 | + reusableRowData.setRow(value); |
| 126 | + return reusableRowData; |
| 127 | + } |
| 128 | + |
| 129 | + private String serializeRecord(TableInfo tableInfo, RecordData record, boolean isDelete) { |
| 130 | + List<Column> columns = tableInfo.schema.getColumns(); |
| 131 | + Preconditions.checkArgument(columns.size() == record.getArity()); |
| 132 | + Map<String, Object> rowMap = new HashMap<>(record.getArity() + 1); |
| 133 | + for (int i = 0; i < record.getArity(); i++) { |
| 134 | + rowMap.put(columns.get(i).getName(), tableInfo.fieldGetters[i].getFieldOrNull(record)); |
| 135 | + } |
| 136 | + rowMap.put("__op", isDelete ? 1 : 0); |
| 137 | + return jsonWrapper.toJSONString(rowMap); |
| 138 | + } |
| 139 | + |
| 140 | + @Override |
| 141 | + public void close() {} |
| 142 | + |
| 143 | + /** Table information. */ |
| 144 | + private static class TableInfo { |
| 145 | + Schema schema; |
| 146 | + RecordData.FieldGetter[] fieldGetters; |
| 147 | + } |
| 148 | +} |
0 commit comments