loserwang1024 commented on code in PR #3442:
URL: https://github.com/apache/flink-cdc/pull/3442#discussion_r1664979827


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresEventDeserializer.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.flink.cdc.connectors.postgres.source;
+
+import org.apache.flink.cdc.common.annotation.Internal;
+import org.apache.flink.cdc.common.event.SchemaChangeEvent;
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.debezium.event.DebeziumEventDeserializationSchema;
+import org.apache.flink.cdc.debezium.event.DebeziumSchemaDataTypeInference;
+import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.debezium.data.Envelope;
+import io.debezium.relational.Tables;
+import org.apache.kafka.connect.data.Schema;
+import org.apache.kafka.connect.data.Struct;
+import org.apache.kafka.connect.source.SourceRecord;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/** Event deserializer for {@link PostgresDataSource}. */
+@Internal
+public class PostgresEventDeserializer extends 
DebeziumEventDeserializationSchema {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final String SCHEMA_CHANGE_EVENT_KEY_NAME =
+            "io.debezium.connector.postgres.SchemaChangeKey";
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    private final boolean includeSchemaChanges;
+
+    private transient Tables tables;
+
+    public PostgresEventDeserializer(
+            DebeziumChangelogMode changelogMode, boolean includeSchemaChanges) 
{
+        super(new DebeziumSchemaDataTypeInference(), changelogMode);
+        this.includeSchemaChanges = includeSchemaChanges;
+    }
+
+    @Override
+    protected List<SchemaChangeEvent> 
deserializeSchemaChangeRecord(SourceRecord record) {
+        return Collections.emptyList();
+    }
+
+    @Override
+    protected boolean isDataChangeRecord(SourceRecord record) {
+        Schema valueSchema = record.valueSchema();
+        Struct value = (Struct) record.value();
+        return value != null
+                && valueSchema != null
+                && valueSchema.field(Envelope.FieldName.OPERATION) != null
+                && value.getString(Envelope.FieldName.OPERATION) != null;
+    }
+
+    @Override
+    protected boolean isSchemaChangeRecord(SourceRecord record) {
+        Schema keySchema = record.keySchema();

Review Comment:
   Does postgres have schema change record?



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/source/reader/PostgresPipelineRecordEmitter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.flink.cdc.connectors.postgres.source.reader;
+
+import org.apache.flink.api.connector.source.SourceOutput;
+import org.apache.flink.cdc.common.event.CreateTableEvent;
+import org.apache.flink.cdc.common.event.Event;
+import org.apache.flink.cdc.common.schema.Schema;
+import org.apache.flink.cdc.connectors.base.options.StartupMode;
+import org.apache.flink.cdc.connectors.base.source.meta.offset.OffsetFactory;
+import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitState;
+import org.apache.flink.cdc.connectors.base.source.metrics.SourceReaderMetrics;
+import 
org.apache.flink.cdc.connectors.base.source.reader.IncrementalSourceRecordEmitter;
+import org.apache.flink.cdc.connectors.postgres.source.PostgresDialect;
+import 
org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceConfig;
+import 
org.apache.flink.cdc.connectors.postgres.source.utils.TableDiscoveryUtils;
+import org.apache.flink.cdc.connectors.postgres.utils.PostgresSchemaUtils;
+import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema;
+import org.apache.flink.connector.base.source.reader.RecordEmitter;
+
+import io.debezium.connector.postgresql.connection.PostgresConnection;
+import io.debezium.relational.TableId;
+import org.apache.kafka.connect.source.SourceRecord;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static 
org.apache.flink.cdc.connectors.base.source.meta.wartermark.WatermarkEvent.isLowWatermarkEvent;
+
+/** The {@link RecordEmitter} implementation for pipeline oracle connector. */
+public class PostgresPipelineRecordEmitter<T> extends 
IncrementalSourceRecordEmitter<T> {
+    private final PostgresSourceConfig sourceConfig;
+    private final PostgresDialect postgresDialect;
+
+    // Used when startup mode is initial
+    private Set<TableId> alreadySendCreateTableTables;
+
+    // Used when startup mode is not initial
+    private boolean alreadySendCreateTableForBinlogSplit = false;
+    private final List<CreateTableEvent> createTableEventCache;
+
+    public PostgresPipelineRecordEmitter(
+            DebeziumDeserializationSchema debeziumDeserializationSchema,
+            SourceReaderMetrics sourceReaderMetrics,
+            PostgresSourceConfig sourceConfig,
+            OffsetFactory offsetFactory,
+            PostgresDialect postgresDialect) {
+        super(
+                debeziumDeserializationSchema,
+                sourceReaderMetrics,
+                sourceConfig.isIncludeSchemaChanges(),
+                offsetFactory);
+        this.sourceConfig = sourceConfig;
+        this.postgresDialect = postgresDialect;
+        this.alreadySendCreateTableTables = new HashSet<>();
+        this.createTableEventCache = new ArrayList<>();
+
+        if 
(!sourceConfig.getStartupOptions().startupMode.equals(StartupMode.INITIAL)) {
+            try (PostgresConnection jdbc = 
postgresDialect.openJdbcConnection()) {
+                List<TableId> capturedTableIds =
+                        TableDiscoveryUtils.listTables(
+                                sourceConfig.getDatabaseList().get(0),
+                                jdbc,
+                                sourceConfig.getTableFilters());
+                for (TableId tableId : capturedTableIds) {
+                    Schema schema = 
PostgresSchemaUtils.getTableSchema(tableId, sourceConfig, jdbc);
+                    createTableEventCache.add(
+                            new CreateTableEvent(
+                                    
org.apache.flink.cdc.common.event.TableId.tableId(
+                                            tableId.schema(), tableId.table()),
+                                    schema));
+                }
+            } catch (SQLException e) {
+                throw new RuntimeException("Cannot start emitter to fetch 
table schema.", e);
+            }
+        }
+    }
+
+    @Override
+    protected void processElement(
+            SourceRecord element, SourceOutput<T> output, SourceSplitState 
splitState)
+            throws Exception {
+        if (isLowWatermarkEvent(element) && splitState.isSnapshotSplitState()) 
{
+            TableId tableId = 
splitState.asSnapshotSplitState().toSourceSplit().getTableId();
+            if (!alreadySendCreateTableTables.contains(tableId)) {
+                try (PostgresConnection jdbc = 
postgresDialect.openJdbcConnection()) {
+                    sendCreateTableEvent(jdbc, tableId, (SourceOutput<Event>) 
output);
+                    alreadySendCreateTableTables.add(tableId);
+                }
+            }
+        } else if (splitState.isStreamSplitState()
+                && !alreadySendCreateTableForBinlogSplit
+                && 
!sourceConfig.getStartupOptions().startupMode.equals(StartupMode.INITIAL)) {
+            for (CreateTableEvent createTableEvent : createTableEventCache) {
+                output.collect((T) createTableEvent);
+            }
+            alreadySendCreateTableForBinlogSplit = true;
+        }
+        super.processElement(element, output, splitState);
+    }
+
+    private void sendCreateTableEvent(
+            PostgresConnection jdbc, TableId tableId, SourceOutput<Event> 
output) {
+        Schema schema = PostgresSchemaUtils.getTableSchema(tableId, 
sourceConfig, jdbc);

Review Comment:
   What if log's schema is different with current table 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to