FIELD_DELIMITER =
diff --git a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQConnectorOptionsUtil.java b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQConnectorOptionsUtil.java
new file mode 100644
index 0000000..c2cefec
--- /dev/null
+++ b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQConnectorOptionsUtil.java
@@ -0,0 +1,151 @@
+/*
+ * 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.rocketmq.flink.source.table;
+
+import org.apache.rocketmq.flink.common.RocketMQOptions;
+import org.apache.rocketmq.flink.common.RocketMQOptions.ValueFieldsStrategy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+import org.apache.flink.util.Preconditions;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.IntStream;
+
+import static org.apache.rocketmq.flink.common.RocketMQOptions.KEY_FIELDS;
+import static org.apache.rocketmq.flink.common.RocketMQOptions.KEY_FIELDS_PREFIX;
+import static org.apache.rocketmq.flink.common.RocketMQOptions.KEY_FORMAT;
+import static org.apache.rocketmq.flink.common.RocketMQOptions.VALUE_FIELDS_INCLUDE;
+
+/** Utilities for {@link RocketMQOptions}. */
+@Internal
+class RocketMQConnectorOptionsUtil {
+
+ /**
+ * Creates an array of indices that determine which physical fields of the table schema to
+ * include in the key format and the order that those fields have in the key format.
+ *
+ * See {@link RocketMQOptions#KEY_FORMAT}, {@link RocketMQOptions#KEY_FIELDS}, and {@link
+ * RocketMQOptions#KEY_FIELDS_PREFIX} for more information.
+ */
+ public static int[] createKeyFormatProjection(
+ ReadableConfig options, DataType physicalDataType) {
+ final LogicalType physicalType = physicalDataType.getLogicalType();
+ Preconditions.checkArgument(
+ physicalType.is(LogicalTypeRoot.ROW), "Row data type expected.");
+ final Optional optionalKeyFormat = options.getOptional(KEY_FORMAT);
+ final Optional> optionalKeyFields = options.getOptional(KEY_FIELDS);
+
+ if (!optionalKeyFormat.isPresent() && optionalKeyFields.isPresent()) {
+ throw new ValidationException(
+ String.format(
+ "The option '%s' can only be declared if a key format is defined using '%s'.",
+ KEY_FIELDS.key(), KEY_FORMAT.key()));
+ } else if (optionalKeyFormat.isPresent()
+ && (!optionalKeyFields.isPresent() || optionalKeyFields.get().size() == 0)) {
+ throw new ValidationException(
+ String.format(
+ "A key format '%s' requires the declaration of one or more of key fields using '%s'.",
+ KEY_FORMAT.key(), KEY_FIELDS.key()));
+ }
+
+ if (!optionalKeyFormat.isPresent()) {
+ return new int[0];
+ }
+
+ final String keyPrefix = options.getOptional(KEY_FIELDS_PREFIX).orElse("");
+
+ final List keyFields = optionalKeyFields.get();
+ final List physicalFields = LogicalTypeChecks.getFieldNames(physicalType);
+ return keyFields.stream()
+ .mapToInt(
+ keyField -> {
+ final int pos = physicalFields.indexOf(keyField);
+ // check that field name exists
+ if (pos < 0) {
+ throw new ValidationException(
+ String.format(
+ "Could not find the field '%s' in the table schema for usage in the key format. "
+ + "A key field must be a regular, physical column. "
+ + "The following columns can be selected in the '%s' option:\n"
+ + "%s",
+ keyField, KEY_FIELDS.key(), physicalFields));
+ }
+ // check that field name is prefixed correctly
+ if (!keyField.startsWith(keyPrefix)) {
+ throw new ValidationException(
+ String.format(
+ "All fields in '%s' must be prefixed with '%s' when option '%s' "
+ + "is set but field '%s' is not prefixed.",
+ KEY_FIELDS.key(),
+ keyPrefix,
+ KEY_FIELDS_PREFIX.key(),
+ keyField));
+ }
+ return pos;
+ })
+ .toArray();
+ }
+ /**
+ * Creates an array of indices that determine which physical fields of the table schema to
+ * include in the value format.
+ *
+ * See {@link RocketMQOptions#VALUE_FORMAT}, {@link RocketMQOptions#VALUE_FIELDS_INCLUDE},
+ * and {@link RocketMQOptions#KEY_FIELDS_PREFIX} for more information.
+ */
+ public static int[] createValueFormatProjection(
+ ReadableConfig options, DataType physicalDataType) {
+ final LogicalType physicalType = physicalDataType.getLogicalType();
+ Preconditions.checkArgument(
+ physicalType.is(LogicalTypeRoot.ROW), "Row data type expected.");
+ final int physicalFieldCount = LogicalTypeChecks.getFieldCount(physicalType);
+ final IntStream physicalFields = IntStream.range(0, physicalFieldCount);
+
+ final String keyPrefix = options.getOptional(KEY_FIELDS_PREFIX).orElse("");
+
+ final ValueFieldsStrategy strategy = options.get(VALUE_FIELDS_INCLUDE);
+ if (strategy == ValueFieldsStrategy.ALL) {
+ if (keyPrefix.length() > 0) {
+ throw new ValidationException(
+ String.format(
+ "A key prefix is not allowed when option '%s' is set to '%s'. "
+ + "Set it to '%s' instead to avoid field overlaps.",
+ VALUE_FIELDS_INCLUDE.key(),
+ ValueFieldsStrategy.ALL,
+ ValueFieldsStrategy.EXCEPT_KEY));
+ }
+ return physicalFields.toArray();
+ } else if (strategy == ValueFieldsStrategy.EXCEPT_KEY) {
+ final int[] keyProjection = createKeyFormatProjection(options, physicalDataType);
+ return physicalFields
+ .filter(pos -> IntStream.of(keyProjection).noneMatch(k -> k == pos))
+ .toArray();
+ }
+ throw new TableException("Unknown value fields strategy:" + strategy);
+ }
+
+ private RocketMQConnectorOptionsUtil() {}
+}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
index 8b4fd52..c0f8e3b 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactory.java
@@ -20,14 +20,23 @@
import org.apache.rocketmq.flink.common.RocketMQOptions;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
import org.apache.flink.table.descriptors.DescriptorProperties;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.FactoryUtil.TableFactoryHelper;
+import org.apache.flink.table.types.DataType;
import org.apache.flink.table.utils.TableSchemaUtils;
+import org.apache.flink.types.RowKind;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.StringUtils;
@@ -36,11 +45,14 @@
import java.text.ParseException;
import java.util.HashSet;
import java.util.Map;
+import java.util.Optional;
import java.util.Set;
import java.util.TimeZone;
+import static org.apache.flink.table.factories.FactoryUtil.FORMAT;
import static org.apache.flink.table.factories.FactoryUtil.createTableFactoryHelper;
import static org.apache.rocketmq.flink.common.RocketMQOptions.CONSUMER_GROUP;
+import static org.apache.rocketmq.flink.common.RocketMQOptions.KEY_FORMAT;
import static org.apache.rocketmq.flink.common.RocketMQOptions.NAME_SERVER_ADDRESS;
import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_ACCESS_KEY;
import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_COLUMN_ERROR_DEBUG;
@@ -61,7 +73,10 @@
import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_TIME_ZONE;
import static org.apache.rocketmq.flink.common.RocketMQOptions.OPTIONAL_USE_NEW_API;
import static org.apache.rocketmq.flink.common.RocketMQOptions.TOPIC;
+import static org.apache.rocketmq.flink.common.RocketMQOptions.VALUE_FORMAT;
import static org.apache.rocketmq.flink.legacy.RocketMQConfig.CONSUMER_OFFSET_LATEST;
+import static org.apache.rocketmq.flink.source.table.RocketMQConnectorOptionsUtil.createKeyFormatProjection;
+import static org.apache.rocketmq.flink.source.table.RocketMQConnectorOptionsUtil.createValueFormatProjection;
/**
* Defines the {@link DynamicTableSourceFactory} implementation to create {@link
@@ -106,12 +121,19 @@ public Set> optionalOptions() {
optionalOptions.add(OPTIONAL_SECRET_KEY);
optionalOptions.add(OPTIONAL_SCAN_STARTUP_MODE);
optionalOptions.add(OPTIONAL_CONSUMER_POLL_MS);
+ optionalOptions.add(FORMAT);
+ optionalOptions.add(KEY_FORMAT);
+ optionalOptions.add(VALUE_FORMAT);
return optionalOptions;
}
@Override
public DynamicTableSource createDynamicTableSource(Context context) {
FactoryUtil.TableFactoryHelper helper = createTableFactoryHelper(this, context);
+ final Optional>> keyDecodingFormat =
+ getKeyDecodingFormat(helper);
+ final Optional>> valueDecodingFormat =
+ getValueDecodingFormat(helper);
helper.validate();
Map rawProperties = context.getCatalogTable().getOptions();
Configuration configuration = Configuration.fromMap(rawProperties);
@@ -183,10 +205,24 @@ public DynamicTableSource createDynamicTableSource(Context context) {
long consumerOffsetTimestamp =
configuration.getLong(
RocketMQOptions.OPTIONAL_OFFSET_FROM_TIMESTAMP, System.currentTimeMillis());
+
+ final ReadableConfig tableOptions = helper.getOptions();
+
+ final DataType physicalDataType =
+ context.getCatalogTable().getResolvedSchema().toPhysicalRowDataType();
+ final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType);
+
+ final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType);
+
return new RocketMQScanTableSource(
configuration.getLong(OPTIONAL_CONSUMER_POLL_MS),
descriptorProperties,
physicalSchema,
+ keyDecodingFormat.orElse(null),
+ valueDecodingFormat.orElse(null),
+ keyProjection,
+ valueProjection,
+ physicalDataType,
topic,
consumerGroup,
nameServerAddress,
@@ -208,4 +244,37 @@ private Long parseDateString(String dateString, String timeZone) throws ParseExc
FastDateFormat.getInstance(DATE_FORMAT, TimeZone.getTimeZone(timeZone));
return simpleDateFormat.parse(dateString).getTime();
}
+
+ private static Optional>> getValueDecodingFormat(
+ TableFactoryHelper helper) {
+ Optional>>
+ deserializationSchemaDecodingFormat =
+ helper.discoverOptionalDecodingFormat(
+ DeserializationFormatFactory.class, FORMAT);
+ if (deserializationSchemaDecodingFormat.isPresent()) {
+ return deserializationSchemaDecodingFormat;
+ }
+
+ return helper.discoverOptionalDecodingFormat(
+ DeserializationFormatFactory.class, VALUE_FORMAT);
+ }
+
+ private static Optional>> getKeyDecodingFormat(
+ TableFactoryHelper helper) {
+ final Optional>> keyDecodingFormat =
+ helper.discoverOptionalDecodingFormat(
+ DeserializationFormatFactory.class, KEY_FORMAT);
+ keyDecodingFormat.ifPresent(
+ format -> {
+ if (!format.getChangelogMode().containsOnly(RowKind.INSERT)) {
+ throw new ValidationException(
+ String.format(
+ "A key format should only deal with INSERT-only records. "
+ + "But %s has a changelog mode of %s.",
+ helper.getOptions().get(KEY_FORMAT),
+ format.getChangelogMode()));
+ }
+ });
+ return keyDecodingFormat;
+ }
}
diff --git a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
index 3eb68df..d549721 100644
--- a/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
+++ b/src/main/java/org/apache/rocketmq/flink/source/table/RocketMQScanTableSource.java
@@ -27,9 +27,13 @@
import org.apache.rocketmq.flink.source.reader.deserializer.RocketMQRowDeserializationSchema;
import org.apache.rocketmq.flink.source.reader.deserializer.RowDeserializationSchema.MetadataConverter;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.connector.format.DecodingFormat;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.connector.source.SourceFunctionProvider;
@@ -39,6 +43,9 @@
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
import java.util.Collections;
import java.util.LinkedHashMap;
@@ -56,6 +63,25 @@ public class RocketMQScanTableSource implements ScanTableSource, SupportsReading
private final DescriptorProperties properties;
private final TableSchema schema;
+ /** Data type that describes the final output of the source. */
+ protected DataType producedDataType;
+
+ /** Optional format for decoding keys from Kafka. */
+ private final @Nullable DecodingFormat<
+ org.apache.flink.api.common.serialization.DeserializationSchema>
+ keyDecodingFormat;
+
+ /** Format for decoding values from Kafka. */
+ private final @Nullable DecodingFormat<
+ org.apache.flink.api.common.serialization.DeserializationSchema>
+ valueDecodingFormat;
+
+ /** Indices that determine the key fields and the target position in the produced row. */
+ protected final int[] keyProjection;
+
+ /** Indices that determine the value fields and the target position in the produced row. */
+ protected final int[] valueProjection;
+
private final String consumerOffsetMode;
private final long consumerOffsetTimestamp;
@@ -81,6 +107,16 @@ public RocketMQScanTableSource(
long pollTime,
DescriptorProperties properties,
TableSchema schema,
+ @Nullable
+ DecodingFormat<
+ org.apache.flink.api.common.serialization.DeserializationSchema<
+ RowData>>
+ keyDecodingFormat,
+ DecodingFormat>
+ valueDecodingFormat,
+ int[] keyProjection,
+ int[] valueProjection,
+ DataType physicalDataType,
String topic,
String consumerGroup,
String nameServerAddress,
@@ -98,6 +134,13 @@ public RocketMQScanTableSource(
this.pollTime = pollTime;
this.properties = properties;
this.schema = schema;
+ this.keyDecodingFormat = keyDecodingFormat;
+ this.valueDecodingFormat = valueDecodingFormat;
+ this.keyProjection =
+ Preconditions.checkNotNull(keyProjection, "Key projection must not be null.");
+ this.valueProjection =
+ Preconditions.checkNotNull(valueProjection, "Value projection must not be null.");
+ this.producedDataType = physicalDataType;
this.topic = topic;
this.consumerGroup = consumerGroup;
this.nameServerAddress = nameServerAddress;
@@ -123,7 +166,19 @@ public ChangelogMode getChangelogMode() {
@Override
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
if (useNewApi) {
- return SourceProvider.of(
+ final org.apache.flink.api.common.serialization.DeserializationSchema
+ keyDeserialization =
+ createDeserialization(scanContext, keyDecodingFormat, keyProjection);
+
+ final org.apache.flink.api.common.serialization.DeserializationSchema
+ valueDeserialization =
+ createDeserialization(
+ scanContext, valueDecodingFormat, valueProjection);
+
+ final TypeInformation producedTypeInfo =
+ scanContext.createTypeInformation(producedDataType);
+
+ RocketMQSource rocketMQSource =
new RocketMQSource<>(
pollTime,
topic,
@@ -138,9 +193,11 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
startMessageOffset < 0 ? 0 : startMessageOffset,
partitionDiscoveryIntervalMs,
isBounded() ? BOUNDED : CONTINUOUS_UNBOUNDED,
- createRocketMQDeserializationSchema(),
+ createRocketMQDeserializationSchema(
+ keyDeserialization, valueDeserialization, producedTypeInfo),
consumerOffsetMode,
- consumerOffsetTimestamp));
+ consumerOffsetTimestamp);
+ return SourceProvider.of(rocketMQSource);
} else {
return SourceFunctionProvider.of(
new RocketMQSourceFunction<>(
@@ -169,6 +226,11 @@ public DynamicTableSource copy() {
pollTime,
properties,
schema,
+ keyDecodingFormat,
+ valueDecodingFormat,
+ keyProjection,
+ valueProjection,
+ producedDataType,
topic,
consumerGroup,
nameServerAddress,
@@ -192,7 +254,11 @@ public String asSummaryString() {
return RocketMQScanTableSource.class.getName();
}
- private RocketMQDeserializationSchema createRocketMQDeserializationSchema() {
+ private RocketMQDeserializationSchema createRocketMQDeserializationSchema(
+ DeserializationSchema keyDeserialization,
+ DeserializationSchema valueDeserialization,
+ TypeInformation producedTypeInfo) {
+
final MetadataConverter[] metadataConverters =
metadataKeys.stream()
.map(
@@ -203,8 +269,14 @@ private RocketMQDeserializationSchema createRocketMQDeserializationSche
.orElseThrow(IllegalStateException::new))
.map(m -> m.converter)
.toArray(MetadataConverter[]::new);
+
return new RocketMQRowDeserializationSchema(
- schema, properties.asMap(), metadataKeys.size() > 0, metadataConverters);
+ schema,
+ keyDeserialization,
+ valueDeserialization,
+ properties.asMap(),
+ metadataKeys.size() > 0,
+ metadataConverters);
}
private boolean isBounded() {
@@ -262,4 +334,22 @@ public Object read(BytesMessage message) {
this.converter = converter;
}
}
+
+ private @Nullable org.apache.flink.api.common.serialization.DeserializationSchema
+ createDeserialization(
+ DynamicTableSource.Context context,
+ @Nullable
+ DecodingFormat<
+ org.apache.flink.api.common.serialization
+ .DeserializationSchema<
+ RowData>>
+ format,
+ int[] projection) {
+ if (format == null) {
+ return null;
+ }
+
+ DataType physicalFormatDataType = Projection.of(projection).project(this.producedDataType);
+ return format.createRuntimeDecoder(context, physicalFormatDataType);
+ }
}
diff --git a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
index 9e78190..30e28df 100644
--- a/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/legacy/RocketMQSourceTest.java
@@ -18,7 +18,6 @@
package org.apache.rocketmq.flink.legacy;
-import java.util.Map;
import org.apache.rocketmq.client.consumer.DefaultLitePullConsumer;
import org.apache.rocketmq.client.consumer.PullResult;
import org.apache.rocketmq.client.consumer.PullStatus;
@@ -36,6 +35,7 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
+import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
diff --git a/src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java b/src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
index a904b04..37cc038 100644
--- a/src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/source/reader/deserializer/RocketMQRowDeserializationSchemaTest.java
@@ -21,11 +21,13 @@
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.flink.api.common.serialization.DeserializationSchema.InitializationContext;
+import org.apache.flink.formats.json.JsonRowDataDeserializationSchema;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.data.RowData;
import org.apache.flink.util.Collector;
+import org.junit.Before;
import org.junit.Test;
import org.powermock.reflect.Whitebox;
@@ -33,6 +35,7 @@
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.doNothing;
@@ -41,9 +44,11 @@
/** Test for {@link RocketMQRowDeserializationSchema}. */
public class RocketMQRowDeserializationSchemaTest {
- @Test
- public void testDeserialize() {
- TableSchema tableSchema =
+ private TableSchema tableSchema;
+
+ @Before
+ public void setup() {
+ tableSchema =
new TableSchema.Builder()
.field("int", DataTypes.INT())
.field("varchar", DataTypes.VARCHAR(100))
@@ -59,8 +64,13 @@ public void testDeserialize() {
.field("time", DataTypes.TIME())
.field("timestamp", DataTypes.TIMESTAMP())
.build();
+ }
+
+ @Test
+ public void testDeserialize() throws Exception {
RocketMQRowDeserializationSchema recordDeserializer =
- new RocketMQRowDeserializationSchema(tableSchema, new HashMap<>(), false, null);
+ new RocketMQRowDeserializationSchema(
+ tableSchema, null, null, new HashMap<>(), false, null);
RowDeserializationSchema sourceDeserializer = mock(RowDeserializationSchema.class);
InitializationContext initializationContext = mock(InitializationContext.class);
doNothing().when(sourceDeserializer).open(initializationContext);
@@ -138,4 +148,95 @@ public void testDeserialize() {
String.valueOf(thirdMsg.getQueueOffset()),
recordDeserializer.getBytesMessages().get(2).getProperty("__queue_offset__"));
}
+
+ @Test
+ public void testJsonDeserialize() throws Exception {
+ Map props = new HashMap<>();
+ props.put("consumergroup", "please_rename_unique_group_name");
+ props.put("nameserveraddress", "10.211.55.5:9876");
+ props.put("schema.0.data-type", "VARCHAR(2147483647)");
+ props.put("connector", "rocketmq");
+ props.put("schema.0.name", "id");
+
+ JsonRowDataDeserializationSchema valueDeserializer =
+ mock(JsonRowDataDeserializationSchema.class);
+ RocketMQRowDeserializationSchema recordDeserializer =
+ new RocketMQRowDeserializationSchema(
+ tableSchema, null, valueDeserializer, props, false, null);
+
+ InitializationContext initializationContext = mock(InitializationContext.class);
+ doNothing().when(valueDeserializer).open(initializationContext);
+ recordDeserializer.open(initializationContext);
+ MessageExt firstMsg =
+ new MessageExt(
+ 1,
+ System.currentTimeMillis(),
+ InetSocketAddress.createUnresolved("localhost", 8080),
+ System.currentTimeMillis(),
+ InetSocketAddress.createUnresolved("localhost", 8088),
+ "184019387");
+ firstMsg.setBody("test_deserializer_raw_messages_1".getBytes());
+ MessageExt secondMsg =
+ new MessageExt(
+ 1,
+ System.currentTimeMillis(),
+ InetSocketAddress.createUnresolved("localhost", 8081),
+ System.currentTimeMillis(),
+ InetSocketAddress.createUnresolved("localhost", 8087),
+ "284019387");
+ secondMsg.setBody("test_deserializer_raw_messages_2".getBytes());
+ MessageExt thirdMsg =
+ new MessageExt(
+ 1,
+ System.currentTimeMillis(),
+ InetSocketAddress.createUnresolved("localhost", 8082),
+ System.currentTimeMillis(),
+ InetSocketAddress.createUnresolved("localhost", 8086),
+ "384019387");
+ thirdMsg.setBody("test_deserializer_raw_messages_3".getBytes());
+ List messages = Arrays.asList(firstMsg, secondMsg, thirdMsg);
+ Collector collector = mock(Collector.class);
+ recordDeserializer.deserialize(messages, collector);
+
+ assertEquals(3, recordDeserializer.getBytesMessages().size());
+ assertEquals(firstMsg.getBody(), recordDeserializer.getBytesMessages().get(0).getData());
+ assertEquals(
+ String.valueOf(firstMsg.getStoreTimestamp()),
+ recordDeserializer.getBytesMessages().get(0).getProperty("__store_timestamp__"));
+ assertEquals(
+ String.valueOf(firstMsg.getBornTimestamp()),
+ recordDeserializer.getBytesMessages().get(0).getProperty("__born_timestamp__"));
+ assertEquals(
+ String.valueOf(firstMsg.getQueueId()),
+ recordDeserializer.getBytesMessages().get(0).getProperty("__queue_id__"));
+ assertEquals(
+ String.valueOf(firstMsg.getQueueOffset()),
+ recordDeserializer.getBytesMessages().get(0).getProperty("__queue_offset__"));
+ assertEquals(secondMsg.getBody(), recordDeserializer.getBytesMessages().get(1).getData());
+ assertEquals(
+ String.valueOf(secondMsg.getStoreTimestamp()),
+ recordDeserializer.getBytesMessages().get(1).getProperty("__store_timestamp__"));
+ assertEquals(
+ String.valueOf(secondMsg.getBornTimestamp()),
+ recordDeserializer.getBytesMessages().get(1).getProperty("__born_timestamp__"));
+ assertEquals(
+ String.valueOf(secondMsg.getQueueId()),
+ recordDeserializer.getBytesMessages().get(1).getProperty("__queue_id__"));
+ assertEquals(
+ String.valueOf(secondMsg.getQueueOffset()),
+ recordDeserializer.getBytesMessages().get(1).getProperty("__queue_offset__"));
+ assertEquals(thirdMsg.getBody(), recordDeserializer.getBytesMessages().get(2).getData());
+ assertEquals(
+ String.valueOf(thirdMsg.getStoreTimestamp()),
+ recordDeserializer.getBytesMessages().get(2).getProperty("__store_timestamp__"));
+ assertEquals(
+ String.valueOf(thirdMsg.getBornTimestamp()),
+ recordDeserializer.getBytesMessages().get(2).getProperty("__born_timestamp__"));
+ assertEquals(
+ String.valueOf(thirdMsg.getQueueId()),
+ recordDeserializer.getBytesMessages().get(2).getProperty("__queue_id__"));
+ assertEquals(
+ String.valueOf(thirdMsg.getQueueOffset()),
+ recordDeserializer.getBytesMessages().get(2).getProperty("__queue_offset__"));
+ }
}
diff --git a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java b/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
index 358e816..5edf02f 100644
--- a/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
+++ b/src/test/java/org/apache/rocketmq/flink/source/table/RocketMQDynamicTableSourceFactoryTest.java
@@ -56,6 +56,8 @@ public class RocketMQDynamicTableSourceFactoryTest {
private static final String CONSUMER_GROUP = "test_consumer";
private static final String NAME_SERVER_ADDRESS = "127.0.0.1:9876";
+ private static final String FORMAT_JSON = "json";
+
@Test
public void testRocketMQDynamicTableSourceWithLegalOption() {
final Map options = new HashMap<>();
@@ -68,6 +70,40 @@ public void testRocketMQDynamicTableSourceWithLegalOption() {
assertEquals(RocketMQScanTableSource.class.getName(), tableSource.asSummaryString());
}
+ @Test
+ public void testRocketMQDynamicTableSourceWithFormatOption() {
+ final Map options = new HashMap<>();
+ options.put("connector", IDENTIFIER);
+ options.put(RocketMQOptions.TOPIC.key(), TOPIC);
+ options.put(RocketMQOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
+ options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
+
+ options.put(FactoryUtil.FORMAT.key(), FORMAT_JSON);
+ final DynamicTableSource tableSource = createTableSource(options);
+
+ assertTrue(tableSource instanceof RocketMQScanTableSource);
+ assertEquals(RocketMQScanTableSource.class.getName(), tableSource.asSummaryString());
+ }
+
+ @Test
+ public void testRocketMQDynamicTableSourceWithJsonOption() {
+ final Map options = new HashMap<>();
+ options.put("connector", IDENTIFIER);
+ options.put(RocketMQOptions.TOPIC.key(), TOPIC);
+ options.put(RocketMQOptions.CONSUMER_GROUP.key(), CONSUMER_GROUP);
+ options.put(RocketMQOptions.NAME_SERVER_ADDRESS.key(), NAME_SERVER_ADDRESS);
+ options.put(FactoryUtil.FORMAT.key(), FORMAT_JSON);
+
+ // json props
+ options.put("json.fail-on-missing-field", "false");
+ options.put("json.ignore-parse-errors", "true");
+ options.put("json.map-null-key.mode", "FAIL");
+ final DynamicTableSource tableSource = createTableSource(options);
+
+ assertTrue(tableSource instanceof RocketMQScanTableSource);
+ assertEquals(RocketMQScanTableSource.class.getName(), tableSource.asSummaryString());
+ }
+
@Test(expected = ValidationException.class)
public void testRocketMQDynamicTableSourceWithoutRequiredOption() {
final Map options = new HashMap<>();