thesumery commented on code in PR #6215: URL: https://github.com/apache/inlong/pull/6215#discussion_r1003192882
########## inlong-sort/sort-connectors/iceberg/src/main/java/org/apache/inlong/sort/iceberg/sink/multiple/IcebergMultipleStreamWriter.java: ########## @@ -0,0 +1,227 @@ +/* + * 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.inlong.sort.iceberg.sink.multiple; + +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.TimestampedCollector; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.OutputTag; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.inlong.sort.iceberg.sink.IcebergStreamWriter; +import org.apache.inlong.sort.iceberg.sink.RowDataTaskWriterFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Map.Entry; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.TableProperties.UPSERT_ENABLED; +import static org.apache.iceberg.TableProperties.UPSERT_ENABLED_DEFAULT; +import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT; + +/** + * Iceberg writer that can distinguish different sink tables and route and distribute data into different IcebergStreamWriter. + */ +public class IcebergMultipleStreamWriter extends IcebergProcessFunction<RecordWithSchema, MultipleWriteResult> + implements CheckpointedFunction, BoundedOneInput { + private static final Logger LOG = LoggerFactory.getLogger(IcebergMultipleStreamWriter.class); + + private final boolean appendMode; + // 可以吧这里的catalogLoad封装在tableLoader中 + private final CatalogLoader catalogLoader; + + private transient Catalog catalog; + // 下面两个的key都是整个db.table的路径 + private transient Map<TableIdentifier, IcebergSingleStreamWriter<RowData>> multipleWriters; + private transient Map<TableIdentifier, Table> multipleTables; + private transient Map<TableIdentifier, Schema> multipleSchemas; + private transient FunctionInitializationContext functionInitializationContext; + + + public IcebergMultipleStreamWriter(boolean appendMode, CatalogLoader catalogLoader) { + this.appendMode = appendMode; + this.catalogLoader = catalogLoader; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.catalog = catalogLoader.loadCatalog(); + this.multipleWriters = new HashMap<>(); + this.multipleTables = new HashMap<>(); + this.multipleSchemas = new HashMap<>(); + } + + @Override + public void close() throws Exception { + if (catalog instanceof Closeable) { + ((Closeable) catalog).close(); + } + } + + @Override + public void endInput() throws Exception { + for (Entry<TableIdentifier, IcebergSingleStreamWriter<RowData>> entry: multipleWriters.entrySet()) { + entry.getValue().endInput(); + } + } + + @Override + public void dispose() throws Exception { + super.dispose(); + for (Entry<TableIdentifier, IcebergSingleStreamWriter<RowData>> entry: multipleWriters.entrySet()) { + entry.getValue().dispose(); + } + multipleWriters.clear(); + multipleTables.clear(); + multipleSchemas.clear(); + } + + @Override + public void processElement(RecordWithSchema recordWithSchema) throws Exception { + TableIdentifier tableId = recordWithSchema.getTableId(); + + if (isSchemaUpdate(recordWithSchema)) { + // 在schema变更时就应该中断之前写入的文件,然后新启动一个writer + if (multipleTables.get(tableId) == null) { + Table table = catalog.loadTable(recordWithSchema.getTableId()); + multipleTables.put(tableId, table); + } + + // refresh some runtime table properties + Table table = multipleTables.get(recordWithSchema.getTableId()); + Map<String, String> tableProperties = table.properties(); + boolean upsertMode = PropertyUtil.propertyAsBoolean(tableProperties, + UPSERT_ENABLED, UPSERT_ENABLED_DEFAULT); + long targetFileSizeBytes = PropertyUtil.propertyAsLong(tableProperties, + WRITE_TARGET_FILE_SIZE_BYTES, WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); + String formatString = tableProperties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + FileFormat fileFormat = FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH)); + List<Integer> equalityFieldIds = recordWithSchema.getPrimaryKeys().stream() + .map(pk -> recordWithSchema.getSchema().findField(pk).fieldId()) + .collect(Collectors.toList()); + // if physical primary key not exist, put all field to logical primary key + if (equalityFieldIds.isEmpty()) { + equalityFieldIds = recordWithSchema.getSchema().columns().stream() + .map(NestedField::fieldId) + .collect(Collectors.toList()); + } + + TaskWriterFactory<RowData> taskWriterFactory = new RowDataTaskWriterFactory( + table, + recordWithSchema.getSchema(), + FlinkSchemaUtil.convert(recordWithSchema.getSchema()), + targetFileSizeBytes, + fileFormat, + equalityFieldIds, + upsertMode, + appendMode); + + if (multipleWriters.get(tableId) == null) { + IcebergSingleStreamWriter<RowData> writer = new IcebergSingleStreamWriter<>( + tableId.toString(), taskWriterFactory, null, null); // todo:后面再考虑metric的兼容 + writer.setup(getRuntimeContext(), + new CallbackCollector<>(writeResult -> + collector.collect(new MultipleWriteResult(tableId, writeResult))), + context); + writer.initializeState(functionInitializationContext); + writer.open(new Configuration()); + multipleWriters.put(tableId, writer); + } else { // only if second times schema will evolute + multipleWriters.get(tableId).schemaEvolution(taskWriterFactory); + } + + } + + if (multipleWriters.get(tableId) != null) { + for (RowData data : recordWithSchema.getData()) { + multipleWriters.get(tableId).processElement(data); + } + } else { + LOG.error("Unregistered table schema for {}.", recordWithSchema.getTableId()); + } + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + for (Entry<TableIdentifier, IcebergSingleStreamWriter<RowData>> entry: multipleWriters.entrySet()) { + entry.getValue().prepareSnapshotPreBarrier(checkpointId); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + for (Entry<TableIdentifier, IcebergSingleStreamWriter<RowData>> entry: multipleWriters.entrySet()) { + entry.getValue().snapshotState(context); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + this.functionInitializationContext = context; + } + + private boolean isSchemaUpdate(RecordWithSchema recordWithSchema) { + TableIdentifier tableId = recordWithSchema.getTableId(); + recordWithSchema.replaceSchema(); + if (multipleSchemas.get(tableId) != null + && multipleSchemas.get(tableId).sameSchema(recordWithSchema.getSchema())) { + return false; + } + LOG.info("Schema evolution with table {}, old schema: {}, new Schema: {}", + tableId, multipleSchemas.get(tableId), recordWithSchema.getSchema()); + multipleSchemas.put(recordWithSchema.getTableId(), recordWithSchema.getSchema()); Review Comment: put schema in cache because every time data come into, it should know scheam update or not. If updated ,the writer will be new, and old wrtier shoule close , so it shoule cache schema to compare data schema and previous schema. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@inlong.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org