Skip to content

Commit

Permalink
[FLINK-36082][pipeline-connector][kafka] Fix lamda NotSerializableExc…
Browse files Browse the repository at this point in the history
…eption in KafkaDataSink

This closes apache#3549
  • Loading branch information
lvyanquan authored and qiaozongmi committed Sep 23, 2024
1 parent f9053db commit 41d5568
Show file tree
Hide file tree
Showing 2 changed files with 58 additions and 34 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,6 @@

import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
import org.apache.flink.cdc.common.event.SchemaChangeEventType;
import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily;
import org.apache.flink.cdc.common.sink.DataSink;
import org.apache.flink.cdc.common.sink.EventSinkProvider;
import org.apache.flink.cdc.common.sink.FlinkSinkProvider;
Expand All @@ -33,10 +30,7 @@
import org.apache.kafka.clients.producer.ProducerConfig;

import java.time.ZoneId;
import java.util.Arrays;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;

/** A {@link DataSink} for "Kafka" connector. */
public class KafkaDataSink implements DataSink {
Expand Down Expand Up @@ -104,33 +98,6 @@ public EventSinkProvider getEventSinkProvider() {

@Override
public MetadataApplier getMetadataApplier() {
return new MetadataApplier() {

private Set<SchemaChangeEventType> enabledEventTypes =
Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet());

@Override
public MetadataApplier setAcceptedSchemaEvolutionTypes(
Set<SchemaChangeEventType> schemaEvolutionTypes) {
enabledEventTypes = schemaEvolutionTypes;
return this;
}

@Override
public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) {
return enabledEventTypes.contains(schemaChangeEventType);
}

@Override
public Set<SchemaChangeEventType> getSupportedSchemaEvolutionTypes() {
// All schema change events are supported.
return Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet());
}

@Override
public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) {
// simply do nothing here because Kafka do not maintain the schemas.
}
};
return new KafkaMetaDataApplier();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.kafka.sink;

import org.apache.flink.cdc.common.event.SchemaChangeEvent;
import org.apache.flink.cdc.common.event.SchemaChangeEventType;
import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily;
import org.apache.flink.cdc.common.sink.MetadataApplier;

import java.util.Arrays;
import java.util.Set;
import java.util.stream.Collectors;

/** Supports {@link KafkaDataSink} to schema evolution. */
public class KafkaMetaDataApplier implements MetadataApplier {

private Set<SchemaChangeEventType> enabledEventTypes =
Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet());

@Override
public MetadataApplier setAcceptedSchemaEvolutionTypes(
Set<SchemaChangeEventType> schemaEvolutionTypes) {
enabledEventTypes = schemaEvolutionTypes;
return this;
}

@Override
public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) {
return enabledEventTypes.contains(schemaChangeEventType);
}

@Override
public Set<SchemaChangeEventType> getSupportedSchemaEvolutionTypes() {
// All schema change events are supported.
return Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet());
}

@Override
public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) {
// simply do nothing here because Kafka do not maintain the schemas.
}
}

0 comments on commit 41d5568

Please sign in to comment.