Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-36958][pipeline-connector][kafka] Support csv/json/debezium-json/canal-json format options #3830

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public class FactoryHelper {
private final Factory factory;
private final Factory.Context context;

private FactoryHelper(Factory factory, Factory.Context context) {
public FactoryHelper(Factory factory, Factory.Context context) {
this.factory = factory;
this.context = context;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,26 +15,17 @@
* limitations under the License.
*/

package org.apache.flink.cdc.connectors.kafka.sink;
package org.apache.flink.cdc.connectors.kafka.format;

import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.factories.Factory;

import org.apache.kafka.clients.producer.ProducerRecord;
/** Factory for creating a {@link SerializationSchema} for {@link Event}. */
public interface FormatFactory extends Factory {

/** Enum class for building {@link SerializationSchema} for {@link ProducerRecord}. */
public enum KeyFormat {
JSON("json"),
SerializationSchema<Event> createEncodingFormat(Context context, Configuration formatOptions);

CSV("csv");

private final String value;

KeyFormat(String value) {
this.value = value;
}

@Override
public String toString() {
return value;
}
// TODO: definition createDecodingFormat method
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,211 @@
/*
* 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.format;

import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.cdc.common.configuration.ConfigOption;
import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.factories.Factory;
import org.apache.flink.cdc.common.factories.FactoryHelper;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.util.Preconditions;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.ServiceLoader;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;

import static org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions.VALUE_FORMAT;
import static org.apache.flink.table.factories.FactoryUtil.FORMAT_SUFFIX;

/** Helper utility for discovering formats and validating all options. */
public class FormatFactoryHelper extends FactoryHelper {

private final Factory factory;
private final Factory.Context context;

protected final Set<String> consumedOptionKeys;
protected final Configuration allOptions;

public FormatFactoryHelper(Factory factory, Factory.Context context) {
super(factory, context);
this.factory = factory;
this.context = context;
this.allOptions = Configuration.fromMap(context.getFactoryConfiguration().toMap());

final List<ConfigOption<?>> consumedOptions = new ArrayList<>();
consumedOptions.addAll(factory.requiredOptions());
consumedOptions.addAll(factory.optionalOptions());
this.consumedOptionKeys =
consumedOptions.stream().map(ConfigOption::key).collect(Collectors.toSet());
}

public static FormatFactoryHelper createFormatFactoryHelper(
Factory factory, Factory.Context context) {
return new FormatFactoryHelper(factory, context);
}

public SerializationSchema<Event> discoverEncodingFormat(
Class<? extends FormatFactory> formatFactoryClass, ConfigOption<String> formatOption) {
return discoverOptionalEncodingFormat(formatFactoryClass, formatOption)
.orElseThrow(
() ->
new ValidationException(
String.format(
"Could not find required sink format '%s'.",
formatOption.key())));
}

public Optional<SerializationSchema<Event>> discoverOptionalEncodingFormat(
Class<? extends FormatFactory> formatFactoryClass, ConfigOption<String> formatOption) {
return discoverOptionalFormatFactory(formatFactoryClass, formatOption)
.map(
formatFactory -> {
String formatPrefix = formatPrefix(formatFactory, formatOption);
try {
return formatFactory.createEncodingFormat(
context,
removePrefix(
context.getFactoryConfiguration(), formatPrefix));
} catch (Throwable t) {
throw new ValidationException(
String.format(
"Error creating scan format '%s' in option space '%s'.",
formatFactory.identifier(), formatPrefix),
t);
}
});
}

public static Configuration removePrefix(Configuration configuration, String formatPrefix) {
return Configuration.fromMap(
configuration.toMap().entrySet().stream()
.map(
entry -> {
if (entry.getKey().startsWith(formatPrefix)) {
return new HashMap.SimpleEntry<>(
entry.getKey().substring(formatPrefix.length()),
entry.getValue());
} else {
return entry;
}
})
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
}

public Optional<FormatFactory> discoverOptionalFormatFactory(
Class<? extends FormatFactory> formatFactoryClass, ConfigOption<String> formatOption) {
final String identifier = allOptions.get(formatOption);
if (identifier == null) {
return Optional.empty();
}

final FormatFactory factory = discoverFactory(identifier, formatFactoryClass);
String formatPrefix = formatPrefix(factory, formatOption);
final List<ConfigOption<?>> consumedOptions = new ArrayList<>();
consumedOptions.addAll(factory.requiredOptions());
consumedOptions.addAll(factory.optionalOptions());

consumedOptions.stream()
.map(option -> formatPrefix + option.key())
.forEach(consumedOptionKeys::add);

return Optional.of(factory);
}

static <T extends Factory> T discoverFactory(String identifier, Class<T> factoryClass) {

final ServiceLoader<Factory> loader = ServiceLoader.load(Factory.class);
final List<Factory> factoryList = new ArrayList<>();

for (Factory factory : loader) {
if (factory != null
&& factory.identifier().equals(identifier)
&& factoryClass.isAssignableFrom(factory.getClass())) {
factoryList.add(factory);
}
}

if (factoryList.isEmpty()) {
throw new RuntimeException(
String.format(
"UnSupported format \"%s\", cannot find factory with identifier \"%s\" in the classpath.\n\n"
+ "Available factory classes are:\n\n"
+ "%s",
identifier,
identifier,
StreamSupport.stream(loader.spliterator(), false)
.map(f -> f.getClass().getName())
.sorted()
.collect(Collectors.joining("\n"))));
}

if (factoryList.size() > 1) {
throw new RuntimeException(
String.format(
"Multiple factories found in the classpath.\n\n"
+ "Ambiguous factory classes are:\n\n"
+ "%s",
factoryList.stream()
.map(f -> f.getClass().getName())
.sorted()
.collect(Collectors.joining("\n"))));
}

return (T) factoryList.get(0);
}

private String formatPrefix(Factory formatFactory, ConfigOption<String> formatOption) {
String formatIdentifier = formatFactory.identifier();
final String formatOptionKey = formatOption.key();
if (formatOptionKey.equals(VALUE_FORMAT.key())) {
return formatIdentifier + ".";
} else if (formatOptionKey.endsWith(FORMAT_SUFFIX)) {
// extract the key prefix, e.g. extract 'key' from 'key.format'
String keyPrefix =
formatOptionKey.substring(0, formatOptionKey.length() - FORMAT_SUFFIX.length());
return keyPrefix + "." + formatIdentifier + ".";
} else {
throw new ValidationException(
"Format identifier key should be 'format' or suffix with '.format', "
+ "don't support format identifier key '"
+ formatOptionKey
+ "'.");
}
}

@Override
public void validateExcept(String... prefixesToSkip) {
Preconditions.checkArgument(
prefixesToSkip.length > 0, "Prefixes to skip can not be empty.");
final List<String> prefixesList = Arrays.asList(prefixesToSkip);
consumedOptionKeys.addAll(
allOptions.keySet().stream()
.filter(key -> prefixesList.stream().anyMatch(key::startsWith))
.collect(Collectors.toSet()));
validateFactoryOptions(factory, allOptions);
validateUnconsumedKeys(factory.identifier(), allOptions.keySet(), consumedOptionKeys);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/*
* 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.format;

import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.cdc.common.configuration.ConfigOption;
import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.factories.FactoryHelper;
import org.apache.flink.cdc.common.pipeline.PipelineOptions;
import org.apache.flink.formats.common.TimestampFormat;

import java.time.ZoneId;
import java.util.Collections;
import java.util.HashSet;
import java.util.Objects;
import java.util.Set;

import static org.apache.flink.cdc.connectors.kafka.format.JsonFormatOptions.ENCODE_DECIMAL_AS_PLAIN_NUMBER;
import static org.apache.flink.cdc.connectors.kafka.format.JsonFormatOptions.FAIL_ON_MISSING_FIELD;
import static org.apache.flink.cdc.connectors.kafka.format.JsonFormatOptions.IGNORE_PARSE_ERRORS;
import static org.apache.flink.cdc.connectors.kafka.format.JsonFormatOptions.MAP_NULL_KEY_LITERAL;
import static org.apache.flink.cdc.connectors.kafka.format.JsonFormatOptions.MAP_NULL_KEY_MODE;
import static org.apache.flink.cdc.connectors.kafka.format.JsonFormatOptions.TIMESTAMP_FORMAT;
import static org.apache.flink.cdc.connectors.kafka.format.debezium.DebeziumJsonFormatOptions.JSON_MAP_NULL_KEY_LITERAL;

/** Table format factory for providing configured instances of JSON to Event. */
public class JsonFormatFactory implements FormatFactory {

public static final String IDENTIFIER = "json";

@Override
public String identifier() {
return IDENTIFIER;
}

@Override
public SerializationSchema<Event> createEncodingFormat(
Context context, Configuration formatOptions) {
FactoryHelper.validateFactoryOptions(this, context.getFactoryConfiguration());
JsonFormatOptionsUtil.validateEncodingFormatOptions(formatOptions);

TimestampFormat timestampFormat = JsonFormatOptionsUtil.getTimestampFormat(formatOptions);
org.apache.flink.formats.json.JsonFormatOptions.MapNullKeyMode mapNullKeyMode =
JsonFormatOptionsUtil.getMapNullKeyMode(formatOptions);
String mapNullKeyLiteral = formatOptions.get(JSON_MAP_NULL_KEY_LITERAL);

Boolean encodeDecimalAsPlainNumber = formatOptions.get(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
ZoneId zoneId = ZoneId.systemDefault();
if (!Objects.equals(
context.getPipelineConfiguration().get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE),
PipelineOptions.PIPELINE_LOCAL_TIME_ZONE.defaultValue())) {
zoneId =
ZoneId.of(
context.getPipelineConfiguration()
.get(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE));
}
return new JsonSerializationSchema(
timestampFormat,
mapNullKeyMode,
mapNullKeyLiteral,
zoneId,
encodeDecimalAsPlainNumber);
}

@Override
public Set<ConfigOption<?>> requiredOptions() {
return Collections.emptySet();
}

@Override
public Set<ConfigOption<?>> optionalOptions() {
Set<ConfigOption<?>> options = new HashSet<>();
options.add(FAIL_ON_MISSING_FIELD);
options.add(IGNORE_PARSE_ERRORS);
options.add(TIMESTAMP_FORMAT);
options.add(MAP_NULL_KEY_MODE);
options.add(MAP_NULL_KEY_LITERAL);
options.add(ENCODE_DECIMAL_AS_PLAIN_NUMBER);
return options;
}
}
Loading
Loading