CloverDew commented on code in PR #10107: URL: https://github.com/apache/seatunnel/pull/10107#discussion_r2623052342
########## seatunnel-translation/seatunnel-translation-flink/seatunnel-translation-flink-common/src/main/java/org/apache/seatunnel/translation/flink/schema/SchemaOperator.java: ########## @@ -0,0 +1,301 @@ +/* + * 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.seatunnel.translation.flink.schema; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.source.SupportSchemaEvolution; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.schema.SchemaChangeType; +import org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent; +import org.apache.seatunnel.api.table.schema.event.TableEvent; +import org.apache.seatunnel.api.table.schema.exception.SchemaValidationException; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.flink.schema.coordinator.LocalSchemaCoordinator; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; + +/** operators added to the source and transformer pipelines to handle schema evolution */ +@Slf4j +public class SchemaOperator extends AbstractStreamOperator<SeaTunnelRow> + implements OneInputStreamOperator<SeaTunnelRow, SeaTunnelRow> { + private final Map<TableIdentifier, CatalogTable> localSchemaState; + private String jobId; + private final SupportSchemaEvolution source; + private final Config pluginConfig; + private volatile Long lastProcessedEventTime; + private transient LocalSchemaCoordinator coordinator; + private transient Map<String, List<BufferedDataRow>> bufferedDataRows; + private volatile boolean schemaChangePending = false; + private volatile CompletableFuture<Boolean> pendingSchemaFuture = null; + + public SchemaOperator(String jobId, SupportSchemaEvolution source, Config pluginConfig) { + this.jobId = jobId; + this.source = source; + this.pluginConfig = pluginConfig; + this.localSchemaState = new ConcurrentHashMap<>(); + } + + @Override + public void open() throws Exception { + super.open(); + String flinkJobId = getRuntimeContext().getJobId().toString(); + if (!flinkJobId.equals(this.jobId)) { + this.jobId = flinkJobId; + } + this.bufferedDataRows = new ConcurrentHashMap<>(); + this.coordinator = LocalSchemaCoordinator.getInstance(this.jobId); + log.info("SchemaOperator opened for job: {}", this.jobId); + } + + @Override + public void processElement(StreamRecord<SeaTunnelRow> streamRecord) { + SeaTunnelRow element = streamRecord.getValue(); + + if (pluginConfig.hasPath("schema-changes.enabled")) { + output.collect(new StreamRecord<>(element, streamRecord.getTimestamp())); + return; + } + + if ("__SCHEMA_CHANGE_EVENT__".equals(element.getTableId()) + && element.getOptions() != null) { + Object object = element.getOptions().get("schema_change_event"); + if (object instanceof SchemaChangeEvent) { + handleSchemaChangeEvent((SchemaChangeEvent) object); + return; + } + } + + if (schemaChangePending && pendingSchemaFuture != null) { + String tableId = element.getTableId(); + if (tableId != null) { + String key = tableId + "#" + System.currentTimeMillis(); + bufferedDataRows + .computeIfAbsent(key, k -> new ArrayList<>()) + .add(new BufferedDataRow(element, streamRecord.getTimestamp())); + return; + } + } + + output.collect(new StreamRecord<>(element, streamRecord.getTimestamp())); + } + + private void handleSchemaChangeEvent(SchemaChangeEvent schemaChangeEvent) { + List<SchemaChangeType> supportedTypes = source.supports(); + if (supportedTypes == null || supportedTypes.isEmpty()) { + log.info( + "Source: {} does not support any schema change types, skipping schema change event", + source); + return; + } + + if (!isSchemaChangeSupported(schemaChangeEvent, supportedTypes)) { + log.warn( + "Schema change type {} not supported by source {}, skipping", + schemaChangeEvent.getEventType(), + source); + return; + } + + processSchemaChangeEvent(schemaChangeEvent); + } + + private boolean isSchemaChangeSupported( + SchemaChangeEvent event, List<SchemaChangeType> supportedTypes) { + switch (event.getEventType()) { + case SCHEMA_CHANGE_ADD_COLUMN: + return supportedTypes.contains(SchemaChangeType.ADD_COLUMN); + case SCHEMA_CHANGE_DROP_COLUMN: + return supportedTypes.contains(SchemaChangeType.DROP_COLUMN); + case SCHEMA_CHANGE_MODIFY_COLUMN: + return supportedTypes.contains(SchemaChangeType.UPDATE_COLUMN); + case SCHEMA_CHANGE_CHANGE_COLUMN: + return supportedTypes.contains(SchemaChangeType.RENAME_COLUMN); + case SCHEMA_CHANGE_UPDATE_COLUMNS: + return supportedTypes.contains(SchemaChangeType.ADD_COLUMN) + || supportedTypes.contains(SchemaChangeType.DROP_COLUMN) + || supportedTypes.contains(SchemaChangeType.UPDATE_COLUMN) + || supportedTypes.contains(SchemaChangeType.RENAME_COLUMN); + default: + log.error("Unknown schema change event type: {}", event.getEventType()); + throw SchemaValidationException.unsupportedChangeType( + event.tableIdentifier(), jobId); + } + } + + private void processSchemaChangeEvent(SchemaChangeEvent schemaChangeEvent) { + TableIdentifier tableId = schemaChangeEvent.tableIdentifier(); + long eventTime = schemaChangeEvent.getCreatedTime(); + + try { + if (lastProcessedEventTime != null && eventTime <= lastProcessedEventTime) { Review Comment: Each time a change event is processed, `lastProcessedEventTime` is updated. Any event with a timestamp less than or equal to the current watermark is rejected; otherwise, it would lead to inconsistencies in the evolution process. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
