diff --git a/flink-format-changelog-json/pom.xml b/flink-format-changelog-json/pom.xml
new file mode 100644
index 000000000..e0de96852
--- /dev/null
+++ b/flink-format-changelog-json/pom.xml
@@ -0,0 +1,76 @@
+
+
+
+
+ flink-cdc-connectors
+ com.alibaba.ververica
+ 1.0.0-SNAPSHOT
+
+ 4.0.0
+
+ flink-format-changelog-json
+ flink-format-changelog-json
+
+ jar
+
+
+
+ org.apache.flink
+ flink-json
+ ${flink.version}
+ provided
+
+
+
+
+
+
+ org.apache.flink
+ flink-table-common
+ ${flink.version}
+ test
+ test-jar
+
+
+
+
+ org.apache.flink
+ flink-table-planner-blink_${scala.binary.version}
+ ${flink.version}
+ test
+
+
+
+
+ org.apache.flink
+ flink-test-utils_${scala.binary.version}
+ ${flink.version}
+ test
+
+
+
+ org.apache.flink
+ flink-table-planner-blink_${scala.binary.version}
+ ${flink.version}
+ test-jar
+ test
+
+
+
\ No newline at end of file
diff --git a/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonDeserializationSchema.java b/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonDeserializationSchema.java
new file mode 100644
index 000000000..5f92968bb
--- /dev/null
+++ b/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonDeserializationSchema.java
@@ -0,0 +1,145 @@
+/*
+ * 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 com.alibaba.ververica.cdc.formats.json;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static java.lang.String.format;
+import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+
+
+/**
+ * Deserialization schema from Changelog Json to Flink Table/SQL internal data structure {@link RowData}.
+ */
+public class ChangelogJsonDeserializationSchema implements DeserializationSchema {
+ private static final long serialVersionUID = -2084214292622004460L;
+
+ /** The deserializer to deserialize Debezium JSON data. */
+ private final JsonRowDataDeserializationSchema jsonDeserializer;
+
+ /** TypeInformation of the produced {@link RowData}. **/
+ private final TypeInformation resultTypeInfo;
+
+ /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */
+ private final boolean ignoreParseErrors;
+
+ public ChangelogJsonDeserializationSchema(
+ RowType rowType,
+ TypeInformation resultTypeInfo,
+ boolean ignoreParseErrors,
+ TimestampFormat timestampFormatOption) {
+ this.resultTypeInfo = resultTypeInfo;
+ this.ignoreParseErrors = ignoreParseErrors;
+ this.jsonDeserializer = new JsonRowDataDeserializationSchema(
+ createJsonRowType(fromLogicalToDataType(rowType)),
+ // the result type is never used, so it's fine to pass in Debezium's result type
+ resultTypeInfo,
+ false, // ignoreParseErrors already contains the functionality of failOnMissingField
+ ignoreParseErrors,
+ timestampFormatOption);
+ }
+
+ @Override
+ public RowData deserialize(byte[] message) throws IOException {
+ throw new RuntimeException(
+ "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead.");
+ }
+
+ @Override
+ public void deserialize(byte[] bytes, Collector out) throws IOException {
+ try {
+ GenericRowData row = (GenericRowData) jsonDeserializer.deserialize(bytes);
+ GenericRowData data = (GenericRowData) row.getField(0);
+ String op = row.getString(1).toString();
+ RowKind rowKind = parseRowKind(op);
+ data.setRowKind(rowKind);
+ out.collect(data);
+ } catch (Throwable t) {
+ // a big try catch to protect the processing.
+ if (!ignoreParseErrors) {
+ throw new IOException(format(
+ "Corrupt Debezium JSON message '%s'.", new String(bytes)), t);
+ }
+ }
+ }
+
+ private static RowKind parseRowKind(String op) {
+ switch (op) {
+ case "+I":
+ return RowKind.INSERT;
+ case "-U":
+ return RowKind.UPDATE_BEFORE;
+ case "+U":
+ return RowKind.UPDATE_AFTER;
+ case "-D":
+ return RowKind.DELETE;
+ default:
+ throw new UnsupportedOperationException("Unsupported operation '" + op + "' for row kind.");
+ }
+ }
+
+ @Override
+ public boolean isEndOfStream(RowData rowData) {
+ return false;
+ }
+
+ @Override
+ public TypeInformation getProducedType() {
+ return resultTypeInfo;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ChangelogJsonDeserializationSchema that = (ChangelogJsonDeserializationSchema) o;
+ return ignoreParseErrors == that.ignoreParseErrors &&
+ Objects.equals(jsonDeserializer, that.jsonDeserializer) &&
+ Objects.equals(resultTypeInfo, that.resultTypeInfo);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(jsonDeserializer, resultTypeInfo, ignoreParseErrors);
+ }
+
+ private static RowType createJsonRowType(DataType databaseSchema) {
+ DataType payload = DataTypes.ROW(
+ DataTypes.FIELD("data", databaseSchema),
+ DataTypes.FIELD("op", DataTypes.STRING()));
+ return (RowType) payload.getLogicalType();
+ }
+}
diff --git a/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonFormatFactory.java b/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonFormatFactory.java
new file mode 100644
index 000000000..828786cc8
--- /dev/null
+++ b/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonFormatFactory.java
@@ -0,0 +1,137 @@
+/*
+ * 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 com.alibaba.ververica.cdc.formats.json;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.formats.json.JsonOptions;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.SerializationFormatFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Format factory for providing configured instances of Debezium JSON to RowData {@link DeserializationSchema}.
+ */
+public class ChangelogJsonFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory {
+
+ public static final String IDENTIFIER = "changelog-json";
+
+ public static final ConfigOption IGNORE_PARSE_ERRORS = JsonOptions.IGNORE_PARSE_ERRORS;
+
+ public static final ConfigOption TIMESTAMP_FORMAT = JsonOptions.TIMESTAMP_FORMAT;
+
+ @Override
+ public DecodingFormat> createDecodingFormat(
+ DynamicTableFactory.Context context, ReadableConfig formatOptions) {
+ FactoryUtil.validateFactoryOptions(this, formatOptions);
+ final boolean ignoreParseErrors = formatOptions.get(IGNORE_PARSE_ERRORS);
+ TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions);
+
+ return new DecodingFormat>() {
+ @SuppressWarnings("unchecked")
+ @Override
+ public DeserializationSchema createRuntimeDecoder(
+ DynamicTableSource.Context context, DataType producedDataType) {
+ final RowType rowType = (RowType) producedDataType.getLogicalType();
+ final TypeInformation rowDataTypeInfo =
+ (TypeInformation) context.createTypeInformation(producedDataType);
+ return new ChangelogJsonDeserializationSchema(
+ rowType,
+ rowDataTypeInfo,
+ ignoreParseErrors,
+ timestampFormat);
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode() {
+ return ChangelogMode.newBuilder()
+ .addContainedKind(RowKind.INSERT)
+ .addContainedKind(RowKind.UPDATE_BEFORE)
+ .addContainedKind(RowKind.UPDATE_AFTER)
+ .addContainedKind(RowKind.DELETE)
+ .build();
+ }
+ };
+ }
+
+ @Override
+ public EncodingFormat> createEncodingFormat(
+ DynamicTableFactory.Context context, ReadableConfig formatOptions) {
+ FactoryUtil.validateFactoryOptions(this, formatOptions);
+ TimestampFormat timestampFormat = JsonOptions.getTimestampFormat(formatOptions);
+
+ return new EncodingFormat>() {
+
+ @Override
+ public ChangelogMode getChangelogMode() {
+ return ChangelogMode.newBuilder()
+ .addContainedKind(RowKind.INSERT)
+ .addContainedKind(RowKind.UPDATE_BEFORE)
+ .addContainedKind(RowKind.UPDATE_AFTER)
+ .addContainedKind(RowKind.DELETE)
+ .build();
+ }
+
+ @Override
+ public SerializationSchema createRuntimeEncoder(DynamicTableSink.Context context, DataType consumedDataType) {
+ final RowType rowType = (RowType) consumedDataType.getLogicalType();
+ return new ChangelogJsonSerializationSchema(
+ rowType,
+ timestampFormat
+ );
+ }
+ };
+ }
+
+ @Override
+ public String factoryIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public Set> requiredOptions() {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public Set> optionalOptions() {
+ Set> options = new HashSet<>();
+ options.add(IGNORE_PARSE_ERRORS);
+ options.add(TIMESTAMP_FORMAT);
+ return options;
+ }
+}
diff --git a/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonSerializationSchema.java b/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonSerializationSchema.java
new file mode 100644
index 000000000..a54064db7
--- /dev/null
+++ b/flink-format-changelog-json/src/main/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonSerializationSchema.java
@@ -0,0 +1,112 @@
+/*
+ * 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 com.alibaba.ververica.cdc.formats.json;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.formats.json.JsonRowDataSerializationSchema;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.RowKind;
+
+import java.util.Objects;
+
+import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+
+/**
+ * Serialization schema from Flink Table/SQL internal data structure {@link RowData} to Changelog Json.
+ */
+public class ChangelogJsonSerializationSchema implements SerializationSchema {
+ private static final long serialVersionUID = -3999450457829887684L;
+
+ private static final StringData OP_INSERT = StringData.fromString("+I");
+ private static final StringData OP_UPDATE_BEFORE = StringData.fromString("-U");
+ private static final StringData OP_UPDATE_AFTER = StringData.fromString("+U");
+ private static final StringData OP_DELETE = StringData.fromString("-D");
+
+ private final JsonRowDataSerializationSchema jsonSerializer;
+
+ /** Timestamp format specification which is used to parse timestamp. */
+ private final TimestampFormat timestampFormat;
+
+ private transient GenericRowData reuse;
+
+ public ChangelogJsonSerializationSchema(RowType rowType, TimestampFormat timestampFormat) {
+ this.jsonSerializer = new JsonRowDataSerializationSchema(
+ createJsonRowType(fromLogicalToDataType(rowType)),
+ timestampFormat);
+ this.timestampFormat = timestampFormat;
+ }
+
+ @Override
+ public void open(InitializationContext context) throws Exception {
+ this.reuse = new GenericRowData(2);
+ }
+
+ @Override
+ public byte[] serialize(RowData rowData) {
+ reuse.setField(0, rowData);
+ reuse.setField(1, stringifyRowKind(rowData.getRowKind()));
+ return jsonSerializer.serialize(reuse);
+ }
+
+ private static StringData stringifyRowKind(RowKind rowKind) {
+ switch (rowKind) {
+ case INSERT:
+ return OP_INSERT;
+ case UPDATE_BEFORE:
+ return OP_UPDATE_BEFORE;
+ case UPDATE_AFTER:
+ return OP_UPDATE_AFTER;
+ case DELETE:
+ return OP_DELETE;
+ default:
+ throw new UnsupportedOperationException("Unsupported operation '" + rowKind + "' for row kind.");
+ }
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ChangelogJsonSerializationSchema that = (ChangelogJsonSerializationSchema) o;
+ return Objects.equals(jsonSerializer, that.jsonSerializer) &&
+ timestampFormat == that.timestampFormat;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(jsonSerializer, timestampFormat);
+ }
+
+ private static RowType createJsonRowType(DataType databaseSchema) {
+ DataType payload = DataTypes.ROW(
+ DataTypes.FIELD("data", databaseSchema),
+ DataTypes.FIELD("op", DataTypes.STRING()));
+ return (RowType) payload.getLogicalType();
+ }
+}
diff --git a/flink-format-changelog-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-format-changelog-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
new file mode 100644
index 000000000..cc04de304
--- /dev/null
+++ b/flink-format-changelog-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -0,0 +1,16 @@
+# 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.
+
+com.alibaba.ververica.cdc.formats.json.ChangelogJsonFormatFactory
diff --git a/flink-format-changelog-json/src/test/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonFormatFactoryTest.java b/flink-format-changelog-json/src/test/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonFormatFactoryTest.java
new file mode 100644
index 000000000..b4891c879
--- /dev/null
+++ b/flink-format-changelog-json/src/test/java/com/alibaba/ververica/cdc/formats/json/ChangelogJsonFormatFactoryTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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 com.alibaba.ververica.cdc.formats.json;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.formats.json.TimestampFormat;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.catalog.CatalogTableImpl;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.TestDynamicTableFactory;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Consumer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link ChangelogJsonFormatFactoryTest}.
+ */
+public class ChangelogJsonFormatFactoryTest extends TestLogger {
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
+
+ private static final TableSchema SCHEMA = TableSchema.builder()
+ .field("a", DataTypes.STRING())
+ .field("b", DataTypes.INT())
+ .field("c", DataTypes.BOOLEAN())
+ .build();
+
+ private static final RowType ROW_TYPE = (RowType) SCHEMA.toRowDataType().getLogicalType();
+
+ @Test
+ public void testSeDeSchema() {
+ final ChangelogJsonDeserializationSchema expectedDeser = new ChangelogJsonDeserializationSchema(
+ ROW_TYPE,
+ RowDataTypeInfo.of(ROW_TYPE),
+ true,
+ TimestampFormat.ISO_8601);
+ final ChangelogJsonSerializationSchema expectedSer = new ChangelogJsonSerializationSchema(
+ ROW_TYPE,
+ TimestampFormat.ISO_8601);
+
+ final Map options = getAllOptions();
+
+ final DynamicTableSource actualSource = createTableSource(options);
+ assert actualSource instanceof TestDynamicTableFactory.DynamicTableSourceMock;
+ TestDynamicTableFactory.DynamicTableSourceMock scanSourceMock =
+ (TestDynamicTableFactory.DynamicTableSourceMock) actualSource;
+
+ DeserializationSchema actualDeser = scanSourceMock.valueFormat
+ .createRuntimeDecoder(
+ ScanRuntimeProviderContext.INSTANCE,
+ SCHEMA.toRowDataType());
+
+ assertEquals(expectedDeser, actualDeser);
+
+ final DynamicTableSink actualSink = createTableSink(options);
+ assert actualSink instanceof TestDynamicTableFactory.DynamicTableSinkMock;
+ TestDynamicTableFactory.DynamicTableSinkMock sinkMock = (TestDynamicTableFactory.DynamicTableSinkMock) actualSink;
+
+ SerializationSchema actualSer = sinkMock.valueFormat
+ .createRuntimeEncoder(
+ new SinkRuntimeProviderContext(false),
+ SCHEMA.toRowDataType());
+
+ assertEquals(expectedSer, actualSer);
+ }
+
+ @Test
+ public void testInvalidIgnoreParseError() {
+ final Map options =
+ getModifiedOptions(opts -> opts.put("changelog-json.ignore-parse-errors", "abc"));
+
+ try {
+ createTableSource(options);
+ } catch (Exception e) {
+ assertTrue(ExceptionUtils.findThrowableWithMessage(
+ e,
+ "Unrecognized option for boolean: abc. Expected either true or false(case insensitive)").isPresent());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ /**
+ * Returns the full options modified by the given consumer {@code optionModifier}.
+ *
+ * @param optionModifier Consumer to modify the options
+ */
+ private Map getModifiedOptions(Consumer